提交 7e8162a7 编写于 作者: S sunby 提交者: yefu.chen

Add dd handler

Signed-off-by: Nsunby <bingyi.sun@zilliz.com>
上级 d5e53aa2
......@@ -98,6 +98,20 @@ type DescribeCollectionResponse struct {
}
```
* *GetCollectionStatistics*
```go
type CollectionStatsRequest struct {
MsgBase
DbName string
CollectionName string
}
type CollectionStatsResponse struct {
Stats []KeyValuePair
}
```
* *ShowCollections*
```go
......@@ -145,6 +159,21 @@ type HasPartitionRequest struct {
}
```
* *GetPartitionStatistics*
```go
type PartitionStatsRequest struct {
MsgBase
DbName string
CollectionName string
PartitionName string
}
type PartitionStatsResponse struct {
Stats []KeyValuePair
}
```
* *ShowPartitions*
```go
......
package dataservice
import (
"context"
"log"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
)
type ddHandler struct {
meta *meta
segAllocator segmentAllocator
}
func newDDHandler(meta *meta, segAllocator segmentAllocator) *ddHandler {
return &ddHandler{
meta: meta,
segAllocator: segAllocator,
}
}
func (handler *ddHandler) Start(ctx context.Context, inputStream ms.MsgStream) {
for {
select {
case msgPack := <-inputStream.Chan():
for _, msg := range msgPack.Msgs {
switch msg.Type() {
case commonpb.MsgType_kCreateCollection:
createCollectionMsg, ok := msg.(*ms.CreateCollectionMsg)
if !ok {
log.Println("message with type MsgType_kCreateCollection can not be cast to CreateCollectionMsg")
continue
}
if err := handler.handleCreateCollection(&createCollectionMsg.CreateCollectionRequest); err != nil {
log.Printf("handle create collection error: %s", err.Error())
}
case commonpb.MsgType_kDropCollection:
dropCollectionMsg, ok := msg.(*ms.DropCollectionMsg)
if !ok {
log.Println("message with type MsgType_kDropCollection can not be cast to DropCollectionMsg")
continue
}
if err := handler.handleDropCollection(&dropCollectionMsg.DropCollectionRequest); err != nil {
log.Printf("handle drop collection error: %s", err.Error())
}
case commonpb.MsgType_kCreatePartition:
createPartitionMsg, ok := msg.(*ms.CreatePartitionMsg)
if !ok {
log.Println("message with type MsgType_kCreatePartition can not be cast to CreatePartitionMsg")
continue
}
if err := handler.handleCreatePartition(&createPartitionMsg.CreatePartitionRequest); err != nil {
log.Printf("handle create partition error: %s", err.Error())
}
case commonpb.MsgType_kDropPartition:
dropPartitionMsg, ok := msg.(*ms.DropPartitionMsg)
if !ok {
log.Println("message with type MsgType_kDropPartition can not be cast to DropPartitionMsg")
continue
}
if err := handler.handleDropPartition(&dropPartitionMsg.DropPartitionRequest); err != nil {
log.Printf("handle drop partition error: %s", err.Error())
}
default:
log.Printf("invalid message type %s", msg.Type())
}
}
case <-ctx.Done():
log.Println("dd handler is shut down.")
break
}
}
}
func (handler *ddHandler) handleCreateCollection(req *internalpb2.CreateCollectionRequest) error {
var schema schemapb.CollectionSchema
if err := proto.UnmarshalMerge(req.Schema, &schema); err != nil {
return err
}
info := &collectionInfo{
ID: req.CollectionID,
Schema: &schema,
}
return handler.meta.AddCollection(info)
}
func (handler *ddHandler) handleDropCollection(req *internalpb2.DropCollectionRequest) error {
if err := handler.meta.DropCollection(req.CollectionID); err != nil {
return err
}
segmentIDs := handler.meta.GetSegmentsByCollectionID(req.CollectionID)
for _, id := range segmentIDs {
if err := handler.meta.DropSegment(id); err != nil {
return err
}
handler.segAllocator.DropSegment(id)
}
return nil
}
func (handler *ddHandler) handleCreatePartition(req *internalpb2.CreatePartitionRequest) error {
return handler.meta.AddPartition(req.CollectionID, req.PartitionID)
}
func (handler *ddHandler) handleDropPartition(req *internalpb2.DropPartitionRequest) error {
if err := handler.meta.DropPartition(req.CollectionID, req.PartitionID); err != nil {
return err
}
ids := handler.meta.GetSegmentsByPartitionID(req.PartitionID)
for _, id := range ids {
if err := handler.meta.DropSegment(id); err != nil {
return err
}
handler.segAllocator.DropSegment(id)
}
return nil
}
......@@ -5,8 +5,6 @@ import (
"strconv"
"sync"
log "github.com/sirupsen/logrus"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
......@@ -21,8 +19,9 @@ type (
UniqueID = typeutil.UniqueID
Timestamp = typeutil.Timestamp
collectionInfo struct {
ID UniqueID
Schema *schemapb.CollectionSchema
ID UniqueID
Schema *schemapb.CollectionSchema
partitions []UniqueID
}
meta struct {
client kv.TxnBase // client of a reliable kv service, i.e. etcd client
......@@ -84,16 +83,6 @@ func (meta *meta) DropCollection(collID UniqueID) error {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
}
delete(meta.collID2Info, collID)
for id, segment := range meta.segID2Info {
if segment.CollectionID != collID {
continue
}
delete(meta.segID2Info, id)
if err := meta.removeSegmentInfo(id); err != nil {
log.Printf("remove segment info failed, %s", err.Error())
_ = meta.reloadFromKV()
}
}
return nil
}
......@@ -103,6 +92,16 @@ func (meta *meta) HasCollection(collID UniqueID) bool {
_, ok := meta.collID2Info[collID]
return ok
}
func (meta *meta) GetCollection(collectionID UniqueID) (*collectionInfo, error) {
meta.ddLock.RLock()
defer meta.ddLock.RUnlock()
collectionInfo, ok := meta.collID2Info[collectionID]
if !ok {
return nil, fmt.Errorf("collection %d not found", collectionID)
}
return collectionInfo, nil
}
func (meta *meta) BuildSegment(collectionID UniqueID, partitionID UniqueID, channelRange []string) (*datapb.SegmentInfo, error) {
id, err := meta.allocator.allocID()
......@@ -152,6 +151,17 @@ func (meta *meta) UpdateSegment(segmentInfo *datapb.SegmentInfo) error {
return nil
}
func (meta *meta) DropSegment(segmentID UniqueID) error {
meta.ddLock.Lock()
meta.ddLock.Unlock()
if _, ok := meta.segID2Info[segmentID]; !ok {
return fmt.Errorf("segment %d not found", segmentID)
}
delete(meta.segID2Info, segmentID)
return nil
}
func (meta *meta) GetSegment(segID UniqueID) (*datapb.SegmentInfo, error) {
meta.ddLock.RLock()
defer meta.ddLock.RUnlock()
......@@ -182,15 +192,72 @@ func (meta *meta) CloseSegment(segID UniqueID, closeTs Timestamp) error {
return nil
}
func (meta *meta) GetCollection(collectionID UniqueID) (*collectionInfo, error) {
func (meta *meta) GetSegmentsByCollectionID(collectionID UniqueID) []UniqueID {
meta.ddLock.RLock()
defer meta.ddLock.RUnlock()
collectionInfo, ok := meta.collID2Info[collectionID]
ret := make([]UniqueID, 0)
for _, info := range meta.segID2Info {
if info.CollectionID == collectionID {
ret = append(ret, info.SegmentID)
}
}
return ret
}
func (meta *meta) GetSegmentsByPartitionID(partitionID UniqueID) []UniqueID {
meta.ddLock.RLock()
defer meta.ddLock.RUnlock()
ret := make([]UniqueID, 0)
for _, info := range meta.segID2Info {
if info.PartitionID == partitionID {
ret = append(ret, info.SegmentID)
}
}
return ret
}
func (meta *meta) AddPartition(collectionID UniqueID, partitionID UniqueID) error {
meta.ddLock.Lock()
defer meta.ddLock.Unlock()
coll, ok := meta.collID2Info[collectionID]
if !ok {
return nil, fmt.Errorf("collection %d not found", collectionID)
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collectionID, 10))
}
return collectionInfo, nil
for _, t := range coll.partitions {
if t == partitionID {
return errors.Errorf("partition %d already exists.", partitionID)
}
}
coll.partitions = append(coll.partitions, partitionID)
return nil
}
func (meta *meta) DropPartition(collID UniqueID, partitionID UniqueID) error {
meta.ddLock.Lock()
defer meta.ddLock.Unlock()
collection, ok := meta.collID2Info[collID]
if !ok {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
}
idx := -1
for i, id := range collection.partitions {
if partitionID == id {
idx = i
break
}
}
if idx == -1 {
return fmt.Errorf("cannot find partition id %d", partitionID)
}
collection.partitions = append(collection.partitions[:idx], collection.partitions[idx+1:]...)
return nil
}
func (meta *meta) saveSegmentInfo(segmentInfo *datapb.SegmentInfo) error {
......
......@@ -6,21 +6,17 @@ import (
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
)
type statsProcessor struct {
meta *meta
segmentThreshold float64
segmentThresholdFactor float64
type statsHandler struct {
meta *meta
}
func newStatsProcessor(meta *meta) *statsProcessor {
return &statsProcessor{
meta: meta,
segmentThreshold: Params.SegmentSize * 1024 * 1024,
segmentThresholdFactor: Params.SegmentSizeFactor,
func newStatsHandler(meta *meta) *statsHandler {
return &statsHandler{
meta: meta,
}
}
func (processor *statsProcessor) ProcessQueryNodeStats(msgPack *msgstream.MsgPack) error {
func (handler *statsHandler) HandleQueryNodeStats(msgPack *msgstream.MsgPack) error {
for _, msg := range msgPack.Msgs {
statsMsg, ok := msg.(*msgstream.QueryNodeStatsMsg)
if !ok {
......@@ -28,7 +24,7 @@ func (processor *statsProcessor) ProcessQueryNodeStats(msgPack *msgstream.MsgPac
}
for _, segStat := range statsMsg.GetSegStats() {
if err := processor.processSegmentStat(segStat); err != nil {
if err := handler.handleSegmentStat(segStat); err != nil {
return err
}
}
......@@ -37,13 +33,12 @@ func (processor *statsProcessor) ProcessQueryNodeStats(msgPack *msgstream.MsgPac
return nil
}
func (processor *statsProcessor) processSegmentStat(segStats *internalpb2.SegmentStats) error {
func (handler *statsHandler) handleSegmentStat(segStats *internalpb2.SegmentStats) error {
if !segStats.GetRecentlyModified() {
return nil
}
segID := segStats.GetSegmentID()
segMeta, err := processor.meta.GetSegment(segID)
segMeta, err := handler.meta.GetSegment(segStats.SegmentID)
if err != nil {
return err
}
......@@ -51,5 +46,5 @@ func (processor *statsProcessor) processSegmentStat(segStats *internalpb2.Segmen
segMeta.NumRows = segStats.NumRows
segMeta.MemSize = segStats.MemorySize
return processor.meta.UpdateSegment(segMeta)
return handler.meta.UpdateSegment(segMeta)
}
......@@ -74,23 +74,24 @@ func (c *GrpcClient) HasCollection(in *milvuspb.HasCollectionRequest) (*milvuspb
func (c *GrpcClient) DescribeCollection(in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
return c.grpcClient.DescribeCollection(context.Background(), in)
}
func (c *GrpcClient) GetCollectionStatistics(in *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) {
return c.grpcClient.GetCollectionStatistics(context.Background(), in)
}
func (c *GrpcClient) ShowCollections(in *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) {
return c.grpcClient.ShowCollections(context.Background(), in)
}
func (c *GrpcClient) CreatePartition(in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) {
return c.grpcClient.CreatePartition(context.Background(), in)
}
func (c *GrpcClient) DropPartition(in *milvuspb.DropPartitionRequest) (*commonpb.Status, error) {
return c.grpcClient.DropPartition(context.Background(), in)
}
func (c *GrpcClient) HasPartition(in *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
return c.grpcClient.HasPartition(context.Background(), in)
}
func (c *GrpcClient) GetPartitionStatistics(in *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) {
return c.grpcClient.GetPartitionStatistics(context.Background(), in)
}
func (c *GrpcClient) ShowPartitions(in *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) {
return c.grpcClient.ShowPartitions(context.Background(), in)
}
......
......@@ -10,7 +10,7 @@ import (
"github.com/stretchr/testify/assert"
cms "github.com/zilliztech/milvus-distributed/internal/masterservice"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
......@@ -35,7 +35,7 @@ func TestGrpcService(t *testing.T) {
cms.Params.StatisticsChannel = fmt.Sprintf("stateChannel%d", randVal)
cms.Params.MaxPartitionNum = 64
cms.Params.DefaultPartitionName = "_default"
cms.Params.DefaultPartitionTag = "_default"
t.Logf("master service port = %d", cms.Params.Port)
......@@ -52,34 +52,54 @@ func TestGrpcService(t *testing.T) {
timeTickArray = append(timeTickArray, ts)
return nil
}
createCollectionArray := make([]*internalpb2.CreateCollectionRequest, 0, 16)
core.DdCreateCollectionReq = func(req *internalpb2.CreateCollectionRequest) error {
t.Logf("Create Colllection %s", req.CollectionName)
createCollectionArray := make([]*cms.CreateCollectionReqTask, 0, 16)
core.DdCreateCollectionReq = func(req *cms.CreateCollectionReqTask) error {
t.Logf("Create Colllection %s", req.Req.CollectionName)
createCollectionArray = append(createCollectionArray, req)
return nil
}
dropCollectionArray := make([]*internalpb2.DropCollectionRequest, 0, 16)
core.DdDropCollectionReq = func(req *internalpb2.DropCollectionRequest) error {
t.Logf("Drop Collection %s", req.CollectionName)
dropCollectionArray := make([]*cms.DropCollectionReqTask, 0, 16)
core.DdDropCollectionReq = func(req *cms.DropCollectionReqTask) error {
t.Logf("Drop Collection %s", req.Req.CollectionName)
dropCollectionArray = append(dropCollectionArray, req)
return nil
}
createPartitionArray := make([]*internalpb2.CreatePartitionRequest, 0, 16)
core.DdCreatePartitionReq = func(req *internalpb2.CreatePartitionRequest) error {
t.Logf("Create Partition %s", req.PartitionName)
createPartitionArray := make([]*cms.CreatePartitionReqTask, 0, 16)
core.DdCreatePartitionReq = func(req *cms.CreatePartitionReqTask) error {
t.Logf("Create Partition %s", req.Req.PartitionName)
createPartitionArray = append(createPartitionArray, req)
return nil
}
dropPartitionArray := make([]*internalpb2.DropPartitionRequest, 0, 16)
core.DdDropPartitionReq = func(req *internalpb2.DropPartitionRequest) error {
t.Logf("Drop Partition %s", req.PartitionName)
dropPartitionArray := make([]*cms.DropPartitionReqTask, 0, 16)
core.DdDropPartitionReq = func(req *cms.DropPartitionReqTask) error {
t.Logf("Drop Partition %s", req.Req.PartitionName)
dropPartitionArray = append(dropPartitionArray, req)
return nil
}
core.GetSegmentMeta = func(id typeutil.UniqueID) (*etcdpb.SegmentMeta, error) {
return &etcdpb.SegmentMeta{
SegmentID: 20,
CollectionID: 10,
PartitionTag: "_default",
ChannelStart: 50,
ChannelEnd: 100,
OpenTime: 1000,
CloseTime: 2000,
NumRows: 16,
MemSize: 1024,
BinlogFilePaths: []*etcdpb.FieldBinlogFiles{
{
FieldID: 101,
BinlogFiles: []string{"/test/binlog/file"},
},
},
}, nil
}
err = svr.Init(&cms.InitParams{ProxyTimeTickChannel: fmt.Sprintf("proxyTimeTick%d", randVal)})
assert.Nil(t, err)
err = svr.Start()
......@@ -131,8 +151,8 @@ func TestGrpcService(t *testing.T) {
assert.Nil(t, err)
assert.Equal(t, len(createCollectionArray), 1)
assert.Equal(t, status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, createCollectionArray[0].Base.MsgType, commonpb.MsgType_kCreateCollection)
assert.Equal(t, createCollectionArray[0].CollectionName, "testColl")
assert.Equal(t, createCollectionArray[0].Req.Base.MsgType, commonpb.MsgType_kCreateCollection)
assert.Equal(t, createCollectionArray[0].Req.CollectionName, "testColl")
})
t.Run("has collection", func(t *testing.T) {
......@@ -199,6 +219,57 @@ func TestGrpcService(t *testing.T) {
assert.Equal(t, rsp.Schema.Name, "testColl")
})
t.Run("get collection statistics", func(t *testing.T) {
req := &milvuspb.CollectionStatsRequest{
Base: &commonpb.MsgBase{
MsgType: 0, //TODO,miss msg type
MsgID: 104,
Timestamp: 104,
SourceID: 104,
},
DbName: "testDb",
CollectionName: "testColl",
}
rsp, err := cli.GetCollectionStatistics(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, len(rsp.Stats), 2)
assert.Equal(t, rsp.Stats[0].Key, "row_count")
assert.Equal(t, rsp.Stats[0].Value, "0")
assert.Equal(t, rsp.Stats[1].Key, "data_size")
assert.Equal(t, rsp.Stats[1].Value, "0")
collMeta, err := core.MetaTable.GetCollectionByName("testColl")
assert.Nil(t, err)
seg := &etcdpb.SegmentMeta{
SegmentID: 101,
CollectionID: collMeta.ID,
PartitionTag: cms.Params.DefaultPartitionTag,
}
err = core.MetaTable.AddSegment(seg)
assert.Nil(t, err)
req = &milvuspb.CollectionStatsRequest{
Base: &commonpb.MsgBase{
MsgType: 0, //TODO,miss msg type
MsgID: 105,
Timestamp: 105,
SourceID: 105,
},
DbName: "testDb",
CollectionName: "testColl",
}
rsp, err = cli.GetCollectionStatistics(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, len(rsp.Stats), 2)
assert.Equal(t, rsp.Stats[0].Key, "row_count")
assert.Equal(t, rsp.Stats[0].Value, "16")
assert.Equal(t, rsp.Stats[1].Key, "data_size")
assert.Equal(t, rsp.Stats[1].Value, "1024")
})
t.Run("show collection", func(t *testing.T) {
req := &milvuspb.ShowCollectionRequest{
Base: &commonpb.MsgBase{
......@@ -234,9 +305,7 @@ func TestGrpcService(t *testing.T) {
collMeta, err := core.MetaTable.GetCollectionByName("testColl")
assert.Nil(t, err)
assert.Equal(t, len(collMeta.PartitionIDs), 2)
partMeta, err := core.MetaTable.GetPartitionByID(collMeta.PartitionIDs[1])
assert.Nil(t, err)
assert.Equal(t, partMeta.PartitionName, "testPartition")
assert.Equal(t, collMeta.PartitionTags[1], "testPartition")
})
......@@ -258,6 +327,28 @@ func TestGrpcService(t *testing.T) {
assert.Equal(t, rsp.Value, true)
})
t.Run("get partition statistics", func(t *testing.T) {
req := &milvuspb.PartitionStatsRequest{
Base: &commonpb.MsgBase{
MsgType: 0, //TODO, msg type
MsgID: 109,
Timestamp: 109,
SourceID: 109,
},
DbName: "testDb",
CollectionName: "testColl",
PartitionName: cms.Params.DefaultPartitionTag,
}
rsp, err := cli.GetPartitionStatistics(req)
assert.Nil(t, err)
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, len(rsp.Stats), 2)
assert.Equal(t, rsp.Stats[0].Key, "row_count")
assert.Equal(t, rsp.Stats[0].Value, "16")
assert.Equal(t, rsp.Stats[1].Key, "data_size")
assert.Equal(t, rsp.Stats[1].Value, "1024")
})
t.Run("show partition", func(t *testing.T) {
req := &milvuspb.ShowPartitionRequest{
Base: &commonpb.MsgBase{
......@@ -293,9 +384,7 @@ func TestGrpcService(t *testing.T) {
collMeta, err := core.MetaTable.GetCollectionByName("testColl")
assert.Nil(t, err)
assert.Equal(t, len(collMeta.PartitionIDs), 1)
partMeta, err := core.MetaTable.GetPartitionByID(collMeta.PartitionIDs[0])
assert.Nil(t, err)
assert.Equal(t, partMeta.PartitionName, cms.Params.DefaultPartitionName)
assert.Equal(t, collMeta.PartitionTags[0], cms.Params.DefaultPartitionTag)
})
t.Run("drop collection", func(t *testing.T) {
......@@ -314,8 +403,8 @@ func TestGrpcService(t *testing.T) {
assert.Nil(t, err)
assert.Equal(t, len(dropCollectionArray), 1)
assert.Equal(t, status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, dropCollectionArray[0].Base.MsgType, commonpb.MsgType_kDropCollection)
assert.Equal(t, dropCollectionArray[0].CollectionName, "testColl")
assert.Equal(t, dropCollectionArray[0].Req.Base.MsgType, commonpb.MsgType_kDropCollection)
assert.Equal(t, dropCollectionArray[0].Req.CollectionName, "testColl")
req = &milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{
......
......@@ -94,6 +94,10 @@ func (s *GrpcServer) DescribeCollection(ctx context.Context, in *milvuspb.Descri
return s.core.DescribeCollection(in)
}
func (s *GrpcServer) GetCollectionStatistics(ctx context.Context, in *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) {
return s.core.GetCollectionStatistics(in)
}
func (s *GrpcServer) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) {
return s.core.ShowCollections(in)
}
......@@ -110,6 +114,10 @@ func (s *GrpcServer) HasPartition(ctx context.Context, in *milvuspb.HasPartition
return s.core.HasPartition(in)
}
func (s *GrpcServer) GetPartitionStatistics(ctx context.Context, in *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) {
return s.core.GetPartitionStatistics(in)
}
func (s *GrpcServer) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) {
return s.core.ShowPartitions(in)
}
......
......@@ -11,6 +11,7 @@ import (
"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"
......@@ -25,7 +26,7 @@ import (
// datapb(data_service)
// indexpb(index_service)
// milvuspb -> servicepb
// masterpb2 -> masterpb (master_service)
// masterpb2 -> masterpb master_service)
type InitParams struct {
ProxyTimeTickChannel string
......@@ -52,10 +53,12 @@ type Interface interface {
DropCollection(in *milvuspb.DropCollectionRequest) (*commonpb.Status, error)
HasCollection(in *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error)
DescribeCollection(in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error)
GetCollectionStatistics(in *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error)
ShowCollections(in *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error)
CreatePartition(in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error)
DropPartition(in *milvuspb.DropPartitionRequest) (*commonpb.Status, error)
HasPartition(in *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error)
GetPartitionStatistics(in *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error)
ShowPartitions(in *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error)
//index builder service
......@@ -120,16 +123,16 @@ type Core struct {
SendTimeTick func(t typeutil.Timestamp) error
//TODO, send create collection into dd channel
DdCreateCollectionReq func(req *internalpb2.CreateCollectionRequest) error
DdCreateCollectionReq func(req *CreateCollectionReqTask) error
//TODO, send drop collection into dd channel, and notify the proxy to delete this collection
DdDropCollectionReq func(req *internalpb2.DropCollectionRequest) error
DdDropCollectionReq func(req *DropCollectionReqTask) error
//TODO, send create partition into dd channel
DdCreatePartitionReq func(req *internalpb2.CreatePartitionRequest) error
DdCreatePartitionReq func(req *CreatePartitionReqTask) error
//TODO, send drop partition into dd channel
DdDropPartitionReq func(req *internalpb2.DropPartitionRequest) error
DdDropPartitionReq func(req *DropPartitionReqTask) error
//dd request scheduler
ddReqQueue chan reqTask //dd request will be push into this chan
......@@ -145,6 +148,9 @@ type Core struct {
initOnce sync.Once
startOnce sync.Once
isInit atomic.Value
//TODO, get segment meta by segment id, from data service by grpc
GetSegmentMeta func(id typeutil.UniqueID) (*etcdpb.SegmentMeta, error)
}
// --------------------- function --------------------------
......@@ -187,6 +193,9 @@ func (c *Core) checkInit() error {
if c.ddReqQueue == nil {
return errors.Errorf("ddReqQueue is nil")
}
if c.GetSegmentMeta == nil {
return errors.Errorf("GetSegmentMeta is nil")
}
if c.DdCreateCollectionReq == nil {
return errors.Errorf("DdCreateCollectionReq is nil")
}
......@@ -449,6 +458,36 @@ func (c *Core) DescribeCollection(in *milvuspb.DescribeCollectionRequest) (*milv
return t.Rsp, nil
}
func (c *Core) GetCollectionStatistics(in *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) {
t := &CollectionStatsReqTask{
baseReqTask: baseReqTask{
cv: make(chan error),
core: c,
},
Req: in,
Rsp: &milvuspb.CollectionStatsResponse{
Stats: nil,
Status: nil,
},
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &milvuspb.CollectionStatsResponse{
Stats: nil,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "GetCollectionStatistics failed: " + err.Error(),
},
}, nil
}
t.Rsp.Status = &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}
return t.Rsp, nil
}
func (c *Core) ShowCollections(in *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) {
t := &ShowCollectionReqTask{
baseReqTask: baseReqTask{
......@@ -551,6 +590,36 @@ func (c *Core) HasPartition(in *milvuspb.HasPartitionRequest) (*milvuspb.BoolRes
}, nil
}
func (c *Core) GetPartitionStatistics(in *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) {
t := &PartitionStatsReqTask{
baseReqTask: baseReqTask{
cv: make(chan error),
core: c,
},
Req: in,
Rsp: &milvuspb.PartitionStatsResponse{
Stats: nil,
Status: nil,
},
}
c.ddReqQueue <- t
err := t.WaitToFinish()
if err != nil {
return &milvuspb.PartitionStatsResponse{
Stats: nil,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "GetPartitionStatistics failed: " + err.Error(),
},
}, nil
}
t.Rsp.Status = &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}
return t.Rsp, nil
}
func (c *Core) ShowPartitions(in *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) {
t := &ShowPartitionReqTask{
baseReqTask: baseReqTask{
......
......@@ -23,6 +23,6 @@ type ParamTable struct {
DdChannel string
StatisticsChannel string
MaxPartitionNum int64
DefaultPartitionName string
MaxPartitionNum int64
DefaultPartitionTag string
}
package masterservice
import (
"fmt"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
......@@ -90,32 +91,19 @@ func (t *CreateCollectionReqTask) Execute() error {
if err != nil {
return err
}
collMeta := etcdpb.CollectionInfo{
ID: collID,
Schema: &schema,
CreateTime: collTs,
PartitionIDs: make([]typeutil.UniqueID, 0, 16),
}
partMeta := etcdpb.PartitionInfo{
PartitionName: Params.DefaultPartitionName,
PartitionID: partitionID,
SegmentIDs: make([]typeutil.UniqueID, 0, 16),
coll := etcdpb.CollectionMeta{
ID: collID,
Schema: &schema,
CreateTime: collTs,
SegmentIDs: make([]typeutil.UniqueID, 0),
PartitionTags: []string{Params.DefaultPartitionTag},
PartitionIDs: []typeutil.UniqueID{partitionID},
}
err = t.core.MetaTable.AddCollection(&collMeta, &partMeta)
err = t.core.MetaTable.AddCollection(&coll)
if err != nil {
return err
}
ddReq := internalpb2.CreateCollectionRequest{
Base: t.Req.Base,
DbName: t.Req.DbName,
CollectionName: t.Req.CollectionName,
DbID: 0, //TODO,not used
CollectionID: collID,
Schema: t.Req.Schema,
}
err = t.core.DdCreateCollectionReq(&ddReq)
err = t.core.DdCreateCollectionReq(t)
if err != nil {
return err
}
......@@ -148,15 +136,7 @@ func (t *DropCollectionReqTask) Execute() error {
//data service should drop segments , which belong to this collection, from the segment manager
ddReq := internalpb2.DropCollectionRequest{
Base: t.Req.Base,
DbName: t.Req.DbName,
CollectionName: t.Req.CollectionName,
DbID: 0, //not used
CollectionID: collMeta.ID,
}
err = t.core.DdDropCollectionReq(&ddReq)
err = t.core.DdDropCollectionReq(t)
if err != nil {
return err
}
......@@ -217,6 +197,50 @@ func (t *DescribeCollectionReqTask) Execute() error {
return nil
}
type CollectionStatsReqTask struct {
baseReqTask
Req *milvuspb.CollectionStatsRequest
Rsp *milvuspb.CollectionStatsResponse
}
func (t *CollectionStatsReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
func (t *CollectionStatsReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
//row_count
//data_size
func (t *CollectionStatsReqTask) Execute() error {
coll, err := t.core.MetaTable.GetCollectionByName(t.Req.CollectionName)
if err != nil {
return err
}
var rowCount int64 = 0
var dataSize int64 = 0
for _, seg := range coll.SegmentIDs {
m, e := t.core.GetSegmentMeta(seg)
if e != nil {
return e
}
rowCount += m.NumRows
dataSize += m.MemSize
}
t.Rsp.Stats = append(t.Rsp.Stats,
&commonpb.KeyValuePair{
Key: "row_count",
Value: fmt.Sprintf("%d", rowCount),
})
t.Rsp.Stats = append(t.Rsp.Stats,
&commonpb.KeyValuePair{
Key: "data_size",
Value: fmt.Sprintf("%d", dataSize),
})
return nil
}
type ShowCollectionReqTask struct {
baseReqTask
Req *milvuspb.ShowCollectionRequest
......@@ -267,17 +291,7 @@ func (t *CreatePartitionReqTask) Execute() error {
return err
}
ddReq := internalpb2.CreatePartitionRequest{
Base: t.Req.Base,
DbName: t.Req.DbName,
CollectionName: t.Req.CollectionName,
PartitionName: t.Req.PartitionName,
DbID: 0, // todo, not used
CollectionID: collMeta.ID,
PartitionID: partitionID,
}
err = t.core.DdCreatePartitionReq(&ddReq)
err = t.core.DdCreatePartitionReq(t)
if err != nil {
return err
}
......@@ -303,22 +317,12 @@ func (t *DropPartitionReqTask) Execute() error {
if err != nil {
return err
}
partID, err := t.core.MetaTable.DeletePartition(coll.ID, t.Req.PartitionName)
err = t.core.MetaTable.DeletePartition(coll.ID, t.Req.PartitionName)
if err != nil {
return err
}
ddReq := internalpb2.DropPartitionRequest{
Base: t.Req.Base,
DbName: t.Req.DbName,
CollectionName: t.Req.CollectionName,
PartitionName: t.Req.PartitionName,
DbID: 0, //todo,not used
CollectionID: coll.ID,
PartitionID: partID,
}
err = t.core.DdDropPartitionReq(&ddReq)
err = t.core.DdDropPartitionReq(t)
if err != nil {
return err
}
......@@ -348,6 +352,50 @@ func (t *HasPartitionReqTask) Execute() error {
return nil
}
type PartitionStatsReqTask struct {
baseReqTask
Req *milvuspb.PartitionStatsRequest
Rsp *milvuspb.PartitionStatsResponse
}
func (t *PartitionStatsReqTask) Type() commonpb.MsgType {
return t.Req.Base.MsgType
}
func (t *PartitionStatsReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
func (t *PartitionStatsReqTask) Execute() error {
coll, err := t.core.MetaTable.GetCollectionByName(t.Req.CollectionName)
if err != nil {
return err
}
var rowCount int64 = 0
var dataSize int64 = 0
for _, seg := range coll.SegmentIDs {
m, e := t.core.GetSegmentMeta(seg)
if e != nil {
return e
}
if m.PartitionTag == t.Req.PartitionName {
rowCount += m.NumRows
dataSize += m.MemSize
}
}
t.Rsp.Stats = append(t.Rsp.Stats,
&commonpb.KeyValuePair{
Key: "row_count",
Value: fmt.Sprintf("%d", rowCount),
})
t.Rsp.Stats = append(t.Rsp.Stats,
&commonpb.KeyValuePair{
Key: "data_size",
Value: fmt.Sprintf("%d", dataSize),
})
return nil
}
type ShowPartitionReqTask struct {
baseReqTask
Req *milvuspb.ShowPartitionRequest
......@@ -362,18 +410,12 @@ func (t *ShowPartitionReqTask) Ts() (typeutil.Timestamp, error) {
return t.Req.Base.Timestamp, nil
}
//TODO,list partition ids and partition tags
func (t *ShowPartitionReqTask) Execute() error {
coll, err := t.core.MetaTable.GetCollectionByName(t.Req.CollectionName)
if err != nil {
return err
}
for _, partID := range coll.PartitionIDs {
partMeta, err := t.core.MetaTable.GetPartitionByID(partID)
if err != nil {
return err
}
t.Rsp.PartitionIDs = append(t.Rsp.PartitionIDs, partMeta.PartitionID)
t.Rsp.PartitionNames = append(t.Rsp.PartitionNames, partMeta.PartitionName)
}
t.Rsp.PartitionNames = append(t.Rsp.PartitionNames, coll.PartitionTags...)
return nil
}
......@@ -67,6 +67,8 @@ service MasterService {
*/
rpc DescribeCollection(milvus.DescribeCollectionRequest) returns (milvus.DescribeCollectionResponse) {}
rpc GetCollectionStatistics(milvus.CollectionStatsRequest) returns (milvus.CollectionStatsResponse) {}
/**
* @brief This method is used to list all collections.
*
......@@ -95,6 +97,8 @@ service MasterService {
*/
rpc HasPartition(milvus.HasPartitionRequest) returns (milvus.BoolResponse) {}
rpc GetPartitionStatistics(milvus.PartitionStatsRequest) returns (milvus.PartitionStatsResponse) {}
/**
* @brief This method is used to show partition information
*
......
......@@ -242,52 +242,55 @@ func init() {
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
var fileDescriptor_f9c348dec43a6705 = []byte{
// 714 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x96, 0x5b, 0x4f, 0xdb, 0x3c,
0x18, 0xc7, 0x39, 0xbd, 0xbc, 0xe2, 0xa1, 0x07, 0xe4, 0x71, 0x81, 0x32, 0x34, 0x58, 0xb7, 0x41,
0x39, 0x2c, 0x9d, 0xe0, 0x0b, 0x8c, 0x92, 0xa9, 0x54, 0x1a, 0xd2, 0xd6, 0x76, 0x9b, 0xd8, 0x84,
0x90, 0x93, 0x5a, 0xad, 0xb5, 0x24, 0xee, 0xe2, 0xa7, 0xb0, 0xf1, 0xd5, 0x77, 0x33, 0xe5, 0xe4,
0x24, 0x90, 0x94, 0x4c, 0xe3, 0xd2, 0xf6, 0xdf, 0xbf, 0xff, 0x73, 0x52, 0x1c, 0xa8, 0x38, 0x54,
0x22, 0xf3, 0xf4, 0x89, 0x27, 0x50, 0x90, 0x27, 0x0e, 0xb7, 0xaf, 0xa7, 0x32, 0x5c, 0xe9, 0xe1,
0x91, 0x56, 0xb1, 0x84, 0xe3, 0x08, 0x37, 0xdc, 0xd4, 0x2a, 0x69, 0x89, 0x56, 0xe3, 0x2e, 0x32,
0xcf, 0xa5, 0x76, 0xb4, 0x26, 0x43, 0x8a, 0xf4, 0x4a, 0x32, 0xef, 0x9a, 0x5b, 0x2c, 0xdc, 0x6b,
0xf4, 0x61, 0xa5, 0x6b, 0xf4, 0xd8, 0x8f, 0x29, 0x93, 0x48, 0xde, 0xc0, 0x92, 0x49, 0x25, 0xdb,
0x98, 0xdf, 0x9e, 0x6f, 0xae, 0x1e, 0x6d, 0xea, 0x19, 0xc3, 0xc8, 0xe8, 0x5c, 0x8e, 0xda, 0x54,
0xb2, 0x5e, 0xa0, 0x24, 0xeb, 0xf0, 0x9f, 0x25, 0xa6, 0x2e, 0x6e, 0x2c, 0x6c, 0xcf, 0x37, 0xab,
0xbd, 0x70, 0xd1, 0x18, 0x01, 0xf8, 0x50, 0x39, 0x11, 0xae, 0x64, 0xe4, 0x18, 0x96, 0x25, 0x52,
0x9c, 0xca, 0x88, 0xfb, 0x34, 0x97, 0xdb, 0x0f, 0x24, 0xbd, 0x48, 0x4a, 0x6a, 0xb0, 0xd0, 0x35,
0x02, 0xea, 0x62, 0x6f, 0xa1, 0x6b, 0x24, 0x46, 0x8b, 0x69, 0xa3, 0x01, 0xc0, 0x40, 0x8a, 0x47,
0x08, 0x3f, 0x43, 0xbd, 0x86, 0xd5, 0x80, 0xfa, 0x2f, 0xf1, 0x6f, 0xc2, 0x0a, 0x72, 0x87, 0x49,
0xa4, 0xce, 0x24, 0x48, 0x63, 0xa9, 0x97, 0x6c, 0xe4, 0xfb, 0x1e, 0xfd, 0xae, 0x43, 0xf5, 0x3c,
0x68, 0x6b, 0x3f, 0xec, 0x11, 0xb9, 0x82, 0xb5, 0x53, 0x8f, 0x51, 0x64, 0xa7, 0xc2, 0xb6, 0x99,
0x85, 0x5c, 0xb8, 0xe4, 0x30, 0x6b, 0x1f, 0x2d, 0xee, 0xca, 0xa2, 0x9a, 0x68, 0xb3, 0x82, 0x6d,
0xcc, 0x91, 0x6f, 0x50, 0x33, 0x3c, 0x31, 0x49, 0xe1, 0xf7, 0x73, 0xf1, 0x59, 0x51, 0x49, 0xf8,
0x15, 0x54, 0xcf, 0xa8, 0x4c, 0xb1, 0xf7, 0x72, 0xd9, 0x19, 0x4d, 0x8c, 0x7e, 0x9e, 0x2b, 0x6d,
0x0b, 0x61, 0xc7, 0x7d, 0x69, 0xcc, 0x91, 0x1b, 0x20, 0x06, 0x93, 0x96, 0xc7, 0xcd, 0x74, 0x81,
0xf4, 0xfc, 0x0c, 0xee, 0x09, 0x63, 0xab, 0x56, 0x69, 0xbd, 0x32, 0xb6, 0xa1, 0xde, 0x1f, 0x8b,
0x9b, 0xe4, 0x4c, 0x16, 0xd4, 0x2d, 0xab, 0x8a, 0x1d, 0x0f, 0x4a, 0x69, 0x95, 0xdb, 0x25, 0xd4,
0xc3, 0xf6, 0x7e, 0xa0, 0x1e, 0xf2, 0x20, 0xc7, 0x83, 0x19, 0x43, 0xa0, 0x54, 0x25, 0xdb, 0x74,
0x01, 0x55, 0xbf, 0xbd, 0x09, 0x7c, 0xaf, 0x70, 0x04, 0xfe, 0x16, 0x7d, 0x09, 0x95, 0x33, 0x2a,
0x13, 0x72, 0xb3, 0x68, 0x00, 0xee, 0x81, 0x4b, 0xf5, 0x9f, 0x43, 0xcd, 0x2f, 0x9a, 0xba, 0x2c,
0x0b, 0x42, 0xcf, 0x88, 0x62, 0x87, 0xfd, 0x32, 0x52, 0x65, 0xe5, 0x42, 0x3d, 0x9e, 0x88, 0x3e,
0x1b, 0x39, 0xcc, 0xc5, 0x82, 0x1e, 0xdc, 0x51, 0xc5, 0x6e, 0x87, 0xe5, 0xc4, 0xca, 0xcf, 0x82,
0x8a, 0x1f, 0x4a, 0x74, 0x20, 0xc9, 0x6e, 0x61, 0xb4, 0x77, 0x8c, 0x9a, 0x0f, 0x0b, 0x95, 0xc9,
0x27, 0x58, 0x0d, 0x47, 0xa6, 0xeb, 0x0e, 0xd9, 0xcf, 0x02, 0x8f, 0x94, 0xa2, 0x64, 0xd7, 0xc7,
0x50, 0x8d, 0x13, 0x0b, 0xc1, 0x7b, 0x33, 0x93, 0xcf, 0xa0, 0xf7, 0xcb, 0x48, 0x55, 0x02, 0x26,
0x54, 0x3b, 0x0c, 0x83, 0x5d, 0xdf, 0x9d, 0x91, 0x9d, 0xdc, 0xeb, 0x89, 0x20, 0xb6, 0xd9, 0x7d,
0x50, 0x97, 0x2a, 0x52, 0xed, 0xc4, 0xb6, 0x85, 0x35, 0x50, 0x5f, 0xef, 0x2d, 0x3d, 0xe7, 0x25,
0xd6, 0x93, 0x87, 0x48, 0xdb, 0x2e, 0x16, 0x28, 0xec, 0x7b, 0xf8, 0x3f, 0xc0, 0x76, 0x0d, 0xf2,
0x2c, 0x57, 0xae, 0x9e, 0x65, 0x6d, 0xab, 0xf0, 0x3c, 0x55, 0x88, 0xfa, 0x89, 0x94, 0x7c, 0xe4,
0x46, 0x4d, 0xee, 0x1a, 0xe4, 0x55, 0xf6, 0x96, 0xff, 0xf6, 0xeb, 0x4a, 0x93, 0xc0, 0x77, 0x1e,
0x92, 0x29, 0x8f, 0xcf, 0xb0, 0xd6, 0x61, 0x18, 0x3d, 0x4d, 0x41, 0x95, 0x24, 0xd1, 0x72, 0x27,
0xe1, 0x9d, 0x33, 0xc1, 0x5f, 0xda, 0xcb, 0xec, 0x99, 0xfa, 0x19, 0xc9, 0x10, 0x1a, 0x73, 0xe4,
0x0b, 0x90, 0x0e, 0x43, 0xbf, 0xbc, 0x03, 0x6e, 0x7d, 0x3f, 0x1d, 0x53, 0xd7, 0x65, 0xf6, 0x4c,
0xf2, 0x8b, 0xfc, 0x19, 0x47, 0x8f, 0xbb, 0xa3, 0x54, 0xc0, 0x1f, 0xa1, 0xd2, 0x61, 0x68, 0x0c,
0x1f, 0x11, 0x79, 0x01, 0xeb, 0x7e, 0x0d, 0x90, 0x22, 0x97, 0xc8, 0x2d, 0xf9, 0x78, 0xe8, 0x76,
0xfb, 0xeb, 0xdb, 0x11, 0xc7, 0xf1, 0xd4, 0xf4, 0x6f, 0xb7, 0x6e, 0xb9, 0x6d, 0xf3, 0x5b, 0x64,
0xd6, 0xb8, 0x15, 0x5e, 0x78, 0x3d, 0xe4, 0x12, 0x3d, 0x6e, 0x4e, 0x91, 0x0d, 0x5b, 0x71, 0x2d,
0x5b, 0x01, 0xb2, 0x15, 0x4e, 0xc4, 0xc4, 0x34, 0x97, 0x83, 0xf5, 0xf1, 0x9f, 0x00, 0x00, 0x00,
0xff, 0xff, 0xcc, 0x29, 0xfa, 0xd0, 0x39, 0x0a, 0x00, 0x00,
// 756 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x96, 0xfb, 0x6e, 0xd3, 0x30,
0x14, 0xc6, 0x77, 0x63, 0x68, 0x67, 0xbd, 0x0c, 0x33, 0xc1, 0x14, 0x26, 0x36, 0x0a, 0x6c, 0xdd,
0x85, 0x14, 0x6d, 0x2f, 0xc0, 0xba, 0xa0, 0xae, 0x12, 0x93, 0xa0, 0x2d, 0xa0, 0x81, 0xa6, 0x29,
0x49, 0xad, 0xd6, 0x22, 0x89, 0x4b, 0x8e, 0xdb, 0xc1, 0x9e, 0x98, 0xc7, 0x40, 0xb9, 0x39, 0x49,
0x97, 0x74, 0x41, 0xec, 0x4f, 0xdb, 0x3f, 0x7f, 0x9f, 0x7d, 0xce, 0xa7, 0xc4, 0x50, 0xb2, 0x75,
0x14, 0xd4, 0x55, 0x47, 0x2e, 0x17, 0x9c, 0x3c, 0xb6, 0x99, 0x35, 0x19, 0x63, 0x30, 0x52, 0x83,
0x25, 0xa5, 0x64, 0x72, 0xdb, 0xe6, 0x4e, 0x30, 0xa9, 0x94, 0x92, 0x88, 0x52, 0x61, 0x8e, 0xa0,
0xae, 0xa3, 0x5b, 0xe1, 0x98, 0xf4, 0x75, 0xa1, 0x5f, 0x21, 0x75, 0x27, 0xcc, 0xa4, 0xc1, 0x5c,
0xad, 0x0b, 0x2b, 0x6d, 0xad, 0x43, 0x7f, 0x8e, 0x29, 0x0a, 0xf2, 0x16, 0x96, 0x0c, 0x1d, 0xe9,
0xc6, 0xfc, 0xf6, 0x7c, 0x7d, 0xf5, 0x68, 0x53, 0x4d, 0x19, 0x86, 0x46, 0xe7, 0x38, 0x68, 0xea,
0x48, 0x3b, 0x3e, 0x49, 0xd6, 0xe1, 0x81, 0xc9, 0xc7, 0x8e, 0xd8, 0x58, 0xd8, 0x9e, 0xaf, 0x97,
0x3b, 0xc1, 0xa0, 0x36, 0x00, 0xf0, 0x44, 0x71, 0xc4, 0x1d, 0xa4, 0xe4, 0x18, 0x96, 0x51, 0xe8,
0x62, 0x8c, 0xa1, 0xee, 0xb3, 0x4c, 0xdd, 0xae, 0x8f, 0x74, 0x42, 0x94, 0x54, 0x60, 0xa1, 0xad,
0xf9, 0xaa, 0x8b, 0x9d, 0x85, 0xb6, 0x16, 0x1b, 0x2d, 0x26, 0x8d, 0x7a, 0x00, 0x3d, 0xe4, 0xf7,
0x70, 0xfc, 0x94, 0xea, 0x04, 0x56, 0x7d, 0xd5, 0xff, 0x39, 0xff, 0x26, 0xac, 0x08, 0x66, 0x53,
0x14, 0xba, 0x3d, 0xf2, 0xaf, 0xb1, 0xd4, 0x89, 0x27, 0xb2, 0x7d, 0x8f, 0xfe, 0x3c, 0x82, 0xf2,
0xb9, 0xdf, 0xd6, 0x6e, 0xd0, 0x23, 0x72, 0x05, 0x6b, 0xa7, 0x2e, 0xd5, 0x05, 0x3d, 0xe5, 0x96,
0x45, 0x4d, 0xc1, 0xb8, 0x43, 0x0e, 0xd3, 0xf6, 0xe1, 0x60, 0x1a, 0x0b, 0x6b, 0xa2, 0xcc, 0x3a,
0x6c, 0x6d, 0x8e, 0x7c, 0x87, 0x8a, 0xe6, 0xf2, 0x51, 0x42, 0x7e, 0x3f, 0x53, 0x3e, 0x0d, 0x15,
0x14, 0xbf, 0x82, 0xf2, 0x99, 0x8e, 0x09, 0xed, 0xbd, 0x4c, 0xed, 0x14, 0x13, 0x49, 0xbf, 0xc8,
0x44, 0x9b, 0x9c, 0x5b, 0x51, 0x5f, 0x6a, 0x73, 0xe4, 0x1a, 0x88, 0x46, 0xd1, 0x74, 0x99, 0x91,
0x2c, 0x90, 0x9a, 0x7d, 0x83, 0x5b, 0x60, 0x64, 0xd5, 0x28, 0xcc, 0x4b, 0xe3, 0x09, 0x3c, 0x6d,
0x51, 0x11, 0x2f, 0x79, 0x57, 0x66, 0x28, 0x98, 0x89, 0xe4, 0x20, 0xbb, 0x3d, 0x29, 0x14, 0x23,
0xeb, 0xc3, 0x62, 0xb0, 0xf4, 0xb5, 0xa0, 0xda, 0x1d, 0xf2, 0xeb, 0x18, 0xc0, 0x9c, 0x7e, 0xa5,
0xa9, 0xc8, 0xee, 0xa0, 0x10, 0x2b, 0xdd, 0x2e, 0xa1, 0x1a, 0xc4, 0xea, 0xa3, 0xee, 0x0a, 0xe6,
0xd7, 0xf6, 0x60, 0x46, 0xf8, 0x24, 0x55, 0x30, 0x1e, 0x17, 0x50, 0xf6, 0x62, 0x15, 0x8b, 0xef,
0xe5, 0x46, 0xef, 0x5f, 0xa5, 0x2f, 0xa1, 0x74, 0xa6, 0x63, 0xac, 0x5c, 0xcf, 0x0b, 0xde, 0x2d,
0xe1, 0x42, 0xb9, 0x43, 0x78, 0xd2, 0xa2, 0x42, 0xee, 0x4d, 0x74, 0x3f, 0xbb, 0x1b, 0x29, 0x12,
0x67, 0x77, 0x63, 0x9a, 0x95, 0xa6, 0x0c, 0x2a, 0x5e, 0xa7, 0xe4, 0x3a, 0xe6, 0xd4, 0x2b, 0x05,
0x45, 0x5e, 0xfb, 0x45, 0x50, 0x69, 0xe5, 0x40, 0x35, 0x8a, 0x7f, 0x97, 0x0e, 0x6c, 0xea, 0x88,
0x9c, 0xc6, 0x4f, 0x51, 0xb3, 0x63, 0x7d, 0x0b, 0x96, 0x7e, 0x26, 0x94, 0xbc, 0xa3, 0x84, 0x0b,
0x48, 0x76, 0x73, 0x4f, 0x3b, 0x65, 0x54, 0xbf, 0x1b, 0x94, 0x26, 0x9f, 0x61, 0x35, 0xc8, 0x69,
0xdb, 0xe9, 0xd3, 0x5f, 0x39, 0x1e, 0x09, 0xa2, 0x60, 0xd4, 0x86, 0x50, 0x8e, 0x2e, 0x16, 0x08,
0xef, 0xcd, 0xbc, 0x7c, 0x4a, 0x7a, 0xbf, 0x08, 0x2a, 0x2f, 0x60, 0x40, 0xb9, 0x45, 0x85, 0x3f,
0xeb, 0xb9, 0x53, 0xb2, 0x93, 0xb9, 0x3d, 0x06, 0x22, 0x9b, 0xdd, 0x3b, 0xb9, 0x44, 0x91, 0x2a,
0x27, 0x96, 0xc5, 0xcd, 0x9e, 0xfc, 0x55, 0x6d, 0xa9, 0x19, 0xcf, 0x0e, 0x35, 0xfe, 0xeb, 0x2a,
0xdb, 0xf9, 0x80, 0x94, 0xfd, 0x00, 0x0f, 0x7d, 0xd9, 0xb6, 0x46, 0x9e, 0x67, 0xe2, 0xf2, 0x0d,
0xa2, 0x6c, 0xe5, 0xae, 0x27, 0x0a, 0x51, 0x3d, 0x41, 0x64, 0x03, 0x27, 0x6c, 0x72, 0x5b, 0x23,
0xaf, 0xd3, 0xbb, 0xbc, 0x87, 0x8e, 0x2a, 0x99, 0x58, 0x7c, 0xe7, 0x2e, 0x4c, 0x7a, 0x7c, 0x81,
0xb5, 0x16, 0x15, 0xe1, 0x7f, 0xd8, 0xaf, 0x12, 0x12, 0x25, 0x33, 0x09, 0xef, 0xed, 0x91, 0xf8,
0xad, 0xbc, 0x4a, 0xaf, 0xc9, 0x97, 0x57, 0x4a, 0xa1, 0x36, 0x47, 0xbe, 0x02, 0x69, 0x51, 0xe1,
0x95, 0xb7, 0xc7, 0xcc, 0x1f, 0xa7, 0x43, 0xdd, 0x71, 0xa8, 0x35, 0x53, 0xf9, 0x65, 0x76, 0xc6,
0x85, 0xcb, 0x9c, 0x41, 0xe2, 0xc0, 0x9f, 0xa0, 0xd4, 0xa2, 0x42, 0xeb, 0xdf, 0xa3, 0xe4, 0x05,
0xac, 0x7b, 0x35, 0x90, 0x5f, 0xb7, 0xfb, 0x93, 0x6e, 0x36, 0xbf, 0xbd, 0x1b, 0x30, 0x31, 0x1c,
0x1b, 0xde, 0xee, 0xc6, 0x0d, 0xb3, 0x2c, 0x76, 0x23, 0xa8, 0x39, 0x6c, 0x04, 0x1b, 0xde, 0xf4,
0x19, 0x0a, 0x97, 0x19, 0x63, 0x41, 0xfb, 0x8d, 0xa8, 0x96, 0x0d, 0x5f, 0xb2, 0x11, 0x24, 0x62,
0x64, 0x18, 0xcb, 0xfe, 0xf8, 0xf8, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xb7, 0xb2, 0x22, 0xf7,
0x26, 0x0b, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
......@@ -330,6 +333,7 @@ type MasterServiceClient interface {
//
// @return CollectionSchema
DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error)
GetCollectionStatistics(ctx context.Context, in *milvuspb.CollectionStatsRequest, opts ...grpc.CallOption) (*milvuspb.CollectionStatsResponse, error)
//*
// @brief This method is used to list all collections.
//
......@@ -350,6 +354,7 @@ type MasterServiceClient interface {
//
// @return BoolResponse
HasPartition(ctx context.Context, in *milvuspb.HasPartitionRequest, opts ...grpc.CallOption) (*milvuspb.BoolResponse, error)
GetPartitionStatistics(ctx context.Context, in *milvuspb.PartitionStatsRequest, opts ...grpc.CallOption) (*milvuspb.PartitionStatsResponse, error)
//*
// @brief This method is used to show partition information
//
......@@ -416,6 +421,15 @@ func (c *masterServiceClient) DescribeCollection(ctx context.Context, in *milvus
return out, nil
}
func (c *masterServiceClient) GetCollectionStatistics(ctx context.Context, in *milvuspb.CollectionStatsRequest, opts ...grpc.CallOption) (*milvuspb.CollectionStatsResponse, error) {
out := new(milvuspb.CollectionStatsResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/GetCollectionStatistics", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *masterServiceClient) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionRequest, opts ...grpc.CallOption) (*milvuspb.ShowCollectionResponse, error) {
out := new(milvuspb.ShowCollectionResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/ShowCollections", in, out, opts...)
......@@ -452,6 +466,15 @@ func (c *masterServiceClient) HasPartition(ctx context.Context, in *milvuspb.Has
return out, nil
}
func (c *masterServiceClient) GetPartitionStatistics(ctx context.Context, in *milvuspb.PartitionStatsRequest, opts ...grpc.CallOption) (*milvuspb.PartitionStatsResponse, error) {
out := new(milvuspb.PartitionStatsResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/GetPartitionStatistics", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *masterServiceClient) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionResponse, error) {
out := new(milvuspb.ShowPartitionResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/ShowPartitions", in, out, opts...)
......@@ -599,6 +622,7 @@ type MasterServiceServer interface {
//
// @return CollectionSchema
DescribeCollection(context.Context, *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error)
GetCollectionStatistics(context.Context, *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error)
//*
// @brief This method is used to list all collections.
//
......@@ -619,6 +643,7 @@ type MasterServiceServer interface {
//
// @return BoolResponse
HasPartition(context.Context, *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error)
GetPartitionStatistics(context.Context, *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error)
//*
// @brief This method is used to show partition information
//
......@@ -657,6 +682,9 @@ func (*UnimplementedMasterServiceServer) HasCollection(ctx context.Context, req
func (*UnimplementedMasterServiceServer) DescribeCollection(ctx context.Context, req *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method DescribeCollection not implemented")
}
func (*UnimplementedMasterServiceServer) GetCollectionStatistics(ctx context.Context, req *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetCollectionStatistics not implemented")
}
func (*UnimplementedMasterServiceServer) ShowCollections(ctx context.Context, req *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method ShowCollections not implemented")
}
......@@ -669,6 +697,9 @@ func (*UnimplementedMasterServiceServer) DropPartition(ctx context.Context, req
func (*UnimplementedMasterServiceServer) HasPartition(ctx context.Context, req *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method HasPartition not implemented")
}
func (*UnimplementedMasterServiceServer) GetPartitionStatistics(ctx context.Context, req *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetPartitionStatistics not implemented")
}
func (*UnimplementedMasterServiceServer) ShowPartitions(ctx context.Context, req *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented")
}
......@@ -785,6 +816,24 @@ func _MasterService_DescribeCollection_Handler(srv interface{}, ctx context.Cont
return interceptor(ctx, in, info, handler)
}
func _MasterService_GetCollectionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(milvuspb.CollectionStatsRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(MasterServiceServer).GetCollectionStatistics(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.master.MasterService/GetCollectionStatistics",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(MasterServiceServer).GetCollectionStatistics(ctx, req.(*milvuspb.CollectionStatsRequest))
}
return interceptor(ctx, in, info, handler)
}
func _MasterService_ShowCollections_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(milvuspb.ShowCollectionRequest)
if err := dec(in); err != nil {
......@@ -857,6 +906,24 @@ func _MasterService_HasPartition_Handler(srv interface{}, ctx context.Context, d
return interceptor(ctx, in, info, handler)
}
func _MasterService_GetPartitionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(milvuspb.PartitionStatsRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(MasterServiceServer).GetPartitionStatistics(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.master.MasterService/GetPartitionStatistics",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(MasterServiceServer).GetPartitionStatistics(ctx, req.(*milvuspb.PartitionStatsRequest))
}
return interceptor(ctx, in, info, handler)
}
func _MasterService_ShowPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(milvuspb.ShowPartitionRequest)
if err := dec(in); err != nil {
......@@ -1111,6 +1178,10 @@ var _MasterService_serviceDesc = grpc.ServiceDesc{
MethodName: "DescribeCollection",
Handler: _MasterService_DescribeCollection_Handler,
},
{
MethodName: "GetCollectionStatistics",
Handler: _MasterService_GetCollectionStatistics_Handler,
},
{
MethodName: "ShowCollections",
Handler: _MasterService_ShowCollections_Handler,
......@@ -1127,6 +1198,10 @@ var _MasterService_serviceDesc = grpc.ServiceDesc{
MethodName: "HasPartition",
Handler: _MasterService_HasPartition_Handler,
},
{
MethodName: "GetPartitionStatistics",
Handler: _MasterService_GetPartitionStatistics_Handler,
},
{
MethodName: "ShowPartitions",
Handler: _MasterService_ShowPartitions_Handler,
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册