Add loadFieldData, use partition id instead of tag

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
pull/4973/head^2
bigsheeper 2021-01-19 14:44:03 +08:00 committed by yefu.chen
parent 76a7684d96
commit 2064b014a1
36 changed files with 6694 additions and 495 deletions

View File

@ -21,6 +21,8 @@ enum SegmentType {
Sealed = 2,
};
typedef enum SegmentType SegmentType;
enum ErrorCode {
Success = 0,
UnexpectedException = 1,
@ -31,6 +33,12 @@ 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

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -21,8 +21,9 @@
#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, int seg_type) {
NewSegment(CCollection collection, uint64_t segment_id, SegmentType seg_type) {
auto col = (milvus::segcore::Collection*)collection;
std::unique_ptr<milvus::segcore::SegmentInterface> segment;
@ -46,6 +47,7 @@ NewSegment(CCollection collection, uint64_t segment_id, int 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;
@ -58,8 +60,86 @@ 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,
@ -123,70 +203,33 @@ 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
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]);
}
LoadFieldData(CSegmentInterface c_segment, CLoadFieldDataInfo load_field_data_info) {
auto segment = (milvus::segcore::SegmentSealed*)c_segment;
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;
}
}
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();
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();
@ -203,7 +246,6 @@ UpdateSegmentIndex(CSegmentInterface c_segment, CLoadIndexInfo c_load_index_info
return status;
}
}
//////////////////////////////////////////////////////////////////
int
Close(CSegmentInterface c_segment) {
@ -223,26 +265,3 @@ 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,8 +26,9 @@ extern "C" {
typedef void* CSegmentInterface;
typedef void* CQueryResult;
////////////////////////////// common interfaces //////////////////////////////
CSegmentInterface
NewSegment(CCollection collection, uint64_t segment_id, int seg_type);
NewSegment(CCollection collection, uint64_t segment_id, SegmentType seg_type);
void
DeleteSegment(CSegmentInterface segment);
@ -35,9 +36,27 @@ 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);
// interface for growing segment
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 //////////////////////////////
CStatus
Insert(CSegmentInterface c_segment,
int64_t reserved_offset,
@ -64,23 +83,13 @@ Delete(CSegmentInterface c_segment,
int64_t
PreDelete(CSegmentInterface c_segment, int64_t size);
// common interface
////////////////////////////// interfaces for growing segment //////////////////////////////
CStatus
Search(CSegmentInterface c_segment,
CPlan plan,
CPlaceholderGroup* placeholder_groups,
uint64_t* timestamps,
int num_groups,
CQueryResult* result);
LoadFieldData(CSegmentInterface c_segment, CLoadFieldDataInfo load_field_data_info);
// common interface
CStatus
FillTargetEntry(CSegmentInterface c_segment, CPlan c_plan, CQueryResult result);
// deprecated
////////////////////////////// deprecated interfaces //////////////////////////////
CStatus
UpdateSegmentIndex(CSegmentInterface c_segment, CLoadIndexInfo c_load_index_info);
//////////////////////////////////////////////////////////////////
// deprecated
int
@ -94,20 +103,6 @@ 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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
// auto segment = NewSegment(collection, 0, Growing);
//
// 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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
// auto segment = NewSegment(collection, 0, Growing);
//
// 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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
// auto segment = NewSegment(collection, 0, Growing);
// 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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
auto segment = NewSegment(collection, 0, Growing);
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, 1);
auto segment = NewSegment(collection, 0, Growing);
auto N = 1000 * 1000;
auto dataset = DataGen(schema, N);

View File

@ -0,0 +1,136 @@
package masterservice
import (
"context"
"time"
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/masterpb"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"google.golang.org/grpc"
)
// grpc client
type GrpcClient struct {
grpcClient masterpb.MasterServiceClient
conn *grpc.ClientConn
//inner member
addr string
timeout time.Duration
retry int
}
func NewGrpcClient(addr string, timeout time.Duration) (*GrpcClient, error) {
return &GrpcClient{
grpcClient: nil,
conn: nil,
addr: addr,
timeout: timeout,
retry: 3,
}, nil
}
func (c *GrpcClient) Init(params *cms.InitParams) error {
ctx, cancel := context.WithTimeout(context.Background(), c.timeout)
defer cancel()
var err error
for i := 0; i < c.retry; i++ {
if c.conn, err = grpc.DialContext(ctx, c.addr, grpc.WithInsecure(), grpc.WithBlock()); err == nil {
break
}
}
if err != nil {
return err
}
c.grpcClient = masterpb.NewMasterServiceClient(c.conn)
return nil
}
func (c *GrpcClient) Start() error {
return nil
}
func (c *GrpcClient) Stop() error {
return c.conn.Close()
}
//TODO, grpc, get service state from server
func (c *GrpcClient) GetServiceStates() (*internalpb2.ServiceStates, error) {
return nil, nil
}
//DDL request
func (c *GrpcClient) CreateCollection(in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
return c.grpcClient.CreateCollection(context.Background(), in)
}
func (c *GrpcClient) DropCollection(in *milvuspb.DropCollectionRequest) (*commonpb.Status, error) {
return c.grpcClient.DropCollection(context.Background(), in)
}
func (c *GrpcClient) HasCollection(in *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) {
return c.grpcClient.HasCollection(context.Background(), in)
}
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)
}
//index builder service
func (c *GrpcClient) CreateIndex(in *milvuspb.CreateIndexRequest) (*commonpb.Status, error) {
return c.grpcClient.CreateIndex(context.Background(), in)
}
func (c *GrpcClient) DescribeIndex(in *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) {
return c.grpcClient.DescribeIndex(context.Background(), in)
}
//global timestamp allocator
func (c *GrpcClient) AllocTimestamp(in *masterpb.TsoRequest) (*masterpb.TsoResponse, error) {
return c.grpcClient.AllocTimestamp(context.Background(), in)
}
func (c *GrpcClient) AllocID(in *masterpb.IDRequest) (*masterpb.IDResponse, error) {
return c.grpcClient.AllocID(context.Background(), in)
}
//receiver time tick from proxy service, and put it into this channel
func (c *GrpcClient) GetTimeTickChannel(empty *commonpb.Empty) (*milvuspb.StringResponse, error) {
return c.grpcClient.GetTimeTickChannel(context.Background(), empty)
}
//receive ddl from rpc and time tick from proxy service, and put them into this channel
func (c *GrpcClient) GetDdChannel(in *commonpb.Empty) (*milvuspb.StringResponse, error) {
return c.grpcClient.GetDdChannel(context.Background(), in)
}
//just define a channel, not used currently
func (c *GrpcClient) GetStatisticsChannel(empty *commonpb.Empty) (*milvuspb.StringResponse, error) {
return c.grpcClient.GetStatisticsChannel(context.Background(), empty)
}
func (c *GrpcClient) DescribeSegment(in *milvuspb.DescribeSegmentRequest) (*milvuspb.DescribeSegmentResponse, error) {
return c.grpcClient.DescribeSegment(context.Background(), in)
}
func (c *GrpcClient) ShowSegments(in *milvuspb.ShowSegmentRequest) (*milvuspb.ShowSegmentResponse, error) {
return c.grpcClient.ShowSegments(context.Background(), in)
}

View File

@ -0,0 +1,430 @@
package masterservice
import (
"fmt"
"math/rand"
"testing"
"time"
"github.com/golang/protobuf/proto"
"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/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"
)
func TestGrpcService(t *testing.T) {
rand.Seed(time.Now().UnixNano())
randVal := rand.Int()
cms.Params.Address = "127.0.0.1"
cms.Params.Port = (randVal % 100) + 10000
cms.Params.NodeID = 0
cms.Params.PulsarAddress = "pulsar://127.0.0.1:6650"
cms.Params.EtcdAddress = "127.0.0.1:2379"
cms.Params.MetaRootPath = fmt.Sprintf("/%d/test/meta", randVal)
cms.Params.KvRootPath = fmt.Sprintf("/%d/test/kv", randVal)
cms.Params.ProxyTimeTickChannel = fmt.Sprintf("proxyTimeTick%d", randVal)
cms.Params.MsgChannelSubName = fmt.Sprintf("msgChannel%d", randVal)
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.MaxPartitionNum = 64
cms.Params.DefaultPartitionTag = "_default"
t.Logf("master service port = %d", cms.Params.Port)
svr, err := NewGrpcServer()
assert.Nil(t, err)
core := svr.core.(*cms.Core)
core.ProxyTimeTickChan = make(chan typeutil.Timestamp, 8)
timeTickArray := make([]typeutil.Timestamp, 0, 16)
core.SendTimeTick = func(ts typeutil.Timestamp) error {
t.Logf("send time tick %d", ts)
timeTickArray = append(timeTickArray, ts)
return nil
}
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([]*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([]*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([]*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)
cli, err := NewGrpcClient(fmt.Sprintf("127.0.0.1:%d", cms.Params.Port), 3*time.Second)
assert.Nil(t, err)
err = cli.Init(&cms.InitParams{ProxyTimeTickChannel: fmt.Sprintf("proxyTimeTick%d", randVal)})
assert.Nil(t, err)
err = cli.Start()
assert.Nil(t, err)
t.Run("create collection", func(t *testing.T) {
schema := schemapb.CollectionSchema{
Name: "testColl",
Description: "testColl",
AutoID: true,
Fields: []*schemapb.FieldSchema{
{
FieldID: 100,
Name: "vector",
IsPrimaryKey: false,
Description: "vector",
DataType: schemapb.DataType_VECTOR_FLOAT,
TypeParams: nil,
IndexParams: nil,
},
},
}
sbf, err := proto.Marshal(&schema)
assert.Nil(t, err)
req := &milvuspb.CreateCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kCreateCollection,
MsgID: 100,
Timestamp: 100,
SourceID: 100,
},
DbName: "testDb",
CollectionName: "testColl",
Schema: sbf,
}
status, err := cli.CreateCollection(req)
assert.Nil(t, err)
assert.Equal(t, len(createCollectionArray), 1)
assert.Equal(t, status.ErrorCode, commonpb.ErrorCode_SUCCESS)
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) {
req := &milvuspb.HasCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kHasCollection,
MsgID: 101,
Timestamp: 101,
SourceID: 101,
},
DbName: "testDb",
CollectionName: "testColl",
}
rsp, err := cli.HasCollection(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, rsp.Value, true)
req = &milvuspb.HasCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kHasCollection,
MsgID: 102,
Timestamp: 102,
SourceID: 102,
},
DbName: "testDb",
CollectionName: "testColl2",
}
rsp, err = cli.HasCollection(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, rsp.Value, false)
req = &milvuspb.HasCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kHasCollection,
MsgID: 102,
Timestamp: 102,
SourceID: 102,
},
DbName: "testDb",
CollectionName: "testColl2",
}
rsp, err = cli.HasCollection(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_UNEXPECTED_ERROR)
})
t.Run("describe collection", func(t *testing.T) {
req := &milvuspb.DescribeCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kDescribeCollection,
MsgID: 103,
Timestamp: 103,
SourceID: 103,
},
DbName: "testDb",
CollectionName: "testColl",
}
rsp, err := cli.DescribeCollection(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
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{
MsgType: commonpb.MsgType_kShowCollections,
MsgID: 106,
Timestamp: 106,
SourceID: 106,
},
DbName: "testDb",
}
rsp, err := cli.ShowCollections(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, rsp.CollectionNames[0], "testColl")
assert.Equal(t, len(rsp.CollectionNames), 1)
})
t.Run("create partition", func(t *testing.T) {
req := &milvuspb.CreatePartitionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kCreatePartition,
MsgID: 107,
Timestamp: 107,
SourceID: 107,
},
DbName: "testDb",
CollectionName: "testColl",
PartitionName: "testPartition",
}
status, err := cli.CreatePartition(req)
assert.Nil(t, err)
assert.Equal(t, status.ErrorCode, commonpb.ErrorCode_SUCCESS)
collMeta, err := core.MetaTable.GetCollectionByName("testColl")
assert.Nil(t, err)
assert.Equal(t, len(collMeta.PartitionIDs), 2)
assert.Equal(t, collMeta.PartitionTags[1], "testPartition")
})
t.Run("has partition", func(t *testing.T) {
req := &milvuspb.HasPartitionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kHasPartition,
MsgID: 108,
Timestamp: 108,
SourceID: 108,
},
DbName: "testDb",
CollectionName: "testColl",
PartitionName: "testPartition",
}
rsp, err := cli.HasPartition(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
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{
MsgType: commonpb.MsgType_kShowPartitions,
MsgID: 110,
Timestamp: 110,
SourceID: 110,
},
DbName: "testDb",
CollectionName: "testColl",
}
rsp, err := cli.ShowPartitions(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, len(rsp.PartitionNames), 2)
})
t.Run("drop partition", func(t *testing.T) {
req := &milvuspb.DropPartitionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kDropPartition,
MsgID: 199,
Timestamp: 199,
SourceID: 199,
},
DbName: "testDb",
CollectionName: "testColl",
PartitionName: "testPartition",
}
status, err := cli.DropPartition(req)
assert.Nil(t, err)
assert.Equal(t, status.ErrorCode, commonpb.ErrorCode_SUCCESS)
collMeta, err := core.MetaTable.GetCollectionByName("testColl")
assert.Nil(t, err)
assert.Equal(t, len(collMeta.PartitionIDs), 1)
assert.Equal(t, collMeta.PartitionTags[0], cms.Params.DefaultPartitionTag)
})
t.Run("drop collection", func(t *testing.T) {
req := &milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kDropCollection,
MsgID: 200,
Timestamp: 200,
SourceID: 200,
},
DbName: "testDb",
CollectionName: "testColl",
}
status, err := cli.DropCollection(req)
assert.Nil(t, err)
assert.Equal(t, len(dropCollectionArray), 1)
assert.Equal(t, status.ErrorCode, commonpb.ErrorCode_SUCCESS)
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{
MsgType: commonpb.MsgType_kDropCollection,
MsgID: 200,
Timestamp: 200,
SourceID: 200,
},
DbName: "testDb",
CollectionName: "testColl",
}
status, err = cli.DropCollection(req)
assert.Nil(t, err)
assert.Equal(t, len(dropCollectionArray), 1)
assert.Equal(t, status.ErrorCode, commonpb.ErrorCode_UNEXPECTED_ERROR)
})
err = cli.Stop()
assert.Nil(t, err)
err = svr.Stop()
assert.Nil(t, err)
}

View File

@ -0,0 +1,174 @@
package masterservice
import (
"context"
"fmt"
"net"
"sync"
cms "github.com/zilliztech/milvus-distributed/internal/masterservice"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"google.golang.org/grpc"
)
// grpc wrapper
type GrpcServer struct {
core cms.Interface
grpcServer *grpc.Server
grpcError error
grpcErrMux sync.Mutex
ctx context.Context
cancel context.CancelFunc
}
func NewGrpcServer() (*GrpcServer, error) {
s := &GrpcServer{}
var err error
s.ctx, s.cancel = context.WithCancel(context.Background())
if s.core, err = cms.NewCore(s.ctx); err != nil {
return nil, err
}
s.grpcServer = grpc.NewServer()
s.grpcError = nil
masterpb.RegisterMasterServiceServer(s.grpcServer, s)
lis, err := net.Listen("tcp", fmt.Sprintf(":%d", cms.Params.Port))
if err != nil {
return nil, err
}
go func() {
if err := s.grpcServer.Serve(lis); err != nil {
s.grpcErrMux.Lock()
defer s.grpcErrMux.Unlock()
s.grpcError = err
}
}()
s.grpcErrMux.Lock()
err = s.grpcError
s.grpcErrMux.Unlock()
if err != nil {
return nil, err
}
return s, nil
}
func (s *GrpcServer) Init(params *cms.InitParams) error {
return s.core.Init(params)
}
func (s *GrpcServer) Start() error {
return s.core.Start()
}
func (s *GrpcServer) Stop() error {
err := s.core.Stop()
s.cancel()
s.grpcServer.GracefulStop()
return err
}
func (s *GrpcServer) GetServiceStates(ctx context.Context, empty *commonpb.Empty) (*internalpb2.ServiceStates, error) {
return nil, nil
}
//DDL request
func (s *GrpcServer) CreateCollection(ctx context.Context, in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
return s.core.CreateCollection(in)
}
func (s *GrpcServer) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRequest) (*commonpb.Status, error) {
return s.core.DropCollection(in)
}
func (s *GrpcServer) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) {
return s.core.HasCollection(in)
}
func (s *GrpcServer) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
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)
}
func (s *GrpcServer) CreatePartition(ctx context.Context, in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) {
return s.core.CreatePartition(in)
}
func (s *GrpcServer) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequest) (*commonpb.Status, error) {
return s.core.DropPartition(in)
}
func (s *GrpcServer) HasPartition(ctx context.Context, in *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
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)
}
//index builder service
func (s *GrpcServer) CreateIndex(ctx context.Context, in *milvuspb.CreateIndexRequest) (*commonpb.Status, error) {
return s.core.CreateIndex(in)
}
func (s *GrpcServer) DescribeIndex(ctx context.Context, in *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) {
return s.core.DescribeIndex(in)
}
//global timestamp allocator
func (s *GrpcServer) AllocTimestamp(ctx context.Context, in *masterpb.TsoRequest) (*masterpb.TsoResponse, error) {
return s.core.AllocTimestamp(in)
}
func (s *GrpcServer) AllocID(ctx context.Context, in *masterpb.IDRequest) (*masterpb.IDResponse, error) {
return s.core.AllocID(in)
}
//receiver time tick from proxy service, and put it into this channel
func (s *GrpcServer) GetTimeTickChannel(ctx context.Context, empty *commonpb.Empty) (*milvuspb.StringResponse, error) {
return s.core.GetTimeTickChannel(empty)
}
//receive ddl from rpc and time tick from proxy service, and put them into this channel
func (s *GrpcServer) GetDdChannel(ctx context.Context, in *commonpb.Empty) (*milvuspb.StringResponse, error) {
return s.core.GetDdChannel(in)
}
//just define a channel, not used currently
func (s *GrpcServer) GetStatisticsChannel(ctx context.Context, empty *commonpb.Empty) (*milvuspb.StringResponse, error) {
return s.core.GetStatisticsChannel(empty)
}
func (s *GrpcServer) DescribeSegment(ctx context.Context, in *milvuspb.DescribeSegmentRequest) (*milvuspb.DescribeSegmentResponse, error) {
return s.core.DescribeSegment(in)
}
func (s *GrpcServer) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentRequest) (*milvuspb.ShowSegmentResponse, error) {
return s.core.ShowSegments(in)
}
//TODO, move to query node
func (s *GrpcServer) GetIndexState(ctx context.Context, request *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error) {
panic("implement me")
}
//TODO, move to data service
func (s *GrpcServer) AssignSegmentID(ctx context.Context, request *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error) {
panic("implement me")
}

View File

@ -495,3 +495,7 @@ func (s *Master) DescribeSegment(ctx context.Context, request *milvuspb.Describe
func (s *Master) ShowSegments(ctx context.Context, request *milvuspb.ShowSegmentRequest) (*milvuspb.ShowSegmentResponse, error) {
panic("implement me")
}
func (s *Master) GetDdChannel(ctx context.Context, empty *commonpb.Empty) (*milvuspb.StringResponse, error) {
panic("implement me")
}

View File

@ -0,0 +1,16 @@
package masterservice
// system filed id:
// 0: unique row id
// 1: timestamp
// 100: first user field id
// 101: second user field id
// 102: ...
const (
StartOfUserFieldID = 100
RowIDField = 0
TimeStampField = 1
RowIDFieldName = "RowID"
TimeStampFieldName = "Timestamp"
)

View File

@ -0,0 +1,118 @@
package masterservice
import (
"log"
"sync/atomic"
"time"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"go.uber.org/zap"
)
// Allocator is a Timestamp Oracle allocator.
type Allocator interface {
// Initialize is used to initialize a TSO allocator.
// It will synchronize TSO with etcd and initialize the
// memory for later allocation work.
Initialize() error
// UpdateTSO is used to update the TSO in memory and the time window in etcd.
UpdateTSO() error
// SetTSO sets the physical part with given tso. It's mainly used for BR restore
// and can not forcibly set the TSO smaller than now.
SetTSO(tso uint64) error
// GenerateTSO is used to generate a given number of TSOs.
// Make sure you have initialized the TSO allocator before calling.
GenerateTSO(count uint32) (uint64, error)
// Reset is used to reset the TSO allocator.
Reset()
}
// GlobalTSOAllocator is the global single point TSO allocator.
type GlobalTSOAllocator struct {
tso *timestampOracle
}
// NewGlobalTSOAllocator creates a new global TSO allocator.
func NewGlobalTSOAllocator(key string, kvBase kv.TxnBase) *GlobalTSOAllocator {
var saveInterval = 3 * time.Second
return &GlobalTSOAllocator{
tso: &timestampOracle{
kvBase: kvBase,
saveInterval: saveInterval,
maxResetTSGap: func() time.Duration { return 3 * time.Second },
key: key,
},
}
}
// Initialize will initialize the created global TSO allocator.
func (gta *GlobalTSOAllocator) Initialize() error {
return gta.tso.InitTimestamp()
}
// UpdateTSO is used to update the TSO in memory and the time window in etcd.
func (gta *GlobalTSOAllocator) UpdateTSO() error {
return gta.tso.UpdateTimestamp()
}
// SetTSO sets the physical part with given tso.
func (gta *GlobalTSOAllocator) SetTSO(tso uint64) error {
return gta.tso.ResetUserTimestamp(tso)
}
// GenerateTSO is used to generate a given number of TSOs.
// Make sure you have initialized the TSO allocator before calling.
func (gta *GlobalTSOAllocator) GenerateTSO(count uint32) (uint64, error) {
var physical, logical int64
if count == 0 {
return 0, errors.New("tso count should be positive")
}
maxRetryCount := 10
for i := 0; i < maxRetryCount; i++ {
current := (*atomicObject)(atomic.LoadPointer(&gta.tso.TSO))
if current == nil || current.physical.Equal(typeutil.ZeroTime) {
// If it's leader, maybe SyncTimestamp hasn't completed yet
log.Println("sync hasn't completed yet, wait for a while")
time.Sleep(200 * time.Millisecond)
continue
}
physical = current.physical.UnixNano() / int64(time.Millisecond)
logical = atomic.AddInt64(&current.logical, int64(count))
if logical >= maxLogical {
log.Println("logical part outside of max logical interval, please check ntp time",
zap.Int("retry-count", i))
time.Sleep(UpdateTimestampStep)
continue
}
return tsoutil.ComposeTS(physical, logical), nil
}
return 0, errors.New("can not get timestamp")
}
func (gta *GlobalTSOAllocator) Alloc(count uint32) (typeutil.Timestamp, error) {
//return gta.tso.SyncTimestamp()
start, err := gta.GenerateTSO(count)
if err != nil {
return typeutil.ZeroTimestamp, err
}
//ret := make([]typeutil.Timestamp, count)
//for i:=uint32(0); i < count; i++{
// ret[i] = start + uint64(i)
//}
return start, err
}
func (gta *GlobalTSOAllocator) AllocOne() (typeutil.Timestamp, error) {
return gta.GenerateTSO(1)
}
// Reset is used to reset the TSO allocator.
func (gta *GlobalTSOAllocator) Reset() {
gta.tso.ResetTimestamp()
}

View File

@ -0,0 +1,47 @@
package masterservice
import (
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
)
// GlobalTSOAllocator is the global single point TSO allocator.
type GlobalIDAllocator struct {
allocator Allocator
}
func NewGlobalIDAllocator(key string, base kv.TxnBase) *GlobalIDAllocator {
return &GlobalIDAllocator{
allocator: NewGlobalTSOAllocator(key, base),
}
}
// Initialize will initialize the created global TSO allocator.
func (gia *GlobalIDAllocator) Initialize() error {
return gia.allocator.Initialize()
}
// GenerateTSO is used to generate a given number of TSOs.
// Make sure you have initialized the TSO allocator before calling.
func (gia *GlobalIDAllocator) Alloc(count uint32) (typeutil.UniqueID, typeutil.UniqueID, error) {
timestamp, err := gia.allocator.GenerateTSO(count)
if err != nil {
return 0, 0, err
}
idStart := typeutil.UniqueID(timestamp)
idEnd := idStart + int64(count)
return idStart, idEnd, nil
}
func (gia *GlobalIDAllocator) AllocOne() (typeutil.UniqueID, error) {
timestamp, err := gia.allocator.GenerateTSO(1)
if err != nil {
return 0, err
}
idStart := typeutil.UniqueID(timestamp)
return idStart, nil
}
func (gia *GlobalIDAllocator) UpdateID() error {
return gia.allocator.UpdateTSO()
}

View File

@ -0,0 +1,715 @@
package masterservice
import (
"context"
"log"
"math/rand"
"sync"
"sync/atomic"
"time"
"github.com/zilliztech/milvus-distributed/internal/errors"
etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd"
"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"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"go.etcd.io/etcd/clientv3"
)
// internalpb2 -> internalpb
// proxypb(proxy_service)
// querypb(query_service)
// datapb(data_service)
// indexpb(index_service)
// milvuspb -> servicepb
// masterpb2 -> masterpb master_service)
type InitParams struct {
ProxyTimeTickChannel string
}
type Service interface {
Init(params *InitParams) error
Start() error
Stop() error
GetServiceStates() (*internalpb2.ServiceStates, error)
GetTimeTickChannel() (string, error)
GetStatesChannel() (string, error)
}
type Interface interface {
//service
Init(params *InitParams) error
Start() error
Stop() error
GetServiceStates(empty *commonpb.Empty) (*internalpb2.ServiceStates, error)
//DDL request
CreateCollection(in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error)
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
CreateIndex(in *milvuspb.CreateIndexRequest) (*commonpb.Status, error)
DescribeIndex(in *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error)
//global timestamp allocator
AllocTimestamp(in *masterpb.TsoRequest) (*masterpb.TsoResponse, error)
AllocID(in *masterpb.IDRequest) (*masterpb.IDResponse, error)
//TODO, master load these channel form config file ?
//receiver time tick from proxy service, and put it into this channel
GetTimeTickChannel(empty *commonpb.Empty) (*milvuspb.StringResponse, error)
//receive ddl from rpc and time tick from proxy service, and put them into this channel
GetDdChannel(empty *commonpb.Empty) (*milvuspb.StringResponse, error)
//just define a channel, not used currently
GetStatisticsChannel(empty *commonpb.Empty) (*milvuspb.StringResponse, error)
//segment
DescribeSegment(in *milvuspb.DescribeSegmentRequest) (*milvuspb.DescribeSegmentResponse, error)
ShowSegments(in *milvuspb.ShowSegmentRequest) (*milvuspb.ShowSegmentResponse, error)
//get system config from master, not used currently
//GetSysConfigs(in *milvuspb.SysConfigRequest)
//GetIndexState(ctx context.Context, request *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error)
}
// ------------------ struct -----------------------
// master core
type Core struct {
//TODO DataService Interface
//TODO IndexService Interface
//TODO ProxyServiceClient Interface, get proxy service time tick channel,InvalidateCollectionMetaCache
//TODO Segment States Channel, from DataService, if create new segment, data service should put the segment id into this channel, and let the master add the segment id to the collection meta
//TODO Segment Flush Watcher, monitor if segment has flushed into disk
//TODO indexBuilder Sch, tell index service to build index
MetaTable *metaTable
//id allocator
idAllocator *GlobalIDAllocator
//tso allocator
tsoAllocator *GlobalTSOAllocator
//inner members
ctx context.Context
cancel context.CancelFunc
etcdCli *clientv3.Client
kvBase *etcdkv.EtcdKV
metaKV *etcdkv.EtcdKV
//TODO, receive time tick from proxy service time tick channel
ProxyTimeTickChan <-chan typeutil.Timestamp
//TODO, send time tick into dd channel and time tick channel
SendTimeTick func(t typeutil.Timestamp) error
//TODO, send create collection into dd channel
DdCreateCollectionReq func(req *CreateCollectionReqTask) error
//TODO, send drop collection into dd channel, and notify the proxy to delete this collection
DdDropCollectionReq func(req *DropCollectionReqTask) error
//TODO, send create partition into dd channel
DdCreatePartitionReq func(req *CreatePartitionReqTask) 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
lastDdTimeStamp typeutil.Timestamp
//time tick loop
lastTimeTick typeutil.Timestamp
//states code
stateCode atomic.Value
//call once
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 --------------------------
func NewCore(c context.Context) (*Core, error) {
ctx, cancel := context.WithCancel(c)
rand.Seed(time.Now().UnixNano())
Params.Init()
core := &Core{
ctx: ctx,
cancel: cancel,
}
core.stateCode.Store(internalpb2.StateCode_INITIALIZING)
core.isInit.Store(false)
return core, nil
}
func (c *Core) checkInit() error {
if c.MetaTable == nil {
return errors.Errorf("MetaTable is nil")
}
if c.idAllocator == nil {
return errors.Errorf("idAllocator is nil")
}
if c.tsoAllocator == nil {
return errors.Errorf("tsoAllocator is nil")
}
if c.etcdCli == nil {
return errors.Errorf("etcdCli is nil")
}
if c.metaKV == nil {
return errors.Errorf("metaKV is nil")
}
if c.kvBase == nil {
return errors.Errorf("kvBase is nil")
}
if c.ProxyTimeTickChan == nil {
return errors.Errorf("ProxyTimeTickChan is nil")
}
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")
}
if c.DdDropCollectionReq == nil {
return errors.Errorf("DdDropCollectionReq is nil")
}
if c.DdCreatePartitionReq == nil {
return errors.Errorf("DdCreatePartitionReq is nil")
}
if c.DdDropPartitionReq == nil {
return errors.Errorf("DdDropPartitionReq is nil")
}
log.Printf("master node id = %d\n", Params.NodeID)
return nil
}
func (c *Core) startDdScheduler() {
for {
select {
case <-c.ctx.Done():
log.Printf("close dd scheduler, exit task execution loop")
return
case task, ok := <-c.ddReqQueue:
if !ok {
log.Printf("dd chan is closed, exit task execution loopo")
return
}
ts, err := task.Ts()
if err != nil {
task.Notify(err)
break
}
if ts <= c.lastDdTimeStamp {
task.Notify(errors.Errorf("input timestamp = %d, last dd time stamp = %d", ts, c.lastDdTimeStamp))
break
}
err = task.Execute()
task.Notify(err)
c.lastDdTimeStamp = ts
}
}
}
func (c *Core) startTimeTickLoop() {
for {
select {
case <-c.ctx.Done():
log.Printf("close master time tick loop")
return
case tt, ok := <-c.ProxyTimeTickChan:
if !ok {
log.Printf("proxyTimeTickStream is closed, exit time tick loop")
return
}
if tt <= c.lastTimeTick {
log.Printf("master time tick go back, last time tick = %d, input time tick = %d", c.lastTimeTick, tt)
}
if err := c.SendTimeTick(tt); err != nil {
log.Printf("master send time tick into dd and time_tick channel failed: %s", err.Error())
}
c.lastTimeTick = tt
}
}
}
func (c *Core) Init(params *InitParams) error {
var initError error = nil
c.initOnce.Do(func() {
Params.ProxyTimeTickChannel = params.ProxyTimeTickChannel
if c.etcdCli, initError = clientv3.New(clientv3.Config{Endpoints: []string{Params.EtcdAddress}, DialTimeout: 5 * time.Second}); initError != nil {
return
}
c.metaKV = etcdkv.NewEtcdKV(c.etcdCli, Params.MetaRootPath)
if c.MetaTable, initError = NewMetaTable(c.metaKV); initError != nil {
return
}
c.kvBase = etcdkv.NewEtcdKV(c.etcdCli, Params.KvRootPath)
c.idAllocator = NewGlobalIDAllocator("idTimestamp", tsoutil.NewTSOKVBase([]string{Params.EtcdAddress}, Params.KvRootPath, "gid"))
if initError = c.idAllocator.Initialize(); initError != nil {
return
}
c.tsoAllocator = NewGlobalTSOAllocator("timestamp", tsoutil.NewTSOKVBase([]string{Params.EtcdAddress}, Params.KvRootPath, "tso"))
if initError = c.tsoAllocator.Initialize(); initError != nil {
return
}
c.ddReqQueue = make(chan reqTask, 1024)
c.isInit.Store(true)
})
return initError
}
func (c *Core) Start() error {
isInit := c.isInit.Load().(bool)
if !isInit {
return errors.Errorf("call init before start")
}
if err := c.checkInit(); err != nil {
return err
}
c.startOnce.Do(func() {
go c.startDdScheduler()
go c.startTimeTickLoop()
c.stateCode.Store(internalpb2.StateCode_HEALTHY)
})
return nil
}
func (c *Core) Stop() error {
c.cancel()
c.stateCode.Store(internalpb2.StateCode_ABNORMAL)
return nil
}
func (c *Core) GetServiceStates(empty *commonpb.Empty) (*internalpb2.ServiceStates, error) {
code := c.stateCode.Load().(internalpb2.StateCode)
return &internalpb2.ServiceStates{
StateCode: code,
NodeStates: []*internalpb2.NodeStates{
{
NodeID: int64(Params.NodeID),
Role: "master",
StateCode: code,
ExtraInfo: nil,
},
},
ExtraInfo: nil,
}, nil
}
func (c *Core) GetTimeTickChannel(empty *commonpb.Empty) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
},
Value: Params.TimeTickChannel,
}, nil
}
func (c *Core) GetDdChannel(empty *commonpb.Empty) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
},
Value: Params.DdChannel,
}, nil
}
func (c *Core) GetStatisticsChannel(empty *commonpb.Empty) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
},
Value: Params.StatisticsChannel,
}, nil
}
func (c *Core) CreateCollection(in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
t := &CreateCollectionReqTask{
baseReqTask: baseReqTask{
cv: make(chan error),
core: c,
},
Req: in,
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "Create collection failed: " + err.Error(),
}, nil
}
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}, nil
}
func (c *Core) DropCollection(in *milvuspb.DropCollectionRequest) (*commonpb.Status, error) {
t := &DropCollectionReqTask{
baseReqTask: baseReqTask{
cv: make(chan error),
core: c,
},
Req: in,
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "Create collection failed: " + err.Error(),
}, nil
}
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}, nil
}
func (c *Core) HasCollection(in *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) {
t := &HasCollectionReqTask{
baseReqTask: baseReqTask{
cv: make(chan error),
core: c,
},
Req: in,
HasCollection: false,
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &milvuspb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "Has collection failed: " + err.Error(),
},
Value: false,
}, nil
}
return &milvuspb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
},
Value: t.HasCollection,
}, nil
}
func (c *Core) DescribeCollection(in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
t := &DescribeCollectionReqTask{
baseReqTask: baseReqTask{
cv: make(chan error),
core: c,
},
Req: in,
Rsp: &milvuspb.DescribeCollectionResponse{},
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &milvuspb.DescribeCollectionResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "describe collection failed: " + err.Error(),
},
Schema: nil,
}, nil
}
t.Rsp.Status = &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}
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{
cv: make(chan error),
core: c,
},
Req: in,
Rsp: &milvuspb.ShowCollectionResponse{
CollectionNames: nil,
},
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &milvuspb.ShowCollectionResponse{
CollectionNames: nil,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "ShowCollections failed: " + err.Error(),
},
}, nil
}
t.Rsp.Status = &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}
return t.Rsp, nil
}
func (c *Core) CreatePartition(in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) {
t := &CreatePartitionReqTask{
baseReqTask: baseReqTask{
cv: make(chan error),
core: c,
},
Req: in,
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "create partition failed: " + err.Error(),
}, nil
}
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}, nil
}
func (c *Core) DropPartition(in *milvuspb.DropPartitionRequest) (*commonpb.Status, error) {
t := &DropPartitionReqTask{
baseReqTask: baseReqTask{
cv: make(chan error),
core: c,
},
Req: in,
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "DropPartition failed: " + err.Error(),
}, nil
}
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}, nil
}
func (c *Core) HasPartition(in *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
t := &HasPartitionReqTask{
baseReqTask: baseReqTask{
cv: make(chan error),
core: c,
},
Req: in,
HasPartition: false,
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &milvuspb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "HasPartition failed: " + err.Error(),
},
Value: false,
}, nil
}
return &milvuspb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
},
Value: t.HasPartition,
}, 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{
cv: make(chan error),
core: c,
},
Req: in,
Rsp: &milvuspb.ShowPartitionResponse{
PartitionNames: nil,
Status: nil,
},
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &milvuspb.ShowPartitionResponse{
PartitionNames: nil,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "ShowPartitions failed: " + err.Error(),
},
}, nil
}
t.Rsp.Status = &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}
return t.Rsp, nil
}
//TODO
func (c *Core) CreateIndex(in *milvuspb.CreateIndexRequest) (*commonpb.Status, error) {
return nil, nil
}
//TODO
func (c *Core) DescribeIndex(in *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) {
return nil, nil
}
//TODO
func (c *Core) DescribeSegment(in *milvuspb.DescribeSegmentRequest) (*milvuspb.DescribeSegmentResponse, error) {
return nil, nil
}
//TODO
func (c *Core) ShowSegments(in *milvuspb.ShowSegmentRequest) (*milvuspb.ShowSegmentResponse, error) {
return nil, nil
}
func (c *Core) AllocTimestamp(in *masterpb.TsoRequest) (*masterpb.TsoResponse, error) {
ts, err := c.tsoAllocator.Alloc(in.Count)
if err != nil {
return &masterpb.TsoResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "AllocTimestamp failed: " + err.Error(),
},
Timestamp: 0,
Count: 0,
}, nil
}
return &masterpb.TsoResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
},
Timestamp: ts,
Count: in.Count,
}, nil
}
func (c *Core) AllocID(in *masterpb.IDRequest) (*masterpb.IDResponse, error) {
start, _, err := c.idAllocator.Alloc(in.Count)
if err != nil {
return &masterpb.IDResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "AllocID failed: " + err.Error(),
},
ID: 0,
Count: in.Count,
}, nil
}
return &masterpb.IDResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
},
ID: start,
Count: in.Count,
}, nil
}

View File

@ -0,0 +1,719 @@
package masterservice
import (
"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/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"
)
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.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) {
mt := &metaTable{
client: kv,
tenantLock: sync.RWMutex{},
proxyLock: sync.RWMutex{},
ddLock: sync.RWMutex{},
}
err := mt.reloadFromKV()
if err != nil {
return nil, err
}
return mt, nil
}
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.CollectionMeta)
mt.collName2ID = make(map[string]typeutil.UniqueID)
mt.segID2Meta = make(map[typeutil.UniqueID]pb.SegmentMeta)
mt.segID2IndexMetas = make(map[typeutil.UniqueID][]pb.FieldIndexMeta)
_, values, err := mt.client.LoadWithPrefix("tenant")
if err != nil {
return err
}
for _, value := range values {
tenantMeta := pb.TenantMeta{}
err := proto.UnmarshalText(value, &tenantMeta)
if err != nil {
return err
}
mt.tenantID2Meta[tenantMeta.ID] = tenantMeta
}
_, values, err = mt.client.LoadWithPrefix("proxy")
if err != nil {
return err
}
for _, value := range values {
proxyMeta := pb.ProxyMeta{}
err = proto.UnmarshalText(value, &proxyMeta)
if err != nil {
return err
}
mt.proxyID2Meta[proxyMeta.ID] = proxyMeta
}
_, values, err = mt.client.LoadWithPrefix("collection")
if err != nil {
return err
}
for _, value := range values {
collectionMeta := pb.CollectionMeta{}
err = proto.UnmarshalText(value, &collectionMeta)
if err != nil {
return err
}
mt.collID2Meta[collectionMeta.ID] = collectionMeta
mt.collName2ID[collectionMeta.Schema.Name] = collectionMeta.ID
}
_, values, err = mt.client.LoadWithPrefix("segment")
if err != nil {
return err
}
for _, value := range values {
segmentMeta := pb.SegmentMeta{}
err = proto.UnmarshalText(value, &segmentMeta)
if err != nil {
return err
}
mt.segID2Meta[segmentMeta.SegmentID] = segmentMeta
}
_, values, err = mt.client.LoadWithPrefix("indexmeta")
if err != nil {
return err
}
for _, v := range values {
indexMeta := pb.FieldIndexMeta{}
err = proto.UnmarshalText(v, &indexMeta)
if err != nil {
return err
}
mt.segID2IndexMetas[indexMeta.SegmentID] = append(mt.segID2IndexMetas[indexMeta.SegmentID], indexMeta)
}
return nil
}
// 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(coll.SegmentIDs) != 0 {
return errors.Errorf("segment should be empty when creating collection")
}
if len(coll.PartitionTags) == 0 {
coll.PartitionTags = append(coll.PartitionTags, Params.DefaultPartitionTag)
}
_, 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
}
return nil
}
func (mt *metaTable) DeleteCollection(collID typeutil.UniqueID) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
collMeta, ok := mt.collID2Meta[collID]
if !ok {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
}
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
}
func (mt *metaTable) HasCollection(collID typeutil.UniqueID) bool {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
_, ok := mt.collID2Meta[collID]
return ok
}
func (mt *metaTable) GetCollectionByName(collectionName string) (*pb.CollectionMeta, error) {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
vid, ok := mt.collName2ID[collectionName]
if !ok {
return nil, errors.Errorf("can't find collection: " + collectionName)
}
col, ok := mt.collID2Meta[vid]
if !ok {
return nil, errors.Errorf("can't find collection: " + collectionName)
}
return &col, nil
}
func (mt *metaTable) ListCollections() ([]string, error) {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
colls := make([]string, 0, len(mt.collName2ID))
for name := range mt.collName2ID {
colls = append(colls, name)
}
return colls, nil
}
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]
if !ok {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
}
// number of partition tags (except _default) should be limited to 4096 by default
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.PartitionTags {
if t == partitionTag {
return errors.Errorf("partition already exists.")
}
}
coll.PartitionTags = append(coll.PartitionTags, partitionTag)
coll.PartitionIDs = append(coll.PartitionIDs, partitionID)
err := mt.saveCollectionMeta(&coll)
if err != nil {
_ = mt.reloadFromKV()
return err
}
return nil
}
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 _, partitionTag := range col.PartitionTags {
if partitionTag == tag {
return true
}
}
return false
}
func (mt *metaTable) DeletePartition(collID typeutil.UniqueID, tag string) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
if tag == Params.DefaultPartitionTag {
return errors.New("default partition cannot be deleted")
}
collMeta, ok := mt.collID2Meta[collID]
if !ok {
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))
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 errors.New("partition " + tag + " does not exist")
}
if len(pt) == len(collMeta.PartitionTags) {
return nil
}
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 {
return errors.Errorf("DeletePartition:can't find segment id = %d", s)
}
if sm.PartitionTag != tag {
seg = append(seg, s)
} else {
toDeleteSeg = append(toDeleteSeg, s)
}
}
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

@ -0,0 +1,28 @@
package masterservice
import (
"github.com/zilliztech/milvus-distributed/internal/util/paramtable"
)
var Params ParamTable
type ParamTable struct {
paramtable.BaseTable
Address string
Port int
NodeID uint64
PulsarAddress string
EtcdAddress string
MetaRootPath string
KvRootPath string
ProxyTimeTickChannel string
MsgChannelSubName string
TimeTickChannel string
DdChannel string
StatisticsChannel string
MaxPartitionNum int64
DefaultPartitionTag string
}

View File

@ -0,0 +1,421 @@
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/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
)
type reqTask interface {
Type() commonpb.MsgType
Ts() (typeutil.Timestamp, error)
Execute() error
WaitToFinish() error
Notify(err error)
}
type baseReqTask struct {
cv chan error
core *Core
}
func (bt *baseReqTask) Notify(err error) {
bt.cv <- err
}
func (bt *baseReqTask) WaitToFinish() error {
select {
case <-bt.core.ctx.Done():
return errors.Errorf("context done")
case err, ok := <-bt.cv:
if !ok {
return errors.Errorf("notify chan closed")
}
return err
}
}
type CreateCollectionReqTask struct {
baseReqTask
Req *milvuspb.CreateCollectionRequest
}
func (t *CreateCollectionReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
func (t *CreateCollectionReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
func (t *CreateCollectionReqTask) Execute() error {
var schema schemapb.CollectionSchema
err := proto.Unmarshal(t.Req.Schema, &schema)
if err != nil {
return err
}
for idx, field := range schema.Fields {
field.FieldID = int64(idx + StartOfUserFieldID)
}
rowIDField := &schemapb.FieldSchema{
FieldID: int64(RowIDField),
Name: RowIDFieldName,
IsPrimaryKey: false,
Description: "row id",
DataType: schemapb.DataType_INT64,
}
timeStampField := &schemapb.FieldSchema{
FieldID: int64(TimeStampField),
Name: TimeStampFieldName,
IsPrimaryKey: false,
Description: "time stamp",
DataType: schemapb.DataType_INT64,
}
schema.Fields = append(schema.Fields, rowIDField, timeStampField)
collID, err := t.core.idAllocator.AllocOne()
if err != nil {
return err
}
collTs, err := t.Ts()
if err != nil {
return err
}
partitionID, err := t.core.idAllocator.AllocOne()
if err != nil {
return err
}
coll := etcdpb.CollectionMeta{
ID: collID,
Schema: &schema,
CreateTime: collTs,
SegmentIDs: make([]typeutil.UniqueID, 0),
PartitionTags: []string{Params.DefaultPartitionTag},
PartitionIDs: []typeutil.UniqueID{partitionID},
}
err = t.core.MetaTable.AddCollection(&coll)
if err != nil {
return err
}
err = t.core.DdCreateCollectionReq(t)
if err != nil {
return err
}
return nil
}
type DropCollectionReqTask struct {
baseReqTask
Req *milvuspb.DropCollectionRequest
}
func (t *DropCollectionReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
func (t *DropCollectionReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
func (t *DropCollectionReqTask) Execute() error {
collMeta, err := t.core.MetaTable.GetCollectionByName(t.Req.CollectionName)
if err != nil {
return err
}
err = t.core.MetaTable.DeleteCollection(collMeta.ID)
if err != nil {
return err
}
//data service should drop segments , which belong to this collection, from the segment manager
err = t.core.DdDropCollectionReq(t)
if err != nil {
return err
}
return nil
}
type HasCollectionReqTask struct {
baseReqTask
Req *milvuspb.HasCollectionRequest
HasCollection bool
}
func (t *HasCollectionReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
func (t *HasCollectionReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
func (t *HasCollectionReqTask) Execute() error {
_, err := t.core.MetaTable.GetCollectionByName(t.Req.CollectionName)
if err == nil {
t.HasCollection = true
} else {
t.HasCollection = false
}
return nil
}
type DescribeCollectionReqTask struct {
baseReqTask
Req *milvuspb.DescribeCollectionRequest
Rsp *milvuspb.DescribeCollectionResponse
}
func (t *DescribeCollectionReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
func (t *DescribeCollectionReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
func (t *DescribeCollectionReqTask) Execute() error {
coll, err := t.core.MetaTable.GetCollectionByName(t.Req.CollectionName)
if err != nil {
return err
}
t.Rsp.Schema = proto.Clone(coll.Schema).(*schemapb.CollectionSchema)
var newField []*schemapb.FieldSchema
for _, field := range t.Rsp.Schema.Fields {
if field.FieldID >= StartOfUserFieldID {
newField = append(newField, field)
}
}
t.Rsp.Schema.Fields = newField
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
Rsp *milvuspb.ShowCollectionResponse
}
func (t *ShowCollectionReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
func (t *ShowCollectionReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
func (t *ShowCollectionReqTask) Execute() error {
coll, err := t.core.MetaTable.ListCollections()
if err != nil {
return err
}
t.Rsp.CollectionNames = coll
return nil
}
type CreatePartitionReqTask struct {
baseReqTask
Req *milvuspb.CreatePartitionRequest
}
func (t *CreatePartitionReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
func (t *CreatePartitionReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
func (t *CreatePartitionReqTask) Execute() error {
collMeta, err := t.core.MetaTable.GetCollectionByName(t.Req.CollectionName)
if err != nil {
return err
}
partitionID, err := t.core.idAllocator.AllocOne()
if err != nil {
return err
}
err = t.core.MetaTable.AddPartition(collMeta.ID, t.Req.PartitionName, partitionID)
if err != nil {
return err
}
err = t.core.DdCreatePartitionReq(t)
if err != nil {
return err
}
return nil
}
type DropPartitionReqTask struct {
baseReqTask
Req *milvuspb.DropPartitionRequest
}
func (t *DropPartitionReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
func (t *DropPartitionReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
func (t *DropPartitionReqTask) Execute() error {
coll, err := t.core.MetaTable.GetCollectionByName(t.Req.CollectionName)
if err != nil {
return err
}
err = t.core.MetaTable.DeletePartition(coll.ID, t.Req.PartitionName)
if err != nil {
return err
}
err = t.core.DdDropPartitionReq(t)
if err != nil {
return err
}
return nil
}
type HasPartitionReqTask struct {
baseReqTask
Req *milvuspb.HasPartitionRequest
HasPartition bool
}
func (t *HasPartitionReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
func (t *HasPartitionReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
func (t *HasPartitionReqTask) Execute() error {
coll, err := t.core.MetaTable.GetCollectionByName(t.Req.CollectionName)
if err != nil {
return err
}
t.HasPartition = t.core.MetaTable.HasPartition(coll.ID, t.Req.PartitionName)
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
Rsp *milvuspb.ShowPartitionResponse
}
func (t *ShowPartitionReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
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
}
t.Rsp.PartitionNames = append(t.Rsp.PartitionNames, coll.PartitionTags...)
return nil
}

View File

@ -0,0 +1,189 @@
package masterservice
import (
"log"
"sync/atomic"
"time"
"unsafe"
"go.uber.org/zap"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
)
const (
// UpdateTimestampStep is used to update timestamp.
UpdateTimestampStep = 50 * time.Millisecond
// updateTimestampGuard is the min timestamp interval.
updateTimestampGuard = time.Millisecond
// maxLogical is the max upper limit for logical time.
// When a TSO's logical time reaches this limit,
// the physical time will be forced to increase.
maxLogical = int64(1 << 18)
)
// atomicObject is used to store the current TSO in memory.
type atomicObject struct {
physical time.Time
logical int64
}
// timestampOracle is used to maintain the logic of tso.
type timestampOracle struct {
key string
kvBase kv.TxnBase
// TODO: remove saveInterval
saveInterval time.Duration
maxResetTSGap func() time.Duration
// For tso, set after the PD becomes a leader.
TSO unsafe.Pointer
lastSavedTime atomic.Value
}
func (t *timestampOracle) loadTimestamp() (time.Time, error) {
strData, err := t.kvBase.Load(t.key)
var binData []byte = []byte(strData)
if err != nil {
return typeutil.ZeroTime, err
}
if len(binData) == 0 {
return typeutil.ZeroTime, nil
}
return typeutil.ParseTimestamp(binData)
}
// save timestamp, if lastTs is 0, we think the timestamp doesn't exist, so create it,
// otherwise, update it.
func (t *timestampOracle) saveTimestamp(ts time.Time) error {
data := typeutil.Uint64ToBytes(uint64(ts.UnixNano()))
err := t.kvBase.Save(t.key, string(data))
if err != nil {
return errors.WithStack(err)
}
t.lastSavedTime.Store(ts)
return nil
}
func (t *timestampOracle) InitTimestamp() error {
//last, err := t.loadTimestamp()
//if err != nil {
// return err
//}
next := time.Now()
// If the current system time minus the saved etcd timestamp is less than `updateTimestampGuard`,
// the timestamp allocation will start from the saved etcd timestamp temporarily.
//if typeutil.SubTimeByWallClock(next, last) < updateTimestampGuard {
// next = last.Add(updateTimestampGuard)
//}
save := next.Add(t.saveInterval)
if err := t.saveTimestamp(save); err != nil {
return err
}
//log.Print("sync and save timestamp", zap.Time("last", last), zap.Time("save", save), zap.Time("next", next))
current := &atomicObject{
physical: next,
}
atomic.StorePointer(&t.TSO, unsafe.Pointer(current))
return nil
}
// ResetUserTimestamp update the physical part with specified tso.
func (t *timestampOracle) ResetUserTimestamp(tso uint64) error {
physical, _ := tsoutil.ParseTS(tso)
next := physical.Add(time.Millisecond)
prev := (*atomicObject)(atomic.LoadPointer(&t.TSO))
// do not update
if typeutil.SubTimeByWallClock(next, prev.physical) <= 3*updateTimestampGuard {
return errors.New("the specified ts too small than now")
}
if typeutil.SubTimeByWallClock(next, prev.physical) >= t.maxResetTSGap() {
return errors.New("the specified ts too large than now")
}
save := next.Add(t.saveInterval)
if err := t.saveTimestamp(save); err != nil {
return err
}
update := &atomicObject{
physical: next,
}
atomic.CompareAndSwapPointer(&t.TSO, unsafe.Pointer(prev), unsafe.Pointer(update))
return nil
}
// UpdateTimestamp is used to update the timestamp.
// This function will do two things:
// 1. When the logical time is going to be used up, increase the current physical time.
// 2. When the time window is not big enough, which means the saved etcd time minus the next physical time
// will be less than or equal to `updateTimestampGuard`, then the time window needs to be updated and
// we also need to save the next physical time plus `TsoSaveInterval` into etcd.
//
// Here is some constraints that this function must satisfy:
// 1. The saved time is monotonically increasing.
// 2. The physical time is monotonically increasing.
// 3. The physical time is always less than the saved timestamp.
func (t *timestampOracle) UpdateTimestamp() error {
prev := (*atomicObject)(atomic.LoadPointer(&t.TSO))
now := time.Now()
jetLag := typeutil.SubTimeByWallClock(now, prev.physical)
if jetLag > 3*UpdateTimestampStep {
log.Print("clock offset", zap.Duration("jet-lag", jetLag), zap.Time("prev-physical", prev.physical), zap.Time("now", now))
}
var next time.Time
prevLogical := atomic.LoadInt64(&prev.logical)
// If the system time is greater, it will be synchronized with the system time.
if jetLag > updateTimestampGuard {
next = now
} else if prevLogical > maxLogical/2 {
// The reason choosing maxLogical/2 here is that it's big enough for common cases.
// Because there is enough timestamp can be allocated before next update.
log.Print("the logical time may be not enough", zap.Int64("prev-logical", prevLogical))
next = prev.physical.Add(time.Millisecond)
} else {
// It will still use the previous physical time to alloc the timestamp.
return nil
}
// It is not safe to increase the physical time to `next`.
// The time window needs to be updated and saved to etcd.
if typeutil.SubTimeByWallClock(t.lastSavedTime.Load().(time.Time), next) <= updateTimestampGuard {
save := next.Add(t.saveInterval)
if err := t.saveTimestamp(save); err != nil {
return err
}
}
current := &atomicObject{
physical: next,
logical: 0,
}
atomic.StorePointer(&t.TSO, unsafe.Pointer(current))
return nil
}
// ResetTimestamp is used to reset the timestamp.
func (t *timestampOracle) ResetTimestamp() {
zero := &atomicObject{
physical: time.Now(),
}
atomic.StorePointer(&t.TSO, unsafe.Pointer(zero))
}

View File

@ -18,6 +18,32 @@ message ProxyMeta {
repeated string result_channelIDs = 3;
}
message PartitionInfo {
string partition_name = 1;
int64 partitionID = 2;
repeated int64 segmentIDs = 3;
}
message CollectionInfo {
int64 ID = 1;
schema.CollectionSchema schema = 2;
uint64 create_time = 3;
repeated int64 partitionIDs = 4;
}
message IndexInfo {
string index_name = 1;
int64 indexID = 2;
repeated common.KeyValuePair index_params = 3;
}
message SegmentIndexInfo {
int64 segmentID = 1;
int64 fieldID = 2;
int64 indexID = 3;
int64 buildID = 4;
}
message CollectionMeta {
int64 ID=1;
schema.CollectionSchema schema=2;

View File

@ -140,6 +140,242 @@ func (m *ProxyMeta) GetResultChannelIDs() []string {
return nil
}
type PartitionInfo struct {
PartitionName string `protobuf:"bytes,1,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"`
PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"`
SegmentIDs []int64 `protobuf:"varint,3,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *PartitionInfo) Reset() { *m = PartitionInfo{} }
func (m *PartitionInfo) String() string { return proto.CompactTextString(m) }
func (*PartitionInfo) ProtoMessage() {}
func (*PartitionInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_975d306d62b73e88, []int{2}
}
func (m *PartitionInfo) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PartitionInfo.Unmarshal(m, b)
}
func (m *PartitionInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_PartitionInfo.Marshal(b, m, deterministic)
}
func (m *PartitionInfo) XXX_Merge(src proto.Message) {
xxx_messageInfo_PartitionInfo.Merge(m, src)
}
func (m *PartitionInfo) XXX_Size() int {
return xxx_messageInfo_PartitionInfo.Size(m)
}
func (m *PartitionInfo) XXX_DiscardUnknown() {
xxx_messageInfo_PartitionInfo.DiscardUnknown(m)
}
var xxx_messageInfo_PartitionInfo proto.InternalMessageInfo
func (m *PartitionInfo) GetPartitionName() string {
if m != nil {
return m.PartitionName
}
return ""
}
func (m *PartitionInfo) GetPartitionID() int64 {
if m != nil {
return m.PartitionID
}
return 0
}
func (m *PartitionInfo) GetSegmentIDs() []int64 {
if m != nil {
return m.SegmentIDs
}
return nil
}
type CollectionInfo struct {
ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
Schema *schemapb.CollectionSchema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"`
CreateTime uint64 `protobuf:"varint,3,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"`
PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *CollectionInfo) Reset() { *m = CollectionInfo{} }
func (m *CollectionInfo) String() string { return proto.CompactTextString(m) }
func (*CollectionInfo) ProtoMessage() {}
func (*CollectionInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_975d306d62b73e88, []int{3}
}
func (m *CollectionInfo) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CollectionInfo.Unmarshal(m, b)
}
func (m *CollectionInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_CollectionInfo.Marshal(b, m, deterministic)
}
func (m *CollectionInfo) XXX_Merge(src proto.Message) {
xxx_messageInfo_CollectionInfo.Merge(m, src)
}
func (m *CollectionInfo) XXX_Size() int {
return xxx_messageInfo_CollectionInfo.Size(m)
}
func (m *CollectionInfo) XXX_DiscardUnknown() {
xxx_messageInfo_CollectionInfo.DiscardUnknown(m)
}
var xxx_messageInfo_CollectionInfo proto.InternalMessageInfo
func (m *CollectionInfo) GetID() int64 {
if m != nil {
return m.ID
}
return 0
}
func (m *CollectionInfo) GetSchema() *schemapb.CollectionSchema {
if m != nil {
return m.Schema
}
return nil
}
func (m *CollectionInfo) GetCreateTime() uint64 {
if m != nil {
return m.CreateTime
}
return 0
}
func (m *CollectionInfo) GetPartitionIDs() []int64 {
if m != nil {
return m.PartitionIDs
}
return nil
}
type IndexInfo struct {
IndexName string `protobuf:"bytes,1,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"`
IndexID int64 `protobuf:"varint,2,opt,name=indexID,proto3" json:"indexID,omitempty"`
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,3,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *IndexInfo) Reset() { *m = IndexInfo{} }
func (m *IndexInfo) String() string { return proto.CompactTextString(m) }
func (*IndexInfo) ProtoMessage() {}
func (*IndexInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_975d306d62b73e88, []int{4}
}
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) GetIndexName() string {
if m != nil {
return m.IndexName
}
return ""
}
func (m *IndexInfo) GetIndexID() int64 {
if m != nil {
return m.IndexID
}
return 0
}
func (m *IndexInfo) GetIndexParams() []*commonpb.KeyValuePair {
if m != nil {
return m.IndexParams
}
return nil
}
type SegmentIndexInfo struct {
SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
FieldID int64 `protobuf:"varint,2,opt,name=fieldID,proto3" json:"fieldID,omitempty"`
IndexID int64 `protobuf:"varint,3,opt,name=indexID,proto3" json:"indexID,omitempty"`
BuildID int64 `protobuf:"varint,4,opt,name=buildID,proto3" json:"buildID,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *SegmentIndexInfo) Reset() { *m = SegmentIndexInfo{} }
func (m *SegmentIndexInfo) String() string { return proto.CompactTextString(m) }
func (*SegmentIndexInfo) ProtoMessage() {}
func (*SegmentIndexInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_975d306d62b73e88, []int{5}
}
func (m *SegmentIndexInfo) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SegmentIndexInfo.Unmarshal(m, b)
}
func (m *SegmentIndexInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_SegmentIndexInfo.Marshal(b, m, deterministic)
}
func (m *SegmentIndexInfo) XXX_Merge(src proto.Message) {
xxx_messageInfo_SegmentIndexInfo.Merge(m, src)
}
func (m *SegmentIndexInfo) XXX_Size() int {
return xxx_messageInfo_SegmentIndexInfo.Size(m)
}
func (m *SegmentIndexInfo) XXX_DiscardUnknown() {
xxx_messageInfo_SegmentIndexInfo.DiscardUnknown(m)
}
var xxx_messageInfo_SegmentIndexInfo proto.InternalMessageInfo
func (m *SegmentIndexInfo) GetSegmentID() int64 {
if m != nil {
return m.SegmentID
}
return 0
}
func (m *SegmentIndexInfo) GetFieldID() int64 {
if m != nil {
return m.FieldID
}
return 0
}
func (m *SegmentIndexInfo) GetIndexID() int64 {
if m != nil {
return m.IndexID
}
return 0
}
func (m *SegmentIndexInfo) GetBuildID() int64 {
if m != nil {
return m.BuildID
}
return 0
}
type CollectionMeta struct {
ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
Schema *schemapb.CollectionSchema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"`
@ -156,7 +392,7 @@ func (m *CollectionMeta) Reset() { *m = CollectionMeta{} }
func (m *CollectionMeta) String() string { return proto.CompactTextString(m) }
func (*CollectionMeta) ProtoMessage() {}
func (*CollectionMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_975d306d62b73e88, []int{2}
return fileDescriptor_975d306d62b73e88, []int{6}
}
func (m *CollectionMeta) XXX_Unmarshal(b []byte) error {
@ -231,7 +467,7 @@ func (m *FieldBinlogFiles) Reset() { *m = FieldBinlogFiles{} }
func (m *FieldBinlogFiles) String() string { return proto.CompactTextString(m) }
func (*FieldBinlogFiles) ProtoMessage() {}
func (*FieldBinlogFiles) Descriptor() ([]byte, []int) {
return fileDescriptor_975d306d62b73e88, []int{3}
return fileDescriptor_975d306d62b73e88, []int{7}
}
func (m *FieldBinlogFiles) XXX_Unmarshal(b []byte) error {
@ -286,7 +522,7 @@ func (m *SegmentMeta) Reset() { *m = SegmentMeta{} }
func (m *SegmentMeta) String() string { return proto.CompactTextString(m) }
func (*SegmentMeta) ProtoMessage() {}
func (*SegmentMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_975d306d62b73e88, []int{4}
return fileDescriptor_975d306d62b73e88, []int{8}
}
func (m *SegmentMeta) XXX_Unmarshal(b []byte) error {
@ -393,7 +629,7 @@ func (m *FieldIndexMeta) Reset() { *m = FieldIndexMeta{} }
func (m *FieldIndexMeta) String() string { return proto.CompactTextString(m) }
func (*FieldIndexMeta) ProtoMessage() {}
func (*FieldIndexMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_975d306d62b73e88, []int{5}
return fileDescriptor_975d306d62b73e88, []int{9}
}
func (m *FieldIndexMeta) XXX_Unmarshal(b []byte) error {
@ -459,6 +695,10 @@ func (m *FieldIndexMeta) GetIndexFilePaths() []string {
func init() {
proto.RegisterType((*TenantMeta)(nil), "milvus.proto.etcd.TenantMeta")
proto.RegisterType((*ProxyMeta)(nil), "milvus.proto.etcd.ProxyMeta")
proto.RegisterType((*PartitionInfo)(nil), "milvus.proto.etcd.PartitionInfo")
proto.RegisterType((*CollectionInfo)(nil), "milvus.proto.etcd.CollectionInfo")
proto.RegisterType((*IndexInfo)(nil), "milvus.proto.etcd.IndexInfo")
proto.RegisterType((*SegmentIndexInfo)(nil), "milvus.proto.etcd.SegmentIndexInfo")
proto.RegisterType((*CollectionMeta)(nil), "milvus.proto.etcd.CollectionMeta")
proto.RegisterType((*FieldBinlogFiles)(nil), "milvus.proto.etcd.FieldBinlogFiles")
proto.RegisterType((*SegmentMeta)(nil), "milvus.proto.etcd.SegmentMeta")
@ -468,50 +708,56 @@ func init() {
func init() { proto.RegisterFile("etcd_meta.proto", fileDescriptor_975d306d62b73e88) }
var fileDescriptor_975d306d62b73e88 = []byte{
// 707 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0x41, 0x6f, 0xda, 0x48,
0x14, 0x16, 0x38, 0x40, 0xfc, 0x20, 0x90, 0xcc, 0xc9, 0x9b, 0xcd, 0x6e, 0x08, 0x51, 0x76, 0x91,
0x56, 0x0b, 0x12, 0xab, 0xed, 0xad, 0x55, 0x9b, 0xd0, 0x48, 0xa8, 0x6a, 0x43, 0x0d, 0xea, 0xa1,
0x17, 0x6b, 0xb0, 0x5f, 0x60, 0x24, 0xcf, 0x98, 0x7a, 0xc6, 0x4d, 0xc2, 0xa9, 0xd7, 0xfe, 0x85,
0xfe, 0xc5, 0xfe, 0x86, 0x4a, 0xd5, 0xcc, 0x38, 0x06, 0x5a, 0x0e, 0xbd, 0x79, 0xbe, 0xf7, 0xbd,
0x37, 0xdf, 0xfb, 0xbe, 0x31, 0xb4, 0x50, 0x85, 0x51, 0xc0, 0x51, 0xd1, 0xde, 0x32, 0x4d, 0x54,
0x42, 0x8e, 0x38, 0x8b, 0x3f, 0x66, 0xd2, 0x9e, 0x7a, 0xba, 0x7a, 0xdc, 0x08, 0x13, 0xce, 0x13,
0x61, 0xa1, 0xe3, 0x86, 0x0c, 0x17, 0xc8, 0x73, 0x7a, 0xe7, 0x4b, 0x09, 0x60, 0x8a, 0x82, 0x0a,
0xf5, 0x1a, 0x15, 0x25, 0x4d, 0x28, 0x8f, 0x86, 0x5e, 0xa9, 0x5d, 0xea, 0x3a, 0x7e, 0x79, 0x34,
0x24, 0x7f, 0x41, 0x4b, 0x64, 0x3c, 0xf8, 0x90, 0x61, 0xfa, 0x10, 0x88, 0x24, 0x42, 0xe9, 0x95,
0x4d, 0xf1, 0x40, 0x64, 0xfc, 0xad, 0x46, 0xdf, 0x68, 0x90, 0xfc, 0x03, 0x47, 0x4c, 0x48, 0x4c,
0x55, 0x10, 0x2e, 0xa8, 0x10, 0x18, 0x8f, 0x86, 0xd2, 0x73, 0xda, 0x4e, 0xd7, 0xf5, 0x0f, 0x6d,
0xe1, 0xaa, 0xc0, 0xc9, 0xdf, 0xd0, 0xb2, 0x03, 0x0b, 0xae, 0xb7, 0xd7, 0x2e, 0x75, 0x5d, 0xbf,
0x69, 0xe0, 0x82, 0xd9, 0xf9, 0x54, 0x02, 0x77, 0x9c, 0x26, 0xf7, 0x0f, 0x3b, 0xb5, 0x3d, 0x81,
0x1a, 0x8d, 0xa2, 0x14, 0xa5, 0xd5, 0x54, 0x1f, 0x9c, 0xf4, 0xb6, 0x76, 0xcf, 0xb7, 0x7e, 0x61,
0x39, 0xfe, 0x23, 0x59, 0x6b, 0x4d, 0x51, 0x66, 0xf1, 0x2e, 0xad, 0xb6, 0xb0, 0xd6, 0xda, 0xf9,
0x5a, 0x82, 0xe6, 0x55, 0x12, 0xc7, 0x18, 0x2a, 0x96, 0x88, 0x9d, 0x3a, 0x9e, 0x42, 0xd5, 0x5a,
0x9a, 0xcb, 0xb8, 0xd8, 0x96, 0x91, 0xdb, 0xbd, 0x1e, 0x32, 0x31, 0x80, 0x9f, 0x37, 0x91, 0x53,
0xa8, 0x87, 0x29, 0x52, 0x85, 0x81, 0x62, 0x1c, 0x3d, 0xa7, 0x5d, 0xea, 0xee, 0xf9, 0x60, 0xa1,
0x29, 0xe3, 0x48, 0xfe, 0x04, 0x90, 0x38, 0xe7, 0x28, 0x94, 0x16, 0xba, 0xd7, 0x76, 0xba, 0x8e,
0xbf, 0x81, 0x90, 0x0b, 0x68, 0x2e, 0x69, 0xaa, 0x98, 0x9e, 0x1d, 0x28, 0x3a, 0x97, 0x5e, 0xc5,
0x2c, 0x73, 0x50, 0xa0, 0x53, 0x3a, 0x97, 0xa4, 0x03, 0x8d, 0x02, 0xd0, 0x83, 0xaa, 0x66, 0xd0,
0x16, 0xd6, 0xb9, 0x81, 0xc3, 0x6b, 0x86, 0x71, 0x74, 0xc9, 0x44, 0x9c, 0xcc, 0xaf, 0x59, 0x8c,
0x92, 0x78, 0x50, 0xbb, 0xd5, 0x58, 0xb1, 0xf3, 0xe3, 0x91, 0x9c, 0x41, 0x63, 0x66, 0x88, 0xc1,
0xad, 0x66, 0x7a, 0x65, 0x73, 0x6d, 0x7d, 0xb6, 0x6e, 0xee, 0x7c, 0x2b, 0x43, 0x7d, 0x62, 0xa5,
0x1a, 0xef, 0x4e, 0xc0, 0x2d, 0x94, 0xe7, 0xe3, 0xd6, 0x80, 0x96, 0x18, 0x16, 0x36, 0x8d, 0x86,
0xf9, 0x53, 0xdb, 0xc2, 0xc8, 0x39, 0x1c, 0x6c, 0x6d, 0x6b, 0x0c, 0x73, 0x37, 0xf6, 0x98, 0xd2,
0xb9, 0x26, 0xe5, 0xd9, 0x06, 0x52, 0xd1, 0x54, 0x99, 0xf7, 0x55, 0xf1, 0x1b, 0x39, 0x38, 0xd1,
0x98, 0x31, 0x3e, 0x27, 0xa1, 0x88, 0xbc, 0x8a, 0xa1, 0x40, 0x0e, 0xbd, 0x14, 0x11, 0xf9, 0x1d,
0xdc, 0x64, 0x89, 0xc2, 0xe6, 0x52, 0x35, 0xb9, 0xec, 0x6b, 0xc0, 0xa4, 0xf2, 0x07, 0x40, 0x18,
0x27, 0x32, 0x4f, 0xad, 0x66, 0xaa, 0xae, 0x41, 0x4c, 0xf9, 0x37, 0xd8, 0xd7, 0x3f, 0x4e, 0x9a,
0xdc, 0x49, 0x6f, 0xdf, 0xda, 0x26, 0x32, 0xee, 0x27, 0x77, 0x52, 0x97, 0x38, 0xf2, 0x40, 0xb2,
0x15, 0x7a, 0xae, 0x2d, 0x71, 0xe4, 0x13, 0xb6, 0x42, 0x72, 0x03, 0x47, 0x1b, 0x8e, 0x06, 0x4b,
0xaa, 0x16, 0xd2, 0x83, 0xb6, 0xd3, 0xad, 0x0f, 0xce, 0x7b, 0x3f, 0xfd, 0xd8, 0xbd, 0x1f, 0xb3,
0xf2, 0x5b, 0x6b, 0xef, 0xc7, 0xba, 0xb7, 0xf3, 0xb9, 0x0c, 0x4d, 0xc3, 0x1a, 0x89, 0x08, 0xef,
0x7f, 0x21, 0x82, 0x8d, 0xb4, 0xcb, 0xdb, 0x69, 0x7b, 0x50, 0x63, 0x7a, 0xc8, 0x68, 0x68, 0x2c,
0x77, 0xfc, 0xc7, 0x23, 0x19, 0x42, 0xc3, 0x7c, 0x06, 0x4b, 0x9a, 0x52, 0x6e, 0x9f, 0x68, 0x7d,
0x70, 0xb6, 0xf3, 0x6f, 0x7c, 0x85, 0x0f, 0xef, 0x68, 0x9c, 0xe1, 0x98, 0xb2, 0xd4, 0xaf, 0x9b,
0xb6, 0xb1, 0xe9, 0x22, 0xff, 0x43, 0x45, 0x2a, 0xaa, 0xd0, 0x04, 0xd1, 0x1c, 0x9c, 0xee, 0x6c,
0x37, 0x6b, 0x4c, 0x34, 0xcd, 0xb7, 0x6c, 0xd2, 0x85, 0x43, 0x7b, 0xf9, 0x86, 0x63, 0x55, 0xf3,
0x10, 0x9b, 0x06, 0x2f, 0xbc, 0xb8, 0x7c, 0xfe, 0xfe, 0xd9, 0x9c, 0xa9, 0x45, 0x36, 0xd3, 0xc3,
0xfa, 0x2b, 0x16, 0xc7, 0x6c, 0xa5, 0x30, 0x5c, 0xf4, 0xed, 0x45, 0xff, 0x46, 0x4c, 0xaa, 0x94,
0xcd, 0x32, 0x85, 0x51, 0x9f, 0x09, 0x85, 0xa9, 0xa0, 0x71, 0xdf, 0xdc, 0xde, 0xd7, 0x6e, 0x2f,
0x67, 0xb3, 0xaa, 0x39, 0xfd, 0xf7, 0x3d, 0x00, 0x00, 0xff, 0xff, 0x57, 0x2c, 0x97, 0xd1, 0x75,
0x05, 0x00, 0x00,
// 809 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0x5d, 0x8f, 0xdb, 0x44,
0x14, 0x95, 0x93, 0x6c, 0x76, 0x7d, 0x93, 0xcd, 0xee, 0xfa, 0xc9, 0x94, 0x96, 0xa6, 0xae, 0x0a,
0x91, 0x10, 0x89, 0xb4, 0x08, 0xde, 0x40, 0xd0, 0x86, 0x4a, 0x11, 0xa2, 0x0d, 0xce, 0x8a, 0x07,
0x5e, 0xac, 0x89, 0x7d, 0x37, 0x19, 0xc9, 0x33, 0x0e, 0x9e, 0x31, 0xdd, 0xdd, 0x07, 0xc4, 0x2b,
0xfc, 0x04, 0x9e, 0xf9, 0x77, 0xfc, 0x06, 0x24, 0x34, 0x77, 0x1c, 0x7f, 0x6c, 0x23, 0x84, 0x2a,
0xf1, 0x78, 0xcf, 0x3d, 0xd7, 0x73, 0xe6, 0x9c, 0x3b, 0x09, 0x9c, 0xa1, 0x8e, 0x93, 0x48, 0xa0,
0x66, 0xd3, 0x5d, 0x9e, 0xe9, 0xcc, 0xbb, 0x10, 0x3c, 0xfd, 0xb9, 0x50, 0xb6, 0x9a, 0x9a, 0xee,
0x83, 0x61, 0x9c, 0x09, 0x91, 0x49, 0x0b, 0x3d, 0x18, 0xaa, 0x78, 0x8b, 0xa2, 0xa4, 0x07, 0x7f,
0x38, 0x00, 0x57, 0x28, 0x99, 0xd4, 0xdf, 0xa1, 0x66, 0xde, 0x08, 0x3a, 0x8b, 0xb9, 0xef, 0x8c,
0x9d, 0x49, 0x37, 0xec, 0x2c, 0xe6, 0xde, 0x87, 0x70, 0x26, 0x0b, 0x11, 0xfd, 0x54, 0x60, 0x7e,
0x1b, 0xc9, 0x2c, 0x41, 0xe5, 0x77, 0xa8, 0x79, 0x2a, 0x0b, 0xf1, 0xbd, 0x41, 0x5f, 0x19, 0xd0,
0xfb, 0x18, 0x2e, 0xb8, 0x54, 0x98, 0xeb, 0x28, 0xde, 0x32, 0x29, 0x31, 0x5d, 0xcc, 0x95, 0xdf,
0x1d, 0x77, 0x27, 0x6e, 0x78, 0x6e, 0x1b, 0x2f, 0x2a, 0xdc, 0xfb, 0x08, 0xce, 0xec, 0x07, 0x2b,
0xae, 0xdf, 0x1b, 0x3b, 0x13, 0x37, 0x1c, 0x11, 0x5c, 0x31, 0x83, 0x5f, 0x1d, 0x70, 0x97, 0x79,
0x76, 0x73, 0x7b, 0x50, 0xdb, 0xe7, 0x70, 0xcc, 0x92, 0x24, 0x47, 0x65, 0x35, 0x0d, 0x2e, 0x1f,
0x4e, 0x5b, 0x77, 0x2f, 0x6f, 0xfd, 0xb5, 0xe5, 0x84, 0x7b, 0xb2, 0xd1, 0x9a, 0xa3, 0x2a, 0xd2,
0x43, 0x5a, 0x6d, 0xa3, 0xd6, 0x1a, 0xdc, 0xc0, 0xe9, 0x92, 0xe5, 0x9a, 0x6b, 0x9e, 0xc9, 0x85,
0xbc, 0xce, 0xbc, 0x67, 0x30, 0xda, 0xed, 0x81, 0x48, 0x32, 0x81, 0xa4, 0xc8, 0x0d, 0x4f, 0x2b,
0xf4, 0x15, 0x13, 0xe8, 0x8d, 0x61, 0x50, 0x01, 0x8b, 0x79, 0x69, 0x5a, 0x13, 0xf2, 0x3e, 0x00,
0x50, 0xb8, 0x11, 0x28, 0xf5, 0xfe, 0xfc, 0x6e, 0xd8, 0x40, 0x82, 0x3f, 0x1d, 0x18, 0xbd, 0xc8,
0xd2, 0x14, 0xe3, 0xea, 0xec, 0xfb, 0x0e, 0x7c, 0x01, 0x7d, 0x1b, 0x66, 0x69, 0xc0, 0xb3, 0xb6,
0x01, 0x65, 0xd0, 0xf5, 0x47, 0x56, 0x04, 0x84, 0xe5, 0x90, 0xf7, 0x18, 0x06, 0x71, 0x8e, 0x4c,
0x63, 0xa4, 0xb9, 0x40, 0xbf, 0x3b, 0x76, 0x26, 0xbd, 0x10, 0x2c, 0x74, 0xc5, 0x05, 0x7a, 0x01,
0x0c, 0x1b, 0x8a, 0x95, 0xdf, 0x23, 0x91, 0x2d, 0x2c, 0xf8, 0xdd, 0x01, 0x77, 0x21, 0x13, 0xbc,
0x21, 0x85, 0x8f, 0x00, 0xb8, 0x29, 0x9a, 0xce, 0xb8, 0x84, 0x90, 0x2b, 0x3e, 0x1c, 0x53, 0x51,
0x39, 0xb2, 0x2f, 0xbd, 0x39, 0x0c, 0xed, 0xe0, 0x8e, 0xe5, 0x4c, 0x58, 0x3f, 0x06, 0x97, 0x4f,
0x0e, 0x26, 0xfa, 0x2d, 0xde, 0xfe, 0xc0, 0xd2, 0x02, 0x97, 0x8c, 0xe7, 0xe1, 0x80, 0xc6, 0x96,
0x34, 0x15, 0xfc, 0x02, 0xe7, 0xab, 0xd2, 0xc1, 0x4a, 0xd2, 0x43, 0x70, 0x2b, 0x57, 0x4b, 0xef,
0x6a, 0xc0, 0x28, 0xba, 0xe6, 0x98, 0x26, 0xb5, 0xa2, 0xb2, 0x6c, 0x6a, 0xed, 0xb6, 0xb5, 0xfa,
0x70, 0xbc, 0x2e, 0x38, 0xcd, 0xf4, 0x6c, 0xa7, 0x2c, 0x83, 0xbf, 0x5a, 0x99, 0x1d, 0xdc, 0xda,
0xff, 0x3b, 0xb3, 0xf6, 0x5a, 0xf5, 0xee, 0xaf, 0x55, 0x7b, 0x7f, 0x35, 0xdb, 0x28, 0xff, 0x88,
0x56, 0xbf, 0xde, 0xdf, 0x2b, 0xb6, 0x51, 0x6f, 0x45, 0xdf, 0x3f, 0x10, 0xfd, 0x6b, 0x38, 0x7f,
0x69, 0xcc, 0x7a, 0xce, 0x65, 0x9a, 0x6d, 0x5e, 0xf2, 0x14, 0x55, 0xd3, 0x4f, 0xa7, 0xed, 0xe7,
0x13, 0x18, 0xae, 0x89, 0x18, 0x5d, 0x1b, 0xa6, 0xdf, 0xa1, 0x63, 0x07, 0xeb, 0x7a, 0x38, 0xf8,
0xbb, 0x03, 0x83, 0x32, 0x3f, 0xf2, 0xee, 0xdf, 0xa3, 0x0b, 0x60, 0x18, 0xd7, 0xef, 0x63, 0x9f,
0x5f, 0x0b, 0xf3, 0x9e, 0xc2, 0x69, 0xeb, 0xb6, 0x64, 0x98, 0xdb, 0xb8, 0xc7, 0x15, 0xdb, 0x18,
0x52, 0xf9, 0x4b, 0x10, 0x29, 0xcd, 0x72, 0x4d, 0xa9, 0x1e, 0x85, 0xc3, 0x12, 0x5c, 0x19, 0x8c,
0x8c, 0x2f, 0x49, 0x28, 0x13, 0xff, 0x88, 0x28, 0x50, 0x42, 0xdf, 0xc8, 0xc4, 0x7b, 0x1f, 0xdc,
0x6c, 0x87, 0xd2, 0xe6, 0xd2, 0xa7, 0x5c, 0x4e, 0x0c, 0x40, 0xa9, 0x3c, 0x02, 0x88, 0xd3, 0x4c,
0x95, 0xa9, 0x1d, 0x53, 0xd7, 0x25, 0x84, 0xda, 0xef, 0xc1, 0x89, 0xf9, 0x99, 0xcd, 0xb3, 0x37,
0xca, 0x3f, 0xb1, 0xb6, 0xc9, 0x42, 0x84, 0xd9, 0x1b, 0x65, 0x5a, 0x02, 0x45, 0xa4, 0xf8, 0x1d,
0xfa, 0xae, 0x6d, 0x09, 0x14, 0x2b, 0x7e, 0x87, 0xde, 0x6b, 0xb8, 0x68, 0x38, 0x1a, 0xed, 0x98,
0xde, 0x2a, 0x1f, 0xe8, 0xe1, 0x3c, 0x9d, 0xbe, 0xf5, 0x37, 0x30, 0xbd, 0x9f, 0x55, 0x78, 0x56,
0x7b, 0xbf, 0x34, 0xb3, 0xc1, 0x6f, 0x1d, 0x18, 0x11, 0x8b, 0x5e, 0xcf, 0x7f, 0x88, 0xe0, 0x5d,
0x5e, 0xcf, 0xfd, 0x97, 0xde, 0x7b, 0x97, 0x97, 0xee, 0x7d, 0x06, 0x47, 0x4a, 0x33, 0x8d, 0x14,
0xc4, 0xe8, 0xf2, 0xf1, 0xc1, 0x71, 0xba, 0xc6, 0xca, 0xd0, 0x42, 0xcb, 0xf6, 0x26, 0x70, 0x6e,
0x0f, 0x6f, 0x38, 0xd6, 0xa7, 0x45, 0x1c, 0x11, 0x5e, 0x79, 0xf1, 0xfc, 0xab, 0x1f, 0xbf, 0xdc,
0x70, 0xbd, 0x2d, 0xd6, 0xe6, 0x63, 0xb3, 0x3b, 0x9e, 0xa6, 0xfc, 0x4e, 0x63, 0xbc, 0x9d, 0xd9,
0x83, 0x3e, 0x49, 0xb8, 0xd2, 0x39, 0x5f, 0x17, 0x1a, 0x93, 0x19, 0x97, 0x1a, 0x73, 0xc9, 0xd2,
0x19, 0x9d, 0x3e, 0x33, 0x6e, 0xef, 0xd6, 0xeb, 0x3e, 0x55, 0x9f, 0xfe, 0x13, 0x00, 0x00, 0xff,
0xff, 0xde, 0xef, 0x54, 0xe3, 0xa3, 0x07, 0x00, 0x00,
}

View File

@ -125,6 +125,8 @@ service MasterService {
rpc GetTimeTickChannel(common.Empty) returns (milvus.StringResponse) {}
rpc GetDdChannel(common.Empty) returns (milvus.StringResponse) {}
rpc GetStatisticsChannel(common.Empty) returns (milvus.StringResponse) {}
}

View File

@ -242,54 +242,55 @@ func init() {
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
var fileDescriptor_f9c348dec43a6705 = []byte{
// 747 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x96, 0xfb, 0x6e, 0xd3, 0x3c,
0x18, 0xc6, 0x77, 0xfa, 0xf6, 0x69, 0xef, 0x7a, 0x18, 0x66, 0x82, 0x29, 0x4c, 0x6c, 0x14, 0xd8,
0xba, 0x03, 0x29, 0xda, 0x6e, 0x80, 0x75, 0x41, 0x5d, 0x25, 0x26, 0xa1, 0xb6, 0x80, 0x06, 0x9a,
0xa6, 0x24, 0xb5, 0x5a, 0x8b, 0x24, 0x2e, 0x79, 0xdd, 0x0e, 0x76, 0x1f, 0xdc, 0x2f, 0xca, 0xc9,
0x49, 0xba, 0xa4, 0x0d, 0x62, 0x7f, 0xda, 0xfe, 0xf9, 0x79, 0xde, 0x93, 0x12, 0x43, 0xc9, 0xd6,
0x51, 0x50, 0x57, 0x1d, 0xb9, 0x5c, 0x70, 0xf2, 0xd8, 0x66, 0xd6, 0x64, 0x8c, 0xc1, 0x4a, 0x0d,
0x8e, 0x94, 0x92, 0xc9, 0x6d, 0x9b, 0x3b, 0xc1, 0xa6, 0x52, 0x4a, 0x22, 0x4a, 0x85, 0x39, 0x82,
0xba, 0x8e, 0x6e, 0x85, 0x6b, 0xd2, 0xd7, 0x85, 0x7e, 0x83, 0xd4, 0x9d, 0x30, 0x93, 0x06, 0x7b,
0xb5, 0x2e, 0xac, 0xb5, 0xb5, 0x0e, 0xfd, 0x31, 0xa6, 0x28, 0xc8, 0x5b, 0x58, 0x31, 0x74, 0xa4,
0x5b, 0x8b, 0xbb, 0x8b, 0xf5, 0xf5, 0x93, 0x6d, 0x35, 0x65, 0x18, 0x1a, 0x5d, 0xe2, 0xa0, 0xa9,
0x23, 0xed, 0xf8, 0x24, 0xd9, 0x84, 0xff, 0x4c, 0x3e, 0x76, 0xc4, 0xd6, 0xd2, 0xee, 0x62, 0xbd,
0xdc, 0x09, 0x16, 0xb5, 0x01, 0x80, 0x27, 0x8a, 0x23, 0xee, 0x20, 0x25, 0xa7, 0xb0, 0x8a, 0x42,
0x17, 0x63, 0x0c, 0x75, 0x9f, 0x65, 0xea, 0x76, 0x7d, 0xa4, 0x13, 0xa2, 0xa4, 0x02, 0x4b, 0x6d,
0xcd, 0x57, 0x5d, 0xee, 0x2c, 0xb5, 0xb5, 0xd8, 0x68, 0x39, 0x69, 0xd4, 0x03, 0xe8, 0x21, 0x7f,
0x80, 0xf0, 0x53, 0xaa, 0x13, 0x58, 0xf7, 0x55, 0xff, 0x25, 0xfe, 0x6d, 0x58, 0x13, 0xcc, 0xa6,
0x28, 0x74, 0x7b, 0xe4, 0xa7, 0xb1, 0xd2, 0x89, 0x37, 0xb2, 0x7d, 0x4f, 0x7e, 0x3f, 0x82, 0xf2,
0xa5, 0xdf, 0xd6, 0x6e, 0xd0, 0x23, 0x72, 0x03, 0x1b, 0xe7, 0x2e, 0xd5, 0x05, 0x3d, 0xe7, 0x96,
0x45, 0x4d, 0xc1, 0xb8, 0x43, 0x8e, 0xd3, 0xf6, 0xe1, 0x62, 0x1a, 0x0b, 0x6b, 0xa2, 0xcc, 0x0a,
0xb6, 0xb6, 0x40, 0xbe, 0x41, 0x45, 0x73, 0xf9, 0x28, 0x21, 0x7f, 0x98, 0x29, 0x9f, 0x86, 0x0a,
0x8a, 0xdf, 0x40, 0xf9, 0x42, 0xc7, 0x84, 0xf6, 0x41, 0xa6, 0x76, 0x8a, 0x89, 0xa4, 0x5f, 0x64,
0xa2, 0x4d, 0xce, 0xad, 0xa8, 0x2f, 0xb5, 0x05, 0x72, 0x0b, 0x44, 0xa3, 0x68, 0xba, 0xcc, 0x48,
0x16, 0x48, 0xcd, 0xce, 0xe0, 0x1e, 0x18, 0x59, 0x35, 0x0a, 0xf3, 0xd2, 0x78, 0x02, 0x4f, 0x5b,
0x54, 0xc4, 0x47, 0x5e, 0xca, 0x0c, 0x05, 0x33, 0x91, 0x1c, 0x65, 0xb7, 0x27, 0x85, 0x62, 0x64,
0x7d, 0x5c, 0x0c, 0x96, 0xbe, 0x16, 0x54, 0xbb, 0x43, 0x7e, 0x1b, 0x03, 0x98, 0xd3, 0xaf, 0x34,
0x15, 0xd9, 0x1d, 0x15, 0x62, 0xa5, 0xdb, 0x35, 0x54, 0x83, 0xb1, 0xfa, 0xa8, 0xbb, 0x82, 0xf9,
0xb5, 0x3d, 0x9a, 0x31, 0x7c, 0x92, 0x2a, 0x38, 0x1e, 0x57, 0x50, 0xf6, 0xc6, 0x2a, 0x16, 0x3f,
0xc8, 0x1d, 0xbd, 0xbf, 0x95, 0xbe, 0x86, 0xd2, 0x85, 0x8e, 0xb1, 0x72, 0x3d, 0x6f, 0xf0, 0xee,
0x09, 0x17, 0x9a, 0x3b, 0x84, 0x27, 0x2d, 0x2a, 0xe4, 0xdd, 0x44, 0xf7, 0xb3, 0xbb, 0x91, 0x22,
0x71, 0x76, 0x37, 0xa6, 0x59, 0x69, 0xca, 0xa0, 0xe2, 0x75, 0x4a, 0x9e, 0x63, 0x4e, 0xbd, 0x52,
0x50, 0xe4, 0x75, 0x58, 0x04, 0x95, 0x56, 0x0e, 0x54, 0xa3, 0xf1, 0xef, 0xd2, 0x81, 0x4d, 0x1d,
0x91, 0xd3, 0xf8, 0x29, 0x6a, 0xf6, 0x58, 0xdf, 0x83, 0xa5, 0x9f, 0x09, 0x25, 0x2f, 0x94, 0xf0,
0x00, 0xc9, 0x7e, 0x6e, 0xb4, 0x53, 0x46, 0xf5, 0xf9, 0xa0, 0x34, 0xf9, 0x04, 0xeb, 0xc1, 0x9c,
0xb6, 0x9d, 0x3e, 0xfd, 0x99, 0xe3, 0x91, 0x20, 0x0a, 0x8e, 0xda, 0x10, 0xca, 0x51, 0x62, 0x81,
0xf0, 0xc1, 0xcc, 0xe4, 0x53, 0xd2, 0x87, 0x45, 0x50, 0x99, 0x80, 0x01, 0xe5, 0x16, 0x15, 0xfe,
0xae, 0xe7, 0x4e, 0xc9, 0x5e, 0xe6, 0xf5, 0x18, 0x88, 0x6c, 0xf6, 0xe7, 0x72, 0x89, 0x22, 0x55,
0xce, 0x2c, 0x8b, 0x9b, 0x3d, 0xf9, 0xab, 0xda, 0x51, 0x33, 0x9e, 0x1d, 0x6a, 0xfc, 0xd7, 0x55,
0x76, 0xf3, 0x01, 0x29, 0xfb, 0x01, 0xfe, 0xf7, 0x65, 0xdb, 0x1a, 0x79, 0x9e, 0x89, 0xcb, 0x37,
0x88, 0xb2, 0x93, 0x7b, 0x9e, 0x28, 0x44, 0xf5, 0x0c, 0x91, 0x0d, 0x9c, 0xb0, 0xc9, 0x6d, 0x8d,
0xbc, 0x4e, 0xdf, 0xf2, 0x1e, 0x3a, 0xaa, 0x64, 0x62, 0xf1, 0xbd, 0x79, 0x98, 0xf4, 0xf8, 0x0c,
0x1b, 0x2d, 0x2a, 0xc2, 0xff, 0xb0, 0x5f, 0x25, 0x24, 0x4a, 0xe6, 0x24, 0xbc, 0xb7, 0x47, 0xe2,
0x97, 0xf2, 0x2a, 0x7d, 0x26, 0x5f, 0x5e, 0x29, 0x85, 0xda, 0x02, 0xf9, 0x02, 0xa4, 0x45, 0x85,
0x57, 0xde, 0x1e, 0x33, 0xbf, 0x9f, 0x0f, 0x75, 0xc7, 0xa1, 0xd6, 0x4c, 0xe5, 0x97, 0xd9, 0x33,
0x2e, 0x5c, 0xe6, 0x0c, 0x12, 0x01, 0x5f, 0xc1, 0xa6, 0x17, 0xb0, 0xfc, 0x14, 0x3d, 0x9c, 0x74,
0xb3, 0xf9, 0xf5, 0xdd, 0x80, 0x89, 0xe1, 0xd8, 0xf0, 0x6e, 0x37, 0xee, 0x98, 0x65, 0xb1, 0x3b,
0x41, 0xcd, 0x61, 0x23, 0xb8, 0xf0, 0xa6, 0xcf, 0x50, 0xb8, 0xcc, 0x18, 0x0b, 0xda, 0x6f, 0x44,
0x89, 0x37, 0x7c, 0xc9, 0x46, 0xd0, 0xbe, 0x91, 0x61, 0xac, 0xfa, 0xeb, 0xd3, 0x3f, 0x01, 0x00,
0x00, 0xff, 0xff, 0x8b, 0xde, 0xd8, 0x75, 0xd3, 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.
@ -372,6 +373,7 @@ type MasterServiceClient interface {
AssignSegmentID(ctx context.Context, in *datapb.AssignSegIDRequest, opts ...grpc.CallOption) (*datapb.AssignSegIDResponse, error)
GetServiceStates(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*internalpb2.ServiceStates, error)
GetTimeTickChannel(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
GetDdChannel(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
GetStatisticsChannel(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
}
@ -572,6 +574,15 @@ func (c *masterServiceClient) GetTimeTickChannel(ctx context.Context, in *common
return out, nil
}
func (c *masterServiceClient) GetDdChannel(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
out := new(milvuspb.StringResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/GetDdChannel", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *masterServiceClient) GetStatisticsChannel(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
out := new(milvuspb.StringResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/GetStatisticsChannel", in, out, opts...)
@ -651,6 +662,7 @@ type MasterServiceServer interface {
AssignSegmentID(context.Context, *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error)
GetServiceStates(context.Context, *commonpb.Empty) (*internalpb2.ServiceStates, error)
GetTimeTickChannel(context.Context, *commonpb.Empty) (*milvuspb.StringResponse, error)
GetDdChannel(context.Context, *commonpb.Empty) (*milvuspb.StringResponse, error)
GetStatisticsChannel(context.Context, *commonpb.Empty) (*milvuspb.StringResponse, error)
}
@ -721,6 +733,9 @@ func (*UnimplementedMasterServiceServer) GetServiceStates(ctx context.Context, r
func (*UnimplementedMasterServiceServer) GetTimeTickChannel(ctx context.Context, req *commonpb.Empty) (*milvuspb.StringResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetTimeTickChannel not implemented")
}
func (*UnimplementedMasterServiceServer) GetDdChannel(ctx context.Context, req *commonpb.Empty) (*milvuspb.StringResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetDdChannel not implemented")
}
func (*UnimplementedMasterServiceServer) GetStatisticsChannel(ctx context.Context, req *commonpb.Empty) (*milvuspb.StringResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetStatisticsChannel not implemented")
}
@ -1107,6 +1122,24 @@ func _MasterService_GetTimeTickChannel_Handler(srv interface{}, ctx context.Cont
return interceptor(ctx, in, info, handler)
}
func _MasterService_GetDdChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(commonpb.Empty)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(MasterServiceServer).GetDdChannel(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.master.MasterService/GetDdChannel",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(MasterServiceServer).GetDdChannel(ctx, req.(*commonpb.Empty))
}
return interceptor(ctx, in, info, handler)
}
func _MasterService_GetStatisticsChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(commonpb.Empty)
if err := dec(in); err != nil {
@ -1213,6 +1246,10 @@ var _MasterService_serviceDesc = grpc.ServiceDesc{
MethodName: "GetTimeTickChannel",
Handler: _MasterService_GetTimeTickChannel_Handler,
},
{
MethodName: "GetDdChannel",
Handler: _MasterService_GetDdChannel_Handler,
},
{
MethodName: "GetStatisticsChannel",
Handler: _MasterService_GetStatisticsChannel_Handler,

View File

@ -119,7 +119,7 @@ message LoadSegmentRequest {
common.MsgBase base = 1;
int64 dbID = 2;
int64 collectionID = 3;
repeated int64 partitionIDs = 4;
int64 partitionID = 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"`
PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"`
PartitionID int64 `protobuf:"varint,4,opt,name=partitionID,proto3" json:"partitionID,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) GetPartitionIDs() []int64 {
func (m *LoadSegmentRequest) GetPartitionID() int64 {
if m != nil {
return m.PartitionIDs
return m.PartitionID
}
return nil
return 0
}
func (m *LoadSegmentRequest) GetSegmentIDs() []int64 {
@ -1297,80 +1297,81 @@ func init() {
func init() { proto.RegisterFile("query_service.proto", fileDescriptor_5fcb6756dc1afb8d) }
var fileDescriptor_5fcb6756dc1afb8d = []byte{
// 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,
// 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,
}
// Reference imports to suppress errors if they are not otherwise used.

View File

@ -51,16 +51,18 @@ 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
addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error
addSegment2(segmentID UniqueID, partitionTag string, collectionID UniqueID, segType segmentType) error
addSegment(segmentID UniqueID, partitionID UniqueID, collectionID UniqueID, segType segmentType) error
removeSegment(segmentID UniqueID) error
getSegmentByID(segmentID UniqueID) (*Segment, error)
hasSegment(segmentID UniqueID) bool
getVecFieldIDsBySegmentID(segmentID UniqueID) ([]int64, error)
getVecFieldsBySegmentID(segmentID UniqueID) (map[int64]string, error)
freeAll()
}
@ -288,6 +290,13 @@ 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 {
@ -303,6 +312,21 @@ 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()
@ -355,7 +379,7 @@ func (colReplica *collectionReplicaImpl) getSegmentStatistics() []*internalpb2.S
return statisticData
}
func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error {
func (colReplica *collectionReplicaImpl) addSegment2(segmentID UniqueID, partitionTag string, collectionID UniqueID, segType segmentType) error {
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
@ -369,7 +393,29 @@ func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, partitio
return err2
}
var newSegment = newSegment(collection, segmentID, partitionTag, collectionID)
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)
colReplica.segments[segmentID] = newSegment
*partition.Segments() = append(*partition.Segments(), newSegment)
@ -430,7 +476,7 @@ func (colReplica *collectionReplicaImpl) hasSegment(segmentID UniqueID) bool {
return ok
}
func (colReplica *collectionReplicaImpl) getVecFieldIDsBySegmentID(segmentID UniqueID) ([]int64, error) {
func (colReplica *collectionReplicaImpl) getVecFieldsBySegmentID(segmentID UniqueID) (map[int64]string, error) {
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
@ -443,16 +489,18 @@ func (colReplica *collectionReplicaImpl) getVecFieldIDsBySegmentID(segmentID Uni
return nil, err2
}
vecFields := make([]int64, 0)
vecFields := make(map[int64]string)
for _, field := range col.Schema().Fields {
if field.DataType == schemapb.DataType_VECTOR_BINARY || field.DataType == schemapb.DataType_VECTOR_FLOAT {
vecFields = append(vecFields, field.FieldID)
vecFields[field.FieldID] = field.Name
}
}
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.addSegment(UniqueID(i), tag, collectionID)
err := node.replica.addSegment2(UniqueID(i), tag, collectionID, segTypeGrowing)
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.addSegment(UniqueID(i), tag, collectionID)
err := node.replica.addSegment2(UniqueID(i), tag, collectionID, segTypeGrowing)
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.addSegment(UniqueID(i), tag, collectionID)
err := node.replica.addSegment2(UniqueID(i), tag, collectionID, segTypeGrowing)
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.addSegment(UniqueID(i), tag, collectionID)
err := node.replica.addSegment2(UniqueID(i), tag, collectionID, segTypeGrowing)
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.addSegment(task.SegmentID, task.PartitionName, collection.ID())
err = iNode.replica.addSegment2(task.SegmentID, task.PartitionName, collection.ID(), segTypeGrowing)
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.addSegment(seg.SegmentID, seg.PartitionTag, seg.CollectionID)
err := mService.replica.addSegment2(seg.SegmentID, seg.PartitionTag, seg.CollectionID, segTypeGrowing)
if err != nil {
log.Println(err)
return

View File

@ -14,9 +14,14 @@ 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.addSegment(UniqueID(i), targetPartition.partitionTag, collection.ID())
err := node.replica.addSegment2(UniqueID(i), targetPartition.partitionTag, collection.ID(), segTypeGrowing)
assert.NoError(t, err)
}

View File

@ -295,10 +295,27 @@ 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: ???
err := node.segManager.loadSegment(segmentID, &fieldIDs)
indexID := UniqueID(0) // TODO: get index id from master
err := node.segManager.loadSegment(segmentID, partitionID, collectionID, &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.addSegment(segmentID, collectionMeta.PartitionTags[0], collectionID)
err = node.replica.addSegment2(segmentID, collectionMeta.PartitionTags[0], collectionID, segTypeGrowing)
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 := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
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,13 +22,21 @@ 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.enum_SegmentType
segmentType C.SegmentType
segmentID UniqueID
partitionTag string // TODO: use partitionID
partitionID UniqueID
collectionID UniqueID
lastMemSize int64
lastRowCount int64
@ -40,10 +48,15 @@ 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()
@ -56,17 +69,13 @@ func (s *Segment) GetRecentlyModified() bool {
return s.recentlyModified
}
//-------------------------------------------------------------------------------------- constructor and destructor
func newSegment(collection *Collection, segmentID int64, partitionTag string, collectionID UniqueID) *Segment {
func newSegment2(collection *Collection, segmentID int64, partitionTag string, collectionID UniqueID, segType segmentType) *Segment {
/*
CSegmentInterface
NewSegment(CCollection collection, uint64_t segment_id, SegmentType seg_type);
*/
initIndexParam := make(map[int64]indexParam)
// 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)
segmentPtr := C.NewSegment(collection.collectionPtr, C.ulong(segmentID), segType)
var newSegment = &Segment{
segmentPtr: segmentPtr,
segmentID: segmentID,
@ -78,6 +87,24 @@ func newSegment(collection *Collection, segmentID int64, partitionTag string, co
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
@ -87,7 +114,6 @@ func deleteSegment(segment *Segment) {
C.DeleteSegment(cPtr)
}
//-------------------------------------------------------------------------------------- stats functions
func (s *Segment) getRowCount() int64 {
/*
long int
@ -116,108 +142,6 @@ 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) {
@ -318,3 +242,138 @@ 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"
"fmt"
"unsafe"
"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, fieldIDs *[]int64) error {
func (s *segmentManager) loadSegment(segmentID UniqueID, partitionID UniqueID, collectionID UniqueID, fieldIDs *[]int64) error {
insertBinlogPathRequest := &datapb.InsertBinlogPathRequest{
SegmentID: segmentID,
}
@ -72,6 +72,12 @@ func (s *segmentManager) loadSegment(segmentID UniqueID, fieldIDs *[]int64) erro
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 {
@ -110,26 +116,34 @@ func (s *segmentManager) loadSegment(segmentID UniqueID, fieldIDs *[]int64) erro
}
for _, value := range insertData.Data {
var numRows int
var data interface{}
switch fieldData := value.(type) {
case storage.BoolFieldData:
numRows := fieldData.NumRows
data := fieldData.Data
fmt.Println(numRows, data, fieldID)
// TODO: s.replica.addSegment()
numRows = fieldData.NumRows
data = fieldData.Data
case storage.Int8FieldData:
// TODO: s.replica.addSegment()
numRows = fieldData.NumRows
data = fieldData.Data
case storage.Int16FieldData:
// TODO: s.replica.addSegment()
numRows = fieldData.NumRows
data = fieldData.Data
case storage.Int32FieldData:
// TODO: s.replica.addSegment()
numRows = fieldData.NumRows
data = fieldData.Data
case storage.Int64FieldData:
// TODO: s.replica.addSegment()
numRows = fieldData.NumRows
data = fieldData.Data
case storage.FloatFieldData:
// TODO: s.replica.addSegment()
numRows = fieldData.NumRows
data = fieldData.Data
case storage.DoubleFieldData:
// TODO: s.replica.addSegment()
numRows = fieldData.NumRows
data = fieldData.Data
case storage.StringFieldData:
// TODO: s.replica.addSegment()
numRows = fieldData.NumRows
data = fieldData.Data
case storage.FloatVectorFieldData:
// segment to be loaded doesn't need vector field,
// so we ignore the type of vector field data
@ -139,6 +153,17 @@ func (s *segmentManager) loadSegment(segmentID UniqueID, fieldIDs *[]int64) erro
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
}
}
}
@ -153,23 +178,17 @@ func (s *segmentManager) loadIndex(segmentID UniqueID, indexID UniqueID) error {
if err != nil || pathResponse.Status.ErrorCode != commonpb.ErrorCode_SUCCESS {
return err
}
targetSegment, err := s.replica.getSegmentByID(segmentID)
if err != nil {
return err
}
// get vector field ids from schema to load index
vecFieldIDs, err := s.replica.getVecFieldIDsBySegmentID(segmentID)
vecFieldIDs, err := s.replica.getVecFieldsBySegmentID(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))
}
for id, name := range vecFieldIDs {
var targetIndexParam indexParam
// TODO: get index param from master
// non-blocking send
go s.sendLoadIndex(pathResponse.IndexFilePaths, segmentID, vecFieldID, "", targetIndexParam)
go s.sendLoadIndex(pathResponse.IndexFilePaths, segmentID, id, name, 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 := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
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 := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
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 := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
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 := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
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 := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
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 := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
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 := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
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 := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
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 := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
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 := newSegment(collection, segmentID, Params.DefaultPartitionTag, collectionID)
segment := newSegment2(collection, segmentID, Params.DefaultPartitionTag, collectionID, segTypeGrowing)
assert.Equal(t, segmentID, segment.segmentID)
ids := []int64{1, 2, 3}