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