Update interface name of GetIndexStates

Signed-off-by: cai.zhang <cai.zhang@zilliz.com>
pull/4973/head^2
cai.zhang 2021-01-19 18:32:57 +08:00 committed by yefu.chen
parent 918e279869
commit fb493cc235
58 changed files with 2995 additions and 1494 deletions

View File

@ -57,3 +57,11 @@ queryNode:
indexBuilder:
address: localhost
port: 31000
indexNode:
address: localhost
port: 21116
indexServer:
address: localhost
port: 21118

View File

@ -81,7 +81,7 @@ type Component interface {
Init()
Start()
Stop()
GetComponentStates() (ComponentStates, error)
GetServiceStates() (ServiceStates, error)
GetTimeTickChannel() (string, error)
GetStatisticsChannel() (string, error)
}
@ -100,16 +100,17 @@ const (
ABNORMAL StateCode = 2
)
type ComponentInfo struct {
type NodeStates struct {
NodeID UniqueID
Role string
StateCode StateCode
ExtraInfo KeyValuePair
}
type ComponentStates struct {
States ComponentInfo
SubcomponentStates []ComponentInfo
type ServiceStates struct {
StateCode StateCode
NodeStates []NodeStates
ExtraInfo KeyValuePair
}
```

View File

@ -98,6 +98,20 @@ type DescribeCollectionResponse struct {
}
```
* *GetCollectionStatistics*
```go
type CollectionStatsRequest struct {
MsgBase
DbName string
CollectionName string
}
type CollectionStatsResponse struct {
Stats []KeyValuePair
}
```
* *ShowCollections*
```go
@ -145,6 +159,21 @@ type HasPartitionRequest struct {
}
```
* *GetPartitionStatistics*
```go
type PartitionStatsRequest struct {
MsgBase
DbName string
CollectionName string
PartitionName string
}
type PartitionStatsResponse struct {
Stats []KeyValuePair
}
```
* *ShowPartitions*
```go

View File

@ -21,8 +21,6 @@ enum SegmentType {
Sealed = 2,
};
typedef enum SegmentType SegmentType;
enum ErrorCode {
Success = 0,
UnexpectedException = 1,
@ -33,12 +31,6 @@ typedef struct CStatus {
const char* error_msg;
} CStatus;
typedef struct CLoadFieldDataInfo {
int64_t field_id;
void* blob;
int64_t row_count;
} CLoadFieldDataInfo;
#ifdef __cplusplus
}
#endif

View File

@ -21,9 +21,8 @@
#include <knowhere/index/vector_index/VecIndex.h>
#include <knowhere/index/vector_index/adapter/VectorAdapter.h>
////////////////////////////// common interfaces //////////////////////////////
CSegmentInterface
NewSegment(CCollection collection, uint64_t segment_id, SegmentType seg_type) {
NewSegment(CCollection collection, uint64_t segment_id, int seg_type) {
auto col = (milvus::segcore::Collection*)collection;
std::unique_ptr<milvus::segcore::SegmentInterface> segment;
@ -47,7 +46,6 @@ NewSegment(CCollection collection, uint64_t segment_id, SegmentType seg_type) {
void
DeleteSegment(CSegmentInterface segment) {
// TODO: use dynamic cast, and return c status
auto s = (milvus::segcore::SegmentGrowing*)segment;
std::cout << "delete segment " << std::endl;
@ -60,86 +58,8 @@ DeleteQueryResult(CQueryResult query_result) {
delete res;
}
CStatus
Search(CSegmentInterface c_segment,
CPlan c_plan,
CPlaceholderGroup* c_placeholder_groups,
uint64_t* timestamps,
int num_groups,
CQueryResult* result) {
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
auto plan = (milvus::query::Plan*)c_plan;
std::vector<const milvus::query::PlaceholderGroup*> placeholder_groups;
for (int i = 0; i < num_groups; ++i) {
placeholder_groups.push_back((const milvus::query::PlaceholderGroup*)c_placeholder_groups[i]);
}
//////////////////////////////////////////////////////////////////
auto query_result = std::make_unique<milvus::QueryResult>();
auto status = CStatus();
try {
*query_result = segment->Search(plan, placeholder_groups.data(), timestamps, num_groups);
if (plan->plan_node_->query_info_.metric_type_ != "IP") {
for (auto& dis : query_result->result_distances_) {
dis *= -1;
}
}
status.error_code = Success;
status.error_msg = "";
} catch (std::exception& e) {
status.error_code = UnexpectedException;
status.error_msg = strdup(e.what());
}
*result = query_result.release();
// result_ids and result_distances have been allocated memory in goLang,
// so we don't need to malloc here.
// memcpy(result_ids, query_result.result_ids_.data(), query_result.get_row_count() * sizeof(long int));
// memcpy(result_distances, query_result.result_distances_.data(), query_result.get_row_count() * sizeof(float));
return status;
}
CStatus
FillTargetEntry(CSegmentInterface c_segment, CPlan c_plan, CQueryResult c_result) {
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
auto plan = (milvus::query::Plan*)c_plan;
auto result = (milvus::QueryResult*)c_result;
auto status = CStatus();
try {
segment->FillTargetEntry(plan, *result);
status.error_code = Success;
status.error_msg = "";
} catch (std::runtime_error& e) {
status.error_code = UnexpectedException;
status.error_msg = strdup(e.what());
}
return status;
}
int64_t
GetMemoryUsageInBytes(CSegmentInterface c_segment) {
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
auto mem_size = segment->GetMemoryUsageInBytes();
return mem_size;
}
int64_t
GetRowCount(CSegmentInterface c_segment) {
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
auto row_count = segment->get_row_count();
return row_count;
}
int64_t
GetDeletedCount(CSegmentInterface c_segment) {
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
auto deleted_count = segment->get_deleted_count();
return deleted_count;
}
////////////////////////////// interfaces for growing segment //////////////////////////////
CStatus
Insert(CSegmentInterface c_segment,
int64_t reserved_offset,
@ -203,33 +123,70 @@ Delete(CSegmentInterface c_segment,
int64_t
PreDelete(CSegmentInterface c_segment, int64_t size) {
// TODO: use dynamic cast, and return c status
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
return segment->PreDelete(size);
}
////////////////////////////// interfaces for growing segment //////////////////////////////
CStatus
LoadFieldData(CSegmentInterface c_segment, CLoadFieldDataInfo load_field_data_info) {
auto segment = (milvus::segcore::SegmentSealed*)c_segment;
Search(CSegmentInterface c_segment,
CPlan c_plan,
CPlaceholderGroup* c_placeholder_groups,
uint64_t* timestamps,
int num_groups,
CQueryResult* result) {
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
auto plan = (milvus::query::Plan*)c_plan;
std::vector<const milvus::query::PlaceholderGroup*> placeholder_groups;
for (int i = 0; i < num_groups; ++i) {
placeholder_groups.push_back((const milvus::query::PlaceholderGroup*)c_placeholder_groups[i]);
}
auto query_result = std::make_unique<milvus::QueryResult>();
auto status = CStatus();
try {
auto load_info =
LoadFieldDataInfo{load_field_data_info.field_id, load_field_data_info.blob, load_field_data_info.row_count};
segment->LoadFieldData(load_info);
auto status = CStatus();
*query_result = segment->Search(plan, placeholder_groups.data(), timestamps, num_groups);
if (plan->plan_node_->query_info_.metric_type_ != "IP") {
for (auto& dis : query_result->result_distances_) {
dis *= -1;
}
}
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedException;
status.error_msg = strdup(e.what());
return status;
}
*result = query_result.release();
// result_ids and result_distances have been allocated memory in goLang,
// so we don't need to malloc here.
// memcpy(result_ids, query_result.result_ids_.data(), query_result.get_row_count() * sizeof(long int));
// memcpy(result_distances, query_result.result_distances_.data(), query_result.get_row_count() * sizeof(float));
return status;
}
CStatus
FillTargetEntry(CSegmentInterface c_segment, CPlan c_plan, CQueryResult c_result) {
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
auto plan = (milvus::query::Plan*)c_plan;
auto result = (milvus::QueryResult*)c_result;
auto status = CStatus();
try {
segment->FillTargetEntry(plan, *result);
status.error_code = Success;
status.error_msg = "";
} catch (std::runtime_error& e) {
status.error_code = UnexpectedException;
status.error_msg = strdup(e.what());
}
return status;
}
////////////////////////////// deprecated interfaces //////////////////////////////
CStatus
UpdateSegmentIndex(CSegmentInterface c_segment, CLoadIndexInfo c_load_index_info) {
auto status = CStatus();
@ -246,6 +203,7 @@ UpdateSegmentIndex(CSegmentInterface c_segment, CLoadIndexInfo c_load_index_info
return status;
}
}
//////////////////////////////////////////////////////////////////
int
Close(CSegmentInterface c_segment) {
@ -265,3 +223,26 @@ IsOpened(CSegmentInterface c_segment) {
auto status = segment->get_state();
return status == milvus::segcore::SegmentGrowing::SegmentState::Open;
}
int64_t
GetMemoryUsageInBytes(CSegmentInterface c_segment) {
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
auto mem_size = segment->GetMemoryUsageInBytes();
return mem_size;
}
//////////////////////////////////////////////////////////////////
int64_t
GetRowCount(CSegmentInterface c_segment) {
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
auto row_count = segment->get_row_count();
return row_count;
}
int64_t
GetDeletedCount(CSegmentInterface c_segment) {
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
auto deleted_count = segment->get_deleted_count();
return deleted_count;
}

View File

@ -26,9 +26,8 @@ extern "C" {
typedef void* CSegmentInterface;
typedef void* CQueryResult;
////////////////////////////// common interfaces //////////////////////////////
CSegmentInterface
NewSegment(CCollection collection, uint64_t segment_id, SegmentType seg_type);
NewSegment(CCollection collection, uint64_t segment_id, int seg_type);
void
DeleteSegment(CSegmentInterface segment);
@ -36,27 +35,9 @@ DeleteSegment(CSegmentInterface segment);
void
DeleteQueryResult(CQueryResult query_result);
CStatus
Search(CSegmentInterface c_segment,
CPlan plan,
CPlaceholderGroup* placeholder_groups,
uint64_t* timestamps,
int num_groups,
CQueryResult* result);
//////////////////////////////////////////////////////////////////
CStatus
FillTargetEntry(CSegmentInterface c_segment, CPlan c_plan, CQueryResult result);
int64_t
GetMemoryUsageInBytes(CSegmentInterface c_segment);
int64_t
GetRowCount(CSegmentInterface c_segment);
int64_t
GetDeletedCount(CSegmentInterface c_segment);
////////////////////////////// interfaces for growing segment //////////////////////////////
// interface for growing segment
CStatus
Insert(CSegmentInterface c_segment,
int64_t reserved_offset,
@ -83,13 +64,23 @@ Delete(CSegmentInterface c_segment,
int64_t
PreDelete(CSegmentInterface c_segment, int64_t size);
////////////////////////////// interfaces for growing segment //////////////////////////////
// common interface
CStatus
LoadFieldData(CSegmentInterface c_segment, CLoadFieldDataInfo load_field_data_info);
Search(CSegmentInterface c_segment,
CPlan plan,
CPlaceholderGroup* placeholder_groups,
uint64_t* timestamps,
int num_groups,
CQueryResult* result);
////////////////////////////// deprecated interfaces //////////////////////////////
// common interface
CStatus
FillTargetEntry(CSegmentInterface c_segment, CPlan c_plan, CQueryResult result);
// deprecated
CStatus
UpdateSegmentIndex(CSegmentInterface c_segment, CLoadIndexInfo c_load_index_info);
//////////////////////////////////////////////////////////////////
// deprecated
int
@ -103,6 +94,20 @@ BuildIndex(CCollection c_collection, CSegmentInterface c_segment);
bool
IsOpened(CSegmentInterface c_segment);
// common interface
int64_t
GetMemoryUsageInBytes(CSegmentInterface c_segment);
//////////////////////////////////////////////////////////////////
// common interface
int64_t
GetRowCount(CSegmentInterface c_segment);
// ???
int64_t
GetDeletedCount(CSegmentInterface c_segment);
#ifdef __cplusplus
}
#endif

View File

@ -52,7 +52,7 @@ TEST(CApiTest, GetCollectionNameTest) {
TEST(CApiTest, SegmentTest) {
auto schema_tmp_conf = "";
auto collection = NewCollection(schema_tmp_conf);
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
DeleteCollection(collection);
DeleteSegment(segment);
}
@ -60,7 +60,7 @@ TEST(CApiTest, SegmentTest) {
TEST(CApiTest, InsertTest) {
auto schema_tmp_conf = "";
auto collection = NewCollection(schema_tmp_conf);
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
std::vector<char> raw_data;
std::vector<uint64_t> timestamps;
@ -95,7 +95,7 @@ TEST(CApiTest, InsertTest) {
TEST(CApiTest, DeleteTest) {
auto schema_tmp_conf = "";
auto collection = NewCollection(schema_tmp_conf);
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
long delete_row_ids[] = {100000, 100001, 100002};
unsigned long delete_timestamps[] = {0, 0, 0};
@ -112,7 +112,7 @@ TEST(CApiTest, DeleteTest) {
TEST(CApiTest, SearchTest) {
auto schema_tmp_conf = "";
auto collection = NewCollection(schema_tmp_conf);
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
std::vector<char> raw_data;
std::vector<uint64_t> timestamps;
@ -201,7 +201,7 @@ TEST(CApiTest, SearchTest) {
// TEST(CApiTest, BuildIndexTest) {
// auto schema_tmp_conf = "";
// auto collection = NewCollection(schema_tmp_conf);
// auto segment = NewSegment(collection, 0, Growing);
// auto segment = NewSegment(collection, 0, 1);
//
// std::vector<char> raw_data;
// std::vector<uint64_t> timestamps;
@ -285,7 +285,7 @@ TEST(CApiTest, SearchTest) {
TEST(CApiTest, IsOpenedTest) {
auto schema_tmp_conf = "";
auto collection = NewCollection(schema_tmp_conf);
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
auto is_opened = IsOpened(segment);
assert(is_opened);
@ -297,7 +297,7 @@ TEST(CApiTest, IsOpenedTest) {
TEST(CApiTest, CloseTest) {
auto schema_tmp_conf = "";
auto collection = NewCollection(schema_tmp_conf);
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
auto status = Close(segment);
assert(status == 0);
@ -309,7 +309,7 @@ TEST(CApiTest, CloseTest) {
TEST(CApiTest, GetMemoryUsageInBytesTest) {
auto schema_tmp_conf = "";
auto collection = NewCollection(schema_tmp_conf);
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
auto old_memory_usage_size = GetMemoryUsageInBytes(segment);
std::cout << "old_memory_usage_size = " << old_memory_usage_size << std::endl;
@ -428,7 +428,7 @@ generate_index(
// TEST(CApiTest, TestSearchPreference) {
// auto schema_tmp_conf = "";
// auto collection = NewCollection(schema_tmp_conf);
// auto segment = NewSegment(collection, 0, Growing);
// auto segment = NewSegment(collection, 0, 1);
//
// auto beg = chrono::high_resolution_clock::now();
// auto next = beg;
@ -547,7 +547,7 @@ generate_index(
TEST(CApiTest, GetDeletedCountTest) {
auto schema_tmp_conf = "";
auto collection = NewCollection(schema_tmp_conf);
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
long delete_row_ids[] = {100000, 100001, 100002};
unsigned long delete_timestamps[] = {0, 0, 0};
@ -568,7 +568,7 @@ TEST(CApiTest, GetDeletedCountTest) {
TEST(CApiTest, GetRowCountTest) {
auto schema_tmp_conf = "";
auto collection = NewCollection(schema_tmp_conf);
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
int N = 10000;
auto [raw_data, timestamps, uids] = generate_data(N);
@ -592,7 +592,7 @@ TEST(CApiTest, GetRowCountTest) {
// "\u003e\ncreate_time: 1600416765\nsegment_ids: 6873737669791618215\npartition_tags: \"default\"\n";
//
// auto collection = NewCollection(schema_string.data());
// auto segment = NewSegment(collection, 0, Growing);
// auto segment = NewSegment(collection, 0, 1);
// DeleteCollection(collection);
// DeleteSegment(segment);
//}
@ -629,7 +629,7 @@ TEST(CApiTest, MergeInto) {
TEST(CApiTest, Reduce) {
auto schema_tmp_conf = "";
auto collection = NewCollection(schema_tmp_conf);
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
std::vector<char> raw_data;
std::vector<uint64_t> timestamps;
@ -845,7 +845,7 @@ TEST(CApiTest, UpdateSegmentIndex_Without_Predicate) {
std::string schema_string = generate_collection_shema("L2", "16", false);
auto collection = NewCollection(schema_string.c_str());
auto schema = ((segcore::Collection*)collection)->get_schema();
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
auto N = 1000 * 1000;
auto dataset = DataGen(schema, N);
@ -970,7 +970,7 @@ TEST(CApiTest, UpdateSegmentIndex_With_float_Predicate_Range) {
std::string schema_string = generate_collection_shema("L2", "16", false);
auto collection = NewCollection(schema_string.c_str());
auto schema = ((segcore::Collection*)collection)->get_schema();
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
auto N = 1000 * 1000;
auto dataset = DataGen(schema, N);
@ -1108,7 +1108,7 @@ TEST(CApiTest, UpdateSegmentIndex_With_float_Predicate_Term) {
std::string schema_string = generate_collection_shema("L2", "16", false);
auto collection = NewCollection(schema_string.c_str());
auto schema = ((segcore::Collection*)collection)->get_schema();
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
auto N = 1000 * 1000;
auto dataset = DataGen(schema, N);
@ -1245,7 +1245,7 @@ TEST(CApiTest, UpdateSegmentIndex_With_binary_Predicate_Range) {
std::string schema_string = generate_collection_shema("JACCARD", "16", true);
auto collection = NewCollection(schema_string.c_str());
auto schema = ((segcore::Collection*)collection)->get_schema();
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
auto N = 1000 * 1000;
auto dataset = DataGen(schema, N);
@ -1384,7 +1384,7 @@ TEST(CApiTest, UpdateSegmentIndex_With_binary_Predicate_Term) {
std::string schema_string = generate_collection_shema("JACCARD", "16", true);
auto collection = NewCollection(schema_string.c_str());
auto schema = ((segcore::Collection*)collection)->get_schema();
auto segment = NewSegment(collection, 0, Growing);
auto segment = NewSegment(collection, 0, 1);
auto N = 1000 * 1000;
auto dataset = DataGen(schema, N);

View File

@ -1,8 +1,14 @@
package grpcindexnode
import (
"context"
"log"
"time"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"google.golang.org/grpc"
)
type Client struct {
@ -10,10 +16,12 @@ type Client struct {
}
func (c Client) Init() {
//TODO:???
panic("implement me")
}
func (c Client) Start() {
//TODO:???
panic("implement me")
}
@ -33,10 +41,22 @@ func (c Client) GetStatisticsChannel() (string, error) {
panic("implement me")
}
func (c Client) BuildIndex(req indexpb.BuildIndexRequest) (indexpb.BuildIndexResponse, error) {
panic("implement me")
func (c Client) BuildIndex(req *indexpb.BuildIndexCmd) (*commonpb.Status, error) {
ctx := context.TODO()
return c.grpcClient.BuildIndex(ctx, req)
}
func NewClient() *Client {
return &Client{}
func NewClient(nodeAddress string) *Client {
ctx1, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()
conn, err := grpc.DialContext(ctx1, nodeAddress, grpc.WithInsecure(), grpc.WithBlock())
if err != nil {
log.Printf("IndexNode connect to IndexService failed, error= %v", err)
}
log.Printf("IndexNode connected to IndexService, IndexService=%s", Params.Address)
return &Client{
grpcClient: indexpb.NewIndexNodeClient(conn),
}
}

View File

@ -0,0 +1,64 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License.
package main
import (
"context"
"log"
"os"
"os/signal"
"syscall"
grpcindexnode "github.com/zilliztech/milvus-distributed/internal/distributed/indexnode"
"go.uber.org/zap"
)
func main() {
grpcindexnode.Init()
ctx, cancel := context.WithCancel(context.Background())
svr, err := grpcindexnode.CreateIndexNode(ctx)
if err != nil {
log.Print("create server failed", zap.Error(err))
}
sc := make(chan os.Signal, 1)
signal.Notify(sc,
syscall.SIGHUP,
syscall.SIGINT,
syscall.SIGTERM,
syscall.SIGQUIT)
var sig os.Signal
go func() {
sig = <-sc
cancel()
}()
if err := svr.Start(); err != nil {
log.Fatal("run builder server failed", zap.Error(err))
}
<-ctx.Done()
log.Print("Got signal to exit", zap.String("signal", sig.String()))
svr.Close()
switch sig {
case syscall.SIGTERM:
exit(0)
default:
exit(1)
}
}
func exit(code int) {
os.Exit(code)
}

View File

@ -0,0 +1,177 @@
package grpcindexnode
import (
"net"
"strconv"
"github.com/zilliztech/milvus-distributed/internal/util/paramtable"
)
type ParamTable struct {
paramtable.BaseTable
Address string
Port int
ServiceAddress string
ServicePort int
NodeID int64
MasterAddress string
EtcdAddress string
MetaRootPath string
MinIOAddress string
MinIOAccessKeyID string
MinIOSecretAccessKey string
MinIOUseSSL bool
MinioBucketName string
}
var Params ParamTable
func (pt *ParamTable) Init() {
pt.BaseTable.Init()
pt.initAddress()
pt.initPort()
pt.initIndexServerAddr()
pt.initIndexServerPort()
pt.initEtcdAddress()
pt.initMasterAddress()
pt.initMetaRootPath()
pt.initMinIOAddress()
pt.initMinIOAccessKeyID()
pt.initMinIOSecretAccessKey()
pt.initMinIOUseSSL()
pt.initMinioBucketName()
}
func (pt *ParamTable) initAddress() {
addr, err := pt.Load("indexNode.address")
if err != nil {
panic(err)
}
hostName, _ := net.LookupHost(addr)
if len(hostName) <= 0 {
if ip := net.ParseIP(addr); ip == nil {
panic("invalid ip indexBuilder.address")
}
}
port, err := pt.Load("indexNode.port")
if err != nil {
panic(err)
}
_, err = strconv.Atoi(port)
if err != nil {
panic(err)
}
pt.Address = addr + ":" + port
}
func (pt *ParamTable) initPort() {
pt.Port = pt.ParseInt("indexNode.port")
}
func (pt *ParamTable) initIndexServerAddr() {
addr, err := pt.Load("indexServer.address")
if err != nil {
panic(err)
}
hostName, _ := net.LookupHost(addr)
if len(hostName) <= 0 {
if ip := net.ParseIP(addr); ip == nil {
panic("invalid ip indexBuilder.address")
}
}
port, err := pt.Load("indexServer.port")
if err != nil {
panic(err)
}
_, err = strconv.Atoi(port)
if err != nil {
panic(err)
}
pt.ServiceAddress = addr + ":" + port
}
func (pt ParamTable) initIndexServerPort() {
pt.ServicePort = pt.ParseInt("indexServer.port")
}
func (pt *ParamTable) initEtcdAddress() {
addr, err := pt.Load("_EtcdAddress")
if err != nil {
panic(err)
}
pt.EtcdAddress = addr
}
func (pt *ParamTable) initMetaRootPath() {
rootPath, err := pt.Load("etcd.rootPath")
if err != nil {
panic(err)
}
subPath, err := pt.Load("etcd.metaSubPath")
if err != nil {
panic(err)
}
pt.MetaRootPath = rootPath + "/" + subPath
}
func (pt *ParamTable) initMasterAddress() {
ret, err := pt.Load("_MasterAddress")
if err != nil {
panic(err)
}
pt.MasterAddress = ret
}
func (pt *ParamTable) initMinIOAddress() {
ret, err := pt.Load("_MinioAddress")
if err != nil {
panic(err)
}
pt.MinIOAddress = ret
}
func (pt *ParamTable) initMinIOAccessKeyID() {
ret, err := pt.Load("minio.accessKeyID")
if err != nil {
panic(err)
}
pt.MinIOAccessKeyID = ret
}
func (pt *ParamTable) initMinIOSecretAccessKey() {
ret, err := pt.Load("minio.secretAccessKey")
if err != nil {
panic(err)
}
pt.MinIOSecretAccessKey = ret
}
func (pt *ParamTable) initMinIOUseSSL() {
ret, err := pt.Load("minio.useSSL")
if err != nil {
panic(err)
}
pt.MinIOUseSSL, err = strconv.ParseBool(ret)
if err != nil {
panic(err)
}
}
func (pt *ParamTable) initMinioBucketName() {
bucketName, err := pt.Load("minio.bucketName")
if err != nil {
panic(err)
}
pt.MinioBucketName = bucketName
}

View File

@ -1,7 +1,16 @@
package grpcindexnode
import (
"context"
"log"
"net"
"strconv"
"sync"
grpcindexservice "github.com/zilliztech/milvus-distributed/internal/distributed/indexservice"
"github.com/zilliztech/milvus-distributed/internal/indexnode"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
"google.golang.org/grpc"
)
@ -9,11 +18,102 @@ type Server struct {
node indexnode.Interface
grpcServer *grpc.Server
indexNodeLoopCtx context.Context
indexNodeLoopCancel func()
indexNodeLoopWg sync.WaitGroup
}
func NewGrpcServer() *Server {
ret := &Server{
node: &indexnode.IndexNode{},
func NewGrpcServer(ctx context.Context, indexID int64) *Server {
return &Server{
node: indexnode.NewIndexNode(ctx, indexID),
}
return ret
}
func registerNode() error {
indexServiceClient := grpcindexservice.NewClient(Params.ServiceAddress)
request := &indexpb.RegisterNodeRequest{
Base: nil,
Address: &commonpb.Address{
Ip: Params.Address,
Port: int64(Params.Port),
},
}
resp, err := indexServiceClient.RegisterNode(request)
if err != nil {
log.Printf("IndexNode connect to IndexService failed, error= %v", err)
return err
}
Params.NodeID = resp.InitParams.NodeID
log.Println("Register indexNode successful with nodeID=", Params.NodeID)
err = Params.LoadFromKVPair(resp.InitParams.StartParams)
return err
}
func (s *Server) grpcLoop() {
defer s.indexNodeLoopWg.Done()
lis, err := net.Listen("tcp", ":"+strconv.Itoa(Params.Port))
if err != nil {
log.Fatalf("IndexNode grpc server fatal error=%v", err)
}
s.grpcServer = grpc.NewServer()
indexpb.RegisterIndexNodeServer(s.grpcServer, s)
if err = s.grpcServer.Serve(lis); err != nil {
log.Fatalf("IndexNode grpc server fatal error=%v", err)
}
log.Println("IndexNode grpc server starting...")
}
func (s *Server) startIndexNode() error {
s.indexNodeLoopWg.Add(1)
//TODO: How to make sure that grpc server has started successfully
go s.grpcLoop()
err := registerNode()
if err != nil {
return err
}
Params.Init()
return nil
}
func Init() {
Params.Init()
}
func CreateIndexNode(ctx context.Context) (*Server, error) {
ctx1, cancel := context.WithCancel(ctx)
s := &Server{
indexNodeLoopCtx: ctx1,
indexNodeLoopCancel: cancel,
}
return s, nil
}
func (s *Server) Start() error {
return s.startIndexNode()
}
func (s *Server) Stop() {
s.indexNodeLoopWg.Wait()
}
func (s *Server) Close() {
s.Stop()
}
func (s *Server) BuildIndex(ctx context.Context, req *indexpb.BuildIndexCmd) (*commonpb.Status, error) {
return s.node.BuildIndex(req)
}

View File

@ -1,6 +1,14 @@
package grpcindexservice
import (
"context"
"log"
"time"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"google.golang.org/grpc"
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
)
@ -33,22 +41,50 @@ func (g Client) GetStatisticsChannel() (string, error) {
panic("implement me")
}
func (g Client) RegisterNode(req indexpb.RegisterNodeRequest) (indexpb.RegisterNodeResponse, error) {
panic("implement me")
func (g Client) RegisterNode(req *indexpb.RegisterNodeRequest) (*indexpb.RegisterNodeResponse, error) {
ctx := context.TODO()
return g.grpcClient.RegisterNode(ctx, req)
}
func (g Client) BuildIndex(req indexpb.BuildIndexRequest) (indexpb.BuildIndexResponse, error) {
panic("implement me")
func (g Client) BuildIndex(req *indexpb.BuildIndexRequest) (*indexpb.BuildIndexResponse, error) {
ctx := context.TODO()
return g.grpcClient.BuildIndex(ctx, req)
}
func (g Client) GetIndexStates(req indexpb.IndexStatesRequest) (indexpb.IndexStatesResponse, error) {
panic("implement me")
func (g Client) GetIndexStates(req *indexpb.IndexStatesRequest) (*indexpb.IndexStatesResponse, error) {
ctx := context.TODO()
return g.grpcClient.GetIndexStates(ctx, req)
}
func (g Client) GetIndexFilePaths(req indexpb.IndexFilePathRequest) (indexpb.IndexFilePathsResponse, error) {
panic("implement me")
func (g Client) GetIndexFilePaths(req *indexpb.IndexFilePathRequest) (*indexpb.IndexFilePathsResponse, error) {
ctx := context.TODO()
return g.grpcClient.GetIndexFilePaths(ctx, req)
}
func NewClient() *Client {
return &Client{}
func (g Client) NotifyBuildIndex(nty *indexpb.BuildIndexNotification) (*commonpb.Status, error) {
ctx := context.TODO()
return g.grpcClient.NotifyBuildIndex(ctx, nty)
}
func NewClient(address string) *Client {
ctx1, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()
conn, err := grpc.DialContext(ctx1, address, grpc.WithInsecure(), grpc.WithBlock())
if err != nil {
log.Printf("IndexNode connect to IndexService failed, error= %v", err)
}
log.Printf("IndexNode connected to IndexService, IndexService=%s", Params.Address)
return &Client{
grpcClient: indexpb.NewIndexServiceClient(conn),
}
}

View File

@ -0,0 +1,64 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License.
package main
import (
"context"
"log"
"os"
"os/signal"
"syscall"
grpcindexserver "github.com/zilliztech/milvus-distributed/internal/distributed/indexservice"
"go.uber.org/zap"
)
func main() {
grpcindexserver.Init()
ctx, cancel := context.WithCancel(context.Background())
svr, err := grpcindexserver.CreateIndexServer(ctx)
if err != nil {
log.Print("create server failed", zap.Error(err))
}
sc := make(chan os.Signal, 1)
signal.Notify(sc,
syscall.SIGHUP,
syscall.SIGINT,
syscall.SIGTERM,
syscall.SIGQUIT)
var sig os.Signal
go func() {
sig = <-sc
cancel()
}()
if err := svr.Start(); err != nil {
log.Fatal("run builder server failed", zap.Error(err))
}
<-ctx.Done()
log.Print("Got signal to exit", zap.String("signal", sig.String()))
svr.Close()
switch sig {
case syscall.SIGTERM:
exit(0)
default:
exit(1)
}
}
func exit(code int) {
os.Exit(code)
}

View File

@ -0,0 +1,144 @@
package grpcindexservice
import (
"net"
"strconv"
"github.com/zilliztech/milvus-distributed/internal/util/paramtable"
)
type ParamTable struct {
paramtable.BaseTable
Address string
Port int
NodeID int64
MasterAddress string
EtcdAddress string
MetaRootPath string
MinIOAddress string
MinIOAccessKeyID string
MinIOSecretAccessKey string
MinIOUseSSL bool
MinioBucketName string
}
var Params ParamTable
func (pt *ParamTable) Init() {
pt.BaseTable.Init()
pt.initAddress()
pt.initPort()
pt.initEtcdAddress()
pt.initMasterAddress()
pt.initMetaRootPath()
pt.initMinIOAddress()
pt.initMinIOAccessKeyID()
pt.initMinIOSecretAccessKey()
pt.initMinIOUseSSL()
pt.initMinioBucketName()
}
func (pt *ParamTable) initAddress() {
addr, err := pt.Load("indexServer.address")
if err != nil {
panic(err)
}
hostName, _ := net.LookupHost(addr)
if len(hostName) <= 0 {
if ip := net.ParseIP(addr); ip == nil {
panic("invalid ip indexBuilder.address")
}
}
port, err := pt.Load("indexServer.port")
if err != nil {
panic(err)
}
_, err = strconv.Atoi(port)
if err != nil {
panic(err)
}
pt.Address = addr + ":" + port
}
func (pt *ParamTable) initPort() {
pt.Port = pt.ParseInt("indexServer.port")
}
func (pt *ParamTable) initEtcdAddress() {
addr, err := pt.Load("_EtcdAddress")
if err != nil {
panic(err)
}
pt.EtcdAddress = addr
}
func (pt *ParamTable) initMetaRootPath() {
rootPath, err := pt.Load("etcd.rootPath")
if err != nil {
panic(err)
}
subPath, err := pt.Load("etcd.metaSubPath")
if err != nil {
panic(err)
}
pt.MetaRootPath = rootPath + "/" + subPath
}
func (pt *ParamTable) initMasterAddress() {
ret, err := pt.Load("_MasterAddress")
if err != nil {
panic(err)
}
pt.MasterAddress = ret
}
func (pt *ParamTable) initMinIOAddress() {
ret, err := pt.Load("_MinioAddress")
if err != nil {
panic(err)
}
pt.MinIOAddress = ret
}
func (pt *ParamTable) initMinIOAccessKeyID() {
ret, err := pt.Load("minio.accessKeyID")
if err != nil {
panic(err)
}
pt.MinIOAccessKeyID = ret
}
func (pt *ParamTable) initMinIOSecretAccessKey() {
ret, err := pt.Load("minio.secretAccessKey")
if err != nil {
panic(err)
}
pt.MinIOSecretAccessKey = ret
}
func (pt *ParamTable) initMinIOUseSSL() {
ret, err := pt.Load("minio.useSSL")
if err != nil {
panic(err)
}
pt.MinIOUseSSL, err = strconv.ParseBool(ret)
if err != nil {
panic(err)
}
}
func (pt *ParamTable) initMinioBucketName() {
bucketName, err := pt.Load("minio.bucketName")
if err != nil {
panic(err)
}
pt.MinioBucketName = bucketName
}

View File

@ -1,62 +1,154 @@
package grpcindexservice
import (
"context"
"log"
"net"
"strconv"
"sync"
"github.com/zilliztech/milvus-distributed/internal/indexservice"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"google.golang.org/grpc"
)
type UniqueID = typeutil.UniqueID
type Timestamp = typeutil.Timestamp
type Server struct {
server indexservice.Interface
grpcServer *grpc.Server
loopCtx context.Context
loopCancel func()
loopWg sync.WaitGroup
}
func (g Server) Init() {
func (s *Server) Init() {
log.Println("initing params ...")
Params.Init()
}
func (s *Server) Start() error {
s.Init()
log.Println("stringing indexserver ...")
return s.startIndexServer()
}
func (s *Server) Stop() {
s.loopWg.Wait()
}
func (s *Server) GetServiceStates() (internalpb2.ServiceStates, error) {
panic("implement me")
}
func (g Server) Start() {
func (s *Server) GetTimeTickChannel() (string, error) {
panic("implement me")
}
func (g Server) Stop() {
func (s *Server) GetStatisticsChannel() (string, error) {
panic("implement me")
}
func (g Server) GetServiceStates() (internalpb2.ServiceStates, error) {
panic("implement me")
func (s *Server) RegisterNode(ctx context.Context, req *indexpb.RegisterNodeRequest) (*indexpb.RegisterNodeResponse, error) {
log.Println("Register IndexNode starting...")
return s.server.RegisterNode(req)
}
func (g Server) GetTimeTickChannel() (string, error) {
panic("implement me")
func (s *Server) BuildIndex(ctx context.Context, req *indexpb.BuildIndexRequest) (*indexpb.BuildIndexResponse, error) {
return s.server.BuildIndex(req)
//indexID := int64(0)
//request := &indexpb.BuildIndexCmd{
// IndexID: indexID,
// Req: req,
//}
//
//indexNodeClient := grpcindexnode.NewClient()
//
//status, err := indexNodeClient.BuildIndex(request)
//response := &indexpb.BuildIndexResponse{
// Status: status,
// IndexID: indexID,
//}
//return response, err
}
func (g Server) GetStatisticsChannel() (string, error) {
panic("implement me")
func (s *Server) GetIndexStates(ctx context.Context, req *indexpb.IndexStatesRequest) (*indexpb.IndexStatesResponse, error) {
return s.server.GetIndexStates(req)
}
func (g Server) RegisterNode(req indexpb.RegisterNodeRequest) (indexpb.RegisterNodeResponse, error) {
panic("implement me")
func (s *Server) GetIndexFilePaths(ctx context.Context, req *indexpb.IndexFilePathRequest) (*indexpb.IndexFilePathsResponse, error) {
return s.server.GetIndexFilePaths(req)
}
func (g Server) BuildIndex(req indexpb.BuildIndexRequest) (indexpb.BuildIndexResponse, error) {
panic("implement me")
}
func (s *Server) NotifyBuildIndex(ctx context.Context, nty *indexpb.BuildIndexNotification) (*commonpb.Status, error) {
func (g Server) GetIndexStates(req indexpb.IndexStatesRequest) (indexpb.IndexStatesResponse, error) {
panic("implement me")
}
func (g Server) GetIndexFilePaths(req indexpb.IndexFilePathRequest) (indexpb.IndexFilePathsResponse, error) {
panic("implement me")
return s.server.NotifyBuildIndex(nty)
}
//varindex
func NewServer() *Server {
return &Server{
server: &indexservice.IndexService{},
//server: &indexservice.IndexService{},
//grpcServer: indexservice.IndexService{},
}
}
func (s *Server) grpcLoop() {
defer s.loopWg.Done()
log.Println("Starting start IndexServer")
lis, err := net.Listen("tcp", ":"+strconv.Itoa(Params.Port))
if err != nil {
log.Fatalf("IndexServer grpc server fatal error=%v", err)
}
s.grpcServer = grpc.NewServer()
indexpb.RegisterIndexServiceServer(s.grpcServer, s)
log.Println("IndexServer register finished")
if err = s.grpcServer.Serve(lis); err != nil {
log.Fatalf("IndexServer grpc server fatal error=%v", err)
}
log.Println("IndexServer grpc server starting...")
}
func (s *Server) startIndexServer() error {
s.loopWg.Add(1)
go s.grpcLoop()
return nil
}
func Init() {
Params.Init()
}
func CreateIndexServer(ctx context.Context) (*Server, error) {
ctx1, cancel := context.WithCancel(ctx)
s := &Server{
loopCtx: ctx1,
loopCancel: cancel,
server: indexservice.NewIndexServiceImpl(ctx),
}
return s, nil
}
func (s *Server) Close() {
s.Stop()
}

View File

@ -74,23 +74,24 @@ func (c *GrpcClient) HasCollection(in *milvuspb.HasCollectionRequest) (*milvuspb
func (c *GrpcClient) DescribeCollection(in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
return c.grpcClient.DescribeCollection(context.Background(), in)
}
func (c *GrpcClient) GetCollectionStatistics(in *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) {
return c.grpcClient.GetCollectionStatistics(context.Background(), in)
}
func (c *GrpcClient) ShowCollections(in *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) {
return c.grpcClient.ShowCollections(context.Background(), in)
}
func (c *GrpcClient) CreatePartition(in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) {
return c.grpcClient.CreatePartition(context.Background(), in)
}
func (c *GrpcClient) DropPartition(in *milvuspb.DropPartitionRequest) (*commonpb.Status, error) {
return c.grpcClient.DropPartition(context.Background(), in)
}
func (c *GrpcClient) HasPartition(in *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
return c.grpcClient.HasPartition(context.Background(), in)
}
func (c *GrpcClient) GetPartitionStatistics(in *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) {
return c.grpcClient.GetPartitionStatistics(context.Background(), in)
}
func (c *GrpcClient) ShowPartitions(in *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) {
return c.grpcClient.ShowPartitions(context.Background(), in)
}

View File

@ -10,7 +10,7 @@ import (
"github.com/stretchr/testify/assert"
cms "github.com/zilliztech/milvus-distributed/internal/masterservice"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
@ -33,10 +33,9 @@ func TestGrpcService(t *testing.T) {
cms.Params.TimeTickChannel = fmt.Sprintf("timeTick%d", randVal)
cms.Params.DdChannel = fmt.Sprintf("ddChannel%d", randVal)
cms.Params.StatisticsChannel = fmt.Sprintf("stateChannel%d", randVal)
cms.Params.DataServiceSegmentChannel = fmt.Sprintf("segmentChannel%d", randVal)
cms.Params.MaxPartitionNum = 64
cms.Params.DefaultPartitionName = "_default"
cms.Params.DefaultPartitionTag = "_default"
t.Logf("master service port = %d", cms.Params.Port)
@ -45,9 +44,6 @@ func TestGrpcService(t *testing.T) {
core := svr.core.(*cms.Core)
err = svr.Init(&cms.InitParams{ProxyTimeTickChannel: fmt.Sprintf("proxyTimeTick%d", randVal)})
assert.Nil(t, err)
core.ProxyTimeTickChan = make(chan typeutil.Timestamp, 8)
timeTickArray := make([]typeutil.Timestamp, 0, 16)
@ -56,34 +52,56 @@ func TestGrpcService(t *testing.T) {
timeTickArray = append(timeTickArray, ts)
return nil
}
createCollectionArray := make([]*internalpb2.CreateCollectionRequest, 0, 16)
core.DdCreateCollectionReq = func(req *internalpb2.CreateCollectionRequest) error {
t.Logf("Create Colllection %s", req.CollectionName)
createCollectionArray := make([]*cms.CreateCollectionReqTask, 0, 16)
core.DdCreateCollectionReq = func(req *cms.CreateCollectionReqTask) error {
t.Logf("Create Colllection %s", req.Req.CollectionName)
createCollectionArray = append(createCollectionArray, req)
return nil
}
dropCollectionArray := make([]*internalpb2.DropCollectionRequest, 0, 16)
core.DdDropCollectionReq = func(req *internalpb2.DropCollectionRequest) error {
t.Logf("Drop Collection %s", req.CollectionName)
dropCollectionArray := make([]*cms.DropCollectionReqTask, 0, 16)
core.DdDropCollectionReq = func(req *cms.DropCollectionReqTask) error {
t.Logf("Drop Collection %s", req.Req.CollectionName)
dropCollectionArray = append(dropCollectionArray, req)
return nil
}
createPartitionArray := make([]*internalpb2.CreatePartitionRequest, 0, 16)
core.DdCreatePartitionReq = func(req *internalpb2.CreatePartitionRequest) error {
t.Logf("Create Partition %s", req.PartitionName)
createPartitionArray := make([]*cms.CreatePartitionReqTask, 0, 16)
core.DdCreatePartitionReq = func(req *cms.CreatePartitionReqTask) error {
t.Logf("Create Partition %s", req.Req.PartitionName)
createPartitionArray = append(createPartitionArray, req)
return nil
}
dropPartitionArray := make([]*internalpb2.DropPartitionRequest, 0, 16)
core.DdDropPartitionReq = func(req *internalpb2.DropPartitionRequest) error {
t.Logf("Drop Partition %s", req.PartitionName)
dropPartitionArray := make([]*cms.DropPartitionReqTask, 0, 16)
core.DdDropPartitionReq = func(req *cms.DropPartitionReqTask) error {
t.Logf("Drop Partition %s", req.Req.PartitionName)
dropPartitionArray = append(dropPartitionArray, req)
return nil
}
core.GetSegmentMeta = func(id typeutil.UniqueID) (*etcdpb.SegmentMeta, error) {
return &etcdpb.SegmentMeta{
SegmentID: 20,
CollectionID: 10,
PartitionTag: "_default",
ChannelStart: 50,
ChannelEnd: 100,
OpenTime: 1000,
CloseTime: 2000,
NumRows: 16,
MemSize: 1024,
BinlogFilePaths: []*etcdpb.FieldBinlogFiles{
{
FieldID: 101,
BinlogFiles: []string{"/test/binlog/file"},
},
},
}, nil
}
err = svr.Init(&cms.InitParams{ProxyTimeTickChannel: fmt.Sprintf("proxyTimeTick%d", randVal)})
assert.Nil(t, err)
err = svr.Start()
assert.Nil(t, err)
@ -133,8 +151,8 @@ func TestGrpcService(t *testing.T) {
assert.Nil(t, err)
assert.Equal(t, len(createCollectionArray), 1)
assert.Equal(t, status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, createCollectionArray[0].Base.MsgType, commonpb.MsgType_kCreateCollection)
assert.Equal(t, createCollectionArray[0].CollectionName, "testColl")
assert.Equal(t, createCollectionArray[0].Req.Base.MsgType, commonpb.MsgType_kCreateCollection)
assert.Equal(t, createCollectionArray[0].Req.CollectionName, "testColl")
})
t.Run("has collection", func(t *testing.T) {
@ -201,6 +219,57 @@ func TestGrpcService(t *testing.T) {
assert.Equal(t, rsp.Schema.Name, "testColl")
})
t.Run("get collection statistics", func(t *testing.T) {
req := &milvuspb.CollectionStatsRequest{
Base: &commonpb.MsgBase{
MsgType: 0, //TODO,miss msg type
MsgID: 104,
Timestamp: 104,
SourceID: 104,
},
DbName: "testDb",
CollectionName: "testColl",
}
rsp, err := cli.GetCollectionStatistics(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, len(rsp.Stats), 2)
assert.Equal(t, rsp.Stats[0].Key, "row_count")
assert.Equal(t, rsp.Stats[0].Value, "0")
assert.Equal(t, rsp.Stats[1].Key, "data_size")
assert.Equal(t, rsp.Stats[1].Value, "0")
collMeta, err := core.MetaTable.GetCollectionByName("testColl")
assert.Nil(t, err)
seg := &etcdpb.SegmentMeta{
SegmentID: 101,
CollectionID: collMeta.ID,
PartitionTag: cms.Params.DefaultPartitionTag,
}
err = core.MetaTable.AddSegment(seg)
assert.Nil(t, err)
req = &milvuspb.CollectionStatsRequest{
Base: &commonpb.MsgBase{
MsgType: 0, //TODO,miss msg type
MsgID: 105,
Timestamp: 105,
SourceID: 105,
},
DbName: "testDb",
CollectionName: "testColl",
}
rsp, err = cli.GetCollectionStatistics(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, len(rsp.Stats), 2)
assert.Equal(t, rsp.Stats[0].Key, "row_count")
assert.Equal(t, rsp.Stats[0].Value, "16")
assert.Equal(t, rsp.Stats[1].Key, "data_size")
assert.Equal(t, rsp.Stats[1].Value, "1024")
})
t.Run("show collection", func(t *testing.T) {
req := &milvuspb.ShowCollectionRequest{
Base: &commonpb.MsgBase{
@ -236,9 +305,7 @@ func TestGrpcService(t *testing.T) {
collMeta, err := core.MetaTable.GetCollectionByName("testColl")
assert.Nil(t, err)
assert.Equal(t, len(collMeta.PartitionIDs), 2)
partMeta, err := core.MetaTable.GetPartitionByID(collMeta.PartitionIDs[1])
assert.Nil(t, err)
assert.Equal(t, partMeta.PartitionName, "testPartition")
assert.Equal(t, collMeta.PartitionTags[1], "testPartition")
})
@ -260,6 +327,28 @@ func TestGrpcService(t *testing.T) {
assert.Equal(t, rsp.Value, true)
})
t.Run("get partition statistics", func(t *testing.T) {
req := &milvuspb.PartitionStatsRequest{
Base: &commonpb.MsgBase{
MsgType: 0, //TODO, msg type
MsgID: 109,
Timestamp: 109,
SourceID: 109,
},
DbName: "testDb",
CollectionName: "testColl",
PartitionName: cms.Params.DefaultPartitionTag,
}
rsp, err := cli.GetPartitionStatistics(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, len(rsp.Stats), 2)
assert.Equal(t, rsp.Stats[0].Key, "row_count")
assert.Equal(t, rsp.Stats[0].Value, "16")
assert.Equal(t, rsp.Stats[1].Key, "data_size")
assert.Equal(t, rsp.Stats[1].Value, "1024")
})
t.Run("show partition", func(t *testing.T) {
req := &milvuspb.ShowPartitionRequest{
Base: &commonpb.MsgBase{
@ -295,9 +384,7 @@ func TestGrpcService(t *testing.T) {
collMeta, err := core.MetaTable.GetCollectionByName("testColl")
assert.Nil(t, err)
assert.Equal(t, len(collMeta.PartitionIDs), 1)
partMeta, err := core.MetaTable.GetPartitionByID(collMeta.PartitionIDs[0])
assert.Nil(t, err)
assert.Equal(t, partMeta.PartitionName, cms.Params.DefaultPartitionName)
assert.Equal(t, collMeta.PartitionTags[0], cms.Params.DefaultPartitionTag)
})
t.Run("drop collection", func(t *testing.T) {
@ -316,8 +403,8 @@ func TestGrpcService(t *testing.T) {
assert.Nil(t, err)
assert.Equal(t, len(dropCollectionArray), 1)
assert.Equal(t, status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, dropCollectionArray[0].Base.MsgType, commonpb.MsgType_kDropCollection)
assert.Equal(t, dropCollectionArray[0].CollectionName, "testColl")
assert.Equal(t, dropCollectionArray[0].Req.Base.MsgType, commonpb.MsgType_kDropCollection)
assert.Equal(t, dropCollectionArray[0].Req.CollectionName, "testColl")
req = &milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{

View File

@ -94,6 +94,10 @@ func (s *GrpcServer) DescribeCollection(ctx context.Context, in *milvuspb.Descri
return s.core.DescribeCollection(in)
}
func (s *GrpcServer) GetCollectionStatistics(ctx context.Context, in *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) {
return s.core.GetCollectionStatistics(in)
}
func (s *GrpcServer) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) {
return s.core.ShowCollections(in)
}
@ -110,6 +114,10 @@ func (s *GrpcServer) HasPartition(ctx context.Context, in *milvuspb.HasPartition
return s.core.HasPartition(in)
}
func (s *GrpcServer) GetPartitionStatistics(ctx context.Context, in *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) {
return s.core.GetPartitionStatistics(in)
}
func (s *GrpcServer) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) {
return s.core.ShowPartitions(in)
}

View File

@ -120,27 +120,17 @@ func (c *Client) BuildIndex(columnDataPaths []string, typeParams map[string]stri
return indexID, err
}
func (c *Client) GetIndexStates(indexID UniqueID) (*indexpb.IndexStatesResponse, error) {
func (c *Client) GetIndexStates(indexIDs []UniqueID) (*indexpb.IndexStatesResponse, error) {
if c.tryConnect() != nil {
panic("DescribeIndex: failed to connect index builder")
}
ctx := context.TODO()
request := &indexpb.IndexStatesRequest{
IndexID: indexID,
}
response, err := c.client.GetIndexStates(ctx, request)
if err != nil {
return &indexpb.IndexStatesResponse{}, err
IndexID: indexIDs,
}
indexDescription := indexpb.IndexStatesResponse{
Status: &commonpb.Status{
ErrorCode: 0,
},
IndexID: indexID,
State: response.State,
}
return &indexDescription, nil
response, err := c.client.GetIndexStates(ctx, request)
return response, err
}
func (c *Client) GetIndexFilePaths(indexID UniqueID) ([]string, error) {

View File

@ -3,6 +3,7 @@ package indexnode
import (
"context"
"errors"
"log"
"time"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
@ -56,14 +57,20 @@ func (b *Builder) BuildIndex(ctx context.Context, request *indexpb.BuildIndexReq
}
func (b *Builder) GetIndexStates(ctx context.Context, request *indexpb.IndexStatesRequest) (*indexpb.IndexStatesResponse, error) {
indexID := request.IndexID
ret, err := b.metaTable.GetIndexStates(indexID)
ret.Status = &commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
ret.IndexID = indexID
if err != nil {
ret.Status.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR
ret.Status.Reason = err.Error()
var indexStates []*indexpb.IndexInfo
for _, indexID := range request.IndexID {
indexState, err := b.metaTable.GetIndexStates(indexID)
log.Println("GetIndexStates error, err=", err)
indexStates = append(indexStates, indexState)
}
ret := &indexpb.IndexStatesResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
},
States: indexStates,
}
return ret, nil
}

View File

@ -9,20 +9,17 @@ import (
"sync"
"time"
"github.com/zilliztech/milvus-distributed/internal/indexservice"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
miniokv "github.com/zilliztech/milvus-distributed/internal/kv/minio"
"go.etcd.io/etcd/clientv3"
"github.com/zilliztech/milvus-distributed/internal/allocator"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/indexservice"
"github.com/zilliztech/milvus-distributed/internal/kv"
etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd"
miniokv "github.com/zilliztech/milvus-distributed/internal/kv/minio"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"go.etcd.io/etcd/clientv3"
"google.golang.org/grpc"
)
@ -30,7 +27,23 @@ type UniqueID = typeutil.UniqueID
type Timestamp = typeutil.Timestamp
type IndexNode struct {
Builder
loopCtx context.Context
loopCancel func()
loopWg sync.WaitGroup
grpcServer *grpc.Server
sched *TaskScheduler
idAllocator *allocator.IDAllocator
kv kv.Base
metaTable *metaTable
// Add callback functions at different stages
startCallbacks []func()
closeCallbacks []func()
indexNodeID int64
serviceClient indexservice.Interface // method factory
}
@ -58,14 +71,64 @@ func (i *IndexNode) GetStatisticsChannel() (string, error) {
panic("implement me")
}
func (i *IndexNode) BuildIndex(req indexpb.BuildIndexRequest) (indexpb.BuildIndexResponse, error) {
panic("implement me")
func (i *IndexNode) BuildIndex(req *indexpb.BuildIndexCmd) (*commonpb.Status, error) {
//TODO: build index in index node
ctx := context.Background()
t := NewIndexAddTask()
t.req = req.Req
t.idAllocator = i.idAllocator
t.buildQueue = i.sched.IndexBuildQueue
t.table = i.metaTable
t.kv = i.kv
var cancel func()
t.ctx, cancel = context.WithTimeout(ctx, reqTimeoutInterval)
defer cancel()
fn := func() error {
select {
case <-ctx.Done():
return errors.New("insert timeout")
default:
return i.sched.IndexAddQueue.Enqueue(t)
}
}
ret := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}
err := fn()
if err != nil {
ret.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR
ret.Reason = err.Error()
return ret, nil
}
err = t.WaitToFinish()
if err != nil {
ret.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR
ret.Reason = err.Error()
return ret, nil
}
return ret, nil
}
func CreateIndexNode(ctx context.Context) (Interface, error) {
func CreateIndexNode(ctx context.Context) (*IndexNode, error) {
return &IndexNode{}, nil
}
func NewIndexNode(ctx context.Context, indexID int64) *IndexNode {
ctx1, cancel := context.WithCancel(ctx)
in := &IndexNode{
loopCtx: ctx1,
loopCancel: cancel,
indexNodeID: indexID,
}
return in
}
type Builder struct {
loopCtx context.Context
loopCancel func()

View File

@ -119,10 +119,10 @@ func TestBuilder_GRPC(t *testing.T) {
time.Sleep(time.Second * 3)
description, err := buildClient.GetIndexStates(indexID)
description, err := buildClient.GetIndexStates([]UniqueID{indexID})
assert.Nil(t, err)
assert.Equal(t, commonpb.IndexState_INPROGRESS, description.State)
assert.Equal(t, indexID, description.IndexID)
assert.Equal(t, commonpb.IndexState_INPROGRESS, description.States[0].State)
assert.Equal(t, indexID, description.States[0].IndexID)
indexDataPaths, err := buildClient.GetIndexFilePaths(indexID)
assert.Nil(t, err)

View File

@ -1,6 +1,7 @@
package indexnode
import (
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
)
@ -8,6 +9,5 @@ import (
type ServiceBase = typeutil.Component
type Interface interface {
ServiceBase
BuildIndex(req indexpb.BuildIndexRequest) (indexpb.BuildIndexResponse, error)
BuildIndex(req *indexpb.BuildIndexCmd) (*commonpb.Status, error)
}

View File

@ -127,14 +127,20 @@ func (mt *metaTable) CompleteIndex(indexID UniqueID, dataPaths []string) error {
return nil
}
func (mt *metaTable) GetIndexStates(indexID UniqueID) (*pb.IndexStatesResponse, error) {
func (mt *metaTable) GetIndexStates(indexID UniqueID) (*pb.IndexInfo, error) {
mt.lock.Lock()
defer mt.lock.Unlock()
ret := &pb.IndexStatesResponse{}
ret := &pb.IndexInfo{
IndexID: indexID,
Reason: "",
}
meta, ok := mt.indexID2Meta[indexID]
if !ok {
return ret, errors.Errorf("index not exists with ID = " + strconv.FormatInt(indexID, 10))
ret.Reason = "index not exists with ID = " + strconv.FormatInt(indexID, 10)
ret.State = commonpb.IndexState_NONE
return ret, nil
}
ret.State = meta.State
return ret, nil
}

View File

@ -1,8 +1,22 @@
package indexservice
import (
"context"
"log"
"strconv"
"sync"
"time"
"github.com/zilliztech/milvus-distributed/internal/allocator"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/kv"
etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"go.etcd.io/etcd/clientv3"
"google.golang.org/grpc"
)
type IndexService struct {
@ -10,49 +24,191 @@ type IndexService struct {
//nodeClients [] .Interface
// factory method
loopCtx context.Context
loopCancel func()
loopWg sync.WaitGroup
grpcServer *grpc.Server
nodeNum int64
idAllocator *allocator.IDAllocator
kv kv.Base
metaTable *metaTable
// Add callback functions at different stages
startCallbacks []func()
closeCallbacks []func()
}
func (i IndexService) Init() {
type UniqueID = typeutil.UniqueID
type Timestamp = typeutil.Timestamp
func (i *IndexService) Init() {
panic("implement me")
}
func (i IndexService) Start() {
func (i *IndexService) Start() {
panic("implement me")
}
func (i IndexService) Stop() {
func (i *IndexService) Stop() {
panic("implement me")
}
func (i IndexService) GetServiceStates() (internalpb2.ServiceStates, error) {
func (i *IndexService) GetServiceStates() (internalpb2.ServiceStates, error) {
panic("implement me")
}
func (i IndexService) GetTimeTickChannel() (string, error) {
func (i *IndexService) GetTimeTickChannel() (string, error) {
panic("implement me")
}
func (i IndexService) GetStatisticsChannel() (string, error) {
func (i *IndexService) GetStatisticsChannel() (string, error) {
panic("implement me")
}
func (i IndexService) RegisterNode(req indexpb.RegisterNodeRequest) (indexpb.RegisterNodeResponse, error) {
func (i *IndexService) RegisterNode(req *indexpb.RegisterNodeRequest) (*indexpb.RegisterNodeResponse, error) {
nodeID := i.nodeNum + 1
//TODO: update meta table
_, ok := i.metaTable.nodeID2Address[nodeID]
if ok {
log.Fatalf("Register IndexNode fatal, IndexNode has already exists with nodeID=%d", nodeID)
}
log.Println("this is register indexNode func")
i.metaTable.nodeID2Address[nodeID] = req.Address
//TODO: register index node params?
var params []*commonpb.KeyValuePair
minioAddress, err := Params.Load("minio.address")
if err != nil {
return nil, err
}
minioPort, err := Params.Load("minio.port")
if err != nil {
return nil, err
}
params = append(params, &commonpb.KeyValuePair{Key: "minio.address", Value: minioAddress})
params = append(params, &commonpb.KeyValuePair{Key: "minio.port", Value: minioPort})
params = append(params, &commonpb.KeyValuePair{Key: "minio.accessKeyID", Value: Params.MinIOAccessKeyID})
params = append(params, &commonpb.KeyValuePair{Key: "minio.secretAccessKey", Value: Params.MinIOSecretAccessKey})
params = append(params, &commonpb.KeyValuePair{Key: "minio.useSSL", Value: strconv.FormatBool(Params.MinIOUseSSL)})
params = append(params, &commonpb.KeyValuePair{Key: "minio.bucketName", Value: Params.MinioBucketName})
i.nodeNum++
return &indexpb.RegisterNodeResponse{
InitParams: &internalpb2.InitParams{
NodeID: nodeID,
StartParams: params,
},
}, nil
}
func (i *IndexService) BuildIndex(req *indexpb.BuildIndexRequest) (*indexpb.BuildIndexResponse, error) {
//TODO: Multiple indexes will build at same time.
//ctx := context.Background()
//indexNodeClient := indexnode.NewIndexNode(ctx, rand.Int63n(i.nodeNum))
//
////TODO: Allocator index ID
//indexID := int64(0)
//
//request := &indexpb.BuildIndexCmd{
// IndexID: indexID,
// Req: req,
//}
//
//status, err := indexNodeClient.BuildIndex(request)
//if err != nil {
// return nil, err
//}
//
//return &indexpb.BuildIndexResponse{
// Status: status,
// IndexID: indexID,
//}, nil
return nil, nil
}
func (i *IndexService) GetIndexStates(req *indexpb.IndexStatesRequest) (*indexpb.IndexStatesResponse, error) {
var indexStates []*indexpb.IndexInfo
for _, indexID := range req.IndexID {
indexState := &indexpb.IndexInfo{
IndexID: indexID,
State: commonpb.IndexState_NONE,
Reason: "",
}
meta, ok := i.metaTable.indexID2Meta[indexID]
if !ok {
indexState.State = commonpb.IndexState_NONE
indexState.Reason = "index does not exists with ID = " + strconv.FormatInt(indexID, 10)
} else {
indexState.State = meta.State
}
indexStates = append(indexStates, indexState)
}
ret := &indexpb.IndexStatesResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
},
States: indexStates,
}
return ret, nil
}
func (i *IndexService) GetIndexFilePaths(req *indexpb.IndexFilePathRequest) (*indexpb.IndexFilePathsResponse, error) {
panic("implement me")
}
func (i IndexService) BuildIndex(req indexpb.BuildIndexRequest) (indexpb.BuildIndexResponse, error) {
panic("implement me")
func (i *IndexService) NotifyBuildIndex(nty *indexpb.BuildIndexNotification) (*commonpb.Status, error) {
//TODO: Multiple indexes are building successfully at same time.
meta, ok := i.metaTable.indexID2Meta[nty.IndexID]
if !ok {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_BUILD_INDEX_ERROR,
Reason: "index already exists with ID = " + strconv.FormatInt(nty.IndexID, 10),
}, errors.Errorf("index already exists with ID = " + strconv.FormatInt(nty.IndexID, 10))
}
meta.State = commonpb.IndexState_FINISHED
meta.IndexFilePaths = nty.IndexFilePaths
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}, nil
}
func (i IndexService) GetIndexStates(req indexpb.IndexStatesRequest) (indexpb.IndexStatesResponse, error) {
panic("implement me")
}
func NewIndexServiceImpl(ctx context.Context) *IndexService {
Params.Init()
ctx1, cancel := context.WithCancel(ctx)
s := &IndexService{
loopCtx: ctx1,
loopCancel: cancel,
}
func (i IndexService) GetIndexFilePaths(req indexpb.IndexFilePathRequest) (indexpb.IndexFilePathsResponse, error) {
panic("implement me")
}
connectEtcdFn := func() error {
etcdAddress := Params.EtcdAddress
etcdClient, err := clientv3.New(clientv3.Config{Endpoints: []string{etcdAddress}})
if err != nil {
return err
}
etcdKV := etcdkv.NewEtcdKV(etcdClient, Params.MetaRootPath)
metakv, err := NewMetaTable(etcdKV)
if err != nil {
return err
}
s.metaTable = metakv
return nil
}
err := Retry(10, time.Millisecond*200, connectEtcdFn)
if err != nil {
return nil
}
func NewIndexServiceImpl() Interface {
return &IndexService{}
s.nodeNum = 0
return s
}

View File

@ -1,6 +1,7 @@
package indexservice
import (
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
)
@ -9,8 +10,9 @@ type ServiceBase = typeutil.Component
type Interface interface {
ServiceBase
RegisterNode(req indexpb.RegisterNodeRequest) (indexpb.RegisterNodeResponse, error)
BuildIndex(req indexpb.BuildIndexRequest) (indexpb.BuildIndexResponse, error)
GetIndexStates(req indexpb.IndexStatesRequest) (indexpb.IndexStatesResponse, error)
GetIndexFilePaths(req indexpb.IndexFilePathRequest) (indexpb.IndexFilePathsResponse, error)
RegisterNode(req *indexpb.RegisterNodeRequest) (*indexpb.RegisterNodeResponse, error)
BuildIndex(req *indexpb.BuildIndexRequest) (*indexpb.BuildIndexResponse, error)
GetIndexStates(req *indexpb.IndexStatesRequest) (*indexpb.IndexStatesResponse, error)
GetIndexFilePaths(req *indexpb.IndexFilePathRequest) (*indexpb.IndexFilePathsResponse, error)
NotifyBuildIndex(nty *indexpb.BuildIndexNotification) (*commonpb.Status, error)
}

View File

@ -0,0 +1,162 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License.
package indexservice
import (
"fmt"
"strconv"
"sync"
"time"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
pb "github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
)
type metaTable struct {
client kv.TxnBase // client of a reliable kv service, i.e. etcd client
indexID2Meta map[UniqueID]pb.IndexMeta // index id to index meta
nodeID2Address map[int64]*commonpb.Address
lock sync.RWMutex
}
func NewMetaTable(kv kv.TxnBase) (*metaTable, error) {
mt := &metaTable{
client: kv,
lock: sync.RWMutex{},
}
err := mt.reloadFromKV()
if err != nil {
return nil, err
}
mt.nodeID2Address = make(map[int64]*commonpb.Address)
return mt, nil
}
func (mt *metaTable) reloadFromKV() error {
mt.indexID2Meta = make(map[UniqueID]pb.IndexMeta)
_, values, err := mt.client.LoadWithPrefix("indexes")
if err != nil {
return err
}
for _, value := range values {
indexMeta := pb.IndexMeta{}
err = proto.UnmarshalText(value, &indexMeta)
if err != nil {
return err
}
mt.indexID2Meta[indexMeta.IndexID] = indexMeta
}
return nil
}
// metaTable.lock.Lock() before call this function
func (mt *metaTable) saveIndexMeta(meta *pb.IndexMeta) error {
value := proto.MarshalTextString(meta)
mt.indexID2Meta[meta.IndexID] = *meta
return mt.client.Save("/indexes/"+strconv.FormatInt(meta.IndexID, 10), value)
}
func (mt *metaTable) AddIndex(indexID UniqueID, req *pb.BuildIndexRequest) error {
mt.lock.Lock()
defer mt.lock.Unlock()
_, ok := mt.indexID2Meta[indexID]
if ok {
return errors.Errorf("index already exists with ID = " + strconv.FormatInt(indexID, 10))
}
meta := &pb.IndexMeta{
State: commonpb.IndexState_INPROGRESS,
IndexID: indexID,
Req: req,
}
return mt.saveIndexMeta(meta)
}
func (mt *metaTable) UpdateIndexStatus(indexID UniqueID, status commonpb.IndexState) error {
mt.lock.Lock()
defer mt.lock.Unlock()
meta, ok := mt.indexID2Meta[indexID]
if !ok {
return errors.Errorf("index not exists with ID = " + strconv.FormatInt(indexID, 10))
}
meta.State = status
return mt.saveIndexMeta(&meta)
}
func (mt *metaTable) UpdateIndexEnqueTime(indexID UniqueID, t time.Time) error {
mt.lock.Lock()
defer mt.lock.Unlock()
meta, ok := mt.indexID2Meta[indexID]
if !ok {
return errors.Errorf("index not exists with ID = " + strconv.FormatInt(indexID, 10))
}
meta.EnqueTime = t.UnixNano()
return mt.saveIndexMeta(&meta)
}
func (mt *metaTable) UpdateIndexScheduleTime(indexID UniqueID, t time.Time) error {
mt.lock.Lock()
defer mt.lock.Unlock()
meta, ok := mt.indexID2Meta[indexID]
if !ok {
return errors.Errorf("index not exists with ID = " + strconv.FormatInt(indexID, 10))
}
meta.ScheduleTime = t.UnixNano()
return mt.saveIndexMeta(&meta)
}
func (mt *metaTable) NotifyBuildIndex(indexID UniqueID, dataPaths []string, state commonpb.IndexState) error {
mt.lock.Lock()
defer mt.lock.Unlock()
meta, ok := mt.indexID2Meta[indexID]
if !ok {
return errors.Errorf("index not exists with ID = " + strconv.FormatInt(indexID, 10))
}
meta.State = state
meta.IndexFilePaths = dataPaths
return mt.saveIndexMeta(&meta)
}
func (mt *metaTable) GetIndexFilePaths(indexID UniqueID) ([]string, error) {
mt.lock.Lock()
defer mt.lock.Unlock()
meta, ok := mt.indexID2Meta[indexID]
if !ok {
return nil, errors.Errorf("index not exists with ID = " + strconv.FormatInt(indexID, 10))
}
return meta.IndexFilePaths, nil
}
func (mt *metaTable) DeleteIndex(indexID UniqueID) error {
mt.lock.Lock()
defer mt.lock.Unlock()
indexMeta, ok := mt.indexID2Meta[indexID]
if !ok {
return errors.Errorf("can't find index. id = " + strconv.FormatInt(indexID, 10))
}
fmt.Print(indexMeta)
return nil
}

View File

@ -0,0 +1,144 @@
package indexservice
import (
"net"
"strconv"
"github.com/zilliztech/milvus-distributed/internal/util/paramtable"
)
type ParamTable struct {
paramtable.BaseTable
Address string
Port int
NodeID int64
MasterAddress string
EtcdAddress string
MetaRootPath string
MinIOAddress string
MinIOAccessKeyID string
MinIOSecretAccessKey string
MinIOUseSSL bool
MinioBucketName string
}
var Params ParamTable
func (pt *ParamTable) Init() {
pt.BaseTable.Init()
pt.initAddress()
pt.initPort()
pt.initEtcdAddress()
pt.initMasterAddress()
pt.initMetaRootPath()
pt.initMinIOAddress()
pt.initMinIOAccessKeyID()
pt.initMinIOSecretAccessKey()
pt.initMinIOUseSSL()
pt.initMinioBucketName()
}
func (pt *ParamTable) initAddress() {
addr, err := pt.Load("indexServer.address")
if err != nil {
panic(err)
}
hostName, _ := net.LookupHost(addr)
if len(hostName) <= 0 {
if ip := net.ParseIP(addr); ip == nil {
panic("invalid ip indexBuilder.address")
}
}
port, err := pt.Load("indexServer.port")
if err != nil {
panic(err)
}
_, err = strconv.Atoi(port)
if err != nil {
panic(err)
}
pt.Address = addr + ":" + port
}
func (pt *ParamTable) initPort() {
pt.Port = pt.ParseInt("indexServer.port")
}
func (pt *ParamTable) initEtcdAddress() {
addr, err := pt.Load("_EtcdAddress")
if err != nil {
panic(err)
}
pt.EtcdAddress = addr
}
func (pt *ParamTable) initMetaRootPath() {
rootPath, err := pt.Load("etcd.rootPath")
if err != nil {
panic(err)
}
subPath, err := pt.Load("etcd.metaSubPath")
if err != nil {
panic(err)
}
pt.MetaRootPath = rootPath + "/" + subPath
}
func (pt *ParamTable) initMasterAddress() {
ret, err := pt.Load("_MasterAddress")
if err != nil {
panic(err)
}
pt.MasterAddress = ret
}
func (pt *ParamTable) initMinIOAddress() {
ret, err := pt.Load("_MinioAddress")
if err != nil {
panic(err)
}
pt.MinIOAddress = ret
}
func (pt *ParamTable) initMinIOAccessKeyID() {
ret, err := pt.Load("minio.accessKeyID")
if err != nil {
panic(err)
}
pt.MinIOAccessKeyID = ret
}
func (pt *ParamTable) initMinIOSecretAccessKey() {
ret, err := pt.Load("minio.secretAccessKey")
if err != nil {
panic(err)
}
pt.MinIOSecretAccessKey = ret
}
func (pt *ParamTable) initMinIOUseSSL() {
ret, err := pt.Load("minio.useSSL")
if err != nil {
panic(err)
}
pt.MinIOUseSSL, err = strconv.ParseBool(ret)
if err != nil {
panic(err)
}
}
func (pt *ParamTable) initMinioBucketName() {
bucketName, err := pt.Load("minio.bucketName")
if err != nil {
panic(err)
}
pt.MinioBucketName = bucketName
}

View File

@ -0,0 +1,40 @@
package indexservice
import (
"log"
"time"
)
// Reference: https://blog.cyeam.com/golang/2018/08/27/retry
func RetryImpl(attempts int, sleep time.Duration, fn func() error, maxSleepTime time.Duration) error {
if err := fn(); err != nil {
if s, ok := err.(InterruptError); ok {
return s.error
}
if attempts--; attempts > 0 {
log.Printf("retry func error: %s. attempts #%d after %s.", err.Error(), attempts, sleep)
time.Sleep(sleep)
if sleep < maxSleepTime {
return RetryImpl(attempts, 2*sleep, fn, maxSleepTime)
}
return RetryImpl(attempts, maxSleepTime, fn, maxSleepTime)
}
return err
}
return nil
}
func Retry(attempts int, sleep time.Duration, fn func() error) error {
maxSleepTime := time.Millisecond * 1000
return RetryImpl(attempts, sleep, fn, maxSleepTime)
}
type InterruptError struct {
error
}
func NoRetryError(err error) InterruptError {
return InterruptError{err}
}

View File

@ -64,7 +64,7 @@ func (m *MockWriteNodeClient) GetInsertBinlogPaths(segmentID UniqueID) (map[Uniq
type BuildIndexClient interface {
BuildIndex(columnDataPaths []string, typeParams map[string]string, indexParams map[string]string) (UniqueID, error)
GetIndexStates(indexID UniqueID) (*indexpb.IndexStatesResponse, error)
GetIndexStates(indexIDs []UniqueID) (*indexpb.IndexStatesResponse, error)
GetIndexFilePaths(indexID UniqueID) ([]string, error)
}
@ -77,18 +77,34 @@ func (m *MockBuildIndexClient) BuildIndex(columnDataPaths []string, typeParams m
return 1, nil
}
func (m *MockBuildIndexClient) GetIndexStates(indexID UniqueID) (*indexpb.IndexStatesResponse, error) {
func (m *MockBuildIndexClient) GetIndexStates(indexIDs []UniqueID) (*indexpb.IndexStatesResponse, error) {
now := time.Now()
if now.Sub(m.buildTime).Seconds() > 2 {
return &indexpb.IndexStatesResponse{
IndexID: indexID,
State: commonpb.IndexState_FINISHED,
}, nil
ret := &indexpb.IndexStatesResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
},
}
return &indexpb.IndexStatesResponse{
IndexID: 1,
State: commonpb.IndexState_INPROGRESS,
}, nil
var indexStates []*indexpb.IndexInfo
if now.Sub(m.buildTime).Seconds() > 2 {
for _, indexID := range indexIDs {
indexState := &indexpb.IndexInfo{
State: commonpb.IndexState_FINISHED,
IndexID: indexID,
}
indexStates = append(indexStates, indexState)
}
ret.States = indexStates
return ret, nil
}
for _, indexID := range indexIDs {
indexState := &indexpb.IndexInfo{
State: commonpb.IndexState_INPROGRESS,
IndexID: indexID,
}
indexStates = append(indexStates, indexState)
}
ret.States = indexStates
return ret, nil
}
func (m *MockBuildIndexClient) GetIndexFilePaths(indexID UniqueID) ([]string, error) {

View File

@ -110,11 +110,11 @@ func (scheduler *IndexBuildScheduler) describe() error {
indexID := channelInfo.id
indexBuildInfo := channelInfo.info
for {
description, err := scheduler.client.GetIndexStates(channelInfo.id)
description, err := scheduler.client.GetIndexStates([]UniqueID{channelInfo.id})
if err != nil {
return err
}
if description.State == commonpb.IndexState_FINISHED {
if description.States[0].State == commonpb.IndexState_FINISHED {
log.Printf("build index for segment %d field %d is finished", indexBuildInfo.segmentID, indexBuildInfo.fieldID)
filePaths, err := scheduler.client.GetIndexFilePaths(indexID)
if err != nil {
@ -167,7 +167,7 @@ func (scheduler *IndexBuildScheduler) describe() error {
FieldID: indexBuildInfo.fieldID,
IndexID: indexID,
IndexParams: channelInfo.indexParams,
State: description.State,
State: description.States[0].State,
})
if err != nil {
return err

View File

@ -96,15 +96,15 @@ func TestPersistenceScheduler(t *testing.T) {
//wait flush segment request sent to build index node
time.Sleep(100 * time.Microsecond)
idxDes, err := buildIndexClient.GetIndexStates(UniqueID(1))
idxDes, err := buildIndexClient.GetIndexStates([]UniqueID{UniqueID(1)})
assert.Nil(t, err)
assert.Equal(t, commonpb.IndexState_INPROGRESS, idxDes.State)
assert.Equal(t, commonpb.IndexState_INPROGRESS, idxDes.States[0].State)
//wait build index to finish
time.Sleep(3 * time.Second)
idxDes, err = buildIndexClient.GetIndexStates(UniqueID(1))
idxDes, err = buildIndexClient.GetIndexStates([]UniqueID{UniqueID(1)})
assert.Nil(t, err)
assert.Equal(t, commonpb.IndexState_FINISHED, idxDes.State)
assert.Equal(t, commonpb.IndexState_FINISHED, idxDes.States[0].State)
}

View File

@ -10,8 +10,8 @@ import (
"github.com/zilliztech/milvus-distributed/internal/errors"
etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd"
ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
@ -26,7 +26,7 @@ import (
// datapb(data_service)
// indexpb(index_service)
// milvuspb -> servicepb
// masterpb2 -> masterpb (master_service)
// masterpb2 -> masterpb master_service)
type InitParams struct {
ProxyTimeTickChannel string
@ -53,10 +53,12 @@ type Interface interface {
DropCollection(in *milvuspb.DropCollectionRequest) (*commonpb.Status, error)
HasCollection(in *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error)
DescribeCollection(in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error)
GetCollectionStatistics(in *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error)
ShowCollections(in *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error)
CreatePartition(in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error)
DropPartition(in *milvuspb.DropPartitionRequest) (*commonpb.Status, error)
HasPartition(in *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error)
GetPartitionStatistics(in *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error)
ShowPartitions(in *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error)
//index builder service
@ -114,23 +116,23 @@ type Core struct {
kvBase *etcdkv.EtcdKV
metaKV *etcdkv.EtcdKV
//setMsgStreams, receive time tick from proxy service time tick channel
ProxyTimeTickChan chan typeutil.Timestamp
//TODO, receive time tick from proxy service time tick channel
ProxyTimeTickChan <-chan typeutil.Timestamp
//setMsgStreams, send time tick into dd channel and time tick channel
//TODO, send time tick into dd channel and time tick channel
SendTimeTick func(t typeutil.Timestamp) error
//setMsgStreams, send create collection into dd channel
DdCreateCollectionReq func(req *internalpb2.CreateCollectionRequest) error
//TODO, send create collection into dd channel
DdCreateCollectionReq func(req *CreateCollectionReqTask) error
//setMsgStreams, send drop collection into dd channel, and notify the proxy to delete this collection
DdDropCollectionReq func(req *internalpb2.DropCollectionRequest) error
//TODO, send drop collection into dd channel, and notify the proxy to delete this collection
DdDropCollectionReq func(req *DropCollectionReqTask) error
//setMsgStreams, send create partition into dd channel
DdCreatePartitionReq func(req *internalpb2.CreatePartitionRequest) error
//TODO, send create partition into dd channel
DdCreatePartitionReq func(req *CreatePartitionReqTask) error
//setMsgStreams, send drop partition into dd channel
DdDropPartitionReq func(req *internalpb2.DropPartitionRequest) error
//TODO, send drop partition into dd channel
DdDropPartitionReq func(req *DropPartitionReqTask) error
//dd request scheduler
ddReqQueue chan reqTask //dd request will be push into this chan
@ -146,6 +148,9 @@ type Core struct {
initOnce sync.Once
startOnce sync.Once
isInit atomic.Value
//TODO, get segment meta by segment id, from data service by grpc
GetSegmentMeta func(id typeutil.UniqueID) (*etcdpb.SegmentMeta, error)
}
// --------------------- function --------------------------
@ -188,6 +193,9 @@ func (c *Core) checkInit() error {
if c.ddReqQueue == nil {
return errors.Errorf("ddReqQueue is nil")
}
if c.GetSegmentMeta == nil {
return errors.Errorf("GetSegmentMeta is nil")
}
if c.DdCreateCollectionReq == nil {
return errors.Errorf("DdCreateCollectionReq is nil")
}
@ -253,185 +261,6 @@ func (c *Core) startTimeTickLoop() {
}
}
func (c *Core) setMsgStreams() error {
//proxy time tick stream,
proxyTimeTickStream := ms.NewPulsarMsgStream(c.ctx, 1024)
proxyTimeTickStream.SetPulsarClient(Params.PulsarAddress)
proxyTimeTickStream.CreatePulsarConsumers([]string{Params.ProxyTimeTickChannel}, Params.MsgChannelSubName, ms.NewUnmarshalDispatcher(), 1024)
proxyTimeTickStream.Start()
// master time tick channel
timeTickStream := ms.NewPulsarMsgStream(c.ctx, 1024)
timeTickStream.SetPulsarClient(Params.PulsarAddress)
timeTickStream.CreatePulsarProducers([]string{Params.TimeTickChannel})
// master dd channel
ddStream := ms.NewPulsarMsgStream(c.ctx, 1024)
ddStream.SetPulsarClient(Params.PulsarAddress)
ddStream.CreatePulsarProducers([]string{Params.DdChannel})
c.SendTimeTick = func(t typeutil.Timestamp) error {
msgPack := ms.MsgPack{}
baseMsg := ms.BaseMsg{
MsgCtx: nil,
BeginTimestamp: t,
EndTimestamp: t,
HashValues: []uint32{0},
}
timeTickResult := internalpb2.TimeTickMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kTimeTick,
MsgID: 0,
Timestamp: t,
SourceID: int64(Params.NodeID),
},
}
timeTickMsg := &ms.TimeTickMsg{
BaseMsg: baseMsg,
TimeTickMsg: timeTickResult,
}
msgPack.Msgs = append(msgPack.Msgs, timeTickMsg)
if err := timeTickStream.Broadcast(&msgPack); err != nil {
return err
}
if err := ddStream.Broadcast(&msgPack); err != nil {
return err
}
return nil
}
c.DdCreateCollectionReq = func(req *internalpb2.CreateCollectionRequest) error {
msgPack := ms.MsgPack{}
baseMsg := ms.BaseMsg{
BeginTimestamp: req.Base.Timestamp,
EndTimestamp: req.Base.Timestamp,
HashValues: []uint32{0},
}
collMsg := &ms.CreateCollectionMsg{
BaseMsg: baseMsg,
CreateCollectionRequest: *req,
}
msgPack.Msgs = append(msgPack.Msgs, collMsg)
if err := ddStream.Broadcast(&msgPack); err != nil {
return err
}
return nil
}
c.DdDropCollectionReq = func(req *internalpb2.DropCollectionRequest) error {
//TODO, notify proxy to delete this collection before send this msg into dd channel
msgPack := ms.MsgPack{}
baseMsg := ms.BaseMsg{
BeginTimestamp: req.Base.Timestamp,
EndTimestamp: req.Base.Timestamp,
HashValues: []uint32{0},
}
collMsg := &ms.DropCollectionMsg{
BaseMsg: baseMsg,
DropCollectionRequest: *req,
}
msgPack.Msgs = append(msgPack.Msgs, collMsg)
if err := ddStream.Broadcast(&msgPack); err != nil {
return err
}
return nil
}
c.DdCreatePartitionReq = func(req *internalpb2.CreatePartitionRequest) error {
msgPack := ms.MsgPack{}
baseMsg := ms.BaseMsg{
BeginTimestamp: req.Base.Timestamp,
EndTimestamp: req.Base.Timestamp,
HashValues: []uint32{0},
}
collMsg := &ms.CreatePartitionMsg{
BaseMsg: baseMsg,
CreatePartitionRequest: *req,
}
msgPack.Msgs = append(msgPack.Msgs, collMsg)
if err := ddStream.Broadcast(&msgPack); err != nil {
return err
}
return nil
}
c.DdDropPartitionReq = func(req *internalpb2.DropPartitionRequest) error {
msgPack := ms.MsgPack{}
baseMsg := ms.BaseMsg{
BeginTimestamp: req.Base.Timestamp,
EndTimestamp: req.Base.Timestamp,
HashValues: []uint32{0},
}
collMsg := &ms.DropPartitionMsg{
BaseMsg: baseMsg,
DropPartitionRequest: *req,
}
msgPack.Msgs = append(msgPack.Msgs, collMsg)
if err := ddStream.Broadcast(&msgPack); err != nil {
return err
}
return nil
}
// receive time tick from msg stream
go func() {
c.ProxyTimeTickChan = make(chan typeutil.Timestamp, 1024)
for {
select {
case <-c.ctx.Done():
return
case ttmsgs, ok := <-proxyTimeTickStream.Chan():
if !ok {
log.Printf("proxy time tick msg stream closed")
return
}
if len(ttmsgs.Msgs) > 0 {
for _, ttm := range ttmsgs.Msgs {
ttmsg, ok := ttm.(*ms.TimeTickMsg)
if !ok {
continue
}
c.ProxyTimeTickChan <- ttmsg.Base.Timestamp
}
}
}
}
}()
//segment channel, data service create segment,or data node flush segment will put msg in this channel
dataServiceStream := ms.NewPulsarMsgStream(c.ctx, 1024)
dataServiceStream.SetPulsarClient(Params.PulsarAddress)
dataServiceStream.CreatePulsarConsumers([]string{Params.DataServiceSegmentChannel}, Params.MsgChannelSubName, ms.NewUnmarshalDispatcher(), 1024)
dataServiceStream.Start()
// receive segment info from msg stream
go func() {
for {
select {
case <-c.ctx.Done():
return
case segMsg, ok := <-dataServiceStream.Chan():
if !ok {
log.Printf("data service segment msg closed")
}
if len(segMsg.Msgs) > 0 {
for _, segm := range segMsg.Msgs {
segInfoMsg, ok := segm.(*ms.SegmentInfoMsg)
if ok {
if err := c.MetaTable.AddSegment(segInfoMsg.Segment); err != nil {
log.Printf("create segment failed, segmentid = %d,colleciont id = %d, error = %s", segInfoMsg.Segment.SegmentID, segInfoMsg.Segment.CollectionID, err.Error())
}
}
//TODO, if data node flush
}
}
}
}
}()
return nil
}
func (c *Core) Init(params *InitParams) error {
var initError error = nil
c.initOnce.Do(func() {
@ -455,7 +284,6 @@ func (c *Core) Init(params *InitParams) error {
return
}
c.ddReqQueue = make(chan reqTask, 1024)
initError = c.setMsgStreams()
c.isInit.Store(true)
})
return initError
@ -630,6 +458,36 @@ func (c *Core) DescribeCollection(in *milvuspb.DescribeCollectionRequest) (*milv
return t.Rsp, nil
}
func (c *Core) GetCollectionStatistics(in *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) {
t := &CollectionStatsReqTask{
baseReqTask: baseReqTask{
cv: make(chan error),
core: c,
},
Req: in,
Rsp: &milvuspb.CollectionStatsResponse{
Stats: nil,
Status: nil,
},
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &milvuspb.CollectionStatsResponse{
Stats: nil,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "GetCollectionStatistics failed: " + err.Error(),
},
}, nil
}
t.Rsp.Status = &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}
return t.Rsp, nil
}
func (c *Core) ShowCollections(in *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) {
t := &ShowCollectionReqTask{
baseReqTask: baseReqTask{
@ -732,6 +590,36 @@ func (c *Core) HasPartition(in *milvuspb.HasPartitionRequest) (*milvuspb.BoolRes
}, nil
}
func (c *Core) GetPartitionStatistics(in *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) {
t := &PartitionStatsReqTask{
baseReqTask: baseReqTask{
cv: make(chan error),
core: c,
},
Req: in,
Rsp: &milvuspb.PartitionStatsResponse{
Stats: nil,
Status: nil,
},
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &milvuspb.PartitionStatsResponse{
Stats: nil,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "GetPartitionStatistics failed: " + err.Error(),
},
}, nil
}
t.Rsp.Status = &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}
return t.Rsp, nil
}
func (c *Core) ShowPartitions(in *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) {
t := &ShowPartitionReqTask{
baseReqTask: baseReqTask{

View File

@ -1,41 +1,32 @@
package masterservice
import (
"log"
"path"
"fmt"
"strconv"
"sync"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
)
const (
TenantMetaPrefix = "tenant"
ProxyMetaPrefix = "proxy"
CollectionMetaPrefix = "collection"
PartitionMetaPrefix = "partition"
SegmentIndexMetaPrefix = "segment-index"
IndexMetaPrefix = "index"
)
type metaTable struct {
client kv.TxnBase // client of a reliable kv service, i.e. etcd client
tenantID2Meta map[typeutil.UniqueID]pb.TenantMeta // tenant id to tenant meta
proxyID2Meta map[typeutil.UniqueID]pb.ProxyMeta // proxy id to proxy meta
collID2Meta map[typeutil.UniqueID]pb.CollectionInfo // collection id to collection meta,
collName2ID map[string]typeutil.UniqueID // collection name to collection id
partitionID2Meta map[typeutil.UniqueID]pb.PartitionInfo //partition id -> partition meta
segID2IndexMeta map[typeutil.UniqueID]*map[typeutil.UniqueID]pb.SegmentIndexInfo // segment id -> index id -> segment index meta
indexID2Meta map[typeutil.UniqueID]pb.IndexInfo //index id ->index meta
client kv.TxnBase // client of a reliable kv service, i.e. etcd client
tenantID2Meta map[typeutil.UniqueID]pb.TenantMeta // tenant id to tenant meta
proxyID2Meta map[typeutil.UniqueID]pb.ProxyMeta // proxy id to proxy meta
collID2Meta map[typeutil.UniqueID]pb.CollectionMeta // collection id to collection meta
collName2ID map[string]typeutil.UniqueID // collection name to collection id
segID2Meta map[typeutil.UniqueID]pb.SegmentMeta // segment id to segment meta
segID2IndexMetas map[typeutil.UniqueID][]pb.FieldIndexMeta // segment id to array of field index meta
tenantLock sync.RWMutex
proxyLock sync.RWMutex
ddLock sync.RWMutex
indexLock sync.RWMutex
}
func NewMetaTable(kv kv.TxnBase) (*metaTable, error) {
@ -56,13 +47,12 @@ func (mt *metaTable) reloadFromKV() error {
mt.tenantID2Meta = make(map[typeutil.UniqueID]pb.TenantMeta)
mt.proxyID2Meta = make(map[typeutil.UniqueID]pb.ProxyMeta)
mt.collID2Meta = make(map[typeutil.UniqueID]pb.CollectionInfo)
mt.collID2Meta = make(map[typeutil.UniqueID]pb.CollectionMeta)
mt.collName2ID = make(map[string]typeutil.UniqueID)
mt.partitionID2Meta = make(map[typeutil.UniqueID]pb.PartitionInfo)
mt.segID2IndexMeta = make(map[typeutil.UniqueID]*map[typeutil.UniqueID]pb.SegmentIndexInfo)
mt.indexID2Meta = make(map[typeutil.UniqueID]pb.IndexInfo)
mt.segID2Meta = make(map[typeutil.UniqueID]pb.SegmentMeta)
mt.segID2IndexMetas = make(map[typeutil.UniqueID][]pb.FieldIndexMeta)
_, values, err := mt.client.LoadWithPrefix(TenantMetaPrefix)
_, values, err := mt.client.LoadWithPrefix("tenant")
if err != nil {
return err
}
@ -76,7 +66,7 @@ func (mt *metaTable) reloadFromKV() error {
mt.tenantID2Meta[tenantMeta.ID] = tenantMeta
}
_, values, err = mt.client.LoadWithPrefix(ProxyMetaPrefix)
_, values, err = mt.client.LoadWithPrefix("proxy")
if err != nil {
return err
}
@ -90,93 +80,160 @@ func (mt *metaTable) reloadFromKV() error {
mt.proxyID2Meta[proxyMeta.ID] = proxyMeta
}
_, values, err = mt.client.LoadWithPrefix(CollectionMetaPrefix)
_, values, err = mt.client.LoadWithPrefix("collection")
if err != nil {
return err
}
for _, value := range values {
collectionInfo := pb.CollectionInfo{}
err = proto.UnmarshalText(value, &collectionInfo)
collectionMeta := pb.CollectionMeta{}
err = proto.UnmarshalText(value, &collectionMeta)
if err != nil {
return err
}
mt.collID2Meta[collectionInfo.ID] = collectionInfo
mt.collName2ID[collectionInfo.Schema.Name] = collectionInfo.ID
mt.collID2Meta[collectionMeta.ID] = collectionMeta
mt.collName2ID[collectionMeta.Schema.Name] = collectionMeta.ID
}
_, values, err = mt.client.LoadWithPrefix(PartitionMetaPrefix)
_, values, err = mt.client.LoadWithPrefix("segment")
if err != nil {
return err
}
for _, value := range values {
partitionInfo := pb.PartitionInfo{}
err = proto.UnmarshalText(value, &partitionInfo)
segmentMeta := pb.SegmentMeta{}
err = proto.UnmarshalText(value, &segmentMeta)
if err != nil {
return err
}
mt.partitionID2Meta[partitionInfo.PartitionID] = partitionInfo
mt.segID2Meta[segmentMeta.SegmentID] = segmentMeta
}
_, values, err = mt.client.LoadWithPrefix(SegmentIndexMetaPrefix)
_, values, err = mt.client.LoadWithPrefix("indexmeta")
if err != nil {
return err
}
for _, value := range values {
segmentIndexInfo := pb.SegmentIndexInfo{}
err = proto.UnmarshalText(value, &segmentIndexInfo)
for _, v := range values {
indexMeta := pb.FieldIndexMeta{}
err = proto.UnmarshalText(v, &indexMeta)
if err != nil {
return err
}
idx, ok := mt.segID2IndexMeta[segmentIndexInfo.SegmentID]
if ok {
(*idx)[segmentIndexInfo.IndexID] = segmentIndexInfo
} else {
meta := make(map[typeutil.UniqueID]pb.SegmentIndexInfo)
meta[segmentIndexInfo.IndexID] = segmentIndexInfo
mt.segID2IndexMeta[segmentIndexInfo.SegmentID] = &meta
}
mt.segID2IndexMetas[indexMeta.SegmentID] = append(mt.segID2IndexMetas[indexMeta.SegmentID], indexMeta)
}
_, values, err = mt.client.LoadWithPrefix(IndexMetaPrefix)
if err != nil {
return err
}
for _, value := range values {
meta := pb.IndexInfo{}
err = proto.UnmarshalText(value, &meta)
if err != nil {
return err
}
mt.indexID2Meta[meta.IndexID] = meta
}
return nil
}
func (mt *metaTable) AddCollection(coll *pb.CollectionInfo, part *pb.PartitionInfo) error {
// MetaTable.ddLock.Lock() before call this function
func (mt *metaTable) saveCollectionMeta(coll *pb.CollectionMeta) error {
collBytes := proto.MarshalTextString(coll)
mt.collID2Meta[coll.ID] = *coll
mt.collName2ID[coll.Schema.Name] = coll.ID
return mt.client.Save("/collection/"+strconv.FormatInt(coll.ID, 10), collBytes)
}
// MetaTable.ddLock.Lock() before call this function
func (mt *metaTable) saveSegmentMeta(seg *pb.SegmentMeta) error {
segBytes := proto.MarshalTextString(seg)
mt.segID2Meta[seg.SegmentID] = *seg
return mt.client.Save("/segment/"+strconv.FormatInt(seg.SegmentID, 10), segBytes)
}
// MetaTable.ddLock.Lock() before call this function
func (mt *metaTable) saveCollectionAndDeleteSegmentsMeta(coll *pb.CollectionMeta, segIDs []typeutil.UniqueID) error {
segIDStrs := make([]string, 0, len(segIDs))
for _, segID := range segIDs {
segIDStrs = append(segIDStrs, "/segment/"+strconv.FormatInt(segID, 10))
}
kvs := make(map[string]string)
collStrs := proto.MarshalTextString(coll)
kvs["/collection/"+strconv.FormatInt(coll.ID, 10)] = collStrs
for _, segID := range segIDs {
_, ok := mt.segID2Meta[segID]
if ok {
delete(mt.segID2Meta, segID)
}
}
mt.collID2Meta[coll.ID] = *coll
return mt.client.MultiSaveAndRemove(kvs, segIDStrs)
}
// MetaTable.ddLock.Lock() before call this function
func (mt *metaTable) saveCollectionsAndSegmentsMeta(coll *pb.CollectionMeta, seg *pb.SegmentMeta) error {
kvs := make(map[string]string)
collBytes := proto.MarshalTextString(coll)
kvs["/collection/"+strconv.FormatInt(coll.ID, 10)] = collBytes
mt.collID2Meta[coll.ID] = *coll
mt.collName2ID[coll.Schema.Name] = coll.ID
segBytes := proto.MarshalTextString(seg)
kvs["/segment/"+strconv.FormatInt(seg.SegmentID, 10)] = segBytes
mt.segID2Meta[seg.SegmentID] = *seg
return mt.client.MultiSave(kvs)
}
// MetaTable.ddLock.Lock() before call this function
func (mt *metaTable) deleteCollectionsAndSegmentsMeta(collID typeutil.UniqueID, segIDs []typeutil.UniqueID) error {
collIDStr := "/collection/" + strconv.FormatInt(collID, 10)
totalIDStrs := make([]string, 0, 1+len(segIDs))
totalIDStrs = append(totalIDStrs, collIDStr)
for _, singleID := range segIDs {
totalIDStrs = append(totalIDStrs, "/segment/"+strconv.FormatInt(singleID, 10))
}
collMeta, ok := mt.collID2Meta[collID]
if ok {
delete(mt.collID2Meta, collID)
}
_, ok = mt.collName2ID[collMeta.Schema.Name]
if ok {
delete(mt.collName2ID, collMeta.Schema.Name)
}
for _, segID := range segIDs {
_, ok := mt.segID2Meta[segID]
if ok {
delete(mt.segID2Meta, segID)
}
}
return mt.client.MultiRemove(totalIDStrs)
}
func (mt *metaTable) AddCollection(coll *pb.CollectionMeta) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
if len(part.SegmentIDs) != 0 {
if len(coll.SegmentIDs) != 0 {
return errors.Errorf("segment should be empty when creating collection")
}
if len(coll.PartitionIDs) != 0 {
return errors.Errorf("partitions should be empty when creating collection")
if len(coll.PartitionTags) == 0 {
coll.PartitionTags = append(coll.PartitionTags, Params.DefaultPartitionTag)
}
coll.PartitionIDs = append(coll.PartitionIDs, part.PartitionID)
mt.collID2Meta[coll.ID] = *coll
mt.collName2ID[coll.Schema.Name] = coll.ID
mt.partitionID2Meta[part.PartitionID] = *part
k1 := path.Join(CollectionMetaPrefix, strconv.FormatInt(coll.ID, 10))
v1 := proto.MarshalTextString(coll)
k2 := path.Join(PartitionMetaPrefix, strconv.FormatInt(part.PartitionID, 10))
v2 := proto.MarshalTextString(part)
meta := map[string]string{k1: v1, k2: v2}
err := mt.client.MultiSave(meta)
_, ok := mt.collName2ID[coll.Schema.Name]
if ok {
return errors.Errorf("collection alread exists with name = " + coll.Schema.Name)
}
err := mt.saveCollectionMeta(coll)
if err != nil {
_ = mt.reloadFromKV()
return err
@ -193,43 +250,18 @@ func (mt *metaTable) DeleteCollection(collID typeutil.UniqueID) error {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
}
metaKeys := []string{path.Join(CollectionMetaPrefix, strconv.FormatInt(collID, 10))}
delete(mt.collID2Meta, collID)
delete(mt.collName2ID, collMeta.Schema.Name)
for _, partID := range collMeta.PartitionIDs {
metaKeys = append(metaKeys, path.Join(PartitionMetaPrefix, strconv.FormatInt(partID, 10)))
partMeta, ok := mt.partitionID2Meta[partID]
if !ok {
log.Printf("partition id = %d not exist", partID)
continue
}
delete(mt.partitionID2Meta, partID)
for _, segID := range partMeta.SegmentIDs {
segIndexMeta, ok := mt.segID2IndexMeta[segID]
if !ok {
log.Printf("segment id = %d not exist", segID)
continue
}
delete(mt.segID2IndexMeta, segID)
for indexID, segIdxMeta := range *segIndexMeta {
metaKeys = append(metaKeys, path.Join(SegmentIndexMetaPrefix, strconv.FormatInt(segID, 10), strconv.FormatInt(indexID, 10)))
indexMeta, ok := mt.indexID2Meta[segIdxMeta.IndexID]
if !ok {
log.Printf("index id = %d not exist", segIdxMeta.IndexID)
continue
}
delete(mt.indexID2Meta, segIdxMeta.IndexID)
metaKeys = append(metaKeys, path.Join(IndexMetaPrefix, strconv.FormatInt(indexMeta.IndexID, 10)))
}
}
}
err := mt.client.MultiRemove(metaKeys)
err := mt.deleteCollectionsAndSegmentsMeta(collID, collMeta.SegmentIDs)
if err != nil {
_ = mt.reloadFromKV()
return err
}
// remove index meta
for _, v := range collMeta.SegmentIDs {
if err := mt.removeSegmentIndexMeta(v); err != nil {
return err
}
}
return nil
}
@ -240,19 +272,19 @@ func (mt *metaTable) HasCollection(collID typeutil.UniqueID) bool {
return ok
}
func (mt *metaTable) GetCollectionByName(collectionName string) (pb.CollectionInfo, error) {
func (mt *metaTable) GetCollectionByName(collectionName string) (*pb.CollectionMeta, error) {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
vid, ok := mt.collName2ID[collectionName]
if !ok {
return pb.CollectionInfo{}, errors.Errorf("can't find collection: " + collectionName)
return nil, errors.Errorf("can't find collection: " + collectionName)
}
col, ok := mt.collID2Meta[vid]
if !ok {
return pb.CollectionInfo{}, errors.Errorf("can't find collection: " + collectionName)
return nil, errors.Errorf("can't find collection: " + collectionName)
}
return col, nil
return &col, nil
}
func (mt *metaTable) ListCollections() ([]string, error) {
@ -266,7 +298,7 @@ func (mt *metaTable) ListCollections() ([]string, error) {
return colls, nil
}
func (mt *metaTable) AddPartition(collID typeutil.UniqueID, partitionName string, partitionID typeutil.UniqueID) error {
func (mt *metaTable) AddPartition(collID typeutil.UniqueID, partitionTag string, partitionID typeutil.UniqueID) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
coll, ok := mt.collID2Meta[collID]
@ -275,39 +307,18 @@ func (mt *metaTable) AddPartition(collID typeutil.UniqueID, partitionName string
}
// number of partition tags (except _default) should be limited to 4096 by default
if int64(len(coll.PartitionIDs)) > Params.MaxPartitionNum {
if int64(len(coll.PartitionTags)) > Params.MaxPartitionNum {
return errors.New("maximum partition's number should be limit to " + strconv.FormatInt(Params.MaxPartitionNum, 10))
}
for _, t := range coll.PartitionIDs {
part, ok := mt.partitionID2Meta[t]
if !ok {
log.Printf("partition id = %d not exist", t)
continue
}
if part.PartitionName == partitionName {
return errors.Errorf("partition name = %s already exists", partitionName)
}
if part.PartitionID == partitionID {
return errors.Errorf("partition id = %d already exists", partitionID)
for _, t := range coll.PartitionTags {
if t == partitionTag {
return errors.Errorf("partition already exists.")
}
}
partMeta := pb.PartitionInfo{
PartitionName: partitionName,
PartitionID: partitionID,
SegmentIDs: make([]typeutil.UniqueID, 0, 16),
}
coll.PartitionTags = append(coll.PartitionTags, partitionTag)
coll.PartitionIDs = append(coll.PartitionIDs, partitionID)
mt.partitionID2Meta[partitionID] = partMeta
mt.collID2Meta[collID] = coll
k1 := path.Join(CollectionMetaPrefix, strconv.FormatInt(coll.ID, 10))
v1 := proto.MarshalTextString(&coll)
k2 := path.Join(PartitionMetaPrefix, strconv.FormatInt(partitionID, 10))
v2 := proto.MarshalTextString(&partMeta)
meta := map[string]string{k1: v1, k2: v2}
err := mt.client.MultiSave(meta)
err := mt.saveCollectionMeta(&coll)
if err != nil {
_ = mt.reloadFromKV()
return err
@ -315,138 +326,394 @@ func (mt *metaTable) AddPartition(collID typeutil.UniqueID, partitionName string
return nil
}
func (mt *metaTable) HasPartition(collID typeutil.UniqueID, partitionName string) bool {
func (mt *metaTable) HasPartition(collID typeutil.UniqueID, tag string) bool {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
col, ok := mt.collID2Meta[collID]
if !ok {
return false
}
for _, partitionID := range col.PartitionIDs {
meta, ok := mt.partitionID2Meta[partitionID]
if ok {
if meta.PartitionName == partitionName {
return true
}
for _, partitionTag := range col.PartitionTags {
if partitionTag == tag {
return true
}
}
return false
}
func (mt *metaTable) DeletePartition(collID typeutil.UniqueID, partitionName string) (typeutil.UniqueID, error) {
func (mt *metaTable) DeletePartition(collID typeutil.UniqueID, tag string) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
if partitionName == Params.DefaultPartitionName {
return 0, errors.New("default partition cannot be deleted")
if tag == Params.DefaultPartitionTag {
return errors.New("default partition cannot be deleted")
}
collMeta, ok := mt.collID2Meta[collID]
if !ok {
return 0, errors.Errorf("can't find collection id = " + strconv.FormatInt(collID, 10))
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
}
// check tag exists
exist := false
pt := make([]string, 0, len(collMeta.PartitionTags))
pd := make([]typeutil.UniqueID, 0, len(collMeta.PartitionIDs))
var partMeta pb.PartitionInfo
for _, t := range collMeta.PartitionIDs {
pm, ok := mt.partitionID2Meta[t]
if ok {
if pm.PartitionName != partitionName {
pd = append(pd, pm.PartitionID)
} else {
partMeta = pm
exist = true
}
for i, t := range collMeta.PartitionTags {
if t != tag {
pt = append(pt, t)
pd = append(pd, collMeta.PartitionIDs[i])
} else {
exist = true
}
}
if !exist {
return 0, errors.New("partition " + partitionName + " does not exist")
return errors.New("partition " + tag + " does not exist")
}
if len(pt) == len(collMeta.PartitionTags) {
return nil
}
delete(mt.partitionID2Meta, partMeta.PartitionID)
collMeta.PartitionIDs = pd
mt.collID2Meta[collID] = collMeta
delMetaKeys := []string{path.Join(PartitionMetaPrefix, strconv.FormatInt(partMeta.PartitionID, 10))}
for _, segID := range partMeta.SegmentIDs {
segIndexMeta, ok := mt.segID2IndexMeta[segID]
toDeleteSeg := make([]typeutil.UniqueID, 0, len(collMeta.SegmentIDs))
seg := make([]typeutil.UniqueID, 0, len(collMeta.SegmentIDs))
for _, s := range collMeta.SegmentIDs {
sm, ok := mt.segID2Meta[s]
if !ok {
log.Printf("segment id = %d not exist", segID)
continue
return errors.Errorf("DeletePartition:can't find segment id = %d", s)
}
delete(mt.segID2IndexMeta, segID)
for indexID, segIdxMeta := range *segIndexMeta {
delMetaKeys = append(delMetaKeys, path.Join(SegmentIndexMetaPrefix, strconv.FormatInt(segID, 10), strconv.FormatInt(indexID, 10)))
indexMeta, ok := mt.indexID2Meta[segIdxMeta.IndexID]
if !ok {
log.Printf("index id = %d not exist", segIdxMeta.IndexID)
continue
}
delete(mt.indexID2Meta, segIdxMeta.IndexID)
delMetaKeys = append(delMetaKeys, path.Join(IndexMetaPrefix, strconv.FormatInt(indexMeta.IndexID, 10)))
if sm.PartitionTag != tag {
seg = append(seg, s)
} else {
toDeleteSeg = append(toDeleteSeg, s)
}
}
collKV := map[string]string{path.Join(CollectionMetaPrefix, strconv.FormatInt(collID, 10)): proto.MarshalTextString(&collMeta)}
err := mt.client.MultiSaveAndRemove(collKV, delMetaKeys)
if err != nil {
_ = mt.reloadFromKV()
return 0, err
}
return partMeta.PartitionID, nil
}
func (mt *metaTable) GetPartitionByID(partitionID typeutil.UniqueID) (pb.PartitionInfo, error) {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
partMeta, ok := mt.partitionID2Meta[partitionID]
if !ok {
return pb.PartitionInfo{}, errors.Errorf("partition id = %d not exist", partitionID)
}
return partMeta, nil
}
func (mt *metaTable) AddSegment(seg *datapb.SegmentInfo) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
collMeta, ok := mt.collID2Meta[seg.CollectionID]
if !ok {
return errors.Errorf("can't find collection id = " + strconv.FormatInt(seg.CollectionID, 10))
}
partMeta, ok := mt.partitionID2Meta[seg.PartitionID]
if !ok {
return errors.Errorf("can't find partition id = " + strconv.FormatInt(seg.PartitionID, 10))
}
exist := false
for _, partID := range collMeta.PartitionIDs {
if partID == seg.PartitionID {
exist = true
break
}
}
if !exist {
return errors.Errorf("partition id = %d, not belong to collection id = %d", seg.PartitionID, seg.CollectionID)
}
exist = false
for _, segID := range partMeta.SegmentIDs {
if segID == seg.SegmentID {
exist = true
}
}
if exist {
return errors.Errorf("segment id = %d exist", seg.SegmentID)
}
partMeta.SegmentIDs = append(partMeta.SegmentIDs, seg.SegmentID)
mt.partitionID2Meta[seg.PartitionID] = partMeta
err := mt.client.Save(path.Join(PartitionMetaPrefix, strconv.FormatInt(seg.PartitionID, 10)), proto.MarshalTextString(&partMeta))
collMeta.PartitionTags = pt
collMeta.PartitionIDs = pd
collMeta.SegmentIDs = seg
err := mt.saveCollectionAndDeleteSegmentsMeta(&collMeta, toDeleteSeg)
if err != nil {
_ = mt.reloadFromKV()
return err
}
return nil
}
func (mt *metaTable) AddSegment(seg *pb.SegmentMeta) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
collID := seg.CollectionID
collMeta := mt.collID2Meta[collID]
collMeta.SegmentIDs = append(collMeta.SegmentIDs, seg.SegmentID)
err := mt.saveCollectionsAndSegmentsMeta(&collMeta, seg)
if err != nil {
_ = mt.reloadFromKV()
return err
}
return nil
}
func (mt *metaTable) UpdateSegment(seg *pb.SegmentMeta) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
collID := seg.CollectionID
collMeta := mt.collID2Meta[collID]
isNewSegID := true
for _, segID := range collMeta.SegmentIDs {
if segID == seg.SegmentID {
isNewSegID = false
break
}
}
if isNewSegID {
collMeta.SegmentIDs = append(collMeta.SegmentIDs, seg.SegmentID)
if err := mt.saveCollectionsAndSegmentsMeta(&collMeta, seg); err != nil {
_ = mt.reloadFromKV()
return err
}
} else {
if err := mt.saveSegmentMeta(seg); err != nil {
_ = mt.reloadFromKV()
return err
}
}
return nil
}
func (mt *metaTable) GetSegmentByID(segID typeutil.UniqueID) (*pb.SegmentMeta, error) {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
sm, ok := mt.segID2Meta[segID]
if !ok {
return nil, errors.Errorf("GetSegmentByID:can't find segment id = %d", segID)
}
return &sm, nil
}
func (mt *metaTable) DeleteSegment(segID typeutil.UniqueID) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
segMeta, ok := mt.segID2Meta[segID]
if !ok {
return errors.Errorf("DeleteSegment:can't find segment. id = " + strconv.FormatInt(segID, 10))
}
collMeta, ok := mt.collID2Meta[segMeta.CollectionID]
if !ok {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(segMeta.CollectionID, 10))
}
for i := 0; i < len(collMeta.SegmentIDs); i++ {
if collMeta.SegmentIDs[i] == segID {
collMeta.SegmentIDs = append(collMeta.SegmentIDs[:i], collMeta.SegmentIDs[i+1:]...)
}
}
err := mt.saveCollectionAndDeleteSegmentsMeta(&collMeta, []typeutil.UniqueID{segID})
if err != nil {
_ = mt.reloadFromKV()
return err
}
return mt.removeSegmentIndexMeta(segID)
}
func (mt *metaTable) CloseSegment(segID typeutil.UniqueID, closeTs typeutil.Timestamp) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
segMeta, ok := mt.segID2Meta[segID]
if !ok {
return errors.Errorf("CloseSegment:can't find segment id = " + strconv.FormatInt(segID, 10))
}
segMeta.CloseTime = closeTs
err := mt.saveSegmentMeta(&segMeta)
if err != nil {
_ = mt.reloadFromKV()
return err
}
return nil
}
func (mt *metaTable) AddFieldIndexMeta(meta *pb.FieldIndexMeta) error {
mt.indexLock.Lock()
defer mt.indexLock.Unlock()
segID := meta.SegmentID
if _, ok := mt.segID2IndexMetas[segID]; !ok {
mt.segID2IndexMetas[segID] = make([]pb.FieldIndexMeta, 0)
}
for _, v := range mt.segID2IndexMetas[segID] {
if v.FieldID == meta.FieldID && typeutil.CompareIndexParams(v.IndexParams, meta.IndexParams) {
return fmt.Errorf("segment %d field id %d's index meta already exist", segID, meta.FieldID)
}
}
mt.segID2IndexMetas[segID] = append(mt.segID2IndexMetas[segID], *meta)
err := mt.saveFieldIndexMetaToEtcd(meta)
if err != nil {
_ = mt.reloadFromKV()
return err
}
return nil
}
func (mt *metaTable) saveFieldIndexMetaToEtcd(meta *pb.FieldIndexMeta) error {
key := "/indexmeta/" + strconv.FormatInt(meta.SegmentID, 10) + strconv.FormatInt(meta.FieldID, 10) + strconv.FormatInt(meta.IndexID, 10)
marshaledMeta := proto.MarshalTextString(meta)
return mt.client.Save(key, marshaledMeta)
}
func (mt *metaTable) DeleteFieldIndexMeta(segID typeutil.UniqueID, fieldID typeutil.UniqueID, indexParams []*commonpb.KeyValuePair) error {
mt.indexLock.Lock()
defer mt.indexLock.Unlock()
if _, ok := mt.segID2IndexMetas[segID]; !ok {
return fmt.Errorf("can not find index meta of segment %d", segID)
}
for i, v := range mt.segID2IndexMetas[segID] {
if v.FieldID == fieldID && typeutil.CompareIndexParams(v.IndexParams, indexParams) {
mt.segID2IndexMetas[segID] = append(mt.segID2IndexMetas[segID][:i], mt.segID2IndexMetas[segID][i+1:]...)
err := mt.deleteFieldIndexMetaToEtcd(segID, fieldID, v.IndexID)
if err != nil {
_ = mt.reloadFromKV()
return err
}
return nil
}
}
return fmt.Errorf("can not find index meta of field %d", fieldID)
}
func (mt *metaTable) deleteFieldIndexMetaToEtcd(segID typeutil.UniqueID, fieldID typeutil.UniqueID, indexID typeutil.UniqueID) error {
key := "/indexmeta/" + strconv.FormatInt(segID, 10) + strconv.FormatInt(fieldID, 10) + strconv.FormatInt(indexID, 10)
return mt.client.Remove(key)
}
func (mt *metaTable) HasFieldIndexMeta(segID typeutil.UniqueID, fieldID typeutil.UniqueID, indexParams []*commonpb.KeyValuePair) (bool, error) {
mt.indexLock.RLock()
defer mt.indexLock.RUnlock()
if _, ok := mt.segID2IndexMetas[segID]; !ok {
return false, nil
}
for _, v := range mt.segID2IndexMetas[segID] {
if v.FieldID == fieldID && typeutil.CompareIndexParams(v.IndexParams, indexParams) {
return true, nil
}
}
return false, nil
}
func (mt *metaTable) GetFieldIndexMeta(segID typeutil.UniqueID, fieldID typeutil.UniqueID, indexParams []*commonpb.KeyValuePair) (*pb.FieldIndexMeta, error) {
mt.indexLock.RLock()
defer mt.indexLock.RUnlock()
if _, ok := mt.segID2IndexMetas[segID]; !ok {
return nil, fmt.Errorf("can not find segment %d", segID)
}
for _, v := range mt.segID2IndexMetas[segID] {
if v.FieldID == fieldID && typeutil.CompareIndexParams(v.IndexParams, indexParams) {
return &v, nil
}
}
return nil, fmt.Errorf("can not find field %d", fieldID)
}
func (mt *metaTable) UpdateFieldIndexMeta(meta *pb.FieldIndexMeta) error {
mt.indexLock.Lock()
defer mt.indexLock.Unlock()
segID := meta.SegmentID
if _, ok := mt.segID2IndexMetas[segID]; !ok {
mt.segID2IndexMetas[segID] = make([]pb.FieldIndexMeta, 0)
}
for i, v := range mt.segID2IndexMetas[segID] {
if v.FieldID == meta.FieldID && typeutil.CompareIndexParams(v.IndexParams, meta.IndexParams) {
mt.segID2IndexMetas[segID][i] = *meta
err := mt.deleteFieldIndexMetaToEtcd(segID, v.FieldID, v.IndexID)
if err != nil {
_ = mt.reloadFromKV()
return err
}
err = mt.saveFieldIndexMetaToEtcd(meta)
if err != nil {
_ = mt.reloadFromKV()
return err
}
return nil
}
}
mt.segID2IndexMetas[segID] = append(mt.segID2IndexMetas[segID], *meta)
err := mt.saveFieldIndexMetaToEtcd(meta)
if err != nil {
_ = mt.reloadFromKV()
return err
}
return nil
}
func (mt *metaTable) removeSegmentIndexMeta(segID typeutil.UniqueID) error {
mt.indexLock.Lock()
defer mt.indexLock.Unlock()
delete(mt.segID2IndexMetas, segID)
keys, _, err := mt.client.LoadWithPrefix("indexmeta/" + strconv.FormatInt(segID, 10))
if err != nil {
_ = mt.reloadFromKV()
return err
}
if err = mt.client.MultiRemove(keys); err != nil {
_ = mt.reloadFromKV()
return err
}
return nil
}
func (mt *metaTable) GetFieldTypeParams(collID typeutil.UniqueID, fieldID typeutil.UniqueID) ([]*commonpb.KeyValuePair, error) {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
if _, ok := mt.collID2Meta[collID]; !ok {
return nil, fmt.Errorf("can not find collection with id %d", collID)
}
for _, fieldSchema := range mt.collID2Meta[collID].Schema.Fields {
if fieldSchema.FieldID == fieldID {
return fieldSchema.TypeParams, nil
}
}
return nil, fmt.Errorf("can not find field %d in collection %d", fieldID, collID)
}
func (mt *metaTable) GetFieldIndexParams(collID typeutil.UniqueID, fieldID typeutil.UniqueID) ([]*commonpb.KeyValuePair, error) {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
if _, ok := mt.collID2Meta[collID]; !ok {
return nil, fmt.Errorf("can not find collection with id %d", collID)
}
for _, fieldSchema := range mt.collID2Meta[collID].Schema.Fields {
if fieldSchema.FieldID == fieldID {
return fieldSchema.IndexParams, nil
}
}
return nil, fmt.Errorf("can not find field %d in collection %d", fieldID, collID)
}
func (mt *metaTable) UpdateFieldIndexParams(collName string, fieldName string, indexParams []*commonpb.KeyValuePair) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
vid, ok := mt.collName2ID[collName]
if !ok {
return errors.Errorf("can't find collection: " + collName)
}
meta, ok := mt.collID2Meta[vid]
if !ok {
return errors.Errorf("can't find collection: " + collName)
}
for _, fieldSchema := range meta.Schema.Fields {
if fieldSchema.Name == fieldName {
fieldSchema.IndexParams = indexParams
if err := mt.saveCollectionMeta(&meta); err != nil {
_ = mt.reloadFromKV()
return err
}
return nil
}
}
return fmt.Errorf("can not find field with id %s", fieldName)
}
func (mt *metaTable) IsIndexable(collID typeutil.UniqueID, fieldID typeutil.UniqueID) (bool, error) {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
if _, ok := mt.collID2Meta[collID]; !ok {
return false, fmt.Errorf("can not find collection with id %d", collID)
}
for _, v := range mt.collID2Meta[collID].Schema.Fields {
// field is vector type and index params is not empty
if v.FieldID == fieldID && (v.DataType == schemapb.DataType_VECTOR_BINARY || v.DataType == schemapb.DataType_VECTOR_FLOAT) &&
len(v.IndexParams) != 0 {
return true, nil
}
}
// fieldID is not in schema(eg: timestamp) or not indexable
return false, nil
}

View File

@ -13,17 +13,16 @@ type ParamTable struct {
Port int
NodeID uint64
PulsarAddress string
EtcdAddress string
MetaRootPath string
KvRootPath string
ProxyTimeTickChannel string
MsgChannelSubName string
TimeTickChannel string
DdChannel string
StatisticsChannel string
DataServiceSegmentChannel string // data service create segment, or data node flush segment
PulsarAddress string
EtcdAddress string
MetaRootPath string
KvRootPath string
ProxyTimeTickChannel string
MsgChannelSubName string
TimeTickChannel string
DdChannel string
StatisticsChannel string
MaxPartitionNum int64
DefaultPartitionName string
MaxPartitionNum int64
DefaultPartitionTag string
}

View File

@ -1,11 +1,12 @@
package masterservice
import (
"fmt"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
@ -90,37 +91,19 @@ func (t *CreateCollectionReqTask) Execute() error {
if err != nil {
return err
}
collMeta := etcdpb.CollectionInfo{
ID: collID,
Schema: &schema,
CreateTime: collTs,
PartitionIDs: make([]typeutil.UniqueID, 0, 16),
coll := etcdpb.CollectionMeta{
ID: collID,
Schema: &schema,
CreateTime: collTs,
SegmentIDs: make([]typeutil.UniqueID, 0),
PartitionTags: []string{Params.DefaultPartitionTag},
PartitionIDs: []typeutil.UniqueID{partitionID},
}
partMeta := etcdpb.PartitionInfo{
PartitionName: Params.DefaultPartitionName,
PartitionID: partitionID,
SegmentIDs: make([]typeutil.UniqueID, 0, 16),
}
err = t.core.MetaTable.AddCollection(&collMeta, &partMeta)
err = t.core.MetaTable.AddCollection(&coll)
if err != nil {
return err
}
schemaBytes, err := proto.Marshal(&schema)
if err != nil {
return err
}
ddReq := internalpb2.CreateCollectionRequest{
Base: t.Req.Base,
DbName: t.Req.DbName,
CollectionName: t.Req.CollectionName,
DbID: 0, //TODO,not used
CollectionID: collID,
Schema: schemaBytes,
}
err = t.core.DdCreateCollectionReq(&ddReq)
err = t.core.DdCreateCollectionReq(t)
if err != nil {
return err
}
@ -153,15 +136,7 @@ func (t *DropCollectionReqTask) Execute() error {
//data service should drop segments , which belong to this collection, from the segment manager
ddReq := internalpb2.DropCollectionRequest{
Base: t.Req.Base,
DbName: t.Req.DbName,
CollectionName: t.Req.CollectionName,
DbID: 0, //not used
CollectionID: collMeta.ID,
}
err = t.core.DdDropCollectionReq(&ddReq)
err = t.core.DdDropCollectionReq(t)
if err != nil {
return err
}
@ -222,6 +197,50 @@ func (t *DescribeCollectionReqTask) Execute() error {
return nil
}
type CollectionStatsReqTask struct {
baseReqTask
Req *milvuspb.CollectionStatsRequest
Rsp *milvuspb.CollectionStatsResponse
}
func (t *CollectionStatsReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
func (t *CollectionStatsReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
//row_count
//data_size
func (t *CollectionStatsReqTask) Execute() error {
coll, err := t.core.MetaTable.GetCollectionByName(t.Req.CollectionName)
if err != nil {
return err
}
var rowCount int64 = 0
var dataSize int64 = 0
for _, seg := range coll.SegmentIDs {
m, e := t.core.GetSegmentMeta(seg)
if e != nil {
return e
}
rowCount += m.NumRows
dataSize += m.MemSize
}
t.Rsp.Stats = append(t.Rsp.Stats,
&commonpb.KeyValuePair{
Key: "row_count",
Value: fmt.Sprintf("%d", rowCount),
})
t.Rsp.Stats = append(t.Rsp.Stats,
&commonpb.KeyValuePair{
Key: "data_size",
Value: fmt.Sprintf("%d", dataSize),
})
return nil
}
type ShowCollectionReqTask struct {
baseReqTask
Req *milvuspb.ShowCollectionRequest
@ -272,17 +291,7 @@ func (t *CreatePartitionReqTask) Execute() error {
return err
}
ddReq := internalpb2.CreatePartitionRequest{
Base: t.Req.Base,
DbName: t.Req.DbName,
CollectionName: t.Req.CollectionName,
PartitionName: t.Req.PartitionName,
DbID: 0, // todo, not used
CollectionID: collMeta.ID,
PartitionID: partitionID,
}
err = t.core.DdCreatePartitionReq(&ddReq)
err = t.core.DdCreatePartitionReq(t)
if err != nil {
return err
}
@ -308,22 +317,12 @@ func (t *DropPartitionReqTask) Execute() error {
if err != nil {
return err
}
partID, err := t.core.MetaTable.DeletePartition(coll.ID, t.Req.PartitionName)
err = t.core.MetaTable.DeletePartition(coll.ID, t.Req.PartitionName)
if err != nil {
return err
}
ddReq := internalpb2.DropPartitionRequest{
Base: t.Req.Base,
DbName: t.Req.DbName,
CollectionName: t.Req.CollectionName,
PartitionName: t.Req.PartitionName,
DbID: 0, //todo,not used
CollectionID: coll.ID,
PartitionID: partID,
}
err = t.core.DdDropPartitionReq(&ddReq)
err = t.core.DdDropPartitionReq(t)
if err != nil {
return err
}
@ -353,6 +352,50 @@ func (t *HasPartitionReqTask) Execute() error {
return nil
}
type PartitionStatsReqTask struct {
baseReqTask
Req *milvuspb.PartitionStatsRequest
Rsp *milvuspb.PartitionStatsResponse
}
func (t *PartitionStatsReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
func (t *PartitionStatsReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
func (t *PartitionStatsReqTask) Execute() error {
coll, err := t.core.MetaTable.GetCollectionByName(t.Req.CollectionName)
if err != nil {
return err
}
var rowCount int64 = 0
var dataSize int64 = 0
for _, seg := range coll.SegmentIDs {
m, e := t.core.GetSegmentMeta(seg)
if e != nil {
return e
}
if m.PartitionTag == t.Req.PartitionName {
rowCount += m.NumRows
dataSize += m.MemSize
}
}
t.Rsp.Stats = append(t.Rsp.Stats,
&commonpb.KeyValuePair{
Key: "row_count",
Value: fmt.Sprintf("%d", rowCount),
})
t.Rsp.Stats = append(t.Rsp.Stats,
&commonpb.KeyValuePair{
Key: "data_size",
Value: fmt.Sprintf("%d", dataSize),
})
return nil
}
type ShowPartitionReqTask struct {
baseReqTask
Req *milvuspb.ShowPartitionRequest
@ -367,18 +410,12 @@ func (t *ShowPartitionReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
//TODO,list partition ids and partition tags
func (t *ShowPartitionReqTask) Execute() error {
coll, err := t.core.MetaTable.GetCollectionByName(t.Req.CollectionName)
if err != nil {
return err
}
for _, partID := range coll.PartitionIDs {
partMeta, err := t.core.MetaTable.GetPartitionByID(partID)
if err != nil {
return err
}
t.Rsp.PartitionIDs = append(t.Rsp.PartitionIDs, partMeta.PartitionID)
t.Rsp.PartitionNames = append(t.Rsp.PartitionNames, partMeta.PartitionName)
}
t.Rsp.PartitionNames = append(t.Rsp.PartitionNames, coll.PartitionTags...)
return nil
}

View File

@ -5,7 +5,6 @@ import (
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
)
@ -559,41 +558,3 @@ func (lim *LoadIndexMsg) Unmarshal(input []byte) (TsMsg, error) {
return loadIndexMsg, nil
}
/////////////////////////////////////////SegmentInfoMsg//////////////////////////////////////////
type SegmentInfoMsg struct {
BaseMsg
datapb.SegmentMsg
}
func (sim *SegmentInfoMsg) Type() MsgType {
return sim.Base.MsgType
}
func (sim *SegmentInfoMsg) GetMsgContext() context.Context {
return sim.MsgCtx
}
func (sim *SegmentInfoMsg) SetMsgContext(ctx context.Context) {
sim.MsgCtx = ctx
}
func (sim *SegmentInfoMsg) Marshal(input TsMsg) ([]byte, error) {
segInfoMsg := input.(*SegmentInfoMsg)
mb, err := proto.Marshal(&segInfoMsg.SegmentMsg)
if err != nil {
return nil, err
}
return mb, nil
}
func (sim *SegmentInfoMsg) Unmarshal(input []byte) (TsMsg, error) {
segMsg := datapb.SegmentMsg{}
err := proto.Unmarshal(input, &segMsg)
if err != nil {
return nil, err
}
return &SegmentInfoMsg{
SegmentMsg: segMsg,
}, nil
}

View File

@ -131,11 +131,6 @@ message SegmentInfo {
int64 mem_size=8;
}
message SegmentMsg{
common.MsgBase base = 1;
SegmentInfo segment = 2;
}
service DataService {
rpc RegisterNode(RegisterNodeRequest) returns (RegisterNodeResponse) {}

View File

@ -1056,53 +1056,6 @@ func (m *SegmentInfo) GetMemSize() int64 {
return 0
}
type SegmentMsg struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
Segment *SegmentInfo `protobuf:"bytes,2,opt,name=segment,proto3" json:"segment,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *SegmentMsg) Reset() { *m = SegmentMsg{} }
func (m *SegmentMsg) String() string { return proto.CompactTextString(m) }
func (*SegmentMsg) ProtoMessage() {}
func (*SegmentMsg) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{17}
}
func (m *SegmentMsg) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SegmentMsg.Unmarshal(m, b)
}
func (m *SegmentMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_SegmentMsg.Marshal(b, m, deterministic)
}
func (m *SegmentMsg) XXX_Merge(src proto.Message) {
xxx_messageInfo_SegmentMsg.Merge(m, src)
}
func (m *SegmentMsg) XXX_Size() int {
return xxx_messageInfo_SegmentMsg.Size(m)
}
func (m *SegmentMsg) XXX_DiscardUnknown() {
xxx_messageInfo_SegmentMsg.DiscardUnknown(m)
}
var xxx_messageInfo_SegmentMsg proto.InternalMessageInfo
func (m *SegmentMsg) GetBase() *commonpb.MsgBase {
if m != nil {
return m.Base
}
return nil
}
func (m *SegmentMsg) GetSegment() *SegmentInfo {
if m != nil {
return m.Segment
}
return nil
}
func init() {
proto.RegisterEnum("milvus.proto.data.SegmentState", SegmentState_name, SegmentState_value)
proto.RegisterType((*RegisterNodeRequest)(nil), "milvus.proto.data.RegisterNodeRequest")
@ -1122,87 +1075,84 @@ func init() {
proto.RegisterType((*WatchDmChannelRequest)(nil), "milvus.proto.data.WatchDmChannelRequest")
proto.RegisterType((*FlushSegRequest)(nil), "milvus.proto.data.FlushSegRequest")
proto.RegisterType((*SegmentInfo)(nil), "milvus.proto.data.SegmentInfo")
proto.RegisterType((*SegmentMsg)(nil), "milvus.proto.data.SegmentMsg")
}
func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad6cfe64) }
var fileDescriptor_3385cd32ad6cfe64 = []byte{
// 1169 bytes of a gzipped FileDescriptorProto
// 1144 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0xdd, 0x6e, 0x1b, 0xc5,
0x17, 0x8f, 0xbf, 0x12, 0xfb, 0xd8, 0xb1, 0x9d, 0x49, 0xfa, 0xff, 0xbb, 0x6e, 0x68, 0xc2, 0x4a,
0x6d, 0xa2, 0x0a, 0x12, 0x94, 0xaa, 0xc0, 0x15, 0xa2, 0xc1, 0x25, 0xb2, 0x20, 0x51, 0x34, 0x46,
0x42, 0x2a, 0x17, 0xd6, 0xda, 0x7b, 0xb2, 0x1e, 0x69, 0x77, 0x66, 0xd9, 0x19, 0x37, 0x51, 0x6e,
0xe0, 0x01, 0xfa, 0x04, 0x70, 0xcf, 0x25, 0x8f, 0x00, 0xaf, 0xc0, 0x23, 0xa1, 0x9d, 0x59, 0xaf,
0x77, 0xfd, 0x51, 0x47, 0x81, 0x72, 0xb7, 0x73, 0xf6, 0x37, 0xe7, 0xeb, 0x77, 0x3e, 0x06, 0x88,
0x63, 0x2b, 0xbb, 0x2f, 0x31, 0x7c, 0xc3, 0x86, 0x78, 0x14, 0x84, 0x42, 0x09, 0xb2, 0xe5, 0x33,
0xef, 0xcd, 0x58, 0x9a, 0xd3, 0x51, 0x04, 0x68, 0xd7, 0x86, 0xc2, 0xf7, 0x05, 0x37, 0xa2, 0x76,
0x9d, 0x71, 0x85, 0x21, 0xb7, 0x3d, 0x73, 0xb6, 0x7e, 0x82, 0x6d, 0x8a, 0x2e, 0x93, 0x0a, 0xc3,
0x0b, 0xe1, 0x20, 0xc5, 0x1f, 0xc7, 0x28, 0x15, 0xf9, 0x04, 0x8a, 0x03, 0x5b, 0x62, 0x2b, 0xb7,
0x9f, 0x3b, 0xac, 0x9e, 0xec, 0x1e, 0x65, 0xd4, 0xc6, 0x0a, 0xcf, 0xa5, 0x7b, 0x6a, 0x4b, 0xa4,
0x1a, 0x49, 0x3e, 0x85, 0x0d, 0xdb, 0x71, 0x42, 0x94, 0xb2, 0x95, 0x7f, 0xc7, 0xa5, 0x97, 0x06,
0x43, 0x27, 0x60, 0xeb, 0x35, 0xec, 0x64, 0x1d, 0x90, 0x81, 0xe0, 0x12, 0xc9, 0x29, 0x54, 0x19,
0x67, 0xaa, 0x1f, 0xd8, 0xa1, 0xed, 0xcb, 0xd8, 0x91, 0x0f, 0xb3, 0x3a, 0x93, 0x58, 0xba, 0x9c,
0xa9, 0x4b, 0x0d, 0xa4, 0xc0, 0x92, 0x6f, 0xeb, 0xaf, 0x1c, 0xd4, 0x7a, 0xe8, 0x76, 0x3b, 0x93,
0xb0, 0x76, 0xa0, 0x34, 0x14, 0x63, 0xae, 0xb4, 0xba, 0x4d, 0x6a, 0x0e, 0x64, 0x1f, 0xaa, 0xc3,
0x91, 0xcd, 0x39, 0x7a, 0x17, 0xb6, 0x8f, 0xda, 0xfd, 0x0a, 0x4d, 0x8b, 0x88, 0x05, 0xb5, 0xa1,
0xf0, 0x3c, 0x1c, 0x2a, 0x26, 0x78, 0xb7, 0xd3, 0x2a, 0xec, 0xe7, 0x0e, 0x0b, 0x34, 0x23, 0x8b,
0xb4, 0x04, 0x76, 0xa8, 0x58, 0x0c, 0x29, 0x6a, 0x48, 0x5a, 0x44, 0x1e, 0x41, 0x25, 0xba, 0xd1,
0xe7, 0x91, 0x95, 0x92, 0xb6, 0x52, 0x8e, 0x04, 0xda, 0xc4, 0x13, 0xa8, 0x27, 0x58, 0x83, 0x58,
0xd7, 0x88, 0xcd, 0x44, 0x1a, 0xc1, 0xac, 0xdf, 0x73, 0x40, 0x5e, 0x4a, 0xc9, 0x5c, 0x9e, 0x09,
0xec, 0x7f, 0xb0, 0xce, 0x85, 0x83, 0xdd, 0x8e, 0x8e, 0xac, 0x40, 0xe3, 0x53, 0x64, 0x32, 0x40,
0x0c, 0xfb, 0xa1, 0xf0, 0x26, 0x81, 0x95, 0x23, 0x01, 0x15, 0x1e, 0x92, 0x5d, 0xa8, 0x28, 0xe6,
0xa3, 0x54, 0xb6, 0x1f, 0xe8, 0x90, 0x8a, 0x74, 0x2a, 0x20, 0xaf, 0x60, 0x53, 0xa6, 0x4c, 0xc8,
0x56, 0x71, 0xbf, 0x70, 0x58, 0x3d, 0xd9, 0x3b, 0x9a, 0x2b, 0xb1, 0xa3, 0xb4, 0x2b, 0x34, 0x7b,
0xcb, 0xfa, 0x33, 0x0f, 0x0d, 0xfd, 0xdf, 0x78, 0xed, 0x23, 0xd7, 0x34, 0x68, 0x50, 0xec, 0xac,
0x39, 0xdc, 0x81, 0x86, 0x84, 0xbe, 0x42, 0x9a, 0xbe, 0x59, 0x72, 0x8a, 0xab, 0xc9, 0x29, 0xcd,
0x93, 0xb3, 0x07, 0x55, 0xbc, 0x09, 0x58, 0x88, 0xfd, 0x28, 0x05, 0x3a, 0xf9, 0x45, 0x0a, 0x46,
0xf4, 0x1d, 0xf3, 0x91, 0x3c, 0x87, 0x75, 0xa9, 0x6c, 0x35, 0x96, 0xad, 0x0d, 0x5d, 0x8b, 0x8f,
0x16, 0xd6, 0x77, 0x4f, 0x43, 0x68, 0x0c, 0xcd, 0x52, 0x5e, 0x5e, 0x49, 0x79, 0x65, 0x11, 0xe5,
0xbf, 0xe4, 0x60, 0x3b, 0x43, 0x79, 0xdc, 0x21, 0x17, 0xd0, 0x94, 0xd9, 0xc4, 0x46, 0x6d, 0x12,
0x71, 0x64, 0x2d, 0xe3, 0x68, 0x0a, 0xa5, 0x73, 0x77, 0x53, 0x01, 0xe6, 0xef, 0x1c, 0xa0, 0x75,
0x03, 0xb5, 0xaf, 0xbd, 0xb1, 0x1c, 0xdd, 0x7f, 0x70, 0x10, 0x28, 0x3a, 0x83, 0x6e, 0x47, 0x1b,
0x2d, 0x50, 0xfd, 0x7d, 0x17, 0x4a, 0xad, 0xb7, 0x39, 0x20, 0xbd, 0x91, 0xb8, 0xee, 0xa1, 0xab,
0x03, 0xba, 0xb7, 0x03, 0xb3, 0xc6, 0xf2, 0xab, 0xeb, 0xa7, 0x30, 0x57, 0x3f, 0xd6, 0x0b, 0xd8,
0xce, 0x78, 0x13, 0x93, 0xf4, 0x18, 0x40, 0x1a, 0x51, 0xb7, 0x63, 0xe8, 0x29, 0xd0, 0x94, 0xc4,
0xba, 0x82, 0x9d, 0xf8, 0x4a, 0x94, 0x58, 0x94, 0xf7, 0x0f, 0x63, 0x17, 0x2a, 0x89, 0xde, 0x38,
0x86, 0xa9, 0xc0, 0xfa, 0x2d, 0x0f, 0x0f, 0x66, 0x0c, 0xc5, 0x1e, 0xbe, 0x80, 0x52, 0xc4, 0xa5,
0x31, 0x55, 0x5f, 0xd6, 0xdf, 0xc9, 0x45, 0x6a, 0xd0, 0x51, 0xbf, 0x0c, 0x43, 0xb4, 0x55, 0xdc,
0x2f, 0x79, 0xd3, 0x2f, 0x46, 0xa4, 0xfb, 0x65, 0x0f, 0xaa, 0x12, 0x6d, 0x0f, 0x1d, 0x03, 0x30,
0xf3, 0x05, 0x8c, 0x48, 0x03, 0xbe, 0x81, 0x86, 0x54, 0x76, 0xa8, 0xfa, 0x81, 0x90, 0x3a, 0x8b,
0x93, 0x11, 0x63, 0x2d, 0x99, 0xf2, 0xe7, 0xd2, 0xbd, 0x8c, 0xa1, 0xb4, 0xae, 0xaf, 0x4e, 0x8e,
0x92, 0x9c, 0xc1, 0x26, 0x72, 0x27, 0xa5, 0xaa, 0x74, 0x67, 0x55, 0x35, 0xe4, 0x4e, 0xa2, 0xc8,
0x62, 0xf0, 0xff, 0x2e, 0x97, 0x18, 0xaa, 0x53, 0xc6, 0x3d, 0xe1, 0x5e, 0xda, 0x6a, 0xf4, 0xbe,
0x38, 0x09, 0xe0, 0xe1, 0xac, 0xa9, 0x29, 0x2d, 0x6d, 0x28, 0x5f, 0x31, 0xf4, 0x9c, 0x69, 0xd9,
0x24, 0x67, 0xf2, 0x19, 0x94, 0x82, 0x08, 0xdc, 0xca, 0xeb, 0x20, 0x97, 0x6d, 0xc5, 0x9e, 0x0a,
0x19, 0x77, 0xbf, 0x65, 0x52, 0x51, 0x83, 0xb7, 0x7e, 0xce, 0xc1, 0x8e, 0x31, 0xf9, 0x95, 0x19,
0xab, 0xef, 0xb7, 0x6d, 0x17, 0xac, 0x49, 0xcb, 0x87, 0x07, 0xdf, 0xdb, 0x6a, 0x38, 0xea, 0xf8,
0xff, 0xd8, 0x85, 0xc8, 0xdc, 0x74, 0x3b, 0x98, 0x6c, 0x54, 0x68, 0x46, 0x66, 0xfd, 0x9a, 0x83,
0x86, 0x1e, 0x50, 0x3d, 0x74, 0xff, 0xf3, 0x60, 0x67, 0xba, 0xbf, 0x38, 0xd7, 0xfd, 0x6f, 0xf3,
0x50, 0x8d, 0x9b, 0xab, 0xcb, 0xaf, 0x44, 0xb6, 0x5e, 0x72, 0x33, 0xf5, 0xf2, 0xef, 0x0c, 0x2a,
0x72, 0x00, 0x0d, 0xa6, 0x4b, 0xa0, 0x1f, 0x27, 0xca, 0x38, 0x56, 0xa1, 0x75, 0x96, 0xae, 0x0c,
0xbd, 0xbb, 0x44, 0x80, 0xdc, 0xb4, 0x6f, 0x49, 0xb7, 0x6f, 0x39, 0x12, 0xe8, 0xe6, 0xfd, 0x00,
0x60, 0xe8, 0x09, 0x99, 0xd9, 0x96, 0x15, 0x2d, 0xd1, 0xbf, 0x1f, 0x42, 0x99, 0x8f, 0xfd, 0x7e,
0x28, 0xae, 0xcd, 0xba, 0x2c, 0xd0, 0x0d, 0x3e, 0xf6, 0xa9, 0xb8, 0x96, 0xd1, 0x2f, 0x1f, 0xfd,
0xbe, 0x64, 0xb7, 0x66, 0x23, 0x16, 0xe8, 0x86, 0x8f, 0x7e, 0x8f, 0xdd, 0xa2, 0x75, 0x03, 0x10,
0x67, 0xe3, 0x5c, 0xba, 0xf7, 0xa0, 0xe9, 0x73, 0xd8, 0x88, 0xb3, 0x15, 0xaf, 0xb0, 0xc7, 0xcb,
0x87, 0x59, 0x94, 0x6f, 0x3a, 0x81, 0x3f, 0xfb, 0x41, 0x3f, 0x14, 0x93, 0x21, 0x47, 0x1a, 0x09,
0x2f, 0x17, 0x82, 0x63, 0x73, 0x8d, 0x6c, 0xeb, 0x57, 0x8c, 0x11, 0xa8, 0x57, 0x37, 0x4c, 0xaa,
0x66, 0x8e, 0x10, 0xa8, 0xc7, 0xc2, 0xb3, 0x50, 0x5c, 0x33, 0xee, 0x36, 0xf3, 0x64, 0x0b, 0x36,
0x27, 0x9a, 0xf4, 0xa8, 0x6b, 0x16, 0x4e, 0xfe, 0x28, 0x41, 0xb5, 0x63, 0x2b, 0xbb, 0x67, 0x9e,
0xea, 0xc4, 0x86, 0x5a, 0xfa, 0xc9, 0x4b, 0x9e, 0x2e, 0xf0, 0x72, 0xc1, 0xa3, 0xbc, 0x7d, 0xb0,
0x12, 0x67, 0x66, 0x87, 0xb5, 0x46, 0xce, 0xa0, 0xa4, 0xab, 0x9e, 0x2c, 0x1a, 0xe7, 0xe9, 0x85,
0xdd, 0x7e, 0xd7, 0x96, 0xb7, 0xd6, 0xc8, 0x00, 0x1a, 0xc9, 0xdb, 0x23, 0x2e, 0xc3, 0x27, 0x0b,
0x54, 0xce, 0x3f, 0x49, 0xdb, 0x4f, 0x57, 0xc1, 0x12, 0x67, 0xfb, 0x50, 0x4b, 0xad, 0x4e, 0xb9,
0xd0, 0xc0, 0xfc, 0xa6, 0x5f, 0x68, 0x60, 0xc1, 0x0a, 0xb6, 0xd6, 0x88, 0x0b, 0xcd, 0x33, 0x54,
0x99, 0xf5, 0x47, 0x0e, 0x56, 0xec, 0xb9, 0xc9, 0x26, 0x6e, 0x1f, 0xae, 0x06, 0x26, 0x86, 0x42,
0xd8, 0x39, 0x43, 0x35, 0x37, 0xd4, 0xc9, 0xb3, 0x05, 0x3a, 0x96, 0x6c, 0x99, 0xf6, 0x47, 0x77,
0xc0, 0xa6, 0x6d, 0xda, 0xb0, 0x95, 0xd8, 0x4c, 0x7a, 0xf7, 0x60, 0xa9, 0x92, 0xec, 0xd4, 0x6d,
0xaf, 0xde, 0x1d, 0xd6, 0xda, 0xe9, 0x97, 0xaf, 0xbf, 0x70, 0x99, 0x1a, 0x8d, 0x07, 0x51, 0x79,
0x1c, 0xdf, 0x32, 0xcf, 0x63, 0xb7, 0x0a, 0x87, 0xa3, 0x63, 0x73, 0xf7, 0x63, 0x87, 0x49, 0x15,
0xb2, 0xc1, 0x58, 0xa1, 0x73, 0x3c, 0xd1, 0x70, 0xac, 0x15, 0x1e, 0x47, 0x96, 0x83, 0xc1, 0x60,
0x5d, 0x9f, 0x9e, 0xff, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x45, 0xdf, 0xe6, 0xdf, 0xb4, 0x0e, 0x00,
0x00,
0x17, 0x8f, 0xbf, 0x12, 0xfb, 0xd8, 0xb1, 0xdd, 0x49, 0xfa, 0xff, 0xbb, 0x6e, 0x21, 0x61, 0xa5,
0x36, 0x51, 0x05, 0x09, 0x4a, 0x55, 0xb8, 0x43, 0x34, 0xb8, 0x44, 0x16, 0x34, 0x8a, 0xc6, 0x48,
0x48, 0xe5, 0xc2, 0x5a, 0x7b, 0x4f, 0xd6, 0x23, 0xed, 0xce, 0x2c, 0x3b, 0xe3, 0x26, 0xca, 0x0d,
0x3c, 0x40, 0x9f, 0x00, 0xee, 0xb9, 0xe4, 0x11, 0xe0, 0x15, 0x78, 0x24, 0xb4, 0x33, 0xeb, 0xf5,
0xae, 0xbd, 0xae, 0xa3, 0x40, 0xb9, 0xdb, 0x39, 0xfb, 0x9b, 0xf3, 0xf5, 0x3b, 0x1f, 0x03, 0xc4,
0xb1, 0x95, 0x3d, 0x94, 0x18, 0xbe, 0x61, 0x63, 0x3c, 0x0a, 0x42, 0xa1, 0x04, 0xb9, 0xe7, 0x33,
0xef, 0xcd, 0x54, 0x9a, 0xd3, 0x51, 0x04, 0xe8, 0x36, 0xc6, 0xc2, 0xf7, 0x05, 0x37, 0xa2, 0x6e,
0x93, 0x71, 0x85, 0x21, 0xb7, 0x3d, 0x73, 0xb6, 0x7e, 0x82, 0x1d, 0x8a, 0x2e, 0x93, 0x0a, 0xc3,
0x73, 0xe1, 0x20, 0xc5, 0x1f, 0xa7, 0x28, 0x15, 0xf9, 0x14, 0xca, 0x23, 0x5b, 0x62, 0xa7, 0xb0,
0x5f, 0x38, 0xac, 0x9f, 0x3c, 0x3a, 0xca, 0xa8, 0x8d, 0x15, 0xbe, 0x92, 0xee, 0xa9, 0x2d, 0x91,
0x6a, 0x24, 0xf9, 0x0c, 0xb6, 0x6c, 0xc7, 0x09, 0x51, 0xca, 0x4e, 0xf1, 0x1d, 0x97, 0x5e, 0x18,
0x0c, 0x9d, 0x81, 0xad, 0xd7, 0xb0, 0x9b, 0x75, 0x40, 0x06, 0x82, 0x4b, 0x24, 0xa7, 0x50, 0x67,
0x9c, 0xa9, 0x61, 0x60, 0x87, 0xb6, 0x2f, 0x63, 0x47, 0x3e, 0xca, 0xea, 0x4c, 0x62, 0xe9, 0x73,
0xa6, 0x2e, 0x34, 0x90, 0x02, 0x4b, 0xbe, 0xad, 0xbf, 0x0a, 0xd0, 0x18, 0xa0, 0xdb, 0xef, 0xcd,
0xc2, 0xda, 0x85, 0xca, 0x58, 0x4c, 0xb9, 0xd2, 0xea, 0xb6, 0xa9, 0x39, 0x90, 0x7d, 0xa8, 0x8f,
0x27, 0x36, 0xe7, 0xe8, 0x9d, 0xdb, 0x3e, 0x6a, 0xf7, 0x6b, 0x34, 0x2d, 0x22, 0x16, 0x34, 0xc6,
0xc2, 0xf3, 0x70, 0xac, 0x98, 0xe0, 0xfd, 0x5e, 0xa7, 0xb4, 0x5f, 0x38, 0x2c, 0xd1, 0x8c, 0x2c,
0xd2, 0x12, 0xd8, 0xa1, 0x62, 0x31, 0xa4, 0xac, 0x21, 0x69, 0x11, 0x79, 0x08, 0xb5, 0xe8, 0xc6,
0x90, 0x47, 0x56, 0x2a, 0xda, 0x4a, 0x35, 0x12, 0x68, 0x13, 0x8f, 0xa1, 0x99, 0x60, 0x0d, 0x62,
0x53, 0x23, 0xb6, 0x13, 0x69, 0x04, 0xb3, 0x7e, 0x2f, 0x00, 0x79, 0x21, 0x25, 0x73, 0x79, 0x26,
0xb0, 0xff, 0xc1, 0x26, 0x17, 0x0e, 0xf6, 0x7b, 0x3a, 0xb2, 0x12, 0x8d, 0x4f, 0x91, 0xc9, 0x00,
0x31, 0x1c, 0x86, 0xc2, 0x9b, 0x05, 0x56, 0x8d, 0x04, 0x54, 0x78, 0x48, 0x1e, 0x41, 0x4d, 0x31,
0x1f, 0xa5, 0xb2, 0xfd, 0x40, 0x87, 0x54, 0xa6, 0x73, 0x01, 0x79, 0x09, 0xdb, 0x32, 0x65, 0x42,
0x76, 0xca, 0xfb, 0xa5, 0xc3, 0xfa, 0xc9, 0xde, 0xd1, 0x52, 0x89, 0x1d, 0xa5, 0x5d, 0xa1, 0xd9,
0x5b, 0xd6, 0x9f, 0x45, 0x68, 0xe9, 0xff, 0xc6, 0x6b, 0x1f, 0xb9, 0xa6, 0x41, 0x83, 0x62, 0x67,
0xcd, 0xe1, 0x16, 0x34, 0x24, 0xf4, 0x95, 0xd2, 0xf4, 0x2d, 0x92, 0x53, 0x5e, 0x4f, 0x4e, 0x65,
0x99, 0x9c, 0x3d, 0xa8, 0xe3, 0x75, 0xc0, 0x42, 0x1c, 0x46, 0x29, 0xd0, 0xc9, 0x2f, 0x53, 0x30,
0xa2, 0xef, 0x98, 0x8f, 0xe4, 0x19, 0x6c, 0x4a, 0x65, 0xab, 0xa9, 0xec, 0x6c, 0xe9, 0x5a, 0x7c,
0x98, 0x5b, 0xdf, 0x03, 0x0d, 0xa1, 0x31, 0x34, 0x4b, 0x79, 0x75, 0x2d, 0xe5, 0xb5, 0x3c, 0xca,
0x7f, 0x29, 0xc0, 0x4e, 0x86, 0xf2, 0xb8, 0x43, 0xce, 0xa1, 0x2d, 0xb3, 0x89, 0x8d, 0xda, 0x24,
0xe2, 0xc8, 0x5a, 0xc5, 0xd1, 0x1c, 0x4a, 0x97, 0xee, 0xa6, 0x02, 0x2c, 0xde, 0x3a, 0x40, 0xeb,
0x1a, 0x1a, 0x5f, 0x7b, 0x53, 0x39, 0xb9, 0xfb, 0xe0, 0x20, 0x50, 0x76, 0x46, 0xfd, 0x9e, 0x36,
0x5a, 0xa2, 0xfa, 0xfb, 0x36, 0x94, 0x5a, 0x6f, 0x0b, 0x40, 0x06, 0x13, 0x71, 0x35, 0x40, 0x57,
0x07, 0x74, 0x67, 0x07, 0x16, 0x8d, 0x15, 0xd7, 0xd7, 0x4f, 0x69, 0xa9, 0x7e, 0xac, 0xe7, 0xb0,
0x93, 0xf1, 0x26, 0x26, 0xe9, 0x43, 0x00, 0x69, 0x44, 0xfd, 0x9e, 0xa1, 0xa7, 0x44, 0x53, 0x12,
0xeb, 0x12, 0x76, 0xe3, 0x2b, 0x51, 0x62, 0x51, 0xde, 0x3d, 0x8c, 0x47, 0x50, 0x4b, 0xf4, 0xc6,
0x31, 0xcc, 0x05, 0xd6, 0x6f, 0x45, 0xb8, 0xbf, 0x60, 0x28, 0xf6, 0xf0, 0x39, 0x54, 0x22, 0x2e,
0x8d, 0xa9, 0xe6, 0xaa, 0xfe, 0x4e, 0x2e, 0x52, 0x83, 0x8e, 0xfa, 0x65, 0x1c, 0xa2, 0xad, 0xe2,
0x7e, 0x29, 0x9a, 0x7e, 0x31, 0x22, 0xdd, 0x2f, 0x7b, 0x50, 0x97, 0x68, 0x7b, 0xe8, 0x18, 0x80,
0x99, 0x2f, 0x60, 0x44, 0x1a, 0xf0, 0x0d, 0xb4, 0xa4, 0xb2, 0x43, 0x35, 0x0c, 0x84, 0xd4, 0x59,
0x9c, 0x8d, 0x18, 0x6b, 0xc5, 0x94, 0x7f, 0x25, 0xdd, 0x8b, 0x18, 0x4a, 0x9b, 0xfa, 0xea, 0xec,
0x28, 0xc9, 0x19, 0x6c, 0x23, 0x77, 0x52, 0xaa, 0x2a, 0xb7, 0x56, 0xd5, 0x40, 0xee, 0x24, 0x8a,
0x2c, 0x06, 0xff, 0xef, 0x73, 0x89, 0xa1, 0x3a, 0x65, 0xdc, 0x13, 0xee, 0x85, 0xad, 0x26, 0xef,
0x8b, 0x93, 0x00, 0x1e, 0x2c, 0x9a, 0x9a, 0xd3, 0xd2, 0x85, 0xea, 0x25, 0x43, 0xcf, 0x99, 0x97,
0x4d, 0x72, 0x26, 0x9f, 0x43, 0x25, 0x88, 0xc0, 0x9d, 0xa2, 0x0e, 0x72, 0xd5, 0x56, 0x1c, 0xa8,
0x90, 0x71, 0xf7, 0x5b, 0x26, 0x15, 0x35, 0x78, 0xeb, 0xe7, 0x02, 0xec, 0x1a, 0x93, 0x5f, 0x99,
0xb1, 0xfa, 0x7e, 0xdb, 0x36, 0x67, 0x4d, 0x5a, 0x3e, 0xdc, 0xff, 0xde, 0x56, 0xe3, 0x49, 0xcf,
0xff, 0xc7, 0x2e, 0x44, 0xe6, 0xe6, 0xdb, 0xc1, 0x64, 0xa3, 0x46, 0x33, 0x32, 0xeb, 0xd7, 0x02,
0xb4, 0xf4, 0x80, 0x1a, 0xa0, 0xfb, 0x9f, 0x07, 0xbb, 0xd0, 0xfd, 0xe5, 0xa5, 0xee, 0x7f, 0x5b,
0x84, 0x7a, 0xdc, 0x5c, 0x7d, 0x7e, 0x29, 0xb2, 0xf5, 0x52, 0x58, 0xa8, 0x97, 0x7f, 0x67, 0x50,
0x91, 0x03, 0x68, 0x31, 0x5d, 0x02, 0xc3, 0x38, 0x51, 0xc6, 0xb1, 0x1a, 0x6d, 0xb2, 0x74, 0x65,
0xe8, 0xdd, 0x25, 0x02, 0xe4, 0xa6, 0x7d, 0x2b, 0xba, 0x7d, 0xab, 0x91, 0x40, 0x37, 0xef, 0x07,
0x00, 0x63, 0x4f, 0xc8, 0xcc, 0xb6, 0xac, 0x69, 0x89, 0xfe, 0xfd, 0x00, 0xaa, 0x7c, 0xea, 0x0f,
0x43, 0x71, 0x65, 0xd6, 0x65, 0x89, 0x6e, 0xf1, 0xa9, 0x4f, 0xc5, 0x95, 0x8c, 0x7e, 0xf9, 0xe8,
0x0f, 0x25, 0xbb, 0x31, 0x1b, 0xb1, 0x44, 0xb7, 0x7c, 0xf4, 0x07, 0xec, 0x06, 0x9f, 0xfe, 0xa0,
0x9f, 0x6b, 0xc9, 0xa8, 0x21, 0xad, 0x24, 0x3b, 0xe7, 0x82, 0x63, 0x7b, 0x83, 0xec, 0xe8, 0xb7,
0x84, 0x11, 0xa8, 0x97, 0xd7, 0x4c, 0xaa, 0x76, 0x81, 0x10, 0x68, 0xc6, 0xc2, 0xb3, 0x50, 0x5c,
0x31, 0xee, 0xb6, 0x8b, 0xe4, 0x1e, 0x6c, 0xcf, 0x34, 0xe9, 0x81, 0xd3, 0x2e, 0x9d, 0xfc, 0x51,
0x81, 0x7a, 0xcf, 0x56, 0xf6, 0xc0, 0x3c, 0x98, 0x89, 0x0d, 0x8d, 0xf4, 0xc3, 0x93, 0x3c, 0xc9,
0x19, 0x7c, 0x39, 0x4f, 0xe3, 0xee, 0xc1, 0x5a, 0x9c, 0xe9, 0x60, 0x6b, 0x83, 0x9c, 0x41, 0x45,
0xd7, 0x1e, 0xc9, 0x1b, 0xaa, 0xe9, 0xb5, 0xd9, 0x7d, 0xd7, 0xae, 0xb5, 0x36, 0xc8, 0x08, 0x5a,
0xc9, 0x0b, 0x20, 0x2e, 0x86, 0xc7, 0x39, 0x2a, 0x97, 0x1f, 0x86, 0xdd, 0x27, 0xeb, 0x60, 0x89,
0xb3, 0x43, 0x68, 0xa4, 0x16, 0x98, 0xcc, 0x35, 0xb0, 0xbc, 0x6f, 0x73, 0x0d, 0xe4, 0x2c, 0x42,
0x6b, 0x83, 0xb8, 0xd0, 0x3e, 0x43, 0x95, 0x59, 0x42, 0xe4, 0x60, 0xcd, 0xb6, 0x99, 0xed, 0xc3,
0xee, 0xe1, 0x7a, 0x60, 0x62, 0x28, 0x84, 0xdd, 0x33, 0x54, 0x4b, 0xa3, 0x95, 0x3c, 0xcd, 0xd1,
0xb1, 0x62, 0xd6, 0x77, 0x3f, 0xbe, 0x05, 0x36, 0x6d, 0xd3, 0x86, 0x7b, 0x89, 0xcd, 0xa4, 0x83,
0x0e, 0x56, 0x2a, 0xc9, 0xce, 0xbe, 0xee, 0xfa, 0x09, 0x6e, 0x6d, 0x9c, 0x7e, 0xf9, 0xfa, 0x0b,
0x97, 0xa9, 0xc9, 0x74, 0x14, 0x95, 0xc7, 0xf1, 0x0d, 0xf3, 0x3c, 0x76, 0xa3, 0x70, 0x3c, 0x39,
0x36, 0x77, 0x3f, 0x71, 0x98, 0x54, 0x21, 0x1b, 0x4d, 0x15, 0x3a, 0xc7, 0x33, 0x0d, 0xc7, 0x5a,
0xe1, 0x71, 0x64, 0x39, 0x18, 0x8d, 0x36, 0xf5, 0xe9, 0xd9, 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff,
0x92, 0x26, 0x37, 0x78, 0x3a, 0x0e, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.

View File

@ -14,17 +14,23 @@ message RegisterNodeRequest {
}
message RegisterNodeResponse {
internal.InitParams init_params = 1;
common.Status status = 1;
internal.InitParams init_params = 2;
}
message IndexStatesRequest {
int64 indexID = 1;
repeated int64 indexID = 1;
}
message IndexInfo {
common.IndexState state = 1;
int64 indexID = 2;
string Reason = 3;
}
message IndexStatesResponse {
common.Status status = 1;
common.IndexState state = 2;
int64 indexID = 3;
repeated IndexInfo states = 2;
}
message BuildIndexRequest {

View File

@ -74,7 +74,8 @@ func (m *RegisterNodeRequest) GetAddress() *commonpb.Address {
}
type RegisterNodeResponse struct {
InitParams *internalpb2.InitParams `protobuf:"bytes,1,opt,name=init_params,json=initParams,proto3" json:"init_params,omitempty"`
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
InitParams *internalpb2.InitParams `protobuf:"bytes,2,opt,name=init_params,json=initParams,proto3" json:"init_params,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -105,6 +106,13 @@ func (m *RegisterNodeResponse) XXX_DiscardUnknown() {
var xxx_messageInfo_RegisterNodeResponse proto.InternalMessageInfo
func (m *RegisterNodeResponse) GetStatus() *commonpb.Status {
if m != nil {
return m.Status
}
return nil
}
func (m *RegisterNodeResponse) GetInitParams() *internalpb2.InitParams {
if m != nil {
return m.InitParams
@ -113,7 +121,7 @@ func (m *RegisterNodeResponse) GetInitParams() *internalpb2.InitParams {
}
type IndexStatesRequest struct {
IndexID int64 `protobuf:"varint,1,opt,name=indexID,proto3" json:"indexID,omitempty"`
IndexID []int64 `protobuf:"varint,1,rep,packed,name=indexID,proto3" json:"indexID,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -144,27 +152,81 @@ func (m *IndexStatesRequest) XXX_DiscardUnknown() {
var xxx_messageInfo_IndexStatesRequest proto.InternalMessageInfo
func (m *IndexStatesRequest) GetIndexID() int64 {
func (m *IndexStatesRequest) GetIndexID() []int64 {
if m != nil {
return m.IndexID
}
return nil
}
type IndexInfo struct {
State commonpb.IndexState `protobuf:"varint,1,opt,name=state,proto3,enum=milvus.proto.common.IndexState" json:"state,omitempty"`
IndexID int64 `protobuf:"varint,2,opt,name=indexID,proto3" json:"indexID,omitempty"`
Reason string `protobuf:"bytes,3,opt,name=Reason,proto3" json:"Reason,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *IndexInfo) Reset() { *m = IndexInfo{} }
func (m *IndexInfo) String() string { return proto.CompactTextString(m) }
func (*IndexInfo) ProtoMessage() {}
func (*IndexInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_a5d2036b4df73e0a, []int{3}
}
func (m *IndexInfo) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_IndexInfo.Unmarshal(m, b)
}
func (m *IndexInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_IndexInfo.Marshal(b, m, deterministic)
}
func (m *IndexInfo) XXX_Merge(src proto.Message) {
xxx_messageInfo_IndexInfo.Merge(m, src)
}
func (m *IndexInfo) XXX_Size() int {
return xxx_messageInfo_IndexInfo.Size(m)
}
func (m *IndexInfo) XXX_DiscardUnknown() {
xxx_messageInfo_IndexInfo.DiscardUnknown(m)
}
var xxx_messageInfo_IndexInfo proto.InternalMessageInfo
func (m *IndexInfo) GetState() commonpb.IndexState {
if m != nil {
return m.State
}
return commonpb.IndexState_NONE
}
func (m *IndexInfo) GetIndexID() int64 {
if m != nil {
return m.IndexID
}
return 0
}
func (m *IndexInfo) GetReason() string {
if m != nil {
return m.Reason
}
return ""
}
type IndexStatesResponse struct {
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
State commonpb.IndexState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.common.IndexState" json:"state,omitempty"`
IndexID int64 `protobuf:"varint,3,opt,name=indexID,proto3" json:"indexID,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
States []*IndexInfo `protobuf:"bytes,2,rep,name=states,proto3" json:"states,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *IndexStatesResponse) Reset() { *m = IndexStatesResponse{} }
func (m *IndexStatesResponse) String() string { return proto.CompactTextString(m) }
func (*IndexStatesResponse) ProtoMessage() {}
func (*IndexStatesResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_a5d2036b4df73e0a, []int{3}
return fileDescriptor_a5d2036b4df73e0a, []int{4}
}
func (m *IndexStatesResponse) XXX_Unmarshal(b []byte) error {
@ -192,18 +254,11 @@ func (m *IndexStatesResponse) GetStatus() *commonpb.Status {
return nil
}
func (m *IndexStatesResponse) GetState() commonpb.IndexState {
func (m *IndexStatesResponse) GetStates() []*IndexInfo {
if m != nil {
return m.State
return m.States
}
return commonpb.IndexState_NONE
}
func (m *IndexStatesResponse) GetIndexID() int64 {
if m != nil {
return m.IndexID
}
return 0
return nil
}
type BuildIndexRequest struct {
@ -219,7 +274,7 @@ func (m *BuildIndexRequest) Reset() { *m = BuildIndexRequest{} }
func (m *BuildIndexRequest) String() string { return proto.CompactTextString(m) }
func (*BuildIndexRequest) ProtoMessage() {}
func (*BuildIndexRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_a5d2036b4df73e0a, []int{4}
return fileDescriptor_a5d2036b4df73e0a, []int{5}
}
func (m *BuildIndexRequest) XXX_Unmarshal(b []byte) error {
@ -273,7 +328,7 @@ func (m *BuildIndexResponse) Reset() { *m = BuildIndexResponse{} }
func (m *BuildIndexResponse) String() string { return proto.CompactTextString(m) }
func (*BuildIndexResponse) ProtoMessage() {}
func (*BuildIndexResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_a5d2036b4df73e0a, []int{5}
return fileDescriptor_a5d2036b4df73e0a, []int{6}
}
func (m *BuildIndexResponse) XXX_Unmarshal(b []byte) error {
@ -320,7 +375,7 @@ func (m *BuildIndexCmd) Reset() { *m = BuildIndexCmd{} }
func (m *BuildIndexCmd) String() string { return proto.CompactTextString(m) }
func (*BuildIndexCmd) ProtoMessage() {}
func (*BuildIndexCmd) Descriptor() ([]byte, []int) {
return fileDescriptor_a5d2036b4df73e0a, []int{6}
return fileDescriptor_a5d2036b4df73e0a, []int{7}
}
func (m *BuildIndexCmd) XXX_Unmarshal(b []byte) error {
@ -368,7 +423,7 @@ func (m *BuildIndexNotification) Reset() { *m = BuildIndexNotification{}
func (m *BuildIndexNotification) String() string { return proto.CompactTextString(m) }
func (*BuildIndexNotification) ProtoMessage() {}
func (*BuildIndexNotification) Descriptor() ([]byte, []int) {
return fileDescriptor_a5d2036b4df73e0a, []int{7}
return fileDescriptor_a5d2036b4df73e0a, []int{8}
}
func (m *BuildIndexNotification) XXX_Unmarshal(b []byte) error {
@ -421,7 +476,7 @@ func (m *IndexFilePathRequest) Reset() { *m = IndexFilePathRequest{} }
func (m *IndexFilePathRequest) String() string { return proto.CompactTextString(m) }
func (*IndexFilePathRequest) ProtoMessage() {}
func (*IndexFilePathRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_a5d2036b4df73e0a, []int{8}
return fileDescriptor_a5d2036b4df73e0a, []int{9}
}
func (m *IndexFilePathRequest) XXX_Unmarshal(b []byte) error {
@ -462,7 +517,7 @@ func (m *IndexFilePathsResponse) Reset() { *m = IndexFilePathsResponse{}
func (m *IndexFilePathsResponse) String() string { return proto.CompactTextString(m) }
func (*IndexFilePathsResponse) ProtoMessage() {}
func (*IndexFilePathsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_a5d2036b4df73e0a, []int{9}
return fileDescriptor_a5d2036b4df73e0a, []int{10}
}
func (m *IndexFilePathsResponse) XXX_Unmarshal(b []byte) error {
@ -521,7 +576,7 @@ func (m *IndexMeta) Reset() { *m = IndexMeta{} }
func (m *IndexMeta) String() string { return proto.CompactTextString(m) }
func (*IndexMeta) ProtoMessage() {}
func (*IndexMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_a5d2036b4df73e0a, []int{10}
return fileDescriptor_a5d2036b4df73e0a, []int{11}
}
func (m *IndexMeta) XXX_Unmarshal(b []byte) error {
@ -595,6 +650,7 @@ func init() {
proto.RegisterType((*RegisterNodeRequest)(nil), "milvus.proto.index.RegisterNodeRequest")
proto.RegisterType((*RegisterNodeResponse)(nil), "milvus.proto.index.RegisterNodeResponse")
proto.RegisterType((*IndexStatesRequest)(nil), "milvus.proto.index.IndexStatesRequest")
proto.RegisterType((*IndexInfo)(nil), "milvus.proto.index.IndexInfo")
proto.RegisterType((*IndexStatesResponse)(nil), "milvus.proto.index.IndexStatesResponse")
proto.RegisterType((*BuildIndexRequest)(nil), "milvus.proto.index.BuildIndexRequest")
proto.RegisterType((*BuildIndexResponse)(nil), "milvus.proto.index.BuildIndexResponse")
@ -608,52 +664,55 @@ func init() {
func init() { proto.RegisterFile("index_service.proto", fileDescriptor_a5d2036b4df73e0a) }
var fileDescriptor_a5d2036b4df73e0a = []byte{
// 714 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0xcd, 0x4e, 0xdb, 0x40,
0x10, 0x26, 0x24, 0x80, 0x32, 0x84, 0x08, 0x36, 0xa8, 0x42, 0x69, 0x51, 0xc1, 0x55, 0x4b, 0x84,
0x54, 0x07, 0x05, 0xb5, 0x3d, 0x56, 0x04, 0xd4, 0x2a, 0xaa, 0x40, 0xc8, 0x54, 0x3d, 0x50, 0x55,
0xd1, 0xda, 0x1e, 0xc8, 0xaa, 0xfe, 0xc3, 0xbb, 0x46, 0x85, 0x4b, 0x0f, 0x7d, 0x82, 0x9e, 0xfb,
0x18, 0xbd, 0xf6, 0xe1, 0x2a, 0xaf, 0xd7, 0x89, 0x4d, 0x4c, 0x02, 0xfd, 0xb9, 0x79, 0x67, 0xbf,
0x99, 0xf9, 0xe6, 0x9b, 0x9d, 0x31, 0x34, 0x98, 0x67, 0xe3, 0x97, 0x3e, 0xc7, 0xf0, 0x92, 0x59,
0xa8, 0x07, 0xa1, 0x2f, 0x7c, 0x42, 0x5c, 0xe6, 0x5c, 0x46, 0x3c, 0x39, 0xe9, 0x12, 0xd1, 0xac,
0x59, 0xbe, 0xeb, 0xfa, 0x5e, 0x62, 0x6b, 0xd6, 0x99, 0x27, 0x30, 0xf4, 0xa8, 0x93, 0x9c, 0xb5,
0xaf, 0xd0, 0x30, 0xf0, 0x9c, 0x71, 0x81, 0xe1, 0x91, 0x6f, 0xa3, 0x81, 0x17, 0x11, 0x72, 0x41,
0x76, 0xa0, 0x62, 0x52, 0x8e, 0x6b, 0xa5, 0x8d, 0x52, 0x6b, 0xb1, 0xf3, 0x48, 0xcf, 0xc5, 0x55,
0x01, 0x0f, 0xf9, 0x79, 0x97, 0x72, 0x34, 0x24, 0x92, 0xbc, 0x84, 0x05, 0x6a, 0xdb, 0x21, 0x72,
0xbe, 0x36, 0x3b, 0xc1, 0x69, 0x2f, 0xc1, 0x18, 0x29, 0x58, 0x3b, 0x85, 0xd5, 0x3c, 0x01, 0x1e,
0xf8, 0x1e, 0x47, 0xd2, 0x85, 0x45, 0xe6, 0x31, 0xd1, 0x0f, 0x68, 0x48, 0x5d, 0xae, 0x88, 0x6c,
0xea, 0x37, 0x0a, 0x54, 0xb5, 0xf4, 0x3c, 0x26, 0x8e, 0x25, 0xd0, 0x00, 0x36, 0xfc, 0xd6, 0x74,
0x20, 0xbd, 0x58, 0x83, 0x13, 0x41, 0x05, 0xf2, 0xb4, 0xb6, 0x35, 0x58, 0x90, 0xca, 0xf4, 0x0e,
0x64, 0xd4, 0xb2, 0x91, 0x1e, 0xb5, 0x1f, 0x25, 0x68, 0xe4, 0x1c, 0x14, 0x97, 0x5d, 0x98, 0xe7,
0x82, 0x8a, 0x28, 0xa5, 0xf1, 0xb0, 0xb0, 0xb4, 0x13, 0x09, 0x31, 0x14, 0x94, 0xbc, 0x80, 0xb9,
0xf8, 0x0b, 0xa5, 0x1c, 0xf5, 0xce, 0xe3, 0x42, 0x9f, 0x51, 0x36, 0x23, 0x41, 0x67, 0xd9, 0x95,
0xf3, 0xec, 0x7e, 0x95, 0x60, 0xa5, 0x1b, 0x31, 0xc7, 0x96, 0x4e, 0x69, 0x35, 0xeb, 0x00, 0x36,
0x15, 0xb4, 0x1f, 0x50, 0x31, 0x88, 0xa5, 0x2f, 0xb7, 0xaa, 0x46, 0x35, 0xb6, 0x1c, 0xc7, 0x86,
0x58, 0x46, 0x71, 0x15, 0x60, 0x2a, 0x63, 0x79, 0xa3, 0x3c, 0x2e, 0xa3, 0xe2, 0xf2, 0x0e, 0xaf,
0x3e, 0x50, 0x27, 0xc2, 0x63, 0xca, 0x42, 0x03, 0x62, 0xaf, 0x44, 0x46, 0x72, 0x00, 0xb5, 0xe4,
0xb1, 0xa9, 0x20, 0x95, 0xbb, 0x06, 0x59, 0x94, 0x6e, 0xaa, 0x19, 0x16, 0x90, 0x2c, 0xfb, 0xbf,
0x91, 0x36, 0xa3, 0xd1, 0x6c, 0x5e, 0x23, 0x13, 0x96, 0x46, 0x49, 0xf6, 0x5d, 0xfb, 0xf6, 0x66,
0x93, 0x57, 0x50, 0x0e, 0xf1, 0x42, 0x3d, 0xd6, 0xa7, 0xfa, 0xf8, 0xe4, 0xe8, 0x63, 0x62, 0x1b,
0xb1, 0x87, 0xf6, 0xbd, 0x04, 0x0f, 0x46, 0x57, 0x47, 0xbe, 0x60, 0x67, 0xcc, 0xa2, 0x82, 0xf9,
0xde, 0x3f, 0xae, 0x86, 0xb4, 0x60, 0x39, 0x11, 0xfe, 0x8c, 0x39, 0xa8, 0x3a, 0x5c, 0x96, 0x1d,
0xae, 0x4b, 0xfb, 0x1b, 0xe6, 0xa0, 0x6c, 0xb3, 0xb6, 0x03, 0xab, 0xbd, 0xac, 0x65, 0xfa, 0x5b,
0x8f, 0xab, 0xc8, 0xb9, 0xf0, 0xff, 0xd4, 0x93, 0x7b, 0x54, 0xf1, 0x73, 0x16, 0xaa, 0x92, 0xd3,
0x21, 0x0a, 0x3a, 0x1a, 0xa0, 0xd2, 0x9f, 0x0e, 0xd0, 0x0d, 0x22, 0xeb, 0x00, 0xe8, 0x5d, 0x44,
0xd8, 0x17, 0xcc, 0x45, 0x35, 0x5d, 0x55, 0x69, 0x79, 0xcf, 0x5c, 0x24, 0x4f, 0x60, 0x89, 0x5b,
0x03, 0xb4, 0x23, 0x47, 0x21, 0x2a, 0x12, 0x51, 0x4b, 0x8d, 0x12, 0xa4, 0x43, 0xc3, 0x8c, 0x7b,
0xdf, 0xb7, 0x7c, 0x37, 0x70, 0x50, 0x28, 0xe8, 0x9c, 0x84, 0xae, 0xc8, 0xab, 0x7d, 0x75, 0x23,
0xf1, 0xea, 0x95, 0xcd, 0xdf, 0xf7, 0x95, 0x15, 0xaa, 0xb6, 0x50, 0xa4, 0x5a, 0xe7, 0x5b, 0x05,
0x6a, 0x89, 0x0c, 0xc9, 0xbf, 0x80, 0x58, 0x50, 0xcb, 0xae, 0x54, 0xb2, 0x55, 0x94, 0xb6, 0x60,
0xeb, 0x37, 0x5b, 0xd3, 0x81, 0xc9, 0x13, 0xd1, 0x66, 0xc8, 0x27, 0x80, 0x11, 0x73, 0x72, 0xb7,
0xca, 0x9a, 0xcf, 0xa6, 0xc1, 0x86, 0xe1, 0x2d, 0xa8, 0xbf, 0x45, 0x91, 0x59, 0xc6, 0xa4, 0xd0,
0x77, 0x7c, 0xbd, 0x37, 0xb7, 0xa6, 0xe2, 0x86, 0x49, 0x3e, 0xc3, 0x4a, 0x9a, 0x64, 0x28, 0x27,
0x69, 0xdd, 0xea, 0x7f, 0x63, 0xb8, 0x9a, 0xdb, 0x53, 0x91, 0x3c, 0x27, 0xd8, 0xb2, 0xdc, 0x15,
0x57, 0x19, 0xd9, 0xb6, 0x27, 0xeb, 0x91, 0xdd, 0x2d, 0xcd, 0x49, 0x53, 0xa8, 0xcd, 0x74, 0x3e,
0xaa, 0xd1, 0x91, 0x1d, 0x3f, 0xca, 0x35, 0x67, 0x73, 0x72, 0x96, 0x7d, 0xd7, 0x9e, 0x12, 0xbc,
0xbb, 0x77, 0xfa, 0xfa, 0x9c, 0x89, 0x41, 0x64, 0xc6, 0x37, 0xed, 0x6b, 0xe6, 0x38, 0xec, 0x5a,
0xa0, 0x35, 0x68, 0x27, 0x5e, 0xcf, 0x6d, 0xc6, 0x45, 0xc8, 0xcc, 0x48, 0xa0, 0xdd, 0x4e, 0x7f,
0xca, 0x6d, 0x19, 0xaa, 0x2d, 0xb3, 0x05, 0xa6, 0x39, 0x2f, 0x8f, 0xbb, 0xbf, 0x03, 0x00, 0x00,
0xff, 0xff, 0x89, 0xd5, 0x9c, 0x9e, 0xb8, 0x08, 0x00, 0x00,
// 757 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0x5d, 0x4f, 0xdb, 0x4a,
0x10, 0xc5, 0x38, 0x04, 0x65, 0x12, 0x22, 0xd8, 0x20, 0x14, 0xe5, 0x5e, 0x74, 0xc1, 0x57, 0x17,
0x22, 0xa4, 0xeb, 0xa0, 0x20, 0xda, 0xc7, 0x8a, 0x80, 0x5a, 0x45, 0x15, 0x08, 0xb9, 0x55, 0x1f,
0x5a, 0x55, 0x91, 0x63, 0x0f, 0x64, 0x55, 0x7f, 0x04, 0xef, 0x1a, 0x15, 0x5e, 0xaa, 0xaa, 0x3f,
0xa0, 0xea, 0x6f, 0xe9, 0x6b, 0x7f, 0x5c, 0xe5, 0xdd, 0x75, 0x12, 0x83, 0x49, 0x40, 0xd0, 0x37,
0xef, 0xee, 0x99, 0x33, 0xb3, 0xe7, 0xcc, 0xac, 0xa1, 0x46, 0x03, 0x17, 0x3f, 0xf7, 0x18, 0x46,
0x97, 0xd4, 0x41, 0x73, 0x18, 0x85, 0x3c, 0x24, 0xc4, 0xa7, 0xde, 0x65, 0xcc, 0xe4, 0xca, 0x14,
0x88, 0x46, 0xc5, 0x09, 0x7d, 0x3f, 0x0c, 0xe4, 0x5e, 0xa3, 0x4a, 0x03, 0x8e, 0x51, 0x60, 0x7b,
0x72, 0x6d, 0x7c, 0x81, 0x9a, 0x85, 0xe7, 0x94, 0x71, 0x8c, 0x4e, 0x42, 0x17, 0x2d, 0xbc, 0x88,
0x91, 0x71, 0xb2, 0x0b, 0x85, 0xbe, 0xcd, 0xb0, 0xae, 0x6d, 0x68, 0xcd, 0x72, 0xfb, 0x6f, 0x33,
0xc3, 0xab, 0x08, 0x8f, 0xd9, 0x79, 0xc7, 0x66, 0x68, 0x09, 0x24, 0x79, 0x06, 0x8b, 0xb6, 0xeb,
0x46, 0xc8, 0x58, 0x7d, 0x7e, 0x4a, 0xd0, 0x81, 0xc4, 0x58, 0x29, 0xd8, 0xf8, 0xae, 0xc1, 0x6a,
0xb6, 0x02, 0x36, 0x0c, 0x03, 0x86, 0x64, 0x0f, 0x8a, 0x8c, 0xdb, 0x3c, 0x66, 0xaa, 0x88, 0xbf,
0x72, 0xf9, 0xde, 0x08, 0x88, 0xa5, 0xa0, 0xa4, 0x03, 0x65, 0x1a, 0x50, 0xde, 0x1b, 0xda, 0x91,
0xed, 0xa7, 0x95, 0x6c, 0x9a, 0x37, 0x64, 0x51, 0x0a, 0x74, 0x03, 0xca, 0x4f, 0x05, 0xd0, 0x02,
0x3a, 0xfa, 0x36, 0x4c, 0x20, 0xdd, 0x44, 0xb9, 0x84, 0x1a, 0x59, 0xaa, 0x48, 0x1d, 0x16, 0x85,
0x9e, 0xdd, 0xa3, 0xba, 0xb6, 0xa1, 0x37, 0x75, 0x2b, 0x5d, 0x1a, 0x1c, 0x4a, 0x02, 0xdf, 0x0d,
0xce, 0x42, 0xb2, 0x0f, 0x0b, 0x49, 0x29, 0x52, 0xb9, 0x6a, 0xfb, 0x9f, 0xdc, 0xa2, 0xc7, 0xf4,
0x96, 0x44, 0x4f, 0xb2, 0x27, 0x35, 0x8f, 0xd9, 0xc9, 0x1a, 0x14, 0x2d, 0xb4, 0x59, 0x18, 0xd4,
0xf5, 0x0d, 0xad, 0x59, 0xb2, 0xd4, 0xca, 0xf8, 0xaa, 0x41, 0x2d, 0x53, 0xe6, 0x63, 0x64, 0xdb,
0x97, 0x41, 0x98, 0x28, 0xa6, 0x37, 0xcb, 0xed, 0x75, 0xf3, 0x76, 0x23, 0x99, 0xa3, 0x4b, 0x5a,
0x0a, 0x6c, 0xfc, 0xd2, 0x60, 0xa5, 0x13, 0x53, 0xcf, 0x15, 0x47, 0xa9, 0x52, 0xeb, 0x00, 0xae,
0xcd, 0xed, 0xde, 0xd0, 0xe6, 0x03, 0x49, 0x58, 0xb2, 0x4a, 0xc9, 0xce, 0x69, 0xb2, 0x91, 0x58,
0xc4, 0xaf, 0x86, 0x98, 0x5a, 0xa4, 0x8b, 0x84, 0x9b, 0xb9, 0x55, 0xbe, 0xc6, 0xab, 0x77, 0xb6,
0x17, 0xe3, 0xa9, 0x4d, 0x23, 0x0b, 0x92, 0x28, 0x69, 0x11, 0x39, 0x82, 0x8a, 0x6c, 0x7f, 0x45,
0x52, 0xb8, 0x2f, 0x49, 0x59, 0x84, 0x29, 0xa3, 0x1d, 0x20, 0x93, 0xd5, 0x3f, 0x46, 0xc0, 0x3b,
0xfd, 0x33, 0xfa, 0xb0, 0x34, 0x4e, 0x72, 0xe8, 0xbb, 0xd9, 0x46, 0xca, 0x58, 0xfd, 0x1c, 0xf4,
0x08, 0x2f, 0x54, 0xd3, 0xfe, 0x97, 0x67, 0xc1, 0x2d, 0xb1, 0xad, 0x24, 0xc2, 0xf8, 0xa1, 0xc1,
0xda, 0xf8, 0xe8, 0x24, 0xe4, 0xf4, 0x8c, 0x3a, 0x36, 0xa7, 0x61, 0xf0, 0xc4, 0xb7, 0x21, 0x4d,
0x58, 0x96, 0xc2, 0x9f, 0x51, 0x0f, 0x95, 0xc3, 0xba, 0x70, 0xb8, 0x2a, 0xf6, 0x5f, 0x52, 0x0f,
0x85, 0xcd, 0xc6, 0x2e, 0xac, 0x76, 0x27, 0x77, 0x72, 0xe7, 0x28, 0xa3, 0x54, 0x72, 0x8b, 0x4c,
0x08, 0xfb, 0x43, 0x9e, 0x3c, 0xe0, 0x16, 0x3f, 0xe7, 0xd5, 0x70, 0x1f, 0x23, 0xb7, 0x9f, 0x7e,
0xb8, 0xd7, 0x01, 0x30, 0xb8, 0x88, 0xb1, 0xc7, 0xa9, 0x8f, 0x62, 0xc0, 0x75, 0xab, 0x24, 0x76,
0xde, 0x52, 0x1f, 0xc9, 0xbf, 0xb0, 0xc4, 0x9c, 0x01, 0xba, 0xb1, 0xa7, 0x10, 0x05, 0x81, 0xa8,
0xa4, 0x9b, 0x02, 0x64, 0x42, 0xad, 0x9f, 0x78, 0xdf, 0x73, 0x42, 0x7f, 0xe8, 0x21, 0x57, 0xd0,
0x05, 0x01, 0x5d, 0x11, 0x47, 0x87, 0xea, 0x44, 0xe0, 0x55, 0x97, 0x15, 0x1f, 0xda, 0x65, 0xb9,
0xaa, 0x2d, 0xe6, 0xa9, 0xd6, 0xfe, 0x56, 0x80, 0x8a, 0x94, 0x41, 0xfe, 0x9d, 0x88, 0x03, 0x95,
0xc9, 0x37, 0x9e, 0x6c, 0xe7, 0xa5, 0xcd, 0xf9, 0x0f, 0x35, 0x9a, 0xb3, 0x81, 0xb2, 0x45, 0x8c,
0x39, 0xf2, 0x11, 0x60, 0x5c, 0x39, 0xb9, 0xdf, 0xcd, 0x1a, 0x5b, 0xb3, 0x60, 0x23, 0x7a, 0x07,
0xaa, 0xaf, 0x90, 0x4f, 0x3c, 0xb9, 0x64, 0xeb, 0xce, 0x57, 0x32, 0xf3, 0xeb, 0x68, 0x6c, 0xcf,
0xc4, 0x8d, 0x92, 0x7c, 0x82, 0x95, 0x34, 0xc9, 0x48, 0x4e, 0xd2, 0xbc, 0x33, 0xfe, 0xc6, 0x70,
0x35, 0x76, 0x66, 0x22, 0x59, 0x46, 0xb0, 0x65, 0xf1, 0x56, 0x5c, 0x4d, 0xc8, 0xb6, 0x33, 0x5d,
0x8f, 0xc9, 0xb7, 0xa5, 0x31, 0x6d, 0x0a, 0x8d, 0xb9, 0xf6, 0x07, 0x35, 0x3a, 0xc2, 0xf1, 0x93,
0x8c, 0x39, 0x9b, 0xd3, 0xb3, 0x1c, 0xfa, 0xee, 0x0c, 0xf2, 0xce, 0xc1, 0xfb, 0x17, 0xe7, 0x94,
0x0f, 0xe2, 0x7e, 0x72, 0xd2, 0xba, 0xa6, 0x9e, 0x47, 0xaf, 0x39, 0x3a, 0x83, 0x96, 0x8c, 0xfa,
0xdf, 0xa5, 0x8c, 0x47, 0xb4, 0x1f, 0x73, 0x74, 0x5b, 0xe9, 0x0f, 0xbf, 0x25, 0xa8, 0x5a, 0x22,
0xdb, 0xb0, 0xdf, 0x2f, 0x8a, 0xe5, 0xde, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0x4d, 0x32, 0xc8,
0x07, 0x4a, 0x09, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.

View File

@ -67,6 +67,8 @@ service MasterService {
*/
rpc DescribeCollection(milvus.DescribeCollectionRequest) returns (milvus.DescribeCollectionResponse) {}
rpc GetCollectionStatistics(milvus.CollectionStatsRequest) returns (milvus.CollectionStatsResponse) {}
/**
* @brief This method is used to list all collections.
*
@ -95,6 +97,8 @@ service MasterService {
*/
rpc HasPartition(milvus.HasPartitionRequest) returns (milvus.BoolResponse) {}
rpc GetPartitionStatistics(milvus.PartitionStatsRequest) returns (milvus.PartitionStatsResponse) {}
/**
* @brief This method is used to show partition information
*

View File

@ -242,52 +242,55 @@ func init() {
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
var fileDescriptor_f9c348dec43a6705 = []byte{
// 714 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x96, 0x5b, 0x4f, 0xdb, 0x3c,
0x18, 0xc7, 0x39, 0xbd, 0xbc, 0xe2, 0xa1, 0x07, 0xe4, 0x71, 0x81, 0x32, 0x34, 0x58, 0xb7, 0x41,
0x39, 0x2c, 0x9d, 0xe0, 0x0b, 0x8c, 0x92, 0xa9, 0x54, 0x1a, 0xd2, 0xd6, 0x76, 0x9b, 0xd8, 0x84,
0x90, 0x93, 0x5a, 0xad, 0xb5, 0x24, 0xee, 0xe2, 0xa7, 0xb0, 0xf1, 0xd5, 0x77, 0x33, 0xe5, 0xe4,
0x24, 0x90, 0x94, 0x4c, 0xe3, 0xd2, 0xf6, 0xdf, 0xbf, 0xff, 0x73, 0x52, 0x1c, 0xa8, 0x38, 0x54,
0x22, 0xf3, 0xf4, 0x89, 0x27, 0x50, 0x90, 0x27, 0x0e, 0xb7, 0xaf, 0xa7, 0x32, 0x5c, 0xe9, 0xe1,
0x91, 0x56, 0xb1, 0x84, 0xe3, 0x08, 0x37, 0xdc, 0xd4, 0x2a, 0x69, 0x89, 0x56, 0xe3, 0x2e, 0x32,
0xcf, 0xa5, 0x76, 0xb4, 0x26, 0x43, 0x8a, 0xf4, 0x4a, 0x32, 0xef, 0x9a, 0x5b, 0x2c, 0xdc, 0x6b,
0xf4, 0x61, 0xa5, 0x6b, 0xf4, 0xd8, 0x8f, 0x29, 0x93, 0x48, 0xde, 0xc0, 0x92, 0x49, 0x25, 0xdb,
0x98, 0xdf, 0x9e, 0x6f, 0xae, 0x1e, 0x6d, 0xea, 0x19, 0xc3, 0xc8, 0xe8, 0x5c, 0x8e, 0xda, 0x54,
0xb2, 0x5e, 0xa0, 0x24, 0xeb, 0xf0, 0x9f, 0x25, 0xa6, 0x2e, 0x6e, 0x2c, 0x6c, 0xcf, 0x37, 0xab,
0xbd, 0x70, 0xd1, 0x18, 0x01, 0xf8, 0x50, 0x39, 0x11, 0xae, 0x64, 0xe4, 0x18, 0x96, 0x25, 0x52,
0x9c, 0xca, 0x88, 0xfb, 0x34, 0x97, 0xdb, 0x0f, 0x24, 0xbd, 0x48, 0x4a, 0x6a, 0xb0, 0xd0, 0x35,
0x02, 0xea, 0x62, 0x6f, 0xa1, 0x6b, 0x24, 0x46, 0x8b, 0x69, 0xa3, 0x01, 0xc0, 0x40, 0x8a, 0x47,
0x08, 0x3f, 0x43, 0xbd, 0x86, 0xd5, 0x80, 0xfa, 0x2f, 0xf1, 0x6f, 0xc2, 0x0a, 0x72, 0x87, 0x49,
0xa4, 0xce, 0x24, 0x48, 0x63, 0xa9, 0x97, 0x6c, 0xe4, 0xfb, 0x1e, 0xfd, 0xae, 0x43, 0xf5, 0x3c,
0x68, 0x6b, 0x3f, 0xec, 0x11, 0xb9, 0x82, 0xb5, 0x53, 0x8f, 0x51, 0x64, 0xa7, 0xc2, 0xb6, 0x99,
0x85, 0x5c, 0xb8, 0xe4, 0x30, 0x6b, 0x1f, 0x2d, 0xee, 0xca, 0xa2, 0x9a, 0x68, 0xb3, 0x82, 0x6d,
0xcc, 0x91, 0x6f, 0x50, 0x33, 0x3c, 0x31, 0x49, 0xe1, 0xf7, 0x73, 0xf1, 0x59, 0x51, 0x49, 0xf8,
0x15, 0x54, 0xcf, 0xa8, 0x4c, 0xb1, 0xf7, 0x72, 0xd9, 0x19, 0x4d, 0x8c, 0x7e, 0x9e, 0x2b, 0x6d,
0x0b, 0x61, 0xc7, 0x7d, 0x69, 0xcc, 0x91, 0x1b, 0x20, 0x06, 0x93, 0x96, 0xc7, 0xcd, 0x74, 0x81,
0xf4, 0xfc, 0x0c, 0xee, 0x09, 0x63, 0xab, 0x56, 0x69, 0xbd, 0x32, 0xb6, 0xa1, 0xde, 0x1f, 0x8b,
0x9b, 0xe4, 0x4c, 0x16, 0xd4, 0x2d, 0xab, 0x8a, 0x1d, 0x0f, 0x4a, 0x69, 0x95, 0xdb, 0x25, 0xd4,
0xc3, 0xf6, 0x7e, 0xa0, 0x1e, 0xf2, 0x20, 0xc7, 0x83, 0x19, 0x43, 0xa0, 0x54, 0x25, 0xdb, 0x74,
0x01, 0x55, 0xbf, 0xbd, 0x09, 0x7c, 0xaf, 0x70, 0x04, 0xfe, 0x16, 0x7d, 0x09, 0x95, 0x33, 0x2a,
0x13, 0x72, 0xb3, 0x68, 0x00, 0xee, 0x81, 0x4b, 0xf5, 0x9f, 0x43, 0xcd, 0x2f, 0x9a, 0xba, 0x2c,
0x0b, 0x42, 0xcf, 0x88, 0x62, 0x87, 0xfd, 0x32, 0x52, 0x65, 0xe5, 0x42, 0x3d, 0x9e, 0x88, 0x3e,
0x1b, 0x39, 0xcc, 0xc5, 0x82, 0x1e, 0xdc, 0x51, 0xc5, 0x6e, 0x87, 0xe5, 0xc4, 0xca, 0xcf, 0x82,
0x8a, 0x1f, 0x4a, 0x74, 0x20, 0xc9, 0x6e, 0x61, 0xb4, 0x77, 0x8c, 0x9a, 0x0f, 0x0b, 0x95, 0xc9,
0x27, 0x58, 0x0d, 0x47, 0xa6, 0xeb, 0x0e, 0xd9, 0xcf, 0x02, 0x8f, 0x94, 0xa2, 0x64, 0xd7, 0xc7,
0x50, 0x8d, 0x13, 0x0b, 0xc1, 0x7b, 0x33, 0x93, 0xcf, 0xa0, 0xf7, 0xcb, 0x48, 0x55, 0x02, 0x26,
0x54, 0x3b, 0x0c, 0x83, 0x5d, 0xdf, 0x9d, 0x91, 0x9d, 0xdc, 0xeb, 0x89, 0x20, 0xb6, 0xd9, 0x7d,
0x50, 0x97, 0x2a, 0x52, 0xed, 0xc4, 0xb6, 0x85, 0x35, 0x50, 0x5f, 0xef, 0x2d, 0x3d, 0xe7, 0x25,
0xd6, 0x93, 0x87, 0x48, 0xdb, 0x2e, 0x16, 0x28, 0xec, 0x7b, 0xf8, 0x3f, 0xc0, 0x76, 0x0d, 0xf2,
0x2c, 0x57, 0xae, 0x9e, 0x65, 0x6d, 0xab, 0xf0, 0x3c, 0x55, 0x88, 0xfa, 0x89, 0x94, 0x7c, 0xe4,
0x46, 0x4d, 0xee, 0x1a, 0xe4, 0x55, 0xf6, 0x96, 0xff, 0xf6, 0xeb, 0x4a, 0x93, 0xc0, 0x77, 0x1e,
0x92, 0x29, 0x8f, 0xcf, 0xb0, 0xd6, 0x61, 0x18, 0x3d, 0x4d, 0x41, 0x95, 0x24, 0xd1, 0x72, 0x27,
0xe1, 0x9d, 0x33, 0xc1, 0x5f, 0xda, 0xcb, 0xec, 0x99, 0xfa, 0x19, 0xc9, 0x10, 0x1a, 0x73, 0xe4,
0x0b, 0x90, 0x0e, 0x43, 0xbf, 0xbc, 0x03, 0x6e, 0x7d, 0x3f, 0x1d, 0x53, 0xd7, 0x65, 0xf6, 0x4c,
0xf2, 0x8b, 0xfc, 0x19, 0x47, 0x8f, 0xbb, 0xa3, 0x54, 0xc0, 0x1f, 0xa1, 0xd2, 0x61, 0x68, 0x0c,
0x1f, 0x11, 0x79, 0x01, 0xeb, 0x7e, 0x0d, 0x90, 0x22, 0x97, 0xc8, 0x2d, 0xf9, 0x78, 0xe8, 0x76,
0xfb, 0xeb, 0xdb, 0x11, 0xc7, 0xf1, 0xd4, 0xf4, 0x6f, 0xb7, 0x6e, 0xb9, 0x6d, 0xf3, 0x5b, 0x64,
0xd6, 0xb8, 0x15, 0x5e, 0x78, 0x3d, 0xe4, 0x12, 0x3d, 0x6e, 0x4e, 0x91, 0x0d, 0x5b, 0x71, 0x2d,
0x5b, 0x01, 0xb2, 0x15, 0x4e, 0xc4, 0xc4, 0x34, 0x97, 0x83, 0xf5, 0xf1, 0x9f, 0x00, 0x00, 0x00,
0xff, 0xff, 0xcc, 0x29, 0xfa, 0xd0, 0x39, 0x0a, 0x00, 0x00,
// 756 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x96, 0xfb, 0x6e, 0xd3, 0x30,
0x14, 0xc6, 0x77, 0x63, 0x68, 0x67, 0xbd, 0x0c, 0x33, 0xc1, 0x14, 0x26, 0x36, 0x0a, 0x6c, 0xdd,
0x85, 0x14, 0x6d, 0x2f, 0xc0, 0xba, 0xa0, 0xae, 0x12, 0x93, 0xa0, 0x2d, 0xa0, 0x81, 0xa6, 0x29,
0x49, 0xad, 0xd6, 0x22, 0x89, 0x4b, 0x8e, 0xdb, 0xc1, 0x9e, 0x98, 0xc7, 0x40, 0xb9, 0x39, 0x49,
0x97, 0x74, 0x41, 0xec, 0x4f, 0xdb, 0x3f, 0x7f, 0x9f, 0x7d, 0xce, 0xa7, 0xc4, 0x50, 0xb2, 0x75,
0x14, 0xd4, 0x55, 0x47, 0x2e, 0x17, 0x9c, 0x3c, 0xb6, 0x99, 0x35, 0x19, 0x63, 0x30, 0x52, 0x83,
0x25, 0xa5, 0x64, 0x72, 0xdb, 0xe6, 0x4e, 0x30, 0xa9, 0x94, 0x92, 0x88, 0x52, 0x61, 0x8e, 0xa0,
0xae, 0xa3, 0x5b, 0xe1, 0x98, 0xf4, 0x75, 0xa1, 0x5f, 0x21, 0x75, 0x27, 0xcc, 0xa4, 0xc1, 0x5c,
0xad, 0x0b, 0x2b, 0x6d, 0xad, 0x43, 0x7f, 0x8e, 0x29, 0x0a, 0xf2, 0x16, 0x96, 0x0c, 0x1d, 0xe9,
0xc6, 0xfc, 0xf6, 0x7c, 0x7d, 0xf5, 0x68, 0x53, 0x4d, 0x19, 0x86, 0x46, 0xe7, 0x38, 0x68, 0xea,
0x48, 0x3b, 0x3e, 0x49, 0xd6, 0xe1, 0x81, 0xc9, 0xc7, 0x8e, 0xd8, 0x58, 0xd8, 0x9e, 0xaf, 0x97,
0x3b, 0xc1, 0xa0, 0x36, 0x00, 0xf0, 0x44, 0x71, 0xc4, 0x1d, 0xa4, 0xe4, 0x18, 0x96, 0x51, 0xe8,
0x62, 0x8c, 0xa1, 0xee, 0xb3, 0x4c, 0xdd, 0xae, 0x8f, 0x74, 0x42, 0x94, 0x54, 0x60, 0xa1, 0xad,
0xf9, 0xaa, 0x8b, 0x9d, 0x85, 0xb6, 0x16, 0x1b, 0x2d, 0x26, 0x8d, 0x7a, 0x00, 0x3d, 0xe4, 0xf7,
0x70, 0xfc, 0x94, 0xea, 0x04, 0x56, 0x7d, 0xd5, 0xff, 0x39, 0xff, 0x26, 0xac, 0x08, 0x66, 0x53,
0x14, 0xba, 0x3d, 0xf2, 0xaf, 0xb1, 0xd4, 0x89, 0x27, 0xb2, 0x7d, 0x8f, 0xfe, 0x3c, 0x82, 0xf2,
0xb9, 0xdf, 0xd6, 0x6e, 0xd0, 0x23, 0x72, 0x05, 0x6b, 0xa7, 0x2e, 0xd5, 0x05, 0x3d, 0xe5, 0x96,
0x45, 0x4d, 0xc1, 0xb8, 0x43, 0x0e, 0xd3, 0xf6, 0xe1, 0x60, 0x1a, 0x0b, 0x6b, 0xa2, 0xcc, 0x3a,
0x6c, 0x6d, 0x8e, 0x7c, 0x87, 0x8a, 0xe6, 0xf2, 0x51, 0x42, 0x7e, 0x3f, 0x53, 0x3e, 0x0d, 0x15,
0x14, 0xbf, 0x82, 0xf2, 0x99, 0x8e, 0x09, 0xed, 0xbd, 0x4c, 0xed, 0x14, 0x13, 0x49, 0xbf, 0xc8,
0x44, 0x9b, 0x9c, 0x5b, 0x51, 0x5f, 0x6a, 0x73, 0xe4, 0x1a, 0x88, 0x46, 0xd1, 0x74, 0x99, 0x91,
0x2c, 0x90, 0x9a, 0x7d, 0x83, 0x5b, 0x60, 0x64, 0xd5, 0x28, 0xcc, 0x4b, 0xe3, 0x09, 0x3c, 0x6d,
0x51, 0x11, 0x2f, 0x79, 0x57, 0x66, 0x28, 0x98, 0x89, 0xe4, 0x20, 0xbb, 0x3d, 0x29, 0x14, 0x23,
0xeb, 0xc3, 0x62, 0xb0, 0xf4, 0xb5, 0xa0, 0xda, 0x1d, 0xf2, 0xeb, 0x18, 0xc0, 0x9c, 0x7e, 0xa5,
0xa9, 0xc8, 0xee, 0xa0, 0x10, 0x2b, 0xdd, 0x2e, 0xa1, 0x1a, 0xc4, 0xea, 0xa3, 0xee, 0x0a, 0xe6,
0xd7, 0xf6, 0x60, 0x46, 0xf8, 0x24, 0x55, 0x30, 0x1e, 0x17, 0x50, 0xf6, 0x62, 0x15, 0x8b, 0xef,
0xe5, 0x46, 0xef, 0x5f, 0xa5, 0x2f, 0xa1, 0x74, 0xa6, 0x63, 0xac, 0x5c, 0xcf, 0x0b, 0xde, 0x2d,
0xe1, 0x42, 0xb9, 0x43, 0x78, 0xd2, 0xa2, 0x42, 0xee, 0x4d, 0x74, 0x3f, 0xbb, 0x1b, 0x29, 0x12,
0x67, 0x77, 0x63, 0x9a, 0x95, 0xa6, 0x0c, 0x2a, 0x5e, 0xa7, 0xe4, 0x3a, 0xe6, 0xd4, 0x2b, 0x05,
0x45, 0x5e, 0xfb, 0x45, 0x50, 0x69, 0xe5, 0x40, 0x35, 0x8a, 0x7f, 0x97, 0x0e, 0x6c, 0xea, 0x88,
0x9c, 0xc6, 0x4f, 0x51, 0xb3, 0x63, 0x7d, 0x0b, 0x96, 0x7e, 0x26, 0x94, 0xbc, 0xa3, 0x84, 0x0b,
0x48, 0x76, 0x73, 0x4f, 0x3b, 0x65, 0x54, 0xbf, 0x1b, 0x94, 0x26, 0x9f, 0x61, 0x35, 0xc8, 0x69,
0xdb, 0xe9, 0xd3, 0x5f, 0x39, 0x1e, 0x09, 0xa2, 0x60, 0xd4, 0x86, 0x50, 0x8e, 0x2e, 0x16, 0x08,
0xef, 0xcd, 0xbc, 0x7c, 0x4a, 0x7a, 0xbf, 0x08, 0x2a, 0x2f, 0x60, 0x40, 0xb9, 0x45, 0x85, 0x3f,
0xeb, 0xb9, 0x53, 0xb2, 0x93, 0xb9, 0x3d, 0x06, 0x22, 0x9b, 0xdd, 0x3b, 0xb9, 0x44, 0x91, 0x2a,
0x27, 0x96, 0xc5, 0xcd, 0x9e, 0xfc, 0x55, 0x6d, 0xa9, 0x19, 0xcf, 0x0e, 0x35, 0xfe, 0xeb, 0x2a,
0xdb, 0xf9, 0x80, 0x94, 0xfd, 0x00, 0x0f, 0x7d, 0xd9, 0xb6, 0x46, 0x9e, 0x67, 0xe2, 0xf2, 0x0d,
0xa2, 0x6c, 0xe5, 0xae, 0x27, 0x0a, 0x51, 0x3d, 0x41, 0x64, 0x03, 0x27, 0x6c, 0x72, 0x5b, 0x23,
0xaf, 0xd3, 0xbb, 0xbc, 0x87, 0x8e, 0x2a, 0x99, 0x58, 0x7c, 0xe7, 0x2e, 0x4c, 0x7a, 0x7c, 0x81,
0xb5, 0x16, 0x15, 0xe1, 0x7f, 0xd8, 0xaf, 0x12, 0x12, 0x25, 0x33, 0x09, 0xef, 0xed, 0x91, 0xf8,
0xad, 0xbc, 0x4a, 0xaf, 0xc9, 0x97, 0x57, 0x4a, 0xa1, 0x36, 0x47, 0xbe, 0x02, 0x69, 0x51, 0xe1,
0x95, 0xb7, 0xc7, 0xcc, 0x1f, 0xa7, 0x43, 0xdd, 0x71, 0xa8, 0x35, 0x53, 0xf9, 0x65, 0x76, 0xc6,
0x85, 0xcb, 0x9c, 0x41, 0xe2, 0xc0, 0x9f, 0xa0, 0xd4, 0xa2, 0x42, 0xeb, 0xdf, 0xa3, 0xe4, 0x05,
0xac, 0x7b, 0x35, 0x90, 0x5f, 0xb7, 0xfb, 0x93, 0x6e, 0x36, 0xbf, 0xbd, 0x1b, 0x30, 0x31, 0x1c,
0x1b, 0xde, 0xee, 0xc6, 0x0d, 0xb3, 0x2c, 0x76, 0x23, 0xa8, 0x39, 0x6c, 0x04, 0x1b, 0xde, 0xf4,
0x19, 0x0a, 0x97, 0x19, 0x63, 0x41, 0xfb, 0x8d, 0xa8, 0x96, 0x0d, 0x5f, 0xb2, 0x11, 0x24, 0x62,
0x64, 0x18, 0xcb, 0xfe, 0xf8, 0xf8, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xb7, 0xb2, 0x22, 0xf7,
0x26, 0x0b, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
@ -330,6 +333,7 @@ type MasterServiceClient interface {
//
// @return CollectionSchema
DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error)
GetCollectionStatistics(ctx context.Context, in *milvuspb.CollectionStatsRequest, opts ...grpc.CallOption) (*milvuspb.CollectionStatsResponse, error)
//*
// @brief This method is used to list all collections.
//
@ -350,6 +354,7 @@ type MasterServiceClient interface {
//
// @return BoolResponse
HasPartition(ctx context.Context, in *milvuspb.HasPartitionRequest, opts ...grpc.CallOption) (*milvuspb.BoolResponse, error)
GetPartitionStatistics(ctx context.Context, in *milvuspb.PartitionStatsRequest, opts ...grpc.CallOption) (*milvuspb.PartitionStatsResponse, error)
//*
// @brief This method is used to show partition information
//
@ -416,6 +421,15 @@ func (c *masterServiceClient) DescribeCollection(ctx context.Context, in *milvus
return out, nil
}
func (c *masterServiceClient) GetCollectionStatistics(ctx context.Context, in *milvuspb.CollectionStatsRequest, opts ...grpc.CallOption) (*milvuspb.CollectionStatsResponse, error) {
out := new(milvuspb.CollectionStatsResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/GetCollectionStatistics", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *masterServiceClient) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionRequest, opts ...grpc.CallOption) (*milvuspb.ShowCollectionResponse, error) {
out := new(milvuspb.ShowCollectionResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/ShowCollections", in, out, opts...)
@ -452,6 +466,15 @@ func (c *masterServiceClient) HasPartition(ctx context.Context, in *milvuspb.Has
return out, nil
}
func (c *masterServiceClient) GetPartitionStatistics(ctx context.Context, in *milvuspb.PartitionStatsRequest, opts ...grpc.CallOption) (*milvuspb.PartitionStatsResponse, error) {
out := new(milvuspb.PartitionStatsResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/GetPartitionStatistics", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *masterServiceClient) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionResponse, error) {
out := new(milvuspb.ShowPartitionResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/ShowPartitions", in, out, opts...)
@ -599,6 +622,7 @@ type MasterServiceServer interface {
//
// @return CollectionSchema
DescribeCollection(context.Context, *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error)
GetCollectionStatistics(context.Context, *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error)
//*
// @brief This method is used to list all collections.
//
@ -619,6 +643,7 @@ type MasterServiceServer interface {
//
// @return BoolResponse
HasPartition(context.Context, *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error)
GetPartitionStatistics(context.Context, *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error)
//*
// @brief This method is used to show partition information
//
@ -657,6 +682,9 @@ func (*UnimplementedMasterServiceServer) HasCollection(ctx context.Context, req
func (*UnimplementedMasterServiceServer) DescribeCollection(ctx context.Context, req *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method DescribeCollection not implemented")
}
func (*UnimplementedMasterServiceServer) GetCollectionStatistics(ctx context.Context, req *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetCollectionStatistics not implemented")
}
func (*UnimplementedMasterServiceServer) ShowCollections(ctx context.Context, req *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method ShowCollections not implemented")
}
@ -669,6 +697,9 @@ func (*UnimplementedMasterServiceServer) DropPartition(ctx context.Context, req
func (*UnimplementedMasterServiceServer) HasPartition(ctx context.Context, req *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method HasPartition not implemented")
}
func (*UnimplementedMasterServiceServer) GetPartitionStatistics(ctx context.Context, req *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetPartitionStatistics not implemented")
}
func (*UnimplementedMasterServiceServer) ShowPartitions(ctx context.Context, req *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented")
}
@ -785,6 +816,24 @@ func _MasterService_DescribeCollection_Handler(srv interface{}, ctx context.Cont
return interceptor(ctx, in, info, handler)
}
func _MasterService_GetCollectionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(milvuspb.CollectionStatsRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(MasterServiceServer).GetCollectionStatistics(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.master.MasterService/GetCollectionStatistics",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(MasterServiceServer).GetCollectionStatistics(ctx, req.(*milvuspb.CollectionStatsRequest))
}
return interceptor(ctx, in, info, handler)
}
func _MasterService_ShowCollections_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(milvuspb.ShowCollectionRequest)
if err := dec(in); err != nil {
@ -857,6 +906,24 @@ func _MasterService_HasPartition_Handler(srv interface{}, ctx context.Context, d
return interceptor(ctx, in, info, handler)
}
func _MasterService_GetPartitionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(milvuspb.PartitionStatsRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(MasterServiceServer).GetPartitionStatistics(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.master.MasterService/GetPartitionStatistics",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(MasterServiceServer).GetPartitionStatistics(ctx, req.(*milvuspb.PartitionStatsRequest))
}
return interceptor(ctx, in, info, handler)
}
func _MasterService_ShowPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(milvuspb.ShowPartitionRequest)
if err := dec(in); err != nil {
@ -1111,6 +1178,10 @@ var _MasterService_serviceDesc = grpc.ServiceDesc{
MethodName: "DescribeCollection",
Handler: _MasterService_DescribeCollection_Handler,
},
{
MethodName: "GetCollectionStatistics",
Handler: _MasterService_GetCollectionStatistics_Handler,
},
{
MethodName: "ShowCollections",
Handler: _MasterService_ShowCollections_Handler,
@ -1127,6 +1198,10 @@ var _MasterService_serviceDesc = grpc.ServiceDesc{
MethodName: "HasPartition",
Handler: _MasterService_HasPartition_Handler,
},
{
MethodName: "GetPartitionStatistics",
Handler: _MasterService_GetPartitionStatistics_Handler,
},
{
MethodName: "ShowPartitions",
Handler: _MasterService_ShowPartitions_Handler,

View File

@ -119,7 +119,7 @@ message LoadSegmentRequest {
common.MsgBase base = 1;
int64 dbID = 2;
int64 collectionID = 3;
int64 partitionID = 4;
repeated int64 partitionIDs = 4;
repeated int64 segmentIDs = 5;
repeated int64 fieldIDs = 6;
}

View File

@ -973,7 +973,7 @@ type LoadSegmentRequest struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"`
CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
PartitionID int64 `protobuf:"varint,4,opt,name=partitionID,proto3" json:"partitionID,omitempty"`
PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"`
SegmentIDs []int64 `protobuf:"varint,5,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"`
FieldIDs []int64 `protobuf:"varint,6,rep,packed,name=fieldIDs,proto3" json:"fieldIDs,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@ -1027,11 +1027,11 @@ func (m *LoadSegmentRequest) GetCollectionID() int64 {
return 0
}
func (m *LoadSegmentRequest) GetPartitionID() int64 {
func (m *LoadSegmentRequest) GetPartitionIDs() []int64 {
if m != nil {
return m.PartitionID
return m.PartitionIDs
}
return 0
return nil
}
func (m *LoadSegmentRequest) GetSegmentIDs() []int64 {
@ -1297,81 +1297,80 @@ func init() {
func init() { proto.RegisterFile("query_service.proto", fileDescriptor_5fcb6756dc1afb8d) }
var fileDescriptor_5fcb6756dc1afb8d = []byte{
// 1170 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x57, 0x5d, 0x6f, 0x1b, 0x45,
0x17, 0xf6, 0xc6, 0x4e, 0xde, 0xfa, 0xc4, 0xb1, 0x9d, 0xc9, 0xa7, 0xf6, 0x45, 0x55, 0x19, 0x0a,
0xcd, 0x07, 0x38, 0x28, 0x95, 0x10, 0x77, 0x28, 0x89, 0xab, 0xc8, 0x12, 0x0d, 0x61, 0xd3, 0x0a,
0x91, 0xb6, 0x32, 0xeb, 0xdd, 0x21, 0x99, 0x76, 0x3f, 0xdc, 0x9d, 0x71, 0x4a, 0x72, 0x03, 0x95,
0xb8, 0xe7, 0x92, 0x6b, 0x04, 0x02, 0x89, 0x1f, 0xc0, 0xff, 0xe0, 0x1a, 0x6e, 0xf8, 0x27, 0x68,
0x67, 0xd6, 0x9b, 0x5d, 0xef, 0x6c, 0xec, 0xd4, 0x69, 0xc3, 0xdd, 0xce, 0xec, 0x99, 0xf3, 0x3c,
0xe7, 0xcc, 0x9c, 0x33, 0xcf, 0xc0, 0xdc, 0xf3, 0x1e, 0x09, 0x4e, 0xdb, 0x8c, 0x04, 0x27, 0xd4,
0x22, 0x8d, 0x6e, 0xe0, 0x73, 0x1f, 0x21, 0x97, 0x3a, 0x27, 0x3d, 0x26, 0x47, 0x0d, 0x61, 0xa1,
0x57, 0x2c, 0xdf, 0x75, 0x7d, 0x4f, 0xce, 0xe9, 0x55, 0xea, 0x71, 0x12, 0x78, 0xa6, 0x23, 0xc7,
0xf8, 0x5b, 0x98, 0x33, 0xc8, 0x11, 0x65, 0x9c, 0x04, 0x7b, 0xbe, 0x4d, 0x0c, 0xf2, 0xbc, 0x47,
0x18, 0x47, 0x1f, 0x42, 0xa9, 0x63, 0x32, 0xb2, 0xac, 0xdd, 0xd2, 0x56, 0xa6, 0x37, 0xdf, 0x6a,
0xa4, 0xfc, 0x46, 0x0e, 0xef, 0xb3, 0xa3, 0x6d, 0x93, 0x11, 0x43, 0x58, 0xa2, 0x8f, 0xe0, 0x7f,
0xa6, 0x6d, 0x07, 0x84, 0xb1, 0xe5, 0x89, 0x0b, 0x16, 0x6d, 0x49, 0x1b, 0xa3, 0x6f, 0x8c, 0x7f,
0xd0, 0x60, 0x3e, 0xcd, 0x80, 0x75, 0x7d, 0x8f, 0x11, 0x74, 0x17, 0xa6, 0x18, 0x37, 0x79, 0x8f,
0x45, 0x24, 0xfe, 0xaf, 0xf4, 0x77, 0x20, 0x4c, 0x8c, 0xc8, 0x14, 0x6d, 0xc3, 0x34, 0xf5, 0x28,
0x6f, 0x77, 0xcd, 0xc0, 0x74, 0xfb, 0x4c, 0xde, 0x4e, 0xaf, 0x8c, 0x33, 0xd0, 0xf2, 0x28, 0xdf,
0x17, 0x86, 0x06, 0xd0, 0xf8, 0x1b, 0x3f, 0x81, 0x85, 0x83, 0x63, 0xff, 0xc5, 0x8e, 0xef, 0x38,
0xc4, 0xe2, 0xd4, 0xf7, 0x5e, 0x3d, 0x29, 0x08, 0x4a, 0x76, 0xa7, 0xd5, 0x14, 0x3c, 0x8a, 0x86,
0xf8, 0xc6, 0x0c, 0x16, 0x07, 0xdd, 0x8f, 0x13, 0xf1, 0x6d, 0x98, 0xb1, 0x62, 0x57, 0xad, 0x66,
0x18, 0x73, 0x71, 0xa5, 0x68, 0xa4, 0x27, 0xf1, 0x4b, 0x0d, 0x16, 0x3e, 0xf5, 0x4d, 0xfb, 0x35,
0x05, 0x85, 0x30, 0x54, 0x92, 0x80, 0xcb, 0x45, 0xf1, 0x2f, 0x35, 0x87, 0xbf, 0xd7, 0x60, 0xd9,
0x20, 0x0e, 0x31, 0x19, 0xb9, 0x4e, 0x1a, 0xdf, 0x69, 0x30, 0x1f, 0x6e, 0xc0, 0xbe, 0x19, 0x70,
0x7a, 0x3d, 0x14, 0xba, 0xf2, 0x84, 0x25, 0x18, 0x8c, 0x73, 0x02, 0x30, 0x54, 0xba, 0x7d, 0x4f,
0xe7, 0x07, 0x20, 0x35, 0x87, 0x5d, 0xa8, 0xc5, 0x68, 0xe1, 0x72, 0xc2, 0xd0, 0x2d, 0x98, 0x4e,
0x98, 0x08, 0xc0, 0xa2, 0x91, 0x9c, 0x42, 0x1f, 0xc3, 0x64, 0x08, 0x41, 0x44, 0x7c, 0xd5, 0x4d,
0xdc, 0xc8, 0x76, 0x97, 0x46, 0xda, 0xab, 0x21, 0x17, 0xe0, 0x5f, 0x35, 0x58, 0x1c, 0xc0, 0x7b,
0xe3, 0x59, 0xce, 0xe4, 0xa5, 0xa4, 0xc8, 0xcb, 0xef, 0x1a, 0x2c, 0x65, 0x88, 0x8e, 0xb3, 0x19,
0x87, 0xb0, 0x18, 0x03, 0xb4, 0x6d, 0xc2, 0xac, 0x80, 0x76, 0xc3, 0x6f, 0xb9, 0x2d, 0xd3, 0x9b,
0xef, 0x0c, 0x4f, 0x22, 0x33, 0x16, 0x62, 0x17, 0xcd, 0x84, 0x07, 0xfc, 0x8b, 0x06, 0xf3, 0x61,
0x11, 0x5f, 0xdf, 0xc9, 0x1d, 0x29, 0xa7, 0xbf, 0x69, 0xb0, 0x14, 0xd5, 0xf9, 0x7f, 0x9c, 0xe9,
0x4f, 0x1a, 0xe8, 0x3b, 0x01, 0x31, 0x39, 0xf9, 0x3c, 0xdc, 0x87, 0x9d, 0x63, 0xd3, 0xf3, 0x88,
0x33, 0xde, 0x01, 0xb8, 0x03, 0xb5, 0x40, 0x06, 0xdb, 0xb6, 0xa4, 0x3f, 0x41, 0xbd, 0x6c, 0x54,
0xa3, 0xe9, 0x08, 0x05, 0xbd, 0x0b, 0xd5, 0x80, 0xb0, 0x9e, 0x73, 0x6e, 0x57, 0x14, 0x76, 0x33,
0x72, 0x36, 0x32, 0xc3, 0x3f, 0x6b, 0xb0, 0xb4, 0x65, 0xdb, 0x49, 0x82, 0x63, 0xd4, 0xd2, 0x3a,
0xcc, 0x0e, 0xb0, 0x8b, 0x52, 0x5b, 0x36, 0xea, 0x69, 0x7e, 0xad, 0x26, 0x5a, 0x85, 0x7a, 0x9a,
0x61, 0x94, 0xea, 0xb2, 0x51, 0x4b, 0x71, 0x6c, 0x35, 0xf1, 0x5f, 0x1a, 0xe8, 0x06, 0x71, 0xfd,
0x13, 0xa2, 0x24, 0xfa, 0x4a, 0x99, 0xec, 0x47, 0x37, 0x31, 0x5e, 0x74, 0xc5, 0x4b, 0x44, 0x57,
0x52, 0x47, 0xf7, 0x14, 0x16, 0xbf, 0x30, 0xb9, 0x75, 0xdc, 0x74, 0xc7, 0xdf, 0x81, 0x9b, 0x00,
0x31, 0x9e, 0x6c, 0x0a, 0x65, 0x23, 0x31, 0x83, 0xff, 0xd6, 0x00, 0x85, 0x45, 0x7e, 0x40, 0x8e,
0x5c, 0xe2, 0xf1, 0x37, 0x5f, 0x38, 0x03, 0xf7, 0x42, 0x29, 0x7b, 0x2f, 0xdc, 0x04, 0x60, 0x92,
0x5d, 0x18, 0xc2, 0xa4, 0x28, 0xac, 0xc4, 0x0c, 0xd2, 0xe1, 0xc6, 0xd7, 0x94, 0x38, 0x76, 0xf8,
0x77, 0x4a, 0xfc, 0x8d, 0xc7, 0xf8, 0x1f, 0x0d, 0x16, 0xa2, 0xe6, 0x70, 0x6d, 0x11, 0x8e, 0xd0,
0x1a, 0xc6, 0x8a, 0xf1, 0xa5, 0x06, 0xfa, 0x2e, 0xe1, 0x0f, 0xa8, 0x4b, 0x1e, 0x50, 0xeb, 0xd9,
0x95, 0xb4, 0x95, 0x06, 0xcc, 0x71, 0xea, 0x92, 0x36, 0xa7, 0xd6, 0xb3, 0x4c, 0xe9, 0xce, 0xf2,
0x34, 0x54, 0xab, 0x89, 0x5f, 0xc0, 0xd2, 0x2e, 0xe1, 0xa1, 0x13, 0x76, 0x55, 0x6d, 0x2d, 0xfc,
0x62, 0x19, 0xec, 0x2a, 0x4b, 0x60, 0xb4, 0x9a, 0xf8, 0x47, 0x0d, 0x16, 0x0e, 0xe4, 0xa3, 0xe4,
0x2a, 0xee, 0xd3, 0x16, 0xcc, 0x84, 0x4f, 0x1c, 0x12, 0xb4, 0x85, 0xb2, 0xe8, 0x4b, 0xfa, 0xdb,
0x39, 0x92, 0x3e, 0x8d, 0x5c, 0x91, 0x4b, 0xe5, 0x68, 0xed, 0x0c, 0xaa, 0xe9, 0x8b, 0x16, 0x55,
0xe0, 0xc6, 0x9e, 0xcf, 0xef, 0x7d, 0x43, 0x19, 0xaf, 0x17, 0x50, 0x15, 0x60, 0xcf, 0xe7, 0xfb,
0x01, 0x61, 0xc4, 0xe3, 0x75, 0x0d, 0x01, 0x4c, 0x7d, 0xe6, 0x35, 0xc3, 0x7f, 0x13, 0x68, 0x2e,
0xd2, 0x4f, 0xa6, 0xd3, 0xf2, 0xee, 0x13, 0xd7, 0x0f, 0x4e, 0xeb, 0xc5, 0x70, 0x79, 0x3c, 0x2a,
0xa1, 0x3a, 0x54, 0x62, 0x93, 0xdd, 0xfd, 0x87, 0xf5, 0x49, 0x54, 0x86, 0x49, 0xf9, 0x39, 0xb5,
0xf9, 0x47, 0x19, 0x2a, 0xa2, 0x33, 0x46, 0x04, 0x91, 0x05, 0x95, 0xe4, 0xab, 0x07, 0xdd, 0x51,
0xe9, 0x02, 0xc5, 0xcb, 0x4c, 0x5f, 0x19, 0x6e, 0x28, 0xf3, 0x8d, 0x0b, 0xe8, 0x29, 0xd4, 0xd2,
0x4f, 0x0d, 0x86, 0x56, 0x55, 0xcb, 0x95, 0xcf, 0x1d, 0x7d, 0x6d, 0x14, 0xd3, 0x18, 0xeb, 0x08,
0xaa, 0x29, 0x4d, 0xcb, 0xd0, 0x4a, 0xde, 0xfa, 0x41, 0x55, 0xa0, 0xaf, 0x8e, 0x60, 0x19, 0x03,
0x7d, 0x09, 0xd5, 0x94, 0x08, 0xca, 0x01, 0x52, 0x09, 0x25, 0xfd, 0xa2, 0x23, 0x87, 0x0b, 0xa8,
0x0d, 0xb3, 0x83, 0xc2, 0x85, 0xa1, 0x75, 0x75, 0xc2, 0x95, 0xfa, 0x66, 0x18, 0xc0, 0xa1, 0xe4,
0x7e, 0x9e, 0x40, 0xf5, 0x7e, 0x28, 0x5f, 0x6a, 0xc3, 0x7c, 0x7f, 0x15, 0x93, 0x4f, 0xb8, 0x7f,
0xff, 0x02, 0xf2, 0x97, 0x46, 0xe8, 0x00, 0xca, 0xaa, 0x25, 0xa4, 0x2b, 0x17, 0xdd, 0x73, 0xbb,
0xfc, 0x54, 0x6f, 0xa8, 0xe0, 0xf3, 0x15, 0x97, 0xc4, 0xc8, 0xb6, 0xce, 0xcb, 0x63, 0xe4, 0xb7,
0x5f, 0x5c, 0x40, 0x8f, 0xa1, 0x36, 0xd0, 0x1b, 0x2f, 0x04, 0x58, 0xcf, 0x01, 0x50, 0x35, 0x57,
0x5c, 0x40, 0xbe, 0x88, 0x60, 0xf0, 0xb5, 0xb5, 0x36, 0x8a, 0xee, 0x8f, 0xb6, 0x61, 0x7d, 0x24,
0xdb, 0x18, 0xf0, 0x11, 0xd4, 0x43, 0x36, 0xc9, 0xce, 0x77, 0x61, 0x3c, 0xea, 0x6a, 0x53, 0xb5,
0x6c, 0x5c, 0xd8, 0xfc, 0xb3, 0x04, 0x65, 0xb1, 0x55, 0xa2, 0x4b, 0x3d, 0x81, 0xda, 0x80, 0x16,
0x55, 0x97, 0x47, 0x8e, 0x60, 0x1d, 0x76, 0xc0, 0x2c, 0x40, 0x59, 0x11, 0x89, 0x1a, 0xea, 0x33,
0x9c, 0x27, 0x36, 0x87, 0x81, 0x3c, 0x86, 0xda, 0x80, 0x98, 0x53, 0x6f, 0x8e, 0x5a, 0xf1, 0x0d,
0xf3, 0xfe, 0x10, 0x2a, 0x09, 0xf5, 0xc6, 0xd0, 0x7b, 0x79, 0xf5, 0x9d, 0x56, 0x3f, 0xc3, 0xdc,
0x3e, 0x82, 0x5a, 0x5a, 0x35, 0xe5, 0x74, 0x72, 0xa5, 0xb4, 0x1a, 0xe6, 0xdc, 0x83, 0xd9, 0xcc,
0x89, 0x7d, 0x8d, 0x07, 0x76, 0x7b, 0xeb, 0xf0, 0x93, 0x23, 0xca, 0x8f, 0x7b, 0x9d, 0x90, 0xc9,
0xc6, 0x19, 0x75, 0x1c, 0x7a, 0xc6, 0x89, 0x75, 0xbc, 0x21, 0xbd, 0x7c, 0x60, 0x53, 0xc6, 0x03,
0xda, 0xe9, 0x71, 0x62, 0x6f, 0xf4, 0x6f, 0xf6, 0x0d, 0xe1, 0x7a, 0x43, 0xb8, 0xee, 0x76, 0x3a,
0x53, 0x62, 0x78, 0xf7, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xb4, 0x7e, 0x65, 0x42, 0x06, 0x15,
0x00, 0x00,
// 1158 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x57, 0x5d, 0x6f, 0x1b, 0x45,
0x17, 0xf6, 0xc6, 0x4e, 0xde, 0xfa, 0xc4, 0xb1, 0x9d, 0xc9, 0xa7, 0xf6, 0x45, 0xa8, 0x0c, 0x85,
0xe6, 0x03, 0x1c, 0x94, 0x4a, 0x88, 0x3b, 0x94, 0xc4, 0x55, 0x64, 0x89, 0x86, 0xb0, 0x69, 0x85,
0x48, 0x5b, 0x99, 0xf5, 0xee, 0x10, 0x4f, 0xbb, 0x1f, 0xee, 0xce, 0x38, 0x25, 0xb9, 0x81, 0x4a,
0xdc, 0x73, 0xc9, 0x35, 0x02, 0x81, 0xc4, 0x0f, 0xe0, 0x7f, 0x70, 0x8d, 0x84, 0xc4, 0x3f, 0x41,
0x3b, 0xb3, 0xde, 0xec, 0xda, 0xb3, 0xb6, 0x53, 0xa7, 0xa4, 0xe2, 0x6e, 0x66, 0xf6, 0xcc, 0x79,
0x9e, 0x73, 0xe6, 0xcc, 0xd9, 0x67, 0x60, 0xe1, 0x59, 0x97, 0x04, 0x67, 0x4d, 0x46, 0x82, 0x53,
0x6a, 0x91, 0x5a, 0x27, 0xf0, 0xb9, 0x8f, 0x90, 0x4b, 0x9d, 0xd3, 0x2e, 0x93, 0xb3, 0x9a, 0xb0,
0xd0, 0x4b, 0x96, 0xef, 0xba, 0xbe, 0x27, 0xd7, 0xf4, 0x32, 0xf5, 0x38, 0x09, 0x3c, 0xd3, 0x91,
0x73, 0xfc, 0x0d, 0x2c, 0x18, 0xe4, 0x84, 0x32, 0x4e, 0x82, 0x03, 0xdf, 0x26, 0x06, 0x79, 0xd6,
0x25, 0x8c, 0xa3, 0x0f, 0xa0, 0xd0, 0x32, 0x19, 0x59, 0xd5, 0x6e, 0x6a, 0x6b, 0xb3, 0xdb, 0x6f,
0xd4, 0x52, 0x7e, 0x23, 0x87, 0xf7, 0xd8, 0xc9, 0xae, 0xc9, 0x88, 0x21, 0x2c, 0xd1, 0x87, 0xf0,
0x3f, 0xd3, 0xb6, 0x03, 0xc2, 0xd8, 0xea, 0xd4, 0x90, 0x4d, 0x3b, 0xd2, 0xc6, 0xe8, 0x19, 0xe3,
0xef, 0x35, 0x58, 0x4c, 0x33, 0x60, 0x1d, 0xdf, 0x63, 0x04, 0xdd, 0x81, 0x19, 0xc6, 0x4d, 0xde,
0x65, 0x11, 0x89, 0xff, 0x2b, 0xfd, 0x1d, 0x09, 0x13, 0x23, 0x32, 0x45, 0xbb, 0x30, 0x4b, 0x3d,
0xca, 0x9b, 0x1d, 0x33, 0x30, 0xdd, 0x1e, 0x93, 0xb7, 0xd2, 0x3b, 0xe3, 0x0c, 0x34, 0x3c, 0xca,
0x0f, 0x85, 0xa1, 0x01, 0x34, 0x1e, 0xe3, 0xc7, 0xb0, 0x74, 0xd4, 0xf6, 0x9f, 0xef, 0xf9, 0x8e,
0x43, 0x2c, 0x4e, 0x7d, 0xef, 0xe5, 0x93, 0x82, 0xa0, 0x60, 0xb7, 0x1a, 0x75, 0xc1, 0x23, 0x6f,
0x88, 0x31, 0x66, 0xb0, 0xdc, 0xef, 0x7e, 0x92, 0x88, 0x6f, 0xc1, 0x9c, 0x15, 0xbb, 0x6a, 0xd4,
0xc3, 0x98, 0xf3, 0x6b, 0x79, 0x23, 0xbd, 0x88, 0x5f, 0x68, 0xb0, 0xf4, 0x89, 0x6f, 0xda, 0xaf,
0x28, 0x28, 0x84, 0xa1, 0x94, 0x04, 0x5c, 0xcd, 0x8b, 0x6f, 0xa9, 0x35, 0xfc, 0x9d, 0x06, 0xab,
0x06, 0x71, 0x88, 0xc9, 0xc8, 0x75, 0xd2, 0xf8, 0x56, 0x83, 0xc5, 0xf0, 0x00, 0x0e, 0xcd, 0x80,
0xd3, 0xeb, 0xa1, 0xd0, 0x91, 0x15, 0x96, 0x60, 0x30, 0x49, 0x05, 0x60, 0x28, 0x75, 0x7a, 0x9e,
0x2e, 0x0a, 0x20, 0xb5, 0x86, 0x5d, 0xa8, 0xc4, 0x68, 0xe1, 0x76, 0xc2, 0xd0, 0x4d, 0x98, 0x4d,
0x98, 0x08, 0xc0, 0xbc, 0x91, 0x5c, 0x42, 0x1f, 0xc1, 0x74, 0x08, 0x41, 0x44, 0x7c, 0xe5, 0x6d,
0x5c, 0x1b, 0xec, 0x2e, 0xb5, 0xb4, 0x57, 0x43, 0x6e, 0xc0, 0xbf, 0x68, 0xb0, 0xdc, 0x87, 0xf7,
0xaf, 0x67, 0x79, 0x20, 0x2f, 0x05, 0x45, 0x5e, 0x7e, 0xd3, 0x60, 0x65, 0x80, 0xe8, 0x24, 0x87,
0x71, 0x0c, 0xcb, 0x31, 0x40, 0xd3, 0x26, 0xcc, 0x0a, 0x68, 0x27, 0x1c, 0xcb, 0x63, 0x99, 0xdd,
0x7e, 0x7b, 0x74, 0x12, 0x99, 0xb1, 0x14, 0xbb, 0xa8, 0x27, 0x3c, 0xe0, 0x9f, 0x35, 0x58, 0x0c,
0x2f, 0xf1, 0xf5, 0x55, 0xee, 0x58, 0x39, 0xfd, 0x55, 0x83, 0x95, 0xe8, 0x9e, 0xbf, 0xe6, 0x4c,
0x7f, 0xd4, 0x40, 0xdf, 0x0b, 0x88, 0xc9, 0xc9, 0x67, 0xe1, 0x39, 0xec, 0xb5, 0x4d, 0xcf, 0x23,
0xce, 0x64, 0x05, 0x70, 0x1b, 0x2a, 0x81, 0x0c, 0xb6, 0x69, 0x49, 0x7f, 0x82, 0x7a, 0xd1, 0x28,
0x47, 0xcb, 0x11, 0x0a, 0x7a, 0x07, 0xca, 0x01, 0x61, 0x5d, 0xe7, 0xc2, 0x2e, 0x2f, 0xec, 0xe6,
0xe4, 0x6a, 0x64, 0x86, 0x7f, 0xd2, 0x60, 0x65, 0xc7, 0xb6, 0x93, 0x04, 0x27, 0xb8, 0x4b, 0x9b,
0x30, 0xdf, 0xc7, 0x2e, 0x4a, 0x6d, 0xd1, 0xa8, 0xa6, 0xf9, 0x35, 0xea, 0x68, 0x1d, 0xaa, 0x69,
0x86, 0x51, 0xaa, 0x8b, 0x46, 0x25, 0xc5, 0xb1, 0x51, 0xc7, 0x7f, 0x6a, 0xa0, 0x1b, 0xc4, 0xf5,
0x4f, 0x89, 0x92, 0xe8, 0x4b, 0x65, 0xb2, 0x17, 0xdd, 0xd4, 0x64, 0xd1, 0xe5, 0x2f, 0x11, 0x5d,
0x41, 0x1d, 0xdd, 0x13, 0x58, 0xfe, 0xdc, 0xe4, 0x56, 0xbb, 0xee, 0x4e, 0x7e, 0x02, 0x6f, 0x02,
0xc4, 0x78, 0xb2, 0x29, 0x14, 0x8d, 0xc4, 0x0a, 0xfe, 0x4b, 0x03, 0x14, 0x5e, 0xf2, 0x23, 0x72,
0xe2, 0x12, 0x8f, 0xbf, 0x96, 0x17, 0x27, 0x0c, 0x82, 0x49, 0x7e, 0xa1, 0xc5, 0xb4, 0xb0, 0x48,
0xac, 0x20, 0x1d, 0x6e, 0x7c, 0x45, 0x89, 0x63, 0x87, 0x5f, 0x67, 0xc4, 0xd7, 0x78, 0x8e, 0xff,
0xd6, 0x60, 0x29, 0x6a, 0x0f, 0xff, 0xd9, 0x18, 0x5f, 0x68, 0xa0, 0xef, 0x13, 0x7e, 0x9f, 0xba,
0xe4, 0x3e, 0xb5, 0x9e, 0x5e, 0x49, 0x63, 0xa9, 0xc1, 0x02, 0xa7, 0x2e, 0x69, 0x72, 0x6a, 0x3d,
0x1d, 0xb8, 0xbc, 0xf3, 0x3c, 0x0d, 0xd5, 0xa8, 0xe3, 0xe7, 0xb0, 0xb2, 0x4f, 0x78, 0xe8, 0x84,
0x5d, 0x55, 0x63, 0x0b, 0x47, 0x6c, 0x00, 0xbb, 0xcc, 0x12, 0x18, 0x8d, 0x3a, 0xfe, 0x41, 0x83,
0xa5, 0x23, 0xf9, 0x2c, 0xb9, 0x8a, 0x3f, 0x6a, 0x03, 0xe6, 0xc2, 0x47, 0x0e, 0x09, 0x9a, 0x42,
0x5b, 0xf4, 0x44, 0xfd, 0xad, 0x0c, 0x51, 0x9f, 0x46, 0x2e, 0xc9, 0xad, 0x72, 0xb6, 0x71, 0x0e,
0xe5, 0xf4, 0xaf, 0x16, 0x95, 0xe0, 0xc6, 0x81, 0xcf, 0xef, 0x7e, 0x4d, 0x19, 0xaf, 0xe6, 0x50,
0x19, 0xe0, 0xc0, 0xe7, 0x87, 0x01, 0x61, 0xc4, 0xe3, 0x55, 0x0d, 0x01, 0xcc, 0x7c, 0xea, 0xd5,
0xc3, 0x6f, 0x53, 0x68, 0x21, 0x52, 0x50, 0xa6, 0xd3, 0xf0, 0xee, 0x11, 0xd7, 0x0f, 0xce, 0xaa,
0xf9, 0x70, 0x7b, 0x3c, 0x2b, 0xa0, 0x2a, 0x94, 0x62, 0x93, 0xfd, 0xc3, 0x07, 0xd5, 0x69, 0x54,
0x84, 0x69, 0x39, 0x9c, 0xd9, 0xfe, 0xbd, 0x08, 0x25, 0xd1, 0x1b, 0x23, 0x82, 0xc8, 0x82, 0x52,
0xf2, 0xdd, 0x83, 0x6e, 0xab, 0x94, 0x81, 0xe2, 0x6d, 0xa6, 0xaf, 0x8d, 0x36, 0x94, 0xf9, 0xc6,
0x39, 0xf4, 0x04, 0x2a, 0xe9, 0xc7, 0x06, 0x43, 0xeb, 0xaa, 0xed, 0xca, 0x07, 0x8f, 0xbe, 0x31,
0x8e, 0x69, 0x8c, 0x75, 0x02, 0xe5, 0x94, 0xaa, 0x65, 0x68, 0x2d, 0x6b, 0x7f, 0xbf, 0x2e, 0xd0,
0xd7, 0xc7, 0xb0, 0x8c, 0x81, 0xbe, 0x80, 0x72, 0x4a, 0x06, 0x65, 0x00, 0xa9, 0xa4, 0x92, 0x3e,
0xac, 0xe4, 0x70, 0x0e, 0x35, 0x61, 0xbe, 0x5f, 0xba, 0x30, 0xb4, 0xa9, 0x4e, 0xb8, 0x52, 0xe1,
0x8c, 0x02, 0x38, 0x96, 0xdc, 0x2f, 0x12, 0xa8, 0x3e, 0x0f, 0xe5, 0x5b, 0x6d, 0x94, 0xef, 0x2f,
0x63, 0xf2, 0x09, 0xf7, 0xef, 0x0d, 0x21, 0x7f, 0x69, 0x84, 0x16, 0xa0, 0x41, 0xbd, 0x84, 0x74,
0xe5, 0xa6, 0xbb, 0x6e, 0x87, 0x9f, 0xe9, 0x35, 0x15, 0x7c, 0xb6, 0xe6, 0x92, 0x18, 0x83, 0xad,
0xf3, 0xf2, 0x18, 0xd9, 0xed, 0x17, 0xe7, 0xd0, 0x23, 0xa8, 0xf4, 0xf5, 0xc6, 0xa1, 0x00, 0x9b,
0x19, 0x00, 0xaa, 0xe6, 0x8a, 0x73, 0xc8, 0x17, 0x11, 0xf4, 0xbf, 0xb7, 0x36, 0xc6, 0x51, 0xfe,
0xd1, 0x31, 0x6c, 0x8e, 0x65, 0x1b, 0x03, 0x3e, 0x84, 0x6a, 0xc8, 0x26, 0xd9, 0xf9, 0x86, 0xc6,
0xa3, 0xbe, 0x6d, 0xaa, 0x96, 0x8d, 0x73, 0xdb, 0x7f, 0x14, 0xa0, 0x28, 0x8e, 0x4a, 0x74, 0xa9,
0xc7, 0x50, 0xe9, 0x53, 0xa3, 0xea, 0xeb, 0x91, 0x21, 0x59, 0x47, 0x15, 0x98, 0x05, 0x68, 0x50,
0x46, 0xa2, 0x9a, 0xba, 0x86, 0xb3, 0xe4, 0xe6, 0x28, 0x90, 0x47, 0x50, 0xe9, 0x93, 0x73, 0xea,
0xc3, 0x51, 0x6b, 0xbe, 0x51, 0xde, 0x1f, 0x40, 0x29, 0xa1, 0xdf, 0x18, 0x7a, 0x37, 0xeb, 0x7e,
0xa7, 0xd5, 0xcf, 0x28, 0xb7, 0x0f, 0xa1, 0x92, 0x56, 0x4d, 0x19, 0x9d, 0x5c, 0x29, 0xad, 0x46,
0x39, 0xf7, 0x60, 0x7e, 0xa0, 0x62, 0x5f, 0x61, 0xc1, 0xee, 0xee, 0x1c, 0x7f, 0x7c, 0x42, 0x79,
0xbb, 0xdb, 0x0a, 0x99, 0x6c, 0x9d, 0x53, 0xc7, 0xa1, 0xe7, 0x9c, 0x58, 0xed, 0x2d, 0xe9, 0xe5,
0x7d, 0x9b, 0x32, 0x1e, 0xd0, 0x56, 0x97, 0x13, 0x7b, 0xab, 0xf7, 0x67, 0xdf, 0x12, 0xae, 0xb7,
0x84, 0xeb, 0x4e, 0xab, 0x35, 0x23, 0xa6, 0x77, 0xfe, 0x09, 0x00, 0x00, 0xff, 0xff, 0x32, 0x8c,
0x07, 0xbf, 0x08, 0x15, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.

View File

@ -51,18 +51,16 @@ type collectionReplica interface {
addPartitionsByCollectionMeta(colMeta *etcdpb.CollectionMeta) error
removePartitionsByCollectionMeta(colMeta *etcdpb.CollectionMeta) error
getPartitionByTag(collectionID UniqueID, partitionTag string) (*Partition, error)
getPartitionByID(collectionID UniqueID, partitionID UniqueID) (*Partition, error)
hasPartition(collectionID UniqueID, partitionTag string) bool
// segment
getSegmentNum() int
getSegmentStatistics() []*internalpb2.SegmentStats
addSegment2(segmentID UniqueID, partitionTag string, collectionID UniqueID, segType segmentType) error
addSegment(segmentID UniqueID, partitionID UniqueID, collectionID UniqueID, segType segmentType) error
addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error
removeSegment(segmentID UniqueID) error
getSegmentByID(segmentID UniqueID) (*Segment, error)
hasSegment(segmentID UniqueID) bool
getVecFieldsBySegmentID(segmentID UniqueID) (map[int64]string, error)
getVecFieldIDsBySegmentID(segmentID UniqueID) ([]int64, error)
freeAll()
}
@ -290,13 +288,6 @@ func (colReplica *collectionReplicaImpl) getPartitionByTag(collectionID UniqueID
return colReplica.getPartitionByTagPrivate(collectionID, partitionTag)
}
func (colReplica *collectionReplicaImpl) getPartitionByID(collectionID UniqueID, partitionID UniqueID) (*Partition, error) {
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
return colReplica.getPartitionByIDPrivate(collectionID, partitionID)
}
func (colReplica *collectionReplicaImpl) getPartitionByTagPrivate(collectionID UniqueID, partitionTag string) (*Partition, error) {
collection, err := colReplica.getCollectionByIDPrivate(collectionID)
if err != nil {
@ -312,21 +303,6 @@ func (colReplica *collectionReplicaImpl) getPartitionByTagPrivate(collectionID U
return nil, errors.New("cannot find partition, tag = " + partitionTag)
}
func (colReplica *collectionReplicaImpl) getPartitionByIDPrivate(collectionID UniqueID, partitionID UniqueID) (*Partition, error) {
collection, err := colReplica.getCollectionByIDPrivate(collectionID)
if err != nil {
return nil, err
}
for _, p := range *collection.Partitions() {
if p.ID() == partitionID {
return p, nil
}
}
return nil, errors.New("cannot find partition, id = " + strconv.FormatInt(partitionID, 10))
}
func (colReplica *collectionReplicaImpl) hasPartition(collectionID UniqueID, partitionTag string) bool {
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
@ -379,7 +355,7 @@ func (colReplica *collectionReplicaImpl) getSegmentStatistics() []*internalpb2.S
return statisticData
}
func (colReplica *collectionReplicaImpl) addSegment2(segmentID UniqueID, partitionTag string, collectionID UniqueID, segType segmentType) error {
func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error {
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
@ -393,29 +369,7 @@ func (colReplica *collectionReplicaImpl) addSegment2(segmentID UniqueID, partiti
return err2
}
var newSegment = newSegment2(collection, segmentID, partitionTag, collectionID, segType)
colReplica.segments[segmentID] = newSegment
*partition.Segments() = append(*partition.Segments(), newSegment)
return nil
}
func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, partitionID UniqueID, collectionID UniqueID, segType segmentType) error {
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
collection, err := colReplica.getCollectionByIDPrivate(collectionID)
if err != nil {
return err
}
partition, err2 := colReplica.getPartitionByIDPrivate(collectionID, partitionID)
if err2 != nil {
return err2
}
var newSegment = newSegment(collection, segmentID, partitionID, collectionID, segType)
var newSegment = newSegment(collection, segmentID, partitionTag, collectionID)
colReplica.segments[segmentID] = newSegment
*partition.Segments() = append(*partition.Segments(), newSegment)
@ -476,7 +430,7 @@ func (colReplica *collectionReplicaImpl) hasSegment(segmentID UniqueID) bool {
return ok
}
func (colReplica *collectionReplicaImpl) getVecFieldsBySegmentID(segmentID UniqueID) (map[int64]string, error) {
func (colReplica *collectionReplicaImpl) getVecFieldIDsBySegmentID(segmentID UniqueID) ([]int64, error) {
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
@ -489,18 +443,16 @@ func (colReplica *collectionReplicaImpl) getVecFieldsBySegmentID(segmentID Uniqu
return nil, err2
}
vecFields := make(map[int64]string)
vecFields := make([]int64, 0)
for _, field := range col.Schema().Fields {
if field.DataType == schemapb.DataType_VECTOR_BINARY || field.DataType == schemapb.DataType_VECTOR_FLOAT {
vecFields[field.FieldID] = field.Name
vecFields = append(vecFields, field.FieldID)
}
}
if len(vecFields) <= 0 {
return nil, errors.New("no vector field in segment " + strconv.FormatInt(segmentID, 10))
}
// return map[fieldID]fieldName
return vecFields, nil
}

View File

@ -226,7 +226,7 @@ func TestCollectionReplica_addSegment(t *testing.T) {
const segmentNum = 3
tag := "default"
for i := 0; i < segmentNum; i++ {
err := node.replica.addSegment2(UniqueID(i), tag, collectionID, segTypeGrowing)
err := node.replica.addSegment(UniqueID(i), tag, collectionID)
assert.NoError(t, err)
targetSeg, err := node.replica.getSegmentByID(UniqueID(i))
assert.NoError(t, err)
@ -246,7 +246,7 @@ func TestCollectionReplica_removeSegment(t *testing.T) {
tag := "default"
for i := 0; i < segmentNum; i++ {
err := node.replica.addSegment2(UniqueID(i), tag, collectionID, segTypeGrowing)
err := node.replica.addSegment(UniqueID(i), tag, collectionID)
assert.NoError(t, err)
targetSeg, err := node.replica.getSegmentByID(UniqueID(i))
assert.NoError(t, err)
@ -268,7 +268,7 @@ func TestCollectionReplica_getSegmentByID(t *testing.T) {
tag := "default"
for i := 0; i < segmentNum; i++ {
err := node.replica.addSegment2(UniqueID(i), tag, collectionID, segTypeGrowing)
err := node.replica.addSegment(UniqueID(i), tag, collectionID)
assert.NoError(t, err)
targetSeg, err := node.replica.getSegmentByID(UniqueID(i))
assert.NoError(t, err)
@ -288,7 +288,7 @@ func TestCollectionReplica_hasSegment(t *testing.T) {
tag := "default"
for i := 0; i < segmentNum; i++ {
err := node.replica.addSegment2(UniqueID(i), tag, collectionID, segTypeGrowing)
err := node.replica.addSegment(UniqueID(i), tag, collectionID)
assert.NoError(t, err)
targetSeg, err := node.replica.getSegmentByID(UniqueID(i))
assert.NoError(t, err)

View File

@ -86,7 +86,7 @@ func (iNode *insertNode) Operate(in []*Msg) []*Msg {
log.Println(err)
continue
}
err = iNode.replica.addSegment2(task.SegmentID, task.PartitionName, collection.ID(), segTypeGrowing)
err = iNode.replica.addSegment(task.SegmentID, task.PartitionName, collection.ID())
if err != nil {
log.Println(err)
continue

View File

@ -160,7 +160,7 @@ func (mService *metaService) processSegmentCreate(id string, value string) {
// TODO: what if seg == nil? We need to notify master and return rpc request failed
if seg != nil {
err := mService.replica.addSegment2(seg.SegmentID, seg.PartitionTag, seg.CollectionID, segTypeGrowing)
err := mService.replica.addSegment(seg.SegmentID, seg.PartitionTag, seg.CollectionID)
if err != nil {
log.Println(err)
return

View File

@ -14,14 +14,9 @@ import "C"
type Partition struct {
partitionTag string
id UniqueID
segments []*Segment
}
func (p *Partition) ID() UniqueID {
return p.id
}
func (p *Partition) Tag() string {
return (*p).partitionTag
}

View File

@ -20,7 +20,7 @@ func TestPartition_Segments(t *testing.T) {
const segmentNum = 3
for i := 0; i < segmentNum; i++ {
err := node.replica.addSegment2(UniqueID(i), targetPartition.partitionTag, collection.ID(), segTypeGrowing)
err := node.replica.addSegment(UniqueID(i), targetPartition.partitionTag, collection.ID())
assert.NoError(t, err)
}

View File

@ -295,27 +295,10 @@ func (node *QueryNode) WatchDmChannels(in *queryPb.WatchDmChannelsRequest) (*com
func (node *QueryNode) LoadSegments(in *queryPb.LoadSegmentRequest) (*commonpb.Status, error) {
// TODO: support db
partitionID := in.PartitionID
collectionID := in.CollectionID
fieldIDs := in.FieldIDs
// TODO: interim solution
if len(fieldIDs) == 0 {
collection, err := node.replica.getCollectionByID(collectionID)
if err != nil {
status := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: err.Error(),
}
return status, err
}
fieldIDs = make([]int64, 0)
for _, field := range collection.Schema().Fields {
fieldIDs = append(fieldIDs, field.FieldID)
}
}
for _, segmentID := range in.SegmentIDs {
indexID := UniqueID(0) // TODO: get index id from master
err := node.segManager.loadSegment(segmentID, partitionID, collectionID, &fieldIDs)
indexID := UniqueID(0) // TODO: ???
err := node.segManager.loadSegment(segmentID, &fieldIDs)
if err != nil {
// TODO: return or continue?
status := &commonpb.Status{

View File

@ -110,7 +110,7 @@ func initTestMeta(t *testing.T, node *QueryNode, collectionName string, collecti
err = node.replica.addPartition(collection.ID(), collectionMeta.PartitionTags[0])
assert.NoError(t, err)
err = node.replica.addSegment2(segmentID, collectionMeta.PartitionTags[0], collectionID, segTypeGrowing)
err = node.replica.addSegment(segmentID, collectionMeta.PartitionTags[0], collectionID)
assert.NoError(t, err)
}

View File

@ -19,7 +19,7 @@ func TestReduce_AllFunc(t *testing.T) {
collectionMeta := genTestCollectionMeta(collectionName, collectionID, false)
collection := newCollection(collectionMeta.ID, collectionMeta.Schema)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
const DIM = 16
var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}

View File

@ -22,21 +22,13 @@ import (
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
)
const (
segTypeInvalid = C.Invalid
segTypeGrowing = C.Growing
segTypeSealed = C.Sealed
)
type segmentType = C.SegmentType
type indexParam = map[string]string
type Segment struct {
segmentPtr C.CSegmentInterface
segmentType C.SegmentType
segmentType C.enum_SegmentType
segmentID UniqueID
partitionTag string // TODO: use partitionID
partitionID UniqueID
collectionID UniqueID
lastMemSize int64
lastRowCount int64
@ -48,15 +40,10 @@ type Segment struct {
indexParam map[int64]indexParam
}
//-------------------------------------------------------------------------------------- common interfaces
func (s *Segment) ID() UniqueID {
return s.segmentID
}
func (s *Segment) Type() segmentType {
return s.segmentType
}
func (s *Segment) SetRecentlyModified(modify bool) {
s.rmMutex.Lock()
defer s.rmMutex.Unlock()
@ -69,13 +56,17 @@ func (s *Segment) GetRecentlyModified() bool {
return s.recentlyModified
}
func newSegment2(collection *Collection, segmentID int64, partitionTag string, collectionID UniqueID, segType segmentType) *Segment {
//-------------------------------------------------------------------------------------- constructor and destructor
func newSegment(collection *Collection, segmentID int64, partitionTag string, collectionID UniqueID) *Segment {
/*
CSegmentInterface
NewSegment(CCollection collection, uint64_t segment_id, SegmentType seg_type);
*/
initIndexParam := make(map[int64]indexParam)
segmentPtr := C.NewSegment(collection.collectionPtr, C.ulong(segmentID), segType)
// TODO: replace by param
//var segmentType C.enum_SegmentType = C.Growing
var segmentType C.int = 1
segmentPtr := C.NewSegment(collection.collectionPtr, C.ulong(segmentID), segmentType)
var newSegment = &Segment{
segmentPtr: segmentPtr,
segmentID: segmentID,
@ -87,24 +78,6 @@ func newSegment2(collection *Collection, segmentID int64, partitionTag string, c
return newSegment
}
func newSegment(collection *Collection, segmentID int64, partitionID UniqueID, collectionID UniqueID, segType segmentType) *Segment {
/*
CSegmentInterface
NewSegment(CCollection collection, uint64_t segment_id, SegmentType seg_type);
*/
initIndexParam := make(map[int64]indexParam)
segmentPtr := C.NewSegment(collection.collectionPtr, C.ulong(segmentID), segType)
var newSegment = &Segment{
segmentPtr: segmentPtr,
segmentID: segmentID,
partitionID: partitionID,
collectionID: collectionID,
indexParam: initIndexParam,
}
return newSegment
}
func deleteSegment(segment *Segment) {
/*
void
@ -114,6 +87,7 @@ func deleteSegment(segment *Segment) {
C.DeleteSegment(cPtr)
}
//-------------------------------------------------------------------------------------- stats functions
func (s *Segment) getRowCount() int64 {
/*
long int
@ -142,6 +116,108 @@ func (s *Segment) getMemSize() int64 {
return int64(memoryUsageInBytes)
}
//-------------------------------------------------------------------------------------- preDm functions
func (s *Segment) segmentPreInsert(numOfRecords int) int64 {
/*
long int
PreInsert(CSegmentInterface c_segment, long int size);
*/
var offset = C.PreInsert(s.segmentPtr, C.long(int64(numOfRecords)))
return int64(offset)
}
func (s *Segment) segmentPreDelete(numOfRecords int) int64 {
/*
long int
PreDelete(CSegmentInterface c_segment, long int size);
*/
var offset = C.PreDelete(s.segmentPtr, C.long(int64(numOfRecords)))
return int64(offset)
}
//-------------------------------------------------------------------------------------- dm & search functions
func (s *Segment) segmentInsert(offset int64, entityIDs *[]UniqueID, timestamps *[]Timestamp, records *[]*commonpb.Blob) error {
/*
CStatus
Insert(CSegmentInterface c_segment,
long int reserved_offset,
signed long int size,
const long* primary_keys,
const unsigned long* timestamps,
void* raw_data,
int sizeof_per_row,
signed long int count);
*/
// Blobs to one big blob
var numOfRow = len(*entityIDs)
var sizeofPerRow = len((*records)[0].Value)
assert.Equal(nil, numOfRow, len(*records))
var rawData = make([]byte, numOfRow*sizeofPerRow)
var copyOffset = 0
for i := 0; i < len(*records); i++ {
copy(rawData[copyOffset:], (*records)[i].Value)
copyOffset += sizeofPerRow
}
var cOffset = C.long(offset)
var cNumOfRows = C.long(numOfRow)
var cEntityIdsPtr = (*C.long)(&(*entityIDs)[0])
var cTimestampsPtr = (*C.ulong)(&(*timestamps)[0])
var cSizeofPerRow = C.int(sizeofPerRow)
var cRawDataVoidPtr = unsafe.Pointer(&rawData[0])
var status = C.Insert(s.segmentPtr,
cOffset,
cNumOfRows,
cEntityIdsPtr,
cTimestampsPtr,
cRawDataVoidPtr,
cSizeofPerRow,
cNumOfRows)
errorCode := status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return errors.New("Insert failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
s.SetRecentlyModified(true)
return nil
}
func (s *Segment) segmentDelete(offset int64, entityIDs *[]UniqueID, timestamps *[]Timestamp) error {
/*
CStatus
Delete(CSegmentInterface c_segment,
long int reserved_offset,
long size,
const long* primary_keys,
const unsigned long* timestamps);
*/
var cOffset = C.long(offset)
var cSize = C.long(len(*entityIDs))
var cEntityIdsPtr = (*C.long)(&(*entityIDs)[0])
var cTimestampsPtr = (*C.ulong)(&(*timestamps)[0])
var status = C.Delete(s.segmentPtr, cOffset, cSize, cEntityIdsPtr, cTimestampsPtr)
errorCode := status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return errors.New("Delete failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
return nil
}
func (s *Segment) segmentSearch(plan *Plan,
placeHolderGroups []*PlaceholderGroup,
timestamp []Timestamp) (*SearchResult, error) {
@ -242,138 +318,3 @@ func (s *Segment) matchIndexParam(fieldID int64, indexParamKv []*commonpb.KeyVal
}
return paramSize == matchCount
}
//-------------------------------------------------------------------------------------- interfaces for growing segment
func (s *Segment) segmentPreInsert(numOfRecords int) int64 {
/*
long int
PreInsert(CSegmentInterface c_segment, long int size);
*/
var offset = C.PreInsert(s.segmentPtr, C.long(int64(numOfRecords)))
return int64(offset)
}
func (s *Segment) segmentPreDelete(numOfRecords int) int64 {
/*
long int
PreDelete(CSegmentInterface c_segment, long int size);
*/
var offset = C.PreDelete(s.segmentPtr, C.long(int64(numOfRecords)))
return int64(offset)
}
func (s *Segment) segmentInsert(offset int64, entityIDs *[]UniqueID, timestamps *[]Timestamp, records *[]*commonpb.Blob) error {
/*
CStatus
Insert(CSegmentInterface c_segment,
long int reserved_offset,
signed long int size,
const long* primary_keys,
const unsigned long* timestamps,
void* raw_data,
int sizeof_per_row,
signed long int count);
*/
// Blobs to one big blob
var numOfRow = len(*entityIDs)
var sizeofPerRow = len((*records)[0].Value)
assert.Equal(nil, numOfRow, len(*records))
var rawData = make([]byte, numOfRow*sizeofPerRow)
var copyOffset = 0
for i := 0; i < len(*records); i++ {
copy(rawData[copyOffset:], (*records)[i].Value)
copyOffset += sizeofPerRow
}
var cOffset = C.long(offset)
var cNumOfRows = C.long(numOfRow)
var cEntityIdsPtr = (*C.long)(&(*entityIDs)[0])
var cTimestampsPtr = (*C.ulong)(&(*timestamps)[0])
var cSizeofPerRow = C.int(sizeofPerRow)
var cRawDataVoidPtr = unsafe.Pointer(&rawData[0])
var status = C.Insert(s.segmentPtr,
cOffset,
cNumOfRows,
cEntityIdsPtr,
cTimestampsPtr,
cRawDataVoidPtr,
cSizeofPerRow,
cNumOfRows)
errorCode := status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return errors.New("Insert failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
s.SetRecentlyModified(true)
return nil
}
func (s *Segment) segmentDelete(offset int64, entityIDs *[]UniqueID, timestamps *[]Timestamp) error {
/*
CStatus
Delete(CSegmentInterface c_segment,
long int reserved_offset,
long size,
const long* primary_keys,
const unsigned long* timestamps);
*/
var cOffset = C.long(offset)
var cSize = C.long(len(*entityIDs))
var cEntityIdsPtr = (*C.long)(&(*entityIDs)[0])
var cTimestampsPtr = (*C.ulong)(&(*timestamps)[0])
var status = C.Delete(s.segmentPtr, cOffset, cSize, cEntityIdsPtr, cTimestampsPtr)
errorCode := status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return errors.New("Delete failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
return nil
}
//-------------------------------------------------------------------------------------- interfaces for sealed segment
func (s *Segment) segmentLoadFieldData(fieldID int64, rowCount int, data unsafe.Pointer) error {
/*
CStatus
LoadFieldData(CSegmentInterface c_segment, CLoadFieldDataInfo load_field_data_info);
*/
if s.segmentType != segTypeSealed {
return errors.New("illegal segment type when loading field data")
}
/*
struct CLoadFieldDataInfo {
int64_t field_id;
void* blob;
int64_t row_count;
};
*/
loadInfo := C.CLoadFieldDataInfo{
field_id: C.int64_t(fieldID),
blob: data,
row_count: C.int64_t(rowCount),
}
var status = C.LoadFieldData(s.segmentPtr, loadInfo)
errorCode := status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return errors.New("LoadFieldData failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
return nil
}

View File

@ -3,7 +3,7 @@ package querynode
import (
"context"
"errors"
"unsafe"
"fmt"
"github.com/zilliztech/milvus-distributed/internal/kv"
miniokv "github.com/zilliztech/milvus-distributed/internal/kv/minio"
@ -58,7 +58,7 @@ func newSegmentManager(ctx context.Context, replica collectionReplica, loadIndex
}
}
func (s *segmentManager) loadSegment(segmentID UniqueID, partitionID UniqueID, collectionID UniqueID, fieldIDs *[]int64) error {
func (s *segmentManager) loadSegment(segmentID UniqueID, fieldIDs *[]int64) error {
insertBinlogPathRequest := &datapb.InsertBinlogPathRequest{
SegmentID: segmentID,
}
@ -72,12 +72,6 @@ func (s *segmentManager) loadSegment(segmentID UniqueID, partitionID UniqueID, c
return errors.New("illegal InsertBinlogPathsResponse")
}
// create segment
err = s.replica.addSegment(segmentID, partitionID, collectionID, segTypeSealed)
if err != nil {
return err
}
containsFunc := func(s []int64, e int64) bool {
for _, a := range s {
if a == e {
@ -116,34 +110,26 @@ func (s *segmentManager) loadSegment(segmentID UniqueID, partitionID UniqueID, c
}
for _, value := range insertData.Data {
var numRows int
var data interface{}
switch fieldData := value.(type) {
case storage.BoolFieldData:
numRows = fieldData.NumRows
data = fieldData.Data
numRows := fieldData.NumRows
data := fieldData.Data
fmt.Println(numRows, data, fieldID)
// TODO: s.replica.addSegment()
case storage.Int8FieldData:
numRows = fieldData.NumRows
data = fieldData.Data
// TODO: s.replica.addSegment()
case storage.Int16FieldData:
numRows = fieldData.NumRows
data = fieldData.Data
// TODO: s.replica.addSegment()
case storage.Int32FieldData:
numRows = fieldData.NumRows
data = fieldData.Data
// TODO: s.replica.addSegment()
case storage.Int64FieldData:
numRows = fieldData.NumRows
data = fieldData.Data
// TODO: s.replica.addSegment()
case storage.FloatFieldData:
numRows = fieldData.NumRows
data = fieldData.Data
// TODO: s.replica.addSegment()
case storage.DoubleFieldData:
numRows = fieldData.NumRows
data = fieldData.Data
// TODO: s.replica.addSegment()
case storage.StringFieldData:
numRows = fieldData.NumRows
data = fieldData.Data
// TODO: s.replica.addSegment()
case storage.FloatVectorFieldData:
// segment to be loaded doesn't need vector field,
// so we ignore the type of vector field data
@ -153,17 +139,6 @@ func (s *segmentManager) loadSegment(segmentID UniqueID, partitionID UniqueID, c
default:
return errors.New("unexpected field data type")
}
segment, err := s.replica.getSegmentByID(segmentID)
if err != nil {
// TODO: return or continue?
return err
}
err = segment.segmentLoadFieldData(fieldID, numRows, unsafe.Pointer(&data))
if err != nil {
// TODO: return or continue?
return err
}
}
}
@ -178,17 +153,23 @@ func (s *segmentManager) loadIndex(segmentID UniqueID, indexID UniqueID) error {
if err != nil || pathResponse.Status.ErrorCode != commonpb.ErrorCode_SUCCESS {
return err
}
// get vector field ids from schema to load index
vecFieldIDs, err := s.replica.getVecFieldsBySegmentID(segmentID)
targetSegment, err := s.replica.getSegmentByID(segmentID)
if err != nil {
return err
}
for id, name := range vecFieldIDs {
var targetIndexParam indexParam
// TODO: get index param from master
// get vector field ids from schema to load index
vecFieldIDs, err := s.replica.getVecFieldIDsBySegmentID(segmentID)
if err != nil {
return err
}
for _, vecFieldID := range vecFieldIDs {
targetIndexParam, ok := targetSegment.indexParam[vecFieldID]
if !ok {
return errors.New(fmt.Sprint("cannot found index params in segment ", segmentID, " with field = ", vecFieldID))
}
// non-blocking send
go s.sendLoadIndex(pathResponse.IndexFilePaths, segmentID, id, name, targetIndexParam)
go s.sendLoadIndex(pathResponse.IndexFilePaths, segmentID, vecFieldID, "", targetIndexParam)
}
return nil

View File

@ -24,7 +24,7 @@ func TestSegment_newSegment(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID)
deleteSegment(segment)
deleteCollection(collection)
@ -40,7 +40,7 @@ func TestSegment_deleteSegment(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID)
deleteSegment(segment)
@ -58,7 +58,7 @@ func TestSegment_getRowCount(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3}
@ -107,7 +107,7 @@ func TestSegment_getDeletedCount(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3}
@ -162,7 +162,7 @@ func TestSegment_getMemSize(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3}
@ -211,7 +211,7 @@ func TestSegment_segmentInsert(t *testing.T) {
assert.Equal(t, collection.Name(), collectionName)
assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3}
@ -256,7 +256,7 @@ func TestSegment_segmentDelete(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3}
@ -307,7 +307,7 @@ func TestSegment_segmentSearch(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3}
@ -390,7 +390,7 @@ func TestSegment_segmentPreInsert(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID)
const DIM = 16
@ -430,7 +430,7 @@ func TestSegment_segmentPreDelete(t *testing.T) {
assert.Equal(t, collection.ID(), collectionID)
segmentID := UniqueID(0)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
segment := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3}

View File

@ -19,6 +19,8 @@ import (
"strconv"
"strings"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/spf13/cast"
"github.com/spf13/viper"
memkv "github.com/zilliztech/milvus-distributed/internal/kv/mem"
@ -61,6 +63,18 @@ func (gp *BaseTable) Init() {
}
func (gp *BaseTable) LoadFromKVPair(kvPairs []*commonpb.KeyValuePair) error {
for _, pair := range kvPairs {
err := gp.Save(pair.Key, pair.Value)
if err != nil {
return err
}
}
return nil
}
func (gp *BaseTable) tryloadFromEnv() {
minioAddress := os.Getenv("MINIO_ADDRESS")