From 67e6ea805143bcbcc65f005b6b72d2b4b8d37b37 Mon Sep 17 00:00:00 2001 From: sunby <bingyi.sun@zilliz.com> Date: Fri, 15 Jan 2021 17:09:41 +0800 Subject: [PATCH] Add data service Signed-off-by: sunby <bingyi.sun@zilliz.com> --- internal/dataservice/channel.go | 47 + internal/dataservice/data_service.go | 5 + internal/dataservice/grpc_service.go | 40 + internal/dataservice/meta.go | 180 +++ internal/dataservice/param.go | 370 ++++++ internal/dataservice/segment_allocator.go | 250 ++++ internal/dataservice/watcher.go | 107 ++ internal/proto/data_service.proto | 10 + internal/proto/datapb/data_service.pb.go | 224 +++- internal/proto/milvus.proto | 75 +- internal/proto/milvuspb/milvus.pb.go | 1367 ++------------------- internal/proxynode/proxy.go | 6 +- internal/proxyservice/interface.go | 16 - internal/proxyservice/proxyservice.go | 54 - internal/querynode/query_node.go | 6 +- internal/timesync/time_sync_producer.go | 141 +++ internal/timesync/timesync.go | 235 ++++ internal/timesync/timetick_watcher.go | 11 + internal/writenode/write_node.go | 6 +- 19 files changed, 1677 insertions(+), 1473 deletions(-) create mode 100644 internal/dataservice/channel.go create mode 100644 internal/dataservice/data_service.go create mode 100644 internal/dataservice/grpc_service.go create mode 100644 internal/dataservice/meta.go create mode 100644 internal/dataservice/param.go create mode 100644 internal/dataservice/segment_allocator.go create mode 100644 internal/dataservice/watcher.go delete mode 100644 internal/proxyservice/interface.go delete mode 100644 internal/proxyservice/proxyservice.go create mode 100644 internal/timesync/time_sync_producer.go create mode 100644 internal/timesync/timesync.go create mode 100644 internal/timesync/timetick_watcher.go diff --git a/internal/dataservice/channel.go b/internal/dataservice/channel.go new file mode 100644 index 000000000..a974d8928 --- /dev/null +++ b/internal/dataservice/channel.go @@ -0,0 +1,47 @@ +package dataservice + +import "fmt" + +type ( + channelRange []string + insertChannelMapper struct { + channelRanges []channelRange + } +) + +func (cr channelRange) Contains(channelName string) bool { + for _, name := range cr { + if name == channelName { + return true + } + } + return false +} + +func newInsertChannelMapper() *insertChannelMapper { + mapper := &insertChannelMapper{channelRanges: make([]channelRange, Params.QueryNodeNum)} + channelNames, numOfChannels, numOfQueryNodes := Params.InsertChannelNames, len(Params.InsertChannelNames), Params.QueryNodeNum + div, rem := numOfChannels/numOfQueryNodes, numOfChannels%numOfQueryNodes + for i, j := 0, 0; i < numOfChannels; j++ { + numOfRange := div + if j < rem { + numOfRange++ + } + cRange := channelRange{} + k := i + numOfRange + for ; i < k; i++ { + cRange = append(cRange, channelNames[i]) + } + mapper.channelRanges = append(mapper.channelRanges, cRange) + } + return mapper +} + +func (mapper *insertChannelMapper) GetChannelRange(channelName string) (channelRange, error) { + for _, cr := range mapper.channelRanges { + if cr.Contains(channelName) { + return cr, nil + } + } + return nil, fmt.Errorf("channel name %s not found", channelName) +} diff --git a/internal/dataservice/data_service.go b/internal/dataservice/data_service.go new file mode 100644 index 000000000..32ebc5d9b --- /dev/null +++ b/internal/dataservice/data_service.go @@ -0,0 +1,5 @@ +package dataservice + +type DataService struct { + segAllocator segmentAllocator +} diff --git a/internal/dataservice/grpc_service.go b/internal/dataservice/grpc_service.go new file mode 100644 index 000000000..e4b9184a7 --- /dev/null +++ b/internal/dataservice/grpc_service.go @@ -0,0 +1,40 @@ +package dataservice + +import ( + "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" + "github.com/zilliztech/milvus-distributed/internal/proto/datapb" + "github.com/zilliztech/milvus-distributed/internal/proto/internalpb2" + "golang.org/x/net/context" +) + +func (ds *DataService) RegisterNode(context.Context, *datapb.RegisterNodeRequest) (*datapb.RegisterNodeResponse, error) { + return nil, nil +} +func (ds *DataService) Flush(context.Context, *datapb.FlushRequest) (*commonpb.Status, error) { + return nil, nil +} +func (ds *DataService) AssignSegmentID(ctx context.Context, request *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error) { + //for _, req := range request.SegIDRequests { + // segmentID, retCount, expireTs, err := ds.segAllocator.AllocSegment(req.CollectionID, req.PartitionID, req.ChannelID, int(req.Count)) + // if err != nil { + // log.Printf() + // } + //} + return nil, nil +} +func (ds *DataService) ShowSegments(context.Context, *datapb.ShowSegmentRequest) (*datapb.ShowSegmentResponse, error) { + return nil, nil + +} +func (ds *DataService) GetSegmentStates(context.Context, *datapb.SegmentStatesRequest) (*datapb.SegmentStatesResponse, error) { + return nil, nil + +} +func (ds *DataService) GetInsertBinlogPaths(context.Context, *datapb.InsertBinlogPathRequest) (*datapb.InsertBinlogPathsResponse, error) { + return nil, nil + +} +func (ds *DataService) GetInsertChannels(context.Context, *datapb.InsertChannelRequest) (*internalpb2.StringList, error) { + return nil, nil + +} diff --git a/internal/dataservice/meta.go b/internal/dataservice/meta.go new file mode 100644 index 000000000..88b058e4b --- /dev/null +++ b/internal/dataservice/meta.go @@ -0,0 +1,180 @@ +package dataservice + +import ( + "fmt" + "strconv" + "sync" + + log "github.com/sirupsen/logrus" + + "github.com/zilliztech/milvus-distributed/internal/proto/datapb" + "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" + + "github.com/zilliztech/milvus-distributed/internal/util/typeutil" + + "github.com/golang/protobuf/proto" + "github.com/zilliztech/milvus-distributed/internal/errors" + "github.com/zilliztech/milvus-distributed/internal/kv" +) + +type ( + UniqueID = typeutil.UniqueID + Timestamp = typeutil.Timestamp + collectionInfo struct { + ID UniqueID + Schema *schemapb.CollectionSchema + } + meta struct { + client kv.TxnBase // client of a reliable kv service, i.e. etcd client + collID2Info map[UniqueID]*collectionInfo // collection id to collection info + segID2Info map[UniqueID]*datapb.SegmentInfo // segment id to segment info + + ddLock sync.RWMutex + } +) + +func NewMetaTable(kv kv.TxnBase) (*meta, error) { + mt := &meta{ + client: kv, + collID2Info: make(map[UniqueID]*collectionInfo), + segID2Info: make(map[UniqueID]*datapb.SegmentInfo), + } + err := mt.reloadFromKV() + if err != nil { + return nil, err + } + return mt, nil +} + +func (mt *meta) reloadFromKV() error { + _, values, err := mt.client.LoadWithPrefix("segment") + if err != nil { + return err + } + + for _, value := range values { + segmentInfo := &datapb.SegmentInfo{} + err = proto.UnmarshalText(value, segmentInfo) + if err != nil { + return err + } + mt.segID2Info[segmentInfo.SegmentID] = segmentInfo + } + + return nil +} + +func (mt *meta) AddCollection(collectionInfo *collectionInfo) error { + mt.ddLock.Lock() + defer mt.ddLock.Unlock() + if _, ok := mt.collID2Info[collectionInfo.ID]; ok { + return fmt.Errorf("collection %s with id %d already exist", collectionInfo.Schema.Name, collectionInfo.ID) + } + mt.collID2Info[collectionInfo.ID] = collectionInfo + return nil +} + +func (mt *meta) DropCollection(collID UniqueID) error { + mt.ddLock.Lock() + defer mt.ddLock.Unlock() + + if _, ok := mt.collID2Info[collID]; !ok { + return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10)) + } + delete(mt.collID2Info, collID) + for id, segment := range mt.segID2Info { + if segment.CollectionID != collID { + continue + } + delete(mt.segID2Info, id) + if err := mt.removeSegmentInfo(id); err != nil { + log.Printf("remove segment info failed, %s", err.Error()) + _ = mt.reloadFromKV() + } + } + return nil +} + +func (mt *meta) HasCollection(collID UniqueID) bool { + mt.ddLock.RLock() + defer mt.ddLock.RUnlock() + _, ok := mt.collID2Info[collID] + return ok +} + +func (mt *meta) AddSegment(segmentInfo *datapb.SegmentInfo) error { + mt.ddLock.Lock() + defer mt.ddLock.Unlock() + if _, ok := mt.segID2Info[segmentInfo.SegmentID]; !ok { + return fmt.Errorf("segment %d already exist", segmentInfo.SegmentID) + } + mt.segID2Info[segmentInfo.SegmentID] = segmentInfo + if err := mt.saveSegmentInfo(segmentInfo); err != nil { + _ = mt.reloadFromKV() + return err + } + return nil +} + +func (mt *meta) UpdateSegment(segmentInfo *datapb.SegmentInfo) error { + mt.ddLock.Lock() + defer mt.ddLock.Unlock() + + mt.segID2Info[segmentInfo.SegmentID] = segmentInfo + if err := mt.saveSegmentInfo(segmentInfo); err != nil { + _ = mt.reloadFromKV() + return err + } + return nil +} + +func (mt *meta) GetSegmentByID(segID UniqueID) (*datapb.SegmentInfo, error) { + mt.ddLock.RLock() + defer mt.ddLock.RUnlock() + + segmentInfo, ok := mt.segID2Info[segID] + if !ok { + return nil, errors.Errorf("GetSegmentByID:can't find segment id = %d", segID) + } + return segmentInfo, nil +} + +func (mt *meta) CloseSegment(segID UniqueID, closeTs Timestamp) error { + mt.ddLock.Lock() + defer mt.ddLock.Unlock() + + segInfo, ok := mt.segID2Info[segID] + if !ok { + return errors.Errorf("DropSegment:can't find segment id = " + strconv.FormatInt(segID, 10)) + } + + segInfo.CloseTime = closeTs + + err := mt.saveSegmentInfo(segInfo) + if err != nil { + _ = mt.reloadFromKV() + return err + } + return nil +} + +func (mt *meta) GetCollection(collectionID UniqueID) (*collectionInfo, error) { + mt.ddLock.RLock() + defer mt.ddLock.RUnlock() + + collectionInfo, ok := mt.collID2Info[collectionID] + if !ok { + return nil, fmt.Errorf("collection %d not found", collectionID) + } + return collectionInfo, nil +} + +func (mt *meta) saveSegmentInfo(segmentInfo *datapb.SegmentInfo) error { + segBytes := proto.MarshalTextString(segmentInfo) + + return mt.client.Save("/segment/"+strconv.FormatInt(segmentInfo.SegmentID, 10), segBytes) +} + +func (mt *meta) removeSegmentInfo(segID UniqueID) error { + return mt.client.Remove("/segment/" + strconv.FormatInt(segID, 10)) +} diff --git a/internal/dataservice/param.go b/internal/dataservice/param.go new file mode 100644 index 000000000..db4ddb6a1 --- /dev/null +++ b/internal/dataservice/param.go @@ -0,0 +1,370 @@ +package dataservice + +import ( + "log" + "strconv" + "strings" + + "github.com/zilliztech/milvus-distributed/internal/util/paramtable" + "github.com/zilliztech/milvus-distributed/internal/util/tsoutil" + "github.com/zilliztech/milvus-distributed/internal/util/typeutil" +) + +type ParamTable struct { + paramtable.BaseTable + + Address string + Port int + + EtcdAddress string + MetaRootPath string + KvRootPath string + WriteNodeSegKvSubPath string + PulsarAddress string + IndexBuilderAddress string + + // nodeID + ProxyIDList []typeutil.UniqueID + WriteNodeIDList []typeutil.UniqueID + + TopicNum int + QueryNodeNum int + SoftTimeTickBarrierInterval typeutil.Timestamp + + // segment + SegmentSize float64 + SegmentSizeFactor float64 + DefaultRecordSize int64 + MinSegIDAssignCnt int64 + MaxSegIDAssignCnt int64 + SegIDAssignExpiration int64 + + // msgChannel + ProxyTimeTickChannelNames []string + WriteNodeTimeTickChannelNames []string + DDChannelNames []string + InsertChannelNames []string + K2SChannelNames []string + QueryNodeStatsChannelName string + MsgChannelSubName string + + MaxPartitionNum int64 + DefaultPartitionTag string + + LoadIndexChannelNames []string +} + +var Params ParamTable + +func (p *ParamTable) Init() { + // load yaml + p.BaseTable.Init() + + err := p.LoadYaml("advanced/master.yaml") + if err != nil { + panic(err) + } + + // set members + p.initAddress() + p.initPort() + + p.initEtcdAddress() + p.initMetaRootPath() + p.initKvRootPath() + p.initWriteNodeSegKvSubPath() + p.initPulsarAddress() + p.initIndexBuilderAddress() + + p.initProxyIDList() + p.initWriteNodeIDList() + + p.initTopicNum() + p.initQueryNodeNum() + p.initSoftTimeTickBarrierInterval() + + p.initSegmentSize() + p.initSegmentSizeFactor() + p.initDefaultRecordSize() + p.initMinSegIDAssignCnt() + p.initMaxSegIDAssignCnt() + p.initSegIDAssignExpiration() + + p.initProxyTimeTickChannelNames() + p.initWriteNodeTimeTickChannelNames() + p.initInsertChannelNames() + p.initDDChannelNames() + p.initK2SChannelNames() + p.initQueryNodeStatsChannelName() + p.initMsgChannelSubName() + p.initMaxPartitionNum() + p.initDefaultPartitionTag() + + p.initLoadIndexChannelNames() +} + +func (p *ParamTable) initAddress() { + masterAddress, err := p.Load("master.address") + if err != nil { + panic(err) + } + p.Address = masterAddress +} + +func (p *ParamTable) initPort() { + p.Port = p.ParseInt("master.port") +} + +func (p *ParamTable) initEtcdAddress() { + addr, err := p.Load("_EtcdAddress") + if err != nil { + panic(err) + } + p.EtcdAddress = addr +} + +func (p *ParamTable) initPulsarAddress() { + addr, err := p.Load("_PulsarAddress") + if err != nil { + panic(err) + } + p.PulsarAddress = addr +} + +func (p *ParamTable) initIndexBuilderAddress() { + ret, err := p.Load("_IndexBuilderAddress") + if err != nil { + panic(err) + } + p.IndexBuilderAddress = ret +} + +func (p *ParamTable) initMetaRootPath() { + rootPath, err := p.Load("etcd.rootPath") + if err != nil { + panic(err) + } + subPath, err := p.Load("etcd.metaSubPath") + if err != nil { + panic(err) + } + p.MetaRootPath = rootPath + "/" + subPath +} + +func (p *ParamTable) initKvRootPath() { + rootPath, err := p.Load("etcd.rootPath") + if err != nil { + panic(err) + } + subPath, err := p.Load("etcd.kvSubPath") + if err != nil { + panic(err) + } + p.KvRootPath = rootPath + "/" + subPath +} + +func (p *ParamTable) initWriteNodeSegKvSubPath() { + subPath, err := p.Load("etcd.writeNodeSegKvSubPath") + if err != nil { + panic(err) + } + p.WriteNodeSegKvSubPath = subPath + "/" +} + +func (p *ParamTable) initTopicNum() { + iRangeStr, err := p.Load("msgChannel.channelRange.insert") + if err != nil { + panic(err) + } + rangeSlice := paramtable.ConvertRangeToIntRange(iRangeStr, ",") + p.TopicNum = rangeSlice[1] - rangeSlice[0] +} + +func (p *ParamTable) initSegmentSize() { + p.SegmentSize = p.ParseFloat("master.segment.size") +} + +func (p *ParamTable) initSegmentSizeFactor() { + p.SegmentSizeFactor = p.ParseFloat("master.segment.sizeFactor") +} + +func (p *ParamTable) initDefaultRecordSize() { + p.DefaultRecordSize = p.ParseInt64("master.segment.defaultSizePerRecord") +} + +func (p *ParamTable) initMinSegIDAssignCnt() { + p.MinSegIDAssignCnt = p.ParseInt64("master.segment.minIDAssignCnt") +} + +func (p *ParamTable) initMaxSegIDAssignCnt() { + p.MaxSegIDAssignCnt = p.ParseInt64("master.segment.maxIDAssignCnt") +} + +func (p *ParamTable) initSegIDAssignExpiration() { + p.SegIDAssignExpiration = p.ParseInt64("master.segment.IDAssignExpiration") +} + +func (p *ParamTable) initQueryNodeNum() { + p.QueryNodeNum = len(p.QueryNodeIDList()) +} + +func (p *ParamTable) initQueryNodeStatsChannelName() { + channels, err := p.Load("msgChannel.chanNamePrefix.queryNodeStats") + if err != nil { + panic(err) + } + p.QueryNodeStatsChannelName = channels +} + +func (p *ParamTable) initProxyIDList() { + p.ProxyIDList = p.BaseTable.ProxyIDList() +} + +func (p *ParamTable) initProxyTimeTickChannelNames() { + ch, err := p.Load("msgChannel.chanNamePrefix.proxyTimeTick") + if err != nil { + log.Panic(err) + } + id, err := p.Load("nodeID.proxyIDList") + if err != nil { + log.Panicf("load proxy id list error, %s", err.Error()) + } + ids := strings.Split(id, ",") + channels := make([]string, 0, len(ids)) + for _, i := range ids { + _, err := strconv.ParseInt(i, 10, 64) + if err != nil { + log.Panicf("load proxy id list error, %s", err.Error()) + } + channels = append(channels, ch+"-"+i) + } + p.ProxyTimeTickChannelNames = channels +} + +func (p *ParamTable) initMsgChannelSubName() { + name, err := p.Load("msgChannel.subNamePrefix.masterSubNamePrefix") + if err != nil { + log.Panic(err) + } + p.MsgChannelSubName = name +} + +func (p *ParamTable) initSoftTimeTickBarrierInterval() { + t, err := p.Load("master.timeSync.softTimeTickBarrierInterval") + if err != nil { + log.Panic(err) + } + v, err := strconv.ParseInt(t, 10, 64) + if err != nil { + log.Panic(err) + } + p.SoftTimeTickBarrierInterval = tsoutil.ComposeTS(v, 0) +} + +func (p *ParamTable) initWriteNodeIDList() { + p.WriteNodeIDList = p.BaseTable.WriteNodeIDList() +} + +func (p *ParamTable) initWriteNodeTimeTickChannelNames() { + ch, err := p.Load("msgChannel.chanNamePrefix.writeNodeTimeTick") + if err != nil { + log.Fatal(err) + } + id, err := p.Load("nodeID.writeNodeIDList") + if err != nil { + log.Panicf("load write node id list error, %s", err.Error()) + } + ids := strings.Split(id, ",") + channels := make([]string, 0, len(ids)) + for _, i := range ids { + _, err := strconv.ParseInt(i, 10, 64) + if err != nil { + log.Panicf("load write node id list error, %s", err.Error()) + } + channels = append(channels, ch+"-"+i) + } + p.WriteNodeTimeTickChannelNames = channels +} + +func (p *ParamTable) initDDChannelNames() { + prefix, err := p.Load("msgChannel.chanNamePrefix.dataDefinition") + if err != nil { + panic(err) + } + prefix += "-" + iRangeStr, err := p.Load("msgChannel.channelRange.dataDefinition") + if err != nil { + panic(err) + } + channelIDs := paramtable.ConvertRangeToIntSlice(iRangeStr, ",") + var ret []string + for _, ID := range channelIDs { + ret = append(ret, prefix+strconv.Itoa(ID)) + } + p.DDChannelNames = ret +} + +func (p *ParamTable) initInsertChannelNames() { + prefix, err := p.Load("msgChannel.chanNamePrefix.insert") + if err != nil { + panic(err) + } + prefix += "-" + iRangeStr, err := p.Load("msgChannel.channelRange.insert") + if err != nil { + panic(err) + } + channelIDs := paramtable.ConvertRangeToIntSlice(iRangeStr, ",") + var ret []string + for _, ID := range channelIDs { + ret = append(ret, prefix+strconv.Itoa(ID)) + } + p.InsertChannelNames = ret +} + +func (p *ParamTable) initK2SChannelNames() { + prefix, err := p.Load("msgChannel.chanNamePrefix.k2s") + if err != nil { + panic(err) + } + prefix += "-" + iRangeStr, err := p.Load("msgChannel.channelRange.k2s") + if err != nil { + panic(err) + } + channelIDs := paramtable.ConvertRangeToIntSlice(iRangeStr, ",") + var ret []string + for _, ID := range channelIDs { + ret = append(ret, prefix+strconv.Itoa(ID)) + } + p.K2SChannelNames = ret +} + +func (p *ParamTable) initMaxPartitionNum() { + str, err := p.Load("master.maxPartitionNum") + if err != nil { + panic(err) + } + maxPartitionNum, err := strconv.ParseInt(str, 10, 64) + if err != nil { + panic(err) + } + p.MaxPartitionNum = maxPartitionNum +} + +func (p *ParamTable) initDefaultPartitionTag() { + defaultTag, err := p.Load("common.defaultPartitionTag") + if err != nil { + panic(err) + } + + p.DefaultPartitionTag = defaultTag +} + +func (p *ParamTable) initLoadIndexChannelNames() { + loadIndexChannelName, err := p.Load("msgChannel.chanNamePrefix.cmd") + if err != nil { + panic(err) + } + p.LoadIndexChannelNames = []string{loadIndexChannelName} +} diff --git a/internal/dataservice/segment_allocator.go b/internal/dataservice/segment_allocator.go new file mode 100644 index 000000000..6c1ac9dea --- /dev/null +++ b/internal/dataservice/segment_allocator.go @@ -0,0 +1,250 @@ +package dataservice + +import ( + "fmt" + "strconv" + "sync" + "time" + + "github.com/zilliztech/milvus-distributed/internal/proto/datapb" + + "github.com/zilliztech/milvus-distributed/internal/util/typeutil" + + "github.com/zilliztech/milvus-distributed/internal/util/tsoutil" +) + +type errRemainInSufficient struct { + requestRows int +} + +func newErrRemainInSufficient(requestRows int) *errRemainInSufficient { + return &errRemainInSufficient{requestRows: requestRows} +} + +func (err *errRemainInSufficient) Error() string { + return "segment remaining is insufficient for" + strconv.Itoa(err.requestRows) +} + +// segmentAllocator is used to allocate rows for segments and record the allocations. +type segmentAllocator interface { + // OpenSegment add the segment to allocator and set it allocatable + OpenSegment(segmentInfo *datapb.SegmentInfo) error + // AllocSegment allocate rows and record the allocation. + AllocSegment(collectionID UniqueID, partitionID UniqueID, channelName string, requestRows int) (UniqueID, int, Timestamp, error) + // GetSealedSegments get all sealed segment. + GetSealedSegments() ([]UniqueID, error) + // SealSegment set segment sealed, the segment will not be allocated anymore. + SealSegment(segmentID UniqueID) + // DropSegment drop the segment from allocator. + DropSegment(segmentID UniqueID) + // ExpireAllocations check all allocations' expire time and remove the expired allocation. + ExpireAllocations(timeTick Timestamp) error + // IsAllocationsExpired check all allocations of segment expired. + IsAllocationsExpired(segmentID UniqueID, ts Timestamp) (bool, error) +} + +type ( + segmentStatus struct { + id UniqueID + collectionID UniqueID + partitionID UniqueID + total int + sealed bool + lastExpireTime Timestamp + allocations []*allocation + cRange channelRange + } + allocation struct { + rowNums int + expireTime Timestamp + } + segmentAllocatorImpl struct { + mt *meta + segments map[UniqueID]*segmentStatus //segment id -> status + cMapper *insertChannelMapper + segmentExpireDuration int64 + defaultSizePerRecord int64 + segmentThreshold float64 + segmentThresholdFactor float64 + numOfChannels int + numOfQueryNodes int + mu sync.RWMutex + globalIDAllocator func() (UniqueID, error) + globalTSOAllocator func() (Timestamp, error) + } +) + +func newSegmentAssigner(metaTable *meta, globalIDAllocator func() (UniqueID, error), + globalTSOAllocator func() (Timestamp, error)) (*segmentAllocatorImpl, error) { + segmentAllocator := &segmentAllocatorImpl{ + mt: metaTable, + segments: make(map[UniqueID]*segmentStatus), + segmentExpireDuration: Params.SegIDAssignExpiration, + defaultSizePerRecord: Params.DefaultRecordSize, + segmentThreshold: Params.SegmentSize * 1024 * 1024, + segmentThresholdFactor: Params.SegmentSizeFactor, + numOfChannels: Params.TopicNum, + numOfQueryNodes: Params.QueryNodeNum, + globalIDAllocator: globalIDAllocator, + globalTSOAllocator: globalTSOAllocator, + } + return segmentAllocator, nil +} + +func (allocator *segmentAllocatorImpl) OpenSegment(segmentInfo *datapb.SegmentInfo) error { + if _, ok := allocator.segments[segmentInfo.SegmentID]; ok { + return fmt.Errorf("segment %d already exist", segmentInfo.SegmentID) + } + totalRows, err := allocator.estimateTotalRows(segmentInfo.CollectionID) + if err != nil { + return err + } + allocator.segments[segmentInfo.SegmentID] = &segmentStatus{ + id: segmentInfo.SegmentID, + collectionID: segmentInfo.CollectionID, + partitionID: segmentInfo.PartitionID, + total: totalRows, + sealed: false, + lastExpireTime: 0, + cRange: segmentInfo.InsertChannels, + } + return nil +} + +func (allocator *segmentAllocatorImpl) AllocSegment(collectionID UniqueID, partitionID UniqueID, channelName string, requestRows int) (segID UniqueID, retCount int, expireTime Timestamp, err error) { + allocator.mu.Lock() + defer allocator.mu.Unlock() + + for _, segStatus := range allocator.segments { + if segStatus.sealed || segStatus.collectionID != collectionID || segStatus.partitionID != partitionID || + !segStatus.cRange.Contains(channelName) { + continue + } + var success bool + success, err = allocator.alloc(segStatus, requestRows) + if err != nil { + return + } + if !success { + continue + } + segID = segStatus.id + retCount = requestRows + expireTime = segStatus.lastExpireTime + return + } + + err = newErrRemainInSufficient(requestRows) + return +} + +func (allocator *segmentAllocatorImpl) alloc(segStatus *segmentStatus, numRows int) (bool, error) { + totalOfAllocations := 0 + for _, allocation := range segStatus.allocations { + totalOfAllocations += allocation.rowNums + } + segMeta, err := allocator.mt.GetSegmentByID(segStatus.id) + if err != nil { + return false, err + } + free := segStatus.total - int(segMeta.NumRows) - totalOfAllocations + if numRows > free { + return false, nil + } + + ts, err := allocator.globalTSOAllocator() + if err != nil { + return false, err + } + physicalTs, logicalTs := tsoutil.ParseTS(ts) + expirePhysicalTs := physicalTs.Add(time.Duration(allocator.segmentExpireDuration) * time.Millisecond) + expireTs := tsoutil.ComposeTS(expirePhysicalTs.UnixNano()/int64(time.Millisecond), int64(logicalTs)) + segStatus.lastExpireTime = expireTs + segStatus.allocations = append(segStatus.allocations, &allocation{ + numRows, + ts, + }) + + return true, nil +} + +func (allocator *segmentAllocatorImpl) estimateTotalRows(collectionID UniqueID) (int, error) { + collMeta, err := allocator.mt.GetCollection(collectionID) + if err != nil { + return -1, err + } + sizePerRecord, err := typeutil.EstimateSizePerRecord(collMeta.Schema) + if err != nil { + return -1, err + } + return int(allocator.segmentThreshold / float64(sizePerRecord)), nil +} + +func (allocator *segmentAllocatorImpl) GetSealedSegments() ([]UniqueID, error) { + allocator.mu.Lock() + defer allocator.mu.Unlock() + keys := make([]UniqueID, 0) + for _, segStatus := range allocator.segments { + if !segStatus.sealed { + sealed, err := allocator.checkSegmentSealed(segStatus) + if err != nil { + return nil, err + } + segStatus.sealed = sealed + } + if segStatus.sealed { + keys = append(keys, segStatus.id) + } + } + return keys, nil +} + +func (allocator *segmentAllocatorImpl) checkSegmentSealed(segStatus *segmentStatus) (bool, error) { + segMeta, err := allocator.mt.GetSegmentByID(segStatus.id) + if err != nil { + return false, err + } + return float64(segMeta.NumRows) >= allocator.segmentThresholdFactor*float64(segStatus.total), nil +} + +func (allocator *segmentAllocatorImpl) SealSegment(segmentID UniqueID) { + allocator.mu.Lock() + defer allocator.mu.Unlock() + status, ok := allocator.segments[segmentID] + if !ok { + return + } + status.sealed = true +} + +func (allocator *segmentAllocatorImpl) DropSegment(segmentID UniqueID) { + allocator.mu.Lock() + defer allocator.mu.Unlock() + delete(allocator.segments, segmentID) +} + +func (allocator *segmentAllocatorImpl) ExpireAllocations(timeTick Timestamp) error { + allocator.mu.Lock() + defer allocator.mu.Unlock() + for _, segStatus := range allocator.segments { + for i := 0; i < len(segStatus.allocations); i++ { + if timeTick < segStatus.allocations[i].expireTime { + continue + } + segStatus.allocations = append(segStatus.allocations[:i], segStatus.allocations[i+1:]...) + i-- + } + } + + return nil +} + +func (allocator *segmentAllocatorImpl) IsAllocationsExpired(segmentID UniqueID, ts Timestamp) (bool, error) { + allocator.mu.RLock() + defer allocator.mu.RUnlock() + status, ok := allocator.segments[segmentID] + if !ok { + return false, fmt.Errorf("segment %d not found", segmentID) + } + return status.lastExpireTime <= ts, nil +} diff --git a/internal/dataservice/watcher.go b/internal/dataservice/watcher.go new file mode 100644 index 000000000..4db2fdfbb --- /dev/null +++ b/internal/dataservice/watcher.go @@ -0,0 +1,107 @@ +package dataservice + +import ( + "log" + + "golang.org/x/net/context" + + "github.com/zilliztech/milvus-distributed/internal/msgstream" +) + +type ( + proxyTimeTickWatcher struct { + allocator segmentAllocator + msgQueue chan *msgstream.TimeTickMsg + ctx context.Context + cancelFunc context.CancelFunc + } + dataNodeTimeTickWatcher struct { + allocator segmentAllocator + msgQueue chan *msgstream.TimeTickMsg + ctx context.Context + cancelFunc context.CancelFunc + } +) + +func newProxyTimeTickWatcher(ctx context.Context, allocator segmentAllocator) *proxyTimeTickWatcher { + cancel, cancelFunc := context.WithCancel(ctx) + return &proxyTimeTickWatcher{ + allocator: allocator, + msgQueue: make(chan *msgstream.TimeTickMsg, 1), + ctx: cancel, + cancelFunc: cancelFunc, + } +} + +func (watcher *proxyTimeTickWatcher) Start() { + go watcher.handleProxyTimeTickMsg() +} + +func (watcher *proxyTimeTickWatcher) Close() { + watcher.cancelFunc() +} + +func (watcher *proxyTimeTickWatcher) Watch(msg *msgstream.TimeTickMsg) { + watcher.msgQueue <- msg +} + +func (watcher *proxyTimeTickWatcher) handleProxyTimeTickMsg() { + for { + select { + case <-watcher.ctx.Done(): + return + case msg := <-watcher.msgQueue: + if err := watcher.allocator.ExpireAllocations(msg.Timestamp); err != nil { + log.Printf("expire allocations error : %s", err.Error()) + } + } + } +} + +func newDataNodeTimeTickWatcher(ctx context.Context, allocator segmentAllocator) *dataNodeTimeTickWatcher { + cancel, cancelFunc := context.WithCancel(ctx) + return &dataNodeTimeTickWatcher{ + allocator: allocator, + msgQueue: make(chan *msgstream.TimeTickMsg, 1), + ctx: cancel, + cancelFunc: cancelFunc, + } +} + +func (watcher *dataNodeTimeTickWatcher) Watch(msg *msgstream.TimeTickMsg) { + watcher.msgQueue <- msg +} + +func (watcher *dataNodeTimeTickWatcher) Start() { + go watcher.handleDataNodeTimeTickMsg() +} + +func (watcher *dataNodeTimeTickWatcher) Close() { + watcher.cancelFunc() +} + +func (watcher *dataNodeTimeTickWatcher) handleDataNodeTimeTickMsg() { + for { + select { + case <-watcher.ctx.Done(): + return + case msg := <-watcher.msgQueue: + segments, err := watcher.allocator.GetSealedSegments() + if err != nil { + log.Printf("get sealed segments error %s", err.Error()) + continue + } + for _, id := range segments { + expired, err := watcher.allocator.IsAllocationsExpired(id, msg.Timestamp) + if err != nil { + log.Printf("check allocations expired error %s", err.Error()) + continue + } + if expired { + // TODO: flush segment + watcher.allocator.DropSegment(id) + } + } + } + } +} diff --git a/internal/proto/data_service.proto b/internal/proto/data_service.proto index d1a4f95d1..29f1f6f99 100644 --- a/internal/proto/data_service.proto +++ b/internal/proto/data_service.proto @@ -113,6 +113,16 @@ message FlushSegRequest { repeated int64 segmentIDs = 4; } +message SegmentInfo { + int64 segmentID=1; + int64 collectionID =2; + int64 partitionID=3; + repeated string insert_channels = 4; + uint64 open_time=5; + uint64 close_time=6; + int64 num_rows=7; + int64 mem_size=8; +} service DataService { diff --git a/internal/proto/datapb/data_service.pb.go b/internal/proto/datapb/data_service.pb.go index 141c5ac8c..c5a9e14cb 100644 --- a/internal/proto/datapb/data_service.pb.go +++ b/internal/proto/datapb/data_service.pb.go @@ -897,6 +897,101 @@ func (m *FlushSegRequest) GetSegmentIDs() []int64 { return nil } +type SegmentInfo struct { + SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,3,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + InsertChannels []string `protobuf:"bytes,4,rep,name=insert_channels,json=insertChannels,proto3" json:"insert_channels,omitempty"` + OpenTime uint64 `protobuf:"varint,5,opt,name=open_time,json=openTime,proto3" json:"open_time,omitempty"` + CloseTime uint64 `protobuf:"varint,6,opt,name=close_time,json=closeTime,proto3" json:"close_time,omitempty"` + NumRows int64 `protobuf:"varint,7,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + MemSize int64 `protobuf:"varint,8,opt,name=mem_size,json=memSize,proto3" json:"mem_size,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SegmentInfo) Reset() { *m = SegmentInfo{} } +func (m *SegmentInfo) String() string { return proto.CompactTextString(m) } +func (*SegmentInfo) ProtoMessage() {} +func (*SegmentInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_3385cd32ad6cfe64, []int{16} +} + +func (m *SegmentInfo) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SegmentInfo.Unmarshal(m, b) +} +func (m *SegmentInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SegmentInfo.Marshal(b, m, deterministic) +} +func (m *SegmentInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_SegmentInfo.Merge(m, src) +} +func (m *SegmentInfo) XXX_Size() int { + return xxx_messageInfo_SegmentInfo.Size(m) +} +func (m *SegmentInfo) XXX_DiscardUnknown() { + xxx_messageInfo_SegmentInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_SegmentInfo proto.InternalMessageInfo + +func (m *SegmentInfo) GetSegmentID() int64 { + if m != nil { + return m.SegmentID + } + return 0 +} + +func (m *SegmentInfo) GetCollectionID() int64 { + if m != nil { + return m.CollectionID + } + return 0 +} + +func (m *SegmentInfo) GetPartitionID() int64 { + if m != nil { + return m.PartitionID + } + return 0 +} + +func (m *SegmentInfo) GetInsertChannels() []string { + if m != nil { + return m.InsertChannels + } + return nil +} + +func (m *SegmentInfo) GetOpenTime() uint64 { + if m != nil { + return m.OpenTime + } + return 0 +} + +func (m *SegmentInfo) GetCloseTime() uint64 { + if m != nil { + return m.CloseTime + } + return 0 +} + +func (m *SegmentInfo) GetNumRows() int64 { + if m != nil { + return m.NumRows + } + return 0 +} + +func (m *SegmentInfo) GetMemSize() int64 { + if m != nil { + return m.MemSize + } + return 0 +} + func init() { proto.RegisterEnum("milvus.proto.data.SegmentState", SegmentState_name, SegmentState_value) proto.RegisterType((*RegisterNodeRequest)(nil), "milvus.proto.data.RegisterNodeRequest") @@ -915,72 +1010,79 @@ func init() { proto.RegisterType((*InsertChannelRequest)(nil), "milvus.proto.data.InsertChannelRequest") proto.RegisterType((*WatchDmChannelRequest)(nil), "milvus.proto.data.WatchDmChannelRequest") proto.RegisterType((*FlushSegRequest)(nil), "milvus.proto.data.FlushSegRequest") + proto.RegisterType((*SegmentInfo)(nil), "milvus.proto.data.SegmentInfo") } func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad6cfe64) } var fileDescriptor_3385cd32ad6cfe64 = []byte{ - // 952 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x56, 0x41, 0x8f, 0xdb, 0x44, - 0x14, 0x8e, 0xe3, 0x64, 0xdb, 0xbc, 0xa4, 0xbb, 0xd9, 0xd9, 0x14, 0x42, 0xba, 0xda, 0x0d, 0x23, - 0xd1, 0x46, 0x15, 0x24, 0xd2, 0x56, 0x85, 0x1b, 0xa2, 0x4b, 0xca, 0x2a, 0x12, 0xac, 0xaa, 0x09, - 0x12, 0x52, 0x39, 0x44, 0x93, 0xf8, 0xe1, 0x8c, 0xea, 0xd8, 0xc6, 0x33, 0x69, 0xab, 0xbd, 0xc1, - 0x89, 0x13, 0xe2, 0x0f, 0xf0, 0x57, 0xf8, 0x5b, 0x5c, 0x91, 0x67, 0x1c, 0xc7, 0x4e, 0x1c, 0x52, - 0x69, 0xa9, 0xb8, 0x65, 0x5e, 0xbe, 0x79, 0xef, 0x9b, 0x6f, 0xde, 0xfb, 0xc6, 0x40, 0x1c, 0xae, - 0xf8, 0x44, 0x62, 0xf4, 0x5a, 0xcc, 0xb0, 0x1f, 0x46, 0x81, 0x0a, 0xc8, 0xf1, 0x42, 0x78, 0xaf, - 0x97, 0xd2, 0xac, 0xfa, 0x31, 0xa0, 0xd3, 0x98, 0x05, 0x8b, 0x45, 0xe0, 0x9b, 0x50, 0xe7, 0x50, - 0xf8, 0x0a, 0x23, 0x9f, 0x7b, 0x66, 0x4d, 0x7f, 0xb1, 0xe0, 0x84, 0xa1, 0x2b, 0xa4, 0xc2, 0xe8, - 0x3a, 0x70, 0x90, 0xe1, 0xcf, 0x4b, 0x94, 0x8a, 0x5c, 0x40, 0x65, 0xca, 0x25, 0xb6, 0xad, 0xae, - 0xd5, 0xab, 0x5f, 0x9c, 0xf5, 0x73, 0x79, 0xd3, 0x1c, 0xdf, 0x49, 0xf7, 0x92, 0x4b, 0x64, 0x1a, - 0x4b, 0x3e, 0x87, 0x3b, 0xdc, 0x71, 0x22, 0x94, 0xb2, 0x5d, 0xd6, 0xdb, 0x4e, 0xf3, 0xdb, 0x12, - 0x22, 0xcf, 0x0c, 0x86, 0xad, 0xc0, 0xf4, 0x25, 0xb4, 0xf2, 0x14, 0x64, 0x18, 0xf8, 0x12, 0xc9, - 0x25, 0xd4, 0x85, 0x2f, 0xd4, 0x24, 0xe4, 0x11, 0x5f, 0xc8, 0x84, 0xca, 0xc7, 0x3b, 0xa8, 0x8c, - 0x7c, 0xa1, 0x5e, 0x68, 0x20, 0x03, 0x91, 0xfe, 0xa6, 0xbf, 0x59, 0xd0, 0x18, 0xa3, 0x3b, 0x1a, - 0xae, 0x0e, 0xd6, 0x82, 0xea, 0x2c, 0x58, 0xfa, 0x4a, 0xa7, 0xab, 0x32, 0xb3, 0x20, 0xa7, 0x50, - 0x9b, 0xcd, 0xb9, 0xef, 0xa3, 0x37, 0x1a, 0x6a, 0xf2, 0x35, 0xb6, 0x0e, 0x10, 0x0a, 0x8d, 0x59, - 0xe0, 0x79, 0x38, 0x53, 0x22, 0xf0, 0x47, 0xc3, 0xb6, 0xdd, 0xb5, 0x7a, 0x36, 0xcb, 0xc5, 0x48, - 0x17, 0xea, 0x21, 0x8f, 0x94, 0x48, 0x20, 0x15, 0x0d, 0xc9, 0x86, 0xe8, 0x1f, 0x16, 0x90, 0x67, - 0x52, 0x0a, 0xd7, 0xcf, 0x11, 0xfa, 0x00, 0x0e, 0xfc, 0xc0, 0xc1, 0xd1, 0x50, 0x33, 0xb2, 0x59, - 0xb2, 0x22, 0x0f, 0xa0, 0x16, 0x22, 0x46, 0x93, 0x28, 0xf0, 0x30, 0xa1, 0x74, 0x37, 0x0e, 0xb0, - 0xc0, 0x43, 0xf2, 0x1c, 0xee, 0xc9, 0x4c, 0x12, 0xd9, 0xb6, 0xbb, 0x76, 0xaf, 0x7e, 0x71, 0xde, - 0xdf, 0xba, 0xff, 0x7e, 0xb6, 0x18, 0xcb, 0xef, 0xa2, 0x7f, 0x5b, 0x70, 0xa4, 0xff, 0x37, 0xbc, - 0x16, 0xe8, 0x6b, 0x81, 0x34, 0x28, 0xa1, 0x63, 0x16, 0xdb, 0x02, 0x55, 0xb3, 0x02, 0xa5, 0xa2, - 0xc6, 0xca, 0xdc, 0x5b, 0x89, 0xba, 0x29, 0x5b, 0x65, 0xbf, 0x6c, 0xd5, 0x2d, 0xd9, 0xc8, 0x39, - 0xd4, 0xf1, 0x6d, 0x28, 0x22, 0x9c, 0x28, 0xb1, 0xc0, 0xf6, 0x41, 0xd7, 0xea, 0x55, 0x18, 0x98, - 0xd0, 0xf7, 0x62, 0x81, 0xe4, 0x09, 0x1c, 0x48, 0xc5, 0xd5, 0x52, 0xb6, 0xef, 0xe8, 0x0e, 0x79, - 0x50, 0xd8, 0x75, 0x63, 0x0d, 0x61, 0x09, 0x94, 0x22, 0x9c, 0xe4, 0xee, 0x22, 0x69, 0xb9, 0x6b, - 0x68, 0xca, 0xbc, 0x1e, 0x71, 0xdf, 0xc5, 0xd2, 0xd2, 0x5d, 0xd2, 0xae, 0xa1, 0x6c, 0x6b, 0x2f, - 0xbd, 0x81, 0xc6, 0x37, 0xde, 0x52, 0xce, 0x6f, 0x33, 0x56, 0x04, 0x2a, 0xce, 0x34, 0x51, 0xdd, - 0x66, 0xfa, 0xf7, 0xbb, 0x48, 0x4b, 0x7f, 0xb7, 0x80, 0x8c, 0xe7, 0xc1, 0x9b, 0x31, 0xba, 0x9a, - 0xdd, 0x2d, 0x28, 0x6c, 0x96, 0x2b, 0xef, 0xbf, 0x49, 0x7b, 0x7b, 0x00, 0x9e, 0xc2, 0x49, 0x8e, - 0x4f, 0xa2, 0xf9, 0x19, 0x80, 0x34, 0xa1, 0xd1, 0xd0, 0xa8, 0x6d, 0xb3, 0x4c, 0x84, 0xce, 0xa1, - 0x95, 0x6c, 0x89, 0xef, 0x10, 0xe5, 0x6d, 0x0e, 0x72, 0x0a, 0xb5, 0x34, 0x73, 0x72, 0x8a, 0x75, - 0x20, 0x9e, 0xd0, 0xfb, 0x1b, 0xa5, 0x12, 0x8e, 0x4f, 0xa1, 0x1a, 0x37, 0x8e, 0x29, 0x76, 0xb8, - 0x6b, 0xce, 0xd2, 0x8d, 0xcc, 0xa0, 0xe3, 0xde, 0x9d, 0x45, 0xc8, 0x55, 0xd2, 0xbb, 0x65, 0xd3, - 0xbb, 0x26, 0xa4, 0x7b, 0xf7, 0x1c, 0xea, 0x12, 0xb9, 0x87, 0x8e, 0x01, 0xd8, 0x06, 0x60, 0x42, - 0x31, 0x80, 0xbe, 0x82, 0x0f, 0x47, 0xbe, 0xc4, 0x48, 0x5d, 0x0a, 0xdf, 0x0b, 0xdc, 0x17, 0x5c, - 0xcd, 0xdf, 0xdf, 0xf9, 0x43, 0xf8, 0x68, 0xb3, 0xd8, 0x5a, 0x82, 0x0e, 0xdc, 0xfd, 0x49, 0xa0, - 0xe7, 0xac, 0x2f, 0x29, 0x5d, 0x93, 0x2f, 0xa0, 0x1a, 0xc6, 0xe0, 0x76, 0x59, 0xcf, 0xca, 0x2e, - 0x8f, 0x1e, 0xab, 0x48, 0xf8, 0xee, 0xb7, 0x42, 0x2a, 0x66, 0xf0, 0xf4, 0x57, 0x0b, 0x5a, 0xa6, - 0xe4, 0xd7, 0xc6, 0x4c, 0xde, 0xf7, 0xa0, 0x14, 0x58, 0x37, 0x7d, 0x05, 0xf7, 0x7f, 0xe0, 0x6a, - 0x36, 0x1f, 0x2e, 0xfe, 0x03, 0x12, 0x67, 0x00, 0xa9, 0x2f, 0x1a, 0x3d, 0x6a, 0x2c, 0x13, 0xa1, - 0x7f, 0x5a, 0x70, 0xa4, 0x2d, 0x61, 0x8c, 0xee, 0xff, 0x70, 0xd8, 0x8d, 0x69, 0xab, 0x6c, 0x4e, - 0xdb, 0xe3, 0x1f, 0xf5, 0x7b, 0x99, 0x76, 0x32, 0x39, 0x82, 0x7a, 0xb2, 0xbe, 0x0e, 0x7c, 0x6c, - 0x96, 0xc8, 0x89, 0x7e, 0x32, 0x4c, 0x40, 0x3d, 0x7f, 0x2b, 0xa4, 0x6a, 0x5a, 0x84, 0xc0, 0x61, - 0x12, 0xbc, 0x8a, 0x82, 0x37, 0xc2, 0x77, 0x9b, 0x65, 0x72, 0x0c, 0xf7, 0x56, 0x99, 0x74, 0x3f, - 0x37, 0xed, 0x8b, 0xbf, 0xaa, 0x50, 0x1f, 0x72, 0xc5, 0xc7, 0xe6, 0xa3, 0x85, 0x70, 0x68, 0x64, - 0x5f, 0x7e, 0xf2, 0xb0, 0x60, 0xae, 0x0a, 0xbe, 0x4e, 0x3a, 0x8f, 0xf6, 0xe2, 0x4c, 0xd3, 0xd2, - 0x12, 0xb9, 0x82, 0xaa, 0x96, 0x9b, 0x14, 0xcd, 0x6c, 0xd6, 0x9b, 0x3b, 0xff, 0xf6, 0x6e, 0xd0, - 0x12, 0x99, 0xc2, 0x51, 0xfa, 0x62, 0x18, 0xb1, 0xc8, 0x27, 0x05, 0x29, 0xb7, 0x5f, 0xf8, 0xce, - 0xc3, 0x7d, 0xb0, 0x94, 0xec, 0x04, 0x1a, 0x19, 0x87, 0x94, 0x85, 0x05, 0xb6, 0x2d, 0xbd, 0xb0, - 0x40, 0x81, 0xd3, 0xd2, 0x12, 0x71, 0xa1, 0x79, 0x85, 0x2a, 0xe7, 0x71, 0xe4, 0xd1, 0x1e, 0x33, - 0x5b, 0x19, 0x6e, 0xa7, 0xb7, 0x1f, 0x98, 0x16, 0x8a, 0xa0, 0x75, 0x85, 0x6a, 0xcb, 0x4d, 0xc8, - 0xe3, 0x82, 0x1c, 0x3b, 0x0c, 0xae, 0xf3, 0xe9, 0x3b, 0x60, 0xb3, 0x35, 0x39, 0x1c, 0xa7, 0x35, - 0x93, 0x49, 0x2e, 0x3e, 0x5d, 0x91, 0xe3, 0x74, 0xf6, 0x9b, 0x16, 0x2d, 0x5d, 0x7e, 0xf5, 0xf2, - 0x4b, 0x57, 0xa8, 0xf9, 0x72, 0x1a, 0xb7, 0xc7, 0xe0, 0x46, 0x78, 0x9e, 0xb8, 0x51, 0x38, 0x9b, - 0x0f, 0xcc, 0xde, 0xcf, 0x1c, 0x21, 0x55, 0x24, 0xa6, 0x4b, 0x85, 0xce, 0x60, 0x95, 0x61, 0xa0, - 0x13, 0x0e, 0xe2, 0xca, 0xe1, 0x74, 0x7a, 0xa0, 0x57, 0x4f, 0xfe, 0x09, 0x00, 0x00, 0xff, 0xff, - 0x63, 0x83, 0xa9, 0x0a, 0xbe, 0x0b, 0x00, 0x00, + // 1051 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x56, 0x51, 0x6f, 0xdb, 0x36, + 0x10, 0x8e, 0x2c, 0x3b, 0x89, 0xcf, 0x4e, 0xe2, 0x30, 0xe9, 0xe6, 0xba, 0x59, 0x92, 0x09, 0x58, + 0x13, 0x14, 0x5b, 0x02, 0xa4, 0xe8, 0xf6, 0x36, 0xac, 0x99, 0xbb, 0xc0, 0xc0, 0x16, 0x14, 0xf4, + 0x80, 0x01, 0xdd, 0x83, 0x21, 0x5b, 0x57, 0x99, 0xa8, 0x44, 0x6a, 0x22, 0xdd, 0x14, 0x79, 0xdb, + 0x9e, 0xf6, 0x30, 0x0c, 0xfb, 0x03, 0xfb, 0x2b, 0xfb, 0x5b, 0x7b, 0x1d, 0x44, 0xca, 0xb2, 0x64, + 0xcb, 0x73, 0x81, 0xb4, 0xe8, 0x9b, 0x78, 0xfa, 0x78, 0x77, 0xfc, 0xf8, 0xdd, 0x1d, 0x81, 0x78, + 0xae, 0x72, 0x07, 0x12, 0xe3, 0xd7, 0x6c, 0x84, 0x67, 0x51, 0x2c, 0x94, 0x20, 0xbb, 0x21, 0x0b, + 0x5e, 0x4f, 0xa4, 0x59, 0x9d, 0x25, 0x80, 0x4e, 0x73, 0x24, 0xc2, 0x50, 0x70, 0x63, 0xea, 0x6c, + 0x33, 0xae, 0x30, 0xe6, 0x6e, 0x60, 0xd6, 0xce, 0xaf, 0x16, 0xec, 0x51, 0xf4, 0x99, 0x54, 0x18, + 0x5f, 0x0b, 0x0f, 0x29, 0xfe, 0x32, 0x41, 0xa9, 0xc8, 0x05, 0x54, 0x87, 0xae, 0xc4, 0xb6, 0x75, + 0x6c, 0x9d, 0x36, 0x2e, 0x0e, 0xcf, 0x0a, 0x7e, 0x33, 0x1f, 0x3f, 0x48, 0xff, 0xd2, 0x95, 0x48, + 0x35, 0x96, 0x7c, 0x09, 0x1b, 0xae, 0xe7, 0xc5, 0x28, 0x65, 0xbb, 0xa2, 0xb7, 0x1d, 0x14, 0xb7, + 0xa5, 0x89, 0x3c, 0x35, 0x18, 0x3a, 0x05, 0x3b, 0x2f, 0x60, 0xbf, 0x98, 0x82, 0x8c, 0x04, 0x97, + 0x48, 0x2e, 0xa1, 0xc1, 0x38, 0x53, 0x83, 0xc8, 0x8d, 0xdd, 0x50, 0xa6, 0xa9, 0x7c, 0xba, 0x24, + 0x95, 0x1e, 0x67, 0xea, 0xb9, 0x06, 0x52, 0x60, 0xd9, 0xb7, 0xf3, 0xbb, 0x05, 0xcd, 0x3e, 0xfa, + 0xbd, 0xee, 0xf4, 0x60, 0xfb, 0x50, 0x1b, 0x89, 0x09, 0x57, 0xda, 0x5d, 0x8d, 0x9a, 0x05, 0x39, + 0x80, 0xfa, 0x68, 0xec, 0x72, 0x8e, 0x41, 0xaf, 0xab, 0x93, 0xaf, 0xd3, 0x99, 0x81, 0x38, 0xd0, + 0x1c, 0x89, 0x20, 0xc0, 0x91, 0x62, 0x82, 0xf7, 0xba, 0x6d, 0xfb, 0xd8, 0x3a, 0xb5, 0x69, 0xc1, + 0x46, 0x8e, 0xa1, 0x11, 0xb9, 0xb1, 0x62, 0x29, 0xa4, 0xaa, 0x21, 0x79, 0x93, 0xf3, 0x97, 0x05, + 0xe4, 0xa9, 0x94, 0xcc, 0xe7, 0x85, 0x84, 0x3e, 0x82, 0x75, 0x2e, 0x3c, 0xec, 0x75, 0x75, 0x46, + 0x36, 0x4d, 0x57, 0xe4, 0x01, 0xd4, 0x23, 0xc4, 0x78, 0x10, 0x8b, 0x00, 0xd3, 0x94, 0x36, 0x13, + 0x03, 0x15, 0x01, 0x92, 0x67, 0xb0, 0x25, 0x73, 0x4e, 0x64, 0xdb, 0x3e, 0xb6, 0x4f, 0x1b, 0x17, + 0x47, 0x67, 0x0b, 0xf7, 0x7f, 0x96, 0x0f, 0x46, 0x8b, 0xbb, 0x9c, 0x7f, 0x2d, 0xd8, 0xd1, 0xff, + 0x4d, 0x5e, 0x21, 0x72, 0x4d, 0x90, 0x06, 0xa5, 0xe9, 0x98, 0xc5, 0x22, 0x41, 0xb5, 0x3c, 0x41, + 0x19, 0xa9, 0x09, 0x33, 0x5b, 0x53, 0x52, 0xe7, 0x69, 0xab, 0xae, 0xa6, 0xad, 0xb6, 0x40, 0x1b, + 0x39, 0x82, 0x06, 0xbe, 0x89, 0x58, 0x8c, 0x03, 0xc5, 0x42, 0x6c, 0xaf, 0x1f, 0x5b, 0xa7, 0x55, + 0x0a, 0xc6, 0xf4, 0x23, 0x0b, 0x91, 0x3c, 0x86, 0x75, 0xa9, 0x5c, 0x35, 0x91, 0xed, 0x0d, 0xad, + 0x90, 0x07, 0xa5, 0xaa, 0xeb, 0x6b, 0x08, 0x4d, 0xa1, 0x0e, 0xc2, 0x5e, 0xe1, 0x2e, 0x52, 0xc9, + 0x5d, 0x43, 0x4b, 0x16, 0xf9, 0x48, 0x74, 0x97, 0x50, 0xeb, 0x2c, 0xa3, 0x76, 0x06, 0xa5, 0x0b, + 0x7b, 0x9d, 0x5b, 0x68, 0x7e, 0x17, 0x4c, 0xe4, 0xf8, 0x2e, 0x65, 0x45, 0xa0, 0xea, 0x0d, 0x53, + 0xd6, 0x6d, 0xaa, 0xbf, 0xdf, 0x86, 0x5a, 0xe7, 0x4f, 0x0b, 0x48, 0x7f, 0x2c, 0x6e, 0xfa, 0xe8, + 0xeb, 0xec, 0xee, 0x90, 0xc2, 0x7c, 0xb8, 0xca, 0xea, 0x9b, 0xb4, 0x17, 0x0b, 0xe0, 0x09, 0xec, + 0x15, 0xf2, 0x49, 0x39, 0x3f, 0x04, 0x90, 0xc6, 0xd4, 0xeb, 0x1a, 0xb6, 0x6d, 0x9a, 0xb3, 0x38, + 0x63, 0xd8, 0x4f, 0xb7, 0x24, 0x77, 0x88, 0xf2, 0x2e, 0x07, 0x39, 0x80, 0x7a, 0xe6, 0x39, 0x3d, + 0xc5, 0xcc, 0x90, 0x54, 0xe8, 0xbd, 0xb9, 0x50, 0x69, 0x8e, 0x4f, 0xa0, 0x96, 0x08, 0xc7, 0x04, + 0xdb, 0x5e, 0x56, 0x67, 0xd9, 0x46, 0x6a, 0xd0, 0x89, 0x76, 0x47, 0x31, 0xba, 0x2a, 0xd5, 0x6e, + 0xc5, 0x68, 0xd7, 0x98, 0xb4, 0x76, 0x8f, 0xa0, 0x21, 0xd1, 0x0d, 0xd0, 0x33, 0x00, 0xdb, 0x00, + 0x8c, 0x29, 0x01, 0x38, 0xaf, 0xe0, 0xe3, 0x1e, 0x97, 0x18, 0xab, 0x4b, 0xc6, 0x03, 0xe1, 0x3f, + 0x77, 0xd5, 0xf8, 0xfd, 0x9d, 0x3f, 0x82, 0xfb, 0xf3, 0xc1, 0x66, 0x14, 0x74, 0x60, 0xf3, 0x25, + 0xc3, 0xc0, 0x9b, 0x5d, 0x52, 0xb6, 0x26, 0x5f, 0x41, 0x2d, 0x4a, 0xc0, 0xed, 0x8a, 0xae, 0x95, + 0x65, 0x3d, 0xba, 0xaf, 0x62, 0xc6, 0xfd, 0xef, 0x99, 0x54, 0xd4, 0xe0, 0x9d, 0xdf, 0x2c, 0xd8, + 0x37, 0x21, 0xbf, 0x35, 0xcd, 0xe4, 0x7d, 0x17, 0x4a, 0x49, 0xeb, 0x76, 0x5e, 0xc1, 0xbd, 0x9f, + 0x5c, 0x35, 0x1a, 0x77, 0xc3, 0x77, 0x90, 0xc4, 0x21, 0x40, 0xd6, 0x17, 0x0d, 0x1f, 0x75, 0x9a, + 0xb3, 0x38, 0x7f, 0x5b, 0xb0, 0xa3, 0x5b, 0x42, 0x1f, 0xfd, 0x0f, 0x70, 0xd8, 0xb9, 0x6a, 0xab, + 0x2e, 0x54, 0xdb, 0x1f, 0x15, 0x68, 0xa4, 0x52, 0xee, 0xf1, 0x97, 0xa2, 0xa8, 0x18, 0x6b, 0x4e, + 0x31, 0xef, 0xa6, 0x31, 0x90, 0x13, 0xd8, 0x61, 0x5a, 0x04, 0x83, 0x94, 0x28, 0x93, 0x58, 0x9d, + 0x6e, 0xb3, 0xbc, 0x36, 0x64, 0x32, 0x13, 0x45, 0x84, 0xdc, 0x14, 0x4b, 0x4d, 0x17, 0xcb, 0x66, + 0x62, 0xd0, 0xb5, 0xf4, 0x09, 0xc0, 0x28, 0x10, 0xb2, 0x30, 0x27, 0xea, 0xda, 0xa2, 0x7f, 0xdf, + 0x87, 0x4d, 0x3e, 0x09, 0x07, 0xb1, 0xb8, 0x31, 0x83, 0xc2, 0xa6, 0x1b, 0x7c, 0x12, 0x52, 0x71, + 0x23, 0x93, 0x5f, 0x21, 0x86, 0x03, 0xc9, 0x6e, 0xb1, 0xbd, 0x69, 0x7e, 0x85, 0x18, 0xf6, 0xd9, + 0x2d, 0x3e, 0xfa, 0x59, 0x3f, 0x1f, 0xb2, 0xc2, 0x26, 0x3b, 0x19, 0x3b, 0xd7, 0x82, 0x63, 0x6b, + 0x8d, 0xec, 0xe9, 0x09, 0x6a, 0x0c, 0xea, 0xd9, 0x1b, 0x26, 0x55, 0xcb, 0x22, 0x04, 0xb6, 0x53, + 0xe3, 0x55, 0x2c, 0x6e, 0x18, 0xf7, 0x5b, 0x15, 0xb2, 0x0b, 0x5b, 0x53, 0x4f, 0xba, 0xbc, 0x5b, + 0xf6, 0xc5, 0x3f, 0x35, 0x68, 0x74, 0x5d, 0xe5, 0xf6, 0xcd, 0x1b, 0x8e, 0xb8, 0xd0, 0xcc, 0x3f, + 0x84, 0xc8, 0xc3, 0x92, 0x36, 0x53, 0xf2, 0x58, 0xeb, 0x9c, 0xac, 0xc4, 0x99, 0x1a, 0x76, 0xd6, + 0xc8, 0x15, 0xd4, 0xb4, 0xfa, 0x48, 0x59, 0x0b, 0xcb, 0x8f, 0xaa, 0xce, 0xff, 0x8d, 0x51, 0x67, + 0x8d, 0x0c, 0x61, 0x27, 0x1b, 0xa0, 0xa9, 0x18, 0x3e, 0x2b, 0x71, 0xb9, 0xf8, 0xe0, 0xe9, 0x3c, + 0x5c, 0x05, 0xcb, 0x92, 0x1d, 0x40, 0x33, 0x37, 0x30, 0x64, 0x69, 0x80, 0xc5, 0x09, 0x57, 0x1a, + 0xa0, 0x64, 0xf0, 0x38, 0x6b, 0xc4, 0x87, 0xd6, 0x15, 0xaa, 0x42, 0xcb, 0x27, 0x27, 0x2b, 0x7a, + 0xfb, 0x74, 0xfe, 0x74, 0x4e, 0x57, 0x03, 0xb3, 0x40, 0x31, 0xec, 0x5f, 0xa1, 0x5a, 0x68, 0xae, + 0xe4, 0x51, 0x89, 0x8f, 0x25, 0xfd, 0xbe, 0xf3, 0xf9, 0x5b, 0x60, 0xf3, 0x31, 0x5d, 0xd8, 0xcd, + 0x62, 0x66, 0x15, 0x74, 0xb2, 0xd4, 0x49, 0xb1, 0xf7, 0x75, 0x56, 0xf7, 0x70, 0x67, 0xed, 0xf2, + 0x9b, 0x17, 0x5f, 0xfb, 0x4c, 0x8d, 0x27, 0xc3, 0x44, 0x1e, 0xe7, 0xb7, 0x2c, 0x08, 0xd8, 0xad, + 0xc2, 0xd1, 0xf8, 0xdc, 0xec, 0xfd, 0xc2, 0x63, 0x52, 0xc5, 0x6c, 0x38, 0x51, 0xe8, 0x9d, 0x4f, + 0x3d, 0x9c, 0x6b, 0x87, 0xe7, 0x49, 0xe4, 0x68, 0x38, 0x5c, 0xd7, 0xab, 0xc7, 0xff, 0x05, 0x00, + 0x00, 0xff, 0xff, 0x45, 0x70, 0x7b, 0x12, 0xcd, 0x0c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/internal/proto/milvus.proto b/internal/proto/milvus.proto index 6be24a638..be31d5e04 100644 --- a/internal/proto/milvus.proto +++ b/internal/proto/milvus.proto @@ -5,13 +5,11 @@ option go_package = "github.com/zilliztech/milvus-distributed/internal/proto/mil import "common.proto"; import "internal.proto"; -import "schema.proto"; message CreateCollectionRequest { internal.MsgBase base = 1; string db_name = 2; string collectionName = 3; - // `schema` is the serialized `schema.CollectionSchema` bytes schema = 4; } @@ -28,11 +26,6 @@ message HasCollectionRequest { string collection_name = 3; } -message BoolResponse { - common.Status status = 1; - bool value = 2; -} - message DescribeCollectionRequest { internal.MsgBase base = 1; string db_name = 2; @@ -40,8 +33,7 @@ message DescribeCollectionRequest { } message DescribeCollectionResponse { - common.Status status = 1; - schema.CollectionSchema schema = 2; + repeated bytes schema = 1; } message LoadCollectionRequest { @@ -64,7 +56,6 @@ message CollectionStatsRequest { message CollectionStatsResponse { repeated common.KeyValuePair stats = 1; - common.Status status = 2; } @@ -75,7 +66,6 @@ message ShowCollectionRequest { message ShowCollectionResponse { repeated string collection_names = 1; - common.Status status = 2; } @@ -124,7 +114,6 @@ message PartitionStatsRequest { message PartitionStatsResponse { repeated common.KeyValuePair stats = 1; - common.Status status = 2; } message ShowPartitionRequest { @@ -135,7 +124,6 @@ message ShowPartitionRequest { message ShowPartitionResponse { repeated string partition_names = 1; - common.Status status = 2; } @@ -161,7 +149,6 @@ message IndexDescription { message DescribeIndexResponse { repeated IndexDescription index_descriptions = 1; - common.Status status = 2; } message InsertRequest { @@ -176,24 +163,6 @@ message InsertRequest { message InsertResponse { int64 rowID_begin = 1; int64 rowID_end = 2; - common.Status status = 3; -} - -enum PlaceholderType { - NONE = 0; - VECTOR_BINARY = 100; - VECTOR_FLOAT = 101; -} - -message PlaceholderValue { - string tag = 1; - PlaceholderType type = 2; - // values is a 2d-array, every array contains a vector - repeated bytes values = 3; -} - -message PlaceholderGroup { - repeated PlaceholderValue placeholders = 1; } message SearchRequest { @@ -202,49 +171,11 @@ message SearchRequest { string collection_name = 3; repeated string partition_names = 4; string dsl = 5; - // serialized `PlaceholderGroup` - bytes placeholder_group = 6; -} - -message Hits { - repeated int64 IDs = 1; - repeated bytes row_data = 2; - repeated float scores = 3; -} - -message QueryResult { - common.Status status = 1; - repeated bytes hits = 2; + repeated bytes placeholder_group = 6; } message FlushRequest { internal.MsgBase base = 1; string db_name = 2; string collection_name = 3; -} - -service MilvusService { - rpc CreateCollection(CreateCollectionRequest) returns (common.Status) {} - rpc DropCollection(DropCollectionRequest) returns (common.Status) {} - rpc HasCollection(HasCollectionRequest) returns (BoolResponse) {} - rpc LoadCollection(LoadCollectionRequest) returns (common.Status) {} - rpc ReleaseCollection(ReleaseCollectionRequest) returns (common.Status) {} - rpc DescribeCollection(DescribeCollectionRequest) returns (DescribeCollectionResponse) {} - rpc GetCollectionStatistics(CollectionStatsRequest) returns (CollectionStatsResponse) {} - rpc ShowCollections(ShowCollectionRequest) returns (ShowCollectionResponse) {} - - rpc CreatePartition(CreatePartitionRequest) returns (common.Status) {} - rpc DropPartition(DropPartitionRequest) returns (common.Status) {} - rpc HasPartition(HasPartitionRequest) returns (BoolResponse) {} - rpc LoadPartitions(LoadPartitonRequest) returns (common.Status) {} - rpc ReleasePartitions(ReleasePartitionRequest) returns (common.Status) {} - rpc GetPartitionStatistics(PartitionStatsRequest) returns (PartitionStatsResponse) {} - rpc ShowPartitions(ShowPartitionRequest) returns (ShowPartitionResponse) {} - - rpc CreateIndex(CreateIndexRequest) returns (common.Status) {} - rpc DescribeIndex(DescribeIndexRequest) returns (DescribeIndexResponse) {} - - rpc Insert(InsertRequest) returns (InsertResponse) {} - rpc Search(SearchRequest) returns (QueryResult) {} - rpc Flush(FlushRequest) returns (common.Status) {} -} +} \ No newline at end of file diff --git a/internal/proto/milvuspb/milvus.pb.go b/internal/proto/milvuspb/milvus.pb.go index 0ff875d43..dc330e557 100644 --- a/internal/proto/milvuspb/milvus.pb.go +++ b/internal/proto/milvuspb/milvus.pb.go @@ -4,15 +4,10 @@ package milvuspb import ( - context "context" fmt "fmt" proto "github.com/golang/protobuf/proto" commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" internalpb2 "github.com/zilliztech/milvus-distributed/internal/proto/internalpb2" - schemapb "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" - grpc "google.golang.org/grpc" - codes "google.golang.org/grpc/codes" - status "google.golang.org/grpc/status" math "math" ) @@ -27,43 +22,14 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package -type PlaceholderType int32 - -const ( - PlaceholderType_NONE PlaceholderType = 0 - PlaceholderType_VECTOR_BINARY PlaceholderType = 100 - PlaceholderType_VECTOR_FLOAT PlaceholderType = 101 -) - -var PlaceholderType_name = map[int32]string{ - 0: "NONE", - 100: "VECTOR_BINARY", - 101: "VECTOR_FLOAT", -} - -var PlaceholderType_value = map[string]int32{ - "NONE": 0, - "VECTOR_BINARY": 100, - "VECTOR_FLOAT": 101, -} - -func (x PlaceholderType) String() string { - return proto.EnumName(PlaceholderType_name, int32(x)) -} - -func (PlaceholderType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{0} -} - type CreateCollectionRequest struct { - Base *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` - CollectionName string `protobuf:"bytes,3,opt,name=collectionName,proto3" json:"collectionName,omitempty"` - // `schema` is the serialized `schema.CollectionSchema` - Schema []byte `protobuf:"bytes,4,opt,name=schema,proto3" json:"schema,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Base *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + CollectionName string `protobuf:"bytes,3,opt,name=collectionName,proto3" json:"collectionName,omitempty"` + Schema []byte `protobuf:"bytes,4,opt,name=schema,proto3" json:"schema,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *CreateCollectionRequest) Reset() { *m = CreateCollectionRequest{} } @@ -229,53 +195,6 @@ func (m *HasCollectionRequest) GetCollectionName() string { return "" } -type BoolResponse struct { - Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` - Value bool `protobuf:"varint,2,opt,name=value,proto3" json:"value,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *BoolResponse) Reset() { *m = BoolResponse{} } -func (m *BoolResponse) String() string { return proto.CompactTextString(m) } -func (*BoolResponse) ProtoMessage() {} -func (*BoolResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{3} -} - -func (m *BoolResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_BoolResponse.Unmarshal(m, b) -} -func (m *BoolResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_BoolResponse.Marshal(b, m, deterministic) -} -func (m *BoolResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_BoolResponse.Merge(m, src) -} -func (m *BoolResponse) XXX_Size() int { - return xxx_messageInfo_BoolResponse.Size(m) -} -func (m *BoolResponse) XXX_DiscardUnknown() { - xxx_messageInfo_BoolResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_BoolResponse proto.InternalMessageInfo - -func (m *BoolResponse) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - -func (m *BoolResponse) GetValue() bool { - if m != nil { - return m.Value - } - return false -} - type DescribeCollectionRequest struct { Base *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` @@ -289,7 +208,7 @@ func (m *DescribeCollectionRequest) Reset() { *m = DescribeCollectionReq func (m *DescribeCollectionRequest) String() string { return proto.CompactTextString(m) } func (*DescribeCollectionRequest) ProtoMessage() {} func (*DescribeCollectionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{4} + return fileDescriptor_02345ba45cc0e303, []int{3} } func (m *DescribeCollectionRequest) XXX_Unmarshal(b []byte) error { @@ -332,18 +251,17 @@ func (m *DescribeCollectionRequest) GetCollectionName() string { } type DescribeCollectionResponse struct { - Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` - Schema *schemapb.CollectionSchema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Schema [][]byte `protobuf:"bytes,1,rep,name=schema,proto3" json:"schema,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *DescribeCollectionResponse) Reset() { *m = DescribeCollectionResponse{} } func (m *DescribeCollectionResponse) String() string { return proto.CompactTextString(m) } func (*DescribeCollectionResponse) ProtoMessage() {} func (*DescribeCollectionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{5} + return fileDescriptor_02345ba45cc0e303, []int{4} } func (m *DescribeCollectionResponse) XXX_Unmarshal(b []byte) error { @@ -364,14 +282,7 @@ func (m *DescribeCollectionResponse) XXX_DiscardUnknown() { var xxx_messageInfo_DescribeCollectionResponse proto.InternalMessageInfo -func (m *DescribeCollectionResponse) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - -func (m *DescribeCollectionResponse) GetSchema() *schemapb.CollectionSchema { +func (m *DescribeCollectionResponse) GetSchema() [][]byte { if m != nil { return m.Schema } @@ -391,7 +302,7 @@ func (m *LoadCollectionRequest) Reset() { *m = LoadCollectionRequest{} } func (m *LoadCollectionRequest) String() string { return proto.CompactTextString(m) } func (*LoadCollectionRequest) ProtoMessage() {} func (*LoadCollectionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{6} + return fileDescriptor_02345ba45cc0e303, []int{5} } func (m *LoadCollectionRequest) XXX_Unmarshal(b []byte) error { @@ -446,7 +357,7 @@ func (m *ReleaseCollectionRequest) Reset() { *m = ReleaseCollectionReque func (m *ReleaseCollectionRequest) String() string { return proto.CompactTextString(m) } func (*ReleaseCollectionRequest) ProtoMessage() {} func (*ReleaseCollectionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{7} + return fileDescriptor_02345ba45cc0e303, []int{6} } func (m *ReleaseCollectionRequest) XXX_Unmarshal(b []byte) error { @@ -501,7 +412,7 @@ func (m *CollectionStatsRequest) Reset() { *m = CollectionStatsRequest{} func (m *CollectionStatsRequest) String() string { return proto.CompactTextString(m) } func (*CollectionStatsRequest) ProtoMessage() {} func (*CollectionStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{8} + return fileDescriptor_02345ba45cc0e303, []int{7} } func (m *CollectionStatsRequest) XXX_Unmarshal(b []byte) error { @@ -545,7 +456,6 @@ func (m *CollectionStatsRequest) GetCollectionName() string { type CollectionStatsResponse struct { Stats []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=stats,proto3" json:"stats,omitempty"` - Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -555,7 +465,7 @@ func (m *CollectionStatsResponse) Reset() { *m = CollectionStatsResponse func (m *CollectionStatsResponse) String() string { return proto.CompactTextString(m) } func (*CollectionStatsResponse) ProtoMessage() {} func (*CollectionStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{9} + return fileDescriptor_02345ba45cc0e303, []int{8} } func (m *CollectionStatsResponse) XXX_Unmarshal(b []byte) error { @@ -583,13 +493,6 @@ func (m *CollectionStatsResponse) GetStats() []*commonpb.KeyValuePair { return nil } -func (m *CollectionStatsResponse) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - type ShowCollectionRequest struct { Base *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` @@ -602,7 +505,7 @@ func (m *ShowCollectionRequest) Reset() { *m = ShowCollectionRequest{} } func (m *ShowCollectionRequest) String() string { return proto.CompactTextString(m) } func (*ShowCollectionRequest) ProtoMessage() {} func (*ShowCollectionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{10} + return fileDescriptor_02345ba45cc0e303, []int{9} } func (m *ShowCollectionRequest) XXX_Unmarshal(b []byte) error { @@ -638,18 +541,17 @@ func (m *ShowCollectionRequest) GetDbName() string { } type ShowCollectionResponse struct { - CollectionNames []string `protobuf:"bytes,1,rep,name=collection_names,json=collectionNames,proto3" json:"collection_names,omitempty"` - Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + CollectionNames []string `protobuf:"bytes,1,rep,name=collection_names,json=collectionNames,proto3" json:"collection_names,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *ShowCollectionResponse) Reset() { *m = ShowCollectionResponse{} } func (m *ShowCollectionResponse) String() string { return proto.CompactTextString(m) } func (*ShowCollectionResponse) ProtoMessage() {} func (*ShowCollectionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{11} + return fileDescriptor_02345ba45cc0e303, []int{10} } func (m *ShowCollectionResponse) XXX_Unmarshal(b []byte) error { @@ -677,13 +579,6 @@ func (m *ShowCollectionResponse) GetCollectionNames() []string { return nil } -func (m *ShowCollectionResponse) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - type CreatePartitionRequest struct { Base *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` @@ -698,7 +593,7 @@ func (m *CreatePartitionRequest) Reset() { *m = CreatePartitionRequest{} func (m *CreatePartitionRequest) String() string { return proto.CompactTextString(m) } func (*CreatePartitionRequest) ProtoMessage() {} func (*CreatePartitionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{12} + return fileDescriptor_02345ba45cc0e303, []int{11} } func (m *CreatePartitionRequest) XXX_Unmarshal(b []byte) error { @@ -761,7 +656,7 @@ func (m *DropPartitionRequest) Reset() { *m = DropPartitionRequest{} } func (m *DropPartitionRequest) String() string { return proto.CompactTextString(m) } func (*DropPartitionRequest) ProtoMessage() {} func (*DropPartitionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{13} + return fileDescriptor_02345ba45cc0e303, []int{12} } func (m *DropPartitionRequest) XXX_Unmarshal(b []byte) error { @@ -824,7 +719,7 @@ func (m *HasPartitionRequest) Reset() { *m = HasPartitionRequest{} } func (m *HasPartitionRequest) String() string { return proto.CompactTextString(m) } func (*HasPartitionRequest) ProtoMessage() {} func (*HasPartitionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{14} + return fileDescriptor_02345ba45cc0e303, []int{13} } func (m *HasPartitionRequest) XXX_Unmarshal(b []byte) error { @@ -887,7 +782,7 @@ func (m *LoadPartitonRequest) Reset() { *m = LoadPartitonRequest{} } func (m *LoadPartitonRequest) String() string { return proto.CompactTextString(m) } func (*LoadPartitonRequest) ProtoMessage() {} func (*LoadPartitonRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{15} + return fileDescriptor_02345ba45cc0e303, []int{14} } func (m *LoadPartitonRequest) XXX_Unmarshal(b []byte) error { @@ -950,7 +845,7 @@ func (m *ReleasePartitionRequest) Reset() { *m = ReleasePartitionRequest func (m *ReleasePartitionRequest) String() string { return proto.CompactTextString(m) } func (*ReleasePartitionRequest) ProtoMessage() {} func (*ReleasePartitionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{16} + return fileDescriptor_02345ba45cc0e303, []int{15} } func (m *ReleasePartitionRequest) XXX_Unmarshal(b []byte) error { @@ -1013,7 +908,7 @@ func (m *PartitionStatsRequest) Reset() { *m = PartitionStatsRequest{} } func (m *PartitionStatsRequest) String() string { return proto.CompactTextString(m) } func (*PartitionStatsRequest) ProtoMessage() {} func (*PartitionStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{17} + return fileDescriptor_02345ba45cc0e303, []int{16} } func (m *PartitionStatsRequest) XXX_Unmarshal(b []byte) error { @@ -1064,7 +959,6 @@ func (m *PartitionStatsRequest) GetPartitionName() string { type PartitionStatsResponse struct { Stats []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=stats,proto3" json:"stats,omitempty"` - Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1074,7 +968,7 @@ func (m *PartitionStatsResponse) Reset() { *m = PartitionStatsResponse{} func (m *PartitionStatsResponse) String() string { return proto.CompactTextString(m) } func (*PartitionStatsResponse) ProtoMessage() {} func (*PartitionStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{18} + return fileDescriptor_02345ba45cc0e303, []int{17} } func (m *PartitionStatsResponse) XXX_Unmarshal(b []byte) error { @@ -1102,13 +996,6 @@ func (m *PartitionStatsResponse) GetStats() []*commonpb.KeyValuePair { return nil } -func (m *PartitionStatsResponse) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - type ShowPartitionRequest struct { Base *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` @@ -1122,7 +1009,7 @@ func (m *ShowPartitionRequest) Reset() { *m = ShowPartitionRequest{} } func (m *ShowPartitionRequest) String() string { return proto.CompactTextString(m) } func (*ShowPartitionRequest) ProtoMessage() {} func (*ShowPartitionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{19} + return fileDescriptor_02345ba45cc0e303, []int{18} } func (m *ShowPartitionRequest) XXX_Unmarshal(b []byte) error { @@ -1165,18 +1052,17 @@ func (m *ShowPartitionRequest) GetCollectionName() string { } type ShowPartitionResponse struct { - PartitionNames []string `protobuf:"bytes,1,rep,name=partition_names,json=partitionNames,proto3" json:"partition_names,omitempty"` - Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + PartitionNames []string `protobuf:"bytes,1,rep,name=partition_names,json=partitionNames,proto3" json:"partition_names,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *ShowPartitionResponse) Reset() { *m = ShowPartitionResponse{} } func (m *ShowPartitionResponse) String() string { return proto.CompactTextString(m) } func (*ShowPartitionResponse) ProtoMessage() {} func (*ShowPartitionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{20} + return fileDescriptor_02345ba45cc0e303, []int{19} } func (m *ShowPartitionResponse) XXX_Unmarshal(b []byte) error { @@ -1204,13 +1090,6 @@ func (m *ShowPartitionResponse) GetPartitionNames() []string { return nil } -func (m *ShowPartitionResponse) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - type CreateIndexRequest struct { Base *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` @@ -1226,7 +1105,7 @@ func (m *CreateIndexRequest) Reset() { *m = CreateIndexRequest{} } func (m *CreateIndexRequest) String() string { return proto.CompactTextString(m) } func (*CreateIndexRequest) ProtoMessage() {} func (*CreateIndexRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{21} + return fileDescriptor_02345ba45cc0e303, []int{20} } func (m *CreateIndexRequest) XXX_Unmarshal(b []byte) error { @@ -1296,7 +1175,7 @@ func (m *DescribeIndexRequest) Reset() { *m = DescribeIndexRequest{} } func (m *DescribeIndexRequest) String() string { return proto.CompactTextString(m) } func (*DescribeIndexRequest) ProtoMessage() {} func (*DescribeIndexRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{22} + return fileDescriptor_02345ba45cc0e303, []int{21} } func (m *DescribeIndexRequest) XXX_Unmarshal(b []byte) error { @@ -1357,7 +1236,7 @@ func (m *IndexDescription) Reset() { *m = IndexDescription{} } func (m *IndexDescription) String() string { return proto.CompactTextString(m) } func (*IndexDescription) ProtoMessage() {} func (*IndexDescription) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{23} + return fileDescriptor_02345ba45cc0e303, []int{22} } func (m *IndexDescription) XXX_Unmarshal(b []byte) error { @@ -1394,7 +1273,6 @@ func (m *IndexDescription) GetParams() []*commonpb.KeyValuePair { type DescribeIndexResponse struct { IndexDescriptions []*IndexDescription `protobuf:"bytes,1,rep,name=index_descriptions,json=indexDescriptions,proto3" json:"index_descriptions,omitempty"` - Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1404,7 +1282,7 @@ func (m *DescribeIndexResponse) Reset() { *m = DescribeIndexResponse{} } func (m *DescribeIndexResponse) String() string { return proto.CompactTextString(m) } func (*DescribeIndexResponse) ProtoMessage() {} func (*DescribeIndexResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{24} + return fileDescriptor_02345ba45cc0e303, []int{23} } func (m *DescribeIndexResponse) XXX_Unmarshal(b []byte) error { @@ -1432,13 +1310,6 @@ func (m *DescribeIndexResponse) GetIndexDescriptions() []*IndexDescription { return nil } -func (m *DescribeIndexResponse) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - type InsertRequest struct { Base *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` @@ -1455,7 +1326,7 @@ func (m *InsertRequest) Reset() { *m = InsertRequest{} } func (m *InsertRequest) String() string { return proto.CompactTextString(m) } func (*InsertRequest) ProtoMessage() {} func (*InsertRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{25} + return fileDescriptor_02345ba45cc0e303, []int{24} } func (m *InsertRequest) XXX_Unmarshal(b []byte) error { @@ -1519,19 +1390,18 @@ func (m *InsertRequest) GetHashKeys() []uint32 { } type InsertResponse struct { - RowIDBegin int64 `protobuf:"varint,1,opt,name=rowID_begin,json=rowIDBegin,proto3" json:"rowID_begin,omitempty"` - RowIDEnd int64 `protobuf:"varint,2,opt,name=rowID_end,json=rowIDEnd,proto3" json:"rowID_end,omitempty"` - Status *commonpb.Status `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + RowIDBegin int64 `protobuf:"varint,1,opt,name=rowID_begin,json=rowIDBegin,proto3" json:"rowID_begin,omitempty"` + RowIDEnd int64 `protobuf:"varint,2,opt,name=rowID_end,json=rowIDEnd,proto3" json:"rowID_end,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *InsertResponse) Reset() { *m = InsertResponse{} } func (m *InsertResponse) String() string { return proto.CompactTextString(m) } func (*InsertResponse) ProtoMessage() {} func (*InsertResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{26} + return fileDescriptor_02345ba45cc0e303, []int{25} } func (m *InsertResponse) XXX_Unmarshal(b []byte) error { @@ -1566,126 +1436,23 @@ func (m *InsertResponse) GetRowIDEnd() int64 { return 0 } -func (m *InsertResponse) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - -type PlaceholderValue struct { - Tag string `protobuf:"bytes,1,opt,name=tag,proto3" json:"tag,omitempty"` - Type PlaceholderType `protobuf:"varint,2,opt,name=type,proto3,enum=milvus.proto.milvus.PlaceholderType" json:"type,omitempty"` - // values is a 2d-array, every array contains a vector - Values [][]byte `protobuf:"bytes,3,rep,name=values,proto3" json:"values,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *PlaceholderValue) Reset() { *m = PlaceholderValue{} } -func (m *PlaceholderValue) String() string { return proto.CompactTextString(m) } -func (*PlaceholderValue) ProtoMessage() {} -func (*PlaceholderValue) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{27} -} - -func (m *PlaceholderValue) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_PlaceholderValue.Unmarshal(m, b) -} -func (m *PlaceholderValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_PlaceholderValue.Marshal(b, m, deterministic) -} -func (m *PlaceholderValue) XXX_Merge(src proto.Message) { - xxx_messageInfo_PlaceholderValue.Merge(m, src) -} -func (m *PlaceholderValue) XXX_Size() int { - return xxx_messageInfo_PlaceholderValue.Size(m) -} -func (m *PlaceholderValue) XXX_DiscardUnknown() { - xxx_messageInfo_PlaceholderValue.DiscardUnknown(m) -} - -var xxx_messageInfo_PlaceholderValue proto.InternalMessageInfo - -func (m *PlaceholderValue) GetTag() string { - if m != nil { - return m.Tag - } - return "" -} - -func (m *PlaceholderValue) GetType() PlaceholderType { - if m != nil { - return m.Type - } - return PlaceholderType_NONE -} - -func (m *PlaceholderValue) GetValues() [][]byte { - if m != nil { - return m.Values - } - return nil -} - -type PlaceholderGroup struct { - Placeholders []*PlaceholderValue `protobuf:"bytes,1,rep,name=placeholders,proto3" json:"placeholders,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *PlaceholderGroup) Reset() { *m = PlaceholderGroup{} } -func (m *PlaceholderGroup) String() string { return proto.CompactTextString(m) } -func (*PlaceholderGroup) ProtoMessage() {} -func (*PlaceholderGroup) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{28} -} - -func (m *PlaceholderGroup) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_PlaceholderGroup.Unmarshal(m, b) -} -func (m *PlaceholderGroup) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_PlaceholderGroup.Marshal(b, m, deterministic) -} -func (m *PlaceholderGroup) XXX_Merge(src proto.Message) { - xxx_messageInfo_PlaceholderGroup.Merge(m, src) -} -func (m *PlaceholderGroup) XXX_Size() int { - return xxx_messageInfo_PlaceholderGroup.Size(m) -} -func (m *PlaceholderGroup) XXX_DiscardUnknown() { - xxx_messageInfo_PlaceholderGroup.DiscardUnknown(m) -} - -var xxx_messageInfo_PlaceholderGroup proto.InternalMessageInfo - -func (m *PlaceholderGroup) GetPlaceholders() []*PlaceholderValue { - if m != nil { - return m.Placeholders - } - return nil -} - type SearchRequest struct { - Base *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` - CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` - PartitionNames []string `protobuf:"bytes,4,rep,name=partition_names,json=partitionNames,proto3" json:"partition_names,omitempty"` - Dsl string `protobuf:"bytes,5,opt,name=dsl,proto3" json:"dsl,omitempty"` - // serialized `PlaceholderGroup` - PlaceholderGroup []byte `protobuf:"bytes,6,opt,name=placeholder_group,json=placeholderGroup,proto3" json:"placeholder_group,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Base *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + PartitionNames []string `protobuf:"bytes,4,rep,name=partition_names,json=partitionNames,proto3" json:"partition_names,omitempty"` + Dsl string `protobuf:"bytes,5,opt,name=dsl,proto3" json:"dsl,omitempty"` + PlaceholderGroup [][]byte `protobuf:"bytes,6,rep,name=placeholder_group,json=placeholderGroup,proto3" json:"placeholder_group,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *SearchRequest) Reset() { *m = SearchRequest{} } func (m *SearchRequest) String() string { return proto.CompactTextString(m) } func (*SearchRequest) ProtoMessage() {} func (*SearchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{29} + return fileDescriptor_02345ba45cc0e303, []int{26} } func (m *SearchRequest) XXX_Unmarshal(b []byte) error { @@ -1741,115 +1508,13 @@ func (m *SearchRequest) GetDsl() string { return "" } -func (m *SearchRequest) GetPlaceholderGroup() []byte { +func (m *SearchRequest) GetPlaceholderGroup() [][]byte { if m != nil { return m.PlaceholderGroup } return nil } -type Hits struct { - IDs []int64 `protobuf:"varint,1,rep,packed,name=IDs,proto3" json:"IDs,omitempty"` - RowData [][]byte `protobuf:"bytes,2,rep,name=row_data,json=rowData,proto3" json:"row_data,omitempty"` - Scores []float32 `protobuf:"fixed32,3,rep,packed,name=scores,proto3" json:"scores,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *Hits) Reset() { *m = Hits{} } -func (m *Hits) String() string { return proto.CompactTextString(m) } -func (*Hits) ProtoMessage() {} -func (*Hits) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{30} -} - -func (m *Hits) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Hits.Unmarshal(m, b) -} -func (m *Hits) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Hits.Marshal(b, m, deterministic) -} -func (m *Hits) XXX_Merge(src proto.Message) { - xxx_messageInfo_Hits.Merge(m, src) -} -func (m *Hits) XXX_Size() int { - return xxx_messageInfo_Hits.Size(m) -} -func (m *Hits) XXX_DiscardUnknown() { - xxx_messageInfo_Hits.DiscardUnknown(m) -} - -var xxx_messageInfo_Hits proto.InternalMessageInfo - -func (m *Hits) GetIDs() []int64 { - if m != nil { - return m.IDs - } - return nil -} - -func (m *Hits) GetRowData() [][]byte { - if m != nil { - return m.RowData - } - return nil -} - -func (m *Hits) GetScores() []float32 { - if m != nil { - return m.Scores - } - return nil -} - -type QueryResult struct { - Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` - Hits [][]byte `protobuf:"bytes,2,rep,name=hits,proto3" json:"hits,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *QueryResult) Reset() { *m = QueryResult{} } -func (m *QueryResult) String() string { return proto.CompactTextString(m) } -func (*QueryResult) ProtoMessage() {} -func (*QueryResult) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{31} -} - -func (m *QueryResult) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_QueryResult.Unmarshal(m, b) -} -func (m *QueryResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_QueryResult.Marshal(b, m, deterministic) -} -func (m *QueryResult) XXX_Merge(src proto.Message) { - xxx_messageInfo_QueryResult.Merge(m, src) -} -func (m *QueryResult) XXX_Size() int { - return xxx_messageInfo_QueryResult.Size(m) -} -func (m *QueryResult) XXX_DiscardUnknown() { - xxx_messageInfo_QueryResult.DiscardUnknown(m) -} - -var xxx_messageInfo_QueryResult proto.InternalMessageInfo - -func (m *QueryResult) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - -func (m *QueryResult) GetHits() [][]byte { - if m != nil { - return m.Hits - } - return nil -} - type FlushRequest struct { Base *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` @@ -1863,7 +1528,7 @@ func (m *FlushRequest) Reset() { *m = FlushRequest{} } func (m *FlushRequest) String() string { return proto.CompactTextString(m) } func (*FlushRequest) ProtoMessage() {} func (*FlushRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_02345ba45cc0e303, []int{32} + return fileDescriptor_02345ba45cc0e303, []int{27} } func (m *FlushRequest) XXX_Unmarshal(b []byte) error { @@ -1906,11 +1571,9 @@ func (m *FlushRequest) GetCollectionName() string { } func init() { - proto.RegisterEnum("milvus.proto.milvus.PlaceholderType", PlaceholderType_name, PlaceholderType_value) proto.RegisterType((*CreateCollectionRequest)(nil), "milvus.proto.milvus.CreateCollectionRequest") proto.RegisterType((*DropCollectionRequest)(nil), "milvus.proto.milvus.DropCollectionRequest") proto.RegisterType((*HasCollectionRequest)(nil), "milvus.proto.milvus.HasCollectionRequest") - proto.RegisterType((*BoolResponse)(nil), "milvus.proto.milvus.BoolResponse") proto.RegisterType((*DescribeCollectionRequest)(nil), "milvus.proto.milvus.DescribeCollectionRequest") proto.RegisterType((*DescribeCollectionResponse)(nil), "milvus.proto.milvus.DescribeCollectionResponse") proto.RegisterType((*LoadCollectionRequest)(nil), "milvus.proto.milvus.LoadCollectionRequest") @@ -1934,869 +1597,63 @@ func init() { proto.RegisterType((*DescribeIndexResponse)(nil), "milvus.proto.milvus.DescribeIndexResponse") proto.RegisterType((*InsertRequest)(nil), "milvus.proto.milvus.InsertRequest") proto.RegisterType((*InsertResponse)(nil), "milvus.proto.milvus.InsertResponse") - proto.RegisterType((*PlaceholderValue)(nil), "milvus.proto.milvus.PlaceholderValue") - proto.RegisterType((*PlaceholderGroup)(nil), "milvus.proto.milvus.PlaceholderGroup") proto.RegisterType((*SearchRequest)(nil), "milvus.proto.milvus.SearchRequest") - proto.RegisterType((*Hits)(nil), "milvus.proto.milvus.Hits") - proto.RegisterType((*QueryResult)(nil), "milvus.proto.milvus.QueryResult") proto.RegisterType((*FlushRequest)(nil), "milvus.proto.milvus.FlushRequest") } func init() { proto.RegisterFile("milvus.proto", fileDescriptor_02345ba45cc0e303) } var fileDescriptor_02345ba45cc0e303 = []byte{ - // 1409 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x58, 0xdd, 0x6f, 0x1b, 0x45, - 0x10, 0xcf, 0xc5, 0x8e, 0x9b, 0x4c, 0xfc, 0x95, 0x6d, 0x93, 0xb8, 0xae, 0x80, 0xf4, 0xa0, 0xd4, - 0xfd, 0x4a, 0xa4, 0x14, 0x09, 0x78, 0x40, 0x6a, 0xdd, 0xf4, 0xc3, 0x6a, 0x9b, 0x86, 0x4b, 0xa8, - 0x14, 0x50, 0x65, 0xd6, 0xbe, 0x25, 0x3e, 0x71, 0xf6, 0x1d, 0xb7, 0xeb, 0xa4, 0xe9, 0x2b, 0x08, - 0x5e, 0x00, 0xc1, 0xbf, 0xc0, 0x03, 0x2f, 0xbc, 0x41, 0x41, 0xfc, 0x53, 0x88, 0x3f, 0x01, 0xb4, - 0x1f, 0xb9, 0xdc, 0x5d, 0xf6, 0x62, 0x13, 0x54, 0xd9, 0x79, 0xf3, 0x8e, 0x67, 0x67, 0x7e, 0x3b, - 0xf3, 0xdb, 0xbd, 0x99, 0x81, 0x7c, 0xd7, 0x71, 0x77, 0xfb, 0x74, 0xd9, 0x0f, 0x3c, 0xe6, 0xa1, - 0xb3, 0xd1, 0xd5, 0xb2, 0x5c, 0x54, 0xf3, 0x6d, 0xaf, 0xdb, 0xf5, 0x7a, 0x52, 0x58, 0x2d, 0x3a, - 0x3d, 0x46, 0x82, 0x1e, 0x76, 0xd5, 0x3a, 0x4f, 0xdb, 0x1d, 0xd2, 0xc5, 0x72, 0x65, 0xfe, 0x6c, - 0xc0, 0xe2, 0x9d, 0x80, 0x60, 0x46, 0xee, 0x78, 0xae, 0x4b, 0xda, 0xcc, 0xf1, 0x7a, 0x16, 0xf9, - 0xa2, 0x4f, 0x28, 0x43, 0xab, 0x90, 0x6d, 0x61, 0x4a, 0x2a, 0xc6, 0x92, 0x51, 0x9b, 0x5d, 0x7d, - 0x7d, 0x39, 0xe6, 0x2b, 0xb4, 0xfa, 0x98, 0xee, 0xd4, 0x31, 0x25, 0x96, 0xd0, 0x45, 0x8b, 0x70, - 0xc6, 0x6e, 0x35, 0x7b, 0xb8, 0x4b, 0x2a, 0x93, 0x4b, 0x46, 0x6d, 0xc6, 0xca, 0xd9, 0xad, 0x75, - 0xdc, 0x25, 0xe8, 0x6d, 0x28, 0xb6, 0x43, 0x0f, 0x5c, 0x52, 0xc9, 0x88, 0xff, 0x13, 0x52, 0xb4, - 0x00, 0x39, 0x09, 0xb0, 0x92, 0x5d, 0x32, 0x6a, 0x79, 0x4b, 0xad, 0xcc, 0x6f, 0x0d, 0x98, 0x5f, - 0x0b, 0x3c, 0x7f, 0x3c, 0x60, 0x72, 0x38, 0xe7, 0x1e, 0x60, 0xfa, 0x8a, 0xd1, 0x5c, 0x86, 0xd2, - 0xa1, 0x5f, 0xa9, 0xa0, 0x87, 0xb3, 0x0d, 0xf9, 0xba, 0xe7, 0xb9, 0x16, 0xa1, 0xbe, 0xd7, 0xa3, - 0x04, 0xdd, 0x84, 0x1c, 0x65, 0x98, 0xf5, 0xa9, 0xc2, 0x71, 0x21, 0x8e, 0x43, 0x11, 0x64, 0x53, - 0xa8, 0x58, 0x4a, 0x15, 0x9d, 0x83, 0xa9, 0x5d, 0xec, 0xf6, 0x25, 0x88, 0x69, 0x4b, 0x2e, 0xcc, - 0x1f, 0x0d, 0x38, 0xbf, 0x46, 0x68, 0x3b, 0x70, 0x5a, 0x64, 0x5c, 0x8e, 0xfb, 0x83, 0x01, 0x55, - 0x1d, 0xa6, 0xff, 0x73, 0xfa, 0x0f, 0x42, 0xe2, 0x4d, 0x8a, 0x4d, 0x97, 0xe2, 0x9b, 0xd4, 0xad, - 0x39, 0xf4, 0xb6, 0x29, 0x04, 0x21, 0x3f, 0xbf, 0x33, 0x60, 0xfe, 0x91, 0x87, 0xed, 0x31, 0x0a, - 0x51, 0xc5, 0x22, 0x2e, 0xc1, 0x74, 0x6c, 0xb2, 0xf6, 0xbd, 0x01, 0x0b, 0x91, 0xf8, 0x31, 0xcc, - 0xe8, 0x68, 0x01, 0x7d, 0xc3, 0x1f, 0xbf, 0x24, 0x20, 0xc5, 0xa1, 0x77, 0x61, 0x8a, 0x13, 0x83, - 0x53, 0x28, 0x53, 0x9b, 0x5d, 0xbd, 0xa8, 0xa5, 0xd0, 0x43, 0xb2, 0xff, 0x94, 0x5f, 0x92, 0x0d, - 0xec, 0x04, 0x96, 0xd4, 0x8f, 0x90, 0x6f, 0x72, 0x68, 0xf2, 0x99, 0x36, 0xcc, 0x6f, 0x76, 0xbc, - 0xbd, 0x57, 0x9b, 0x29, 0xf3, 0x39, 0x2c, 0x24, 0xbd, 0xa8, 0xd3, 0x5e, 0x81, 0x72, 0x22, 0x64, - 0xf2, 0xe0, 0x33, 0x56, 0x29, 0x1e, 0xb3, 0x13, 0x9e, 0xef, 0x77, 0x9e, 0x7a, 0xf1, 0x99, 0xd9, - 0xc0, 0x01, 0x73, 0x46, 0xce, 0x45, 0x74, 0x09, 0x8a, 0xfe, 0x01, 0x12, 0xa9, 0x97, 0x15, 0x7a, - 0x85, 0x50, 0x2a, 0x22, 0xf6, 0x9b, 0x01, 0xe7, 0xf8, 0x57, 0xe7, 0x74, 0xa1, 0xfe, 0xd5, 0x80, - 0xb3, 0x0f, 0x30, 0x3d, 0x5d, 0xa0, 0x5f, 0x1a, 0x70, 0x96, 0x3f, 0xa0, 0x12, 0xf5, 0xa8, 0x41, - 0x5f, 0x86, 0x52, 0x1c, 0x34, 0xad, 0x64, 0xc5, 0x7d, 0x28, 0xc6, 0x50, 0x53, 0xf3, 0x4f, 0x03, - 0x16, 0xd5, 0x3b, 0x3b, 0x26, 0xf1, 0x1e, 0x1a, 0xfa, 0x4b, 0x03, 0xe6, 0x43, 0xcc, 0xa3, 0x7f, - 0x8e, 0x87, 0x25, 0xca, 0xd7, 0x06, 0x2c, 0x24, 0x61, 0x8f, 0xe4, 0xd1, 0xe6, 0x35, 0x20, 0x7f, - 0x4f, 0xc7, 0x23, 0xef, 0x66, 0x5f, 0x7e, 0x43, 0x22, 0x68, 0x54, 0x54, 0x34, 0x84, 0x30, 0x74, - 0x84, 0x38, 0x59, 0x14, 0xfe, 0x36, 0x00, 0xc9, 0xa7, 0xbd, 0xd1, 0xb3, 0xc9, 0xf3, 0xd1, 0x52, - 0xe8, 0x35, 0x80, 0xcf, 0x1c, 0xe2, 0xda, 0x51, 0xfa, 0xcc, 0x08, 0x89, 0xf8, 0x7b, 0x0d, 0xf2, - 0xe4, 0x39, 0x0b, 0x70, 0xd3, 0xc7, 0x01, 0xee, 0xd2, 0xca, 0xd4, 0xb0, 0x34, 0x99, 0x15, 0xdb, - 0x36, 0xc4, 0x2e, 0xf3, 0x17, 0xfe, 0x51, 0x50, 0xd5, 0xe7, 0xd8, 0x9f, 0xd9, 0x74, 0xa1, 0x2c, - 0x40, 0x4a, 0xc4, 0x3e, 0xdf, 0xc6, 0xb7, 0x38, 0x5c, 0x26, 0xb7, 0x18, 0x72, 0x8b, 0x90, 0x08, - 0x8b, 0xef, 0x43, 0x4e, 0x05, 0x68, 0x72, 0xd8, 0x00, 0xa9, 0x0d, 0xe6, 0x4f, 0xbc, 0x4d, 0x8b, - 0xc7, 0x46, 0xb1, 0x70, 0x0b, 0x90, 0xf4, 0x69, 0x1f, 0x02, 0x39, 0xb8, 0xa8, 0x89, 0x5a, 0x5b, - 0x2d, 0x92, 0xb0, 0xad, 0x39, 0x27, 0x21, 0x39, 0x21, 0x65, 0xff, 0x31, 0xa0, 0xd0, 0xe8, 0x51, - 0x12, 0xb0, 0xd3, 0xf0, 0xe0, 0xa1, 0x77, 0x60, 0x3a, 0xf0, 0xf6, 0x9a, 0x36, 0x66, 0x58, 0x31, - 0xf6, 0xbc, 0xf6, 0x94, 0x75, 0xd7, 0x6b, 0x59, 0x67, 0x02, 0x6f, 0x6f, 0x0d, 0x33, 0x8c, 0x2e, - 0xc0, 0x4c, 0x07, 0xd3, 0x4e, 0xf3, 0x73, 0xb2, 0x4f, 0x2b, 0xb9, 0xa5, 0x4c, 0xad, 0x60, 0x4d, - 0x73, 0xc1, 0x43, 0xb2, 0x4f, 0xcd, 0x2f, 0x0d, 0x28, 0x1e, 0x44, 0x40, 0xe5, 0xe7, 0x0d, 0x98, - 0x0d, 0xbc, 0xbd, 0xc6, 0x5a, 0xb3, 0x45, 0x76, 0x9c, 0x9e, 0x88, 0x44, 0xc6, 0x02, 0x21, 0xaa, - 0x73, 0x09, 0x37, 0x28, 0x15, 0x48, 0xcf, 0x16, 0x27, 0xce, 0x58, 0xd3, 0x42, 0x70, 0xb7, 0x67, - 0x47, 0xf2, 0x90, 0x19, 0x3e, 0x0f, 0xbb, 0x50, 0xde, 0x70, 0x71, 0x9b, 0x74, 0x3c, 0xd7, 0x26, - 0x81, 0x20, 0x13, 0x2a, 0x43, 0x86, 0xe1, 0x1d, 0xc5, 0x49, 0xfe, 0x13, 0xbd, 0x07, 0x59, 0xb6, - 0xef, 0xcb, 0x20, 0x17, 0x57, 0xdf, 0xd2, 0x52, 0x25, 0x62, 0x66, 0x6b, 0xdf, 0x27, 0x96, 0xd8, - 0x81, 0x16, 0x20, 0x27, 0x7a, 0x58, 0x0e, 0x2a, 0x53, 0xcb, 0x5b, 0x6a, 0x65, 0x3e, 0x8b, 0xf9, - 0xbd, 0x1f, 0x78, 0x7d, 0x1f, 0x35, 0x20, 0xef, 0x1f, 0xca, 0x8e, 0x27, 0x66, 0x12, 0xb4, 0x15, - 0xdb, 0x6a, 0xfe, 0x65, 0x40, 0x61, 0x93, 0xe0, 0xa0, 0xdd, 0x39, 0x1d, 0x85, 0x00, 0x8f, 0xb9, - 0x4d, 0xdd, 0xca, 0x94, 0x8c, 0xb9, 0x4d, 0x5d, 0x74, 0x0d, 0xe6, 0x22, 0x47, 0x6a, 0xee, 0xf0, - 0x10, 0x55, 0x72, 0x62, 0x20, 0x53, 0xf6, 0x13, 0xa1, 0x33, 0x1f, 0x42, 0xf6, 0x81, 0xc3, 0x84, - 0x99, 0xc6, 0x9a, 0x8c, 0x5c, 0xc6, 0xe2, 0x3f, 0xd1, 0xf9, 0x08, 0x73, 0x27, 0x45, 0x0a, 0x42, - 0x7a, 0x8a, 0x39, 0x8f, 0x17, 0xa8, 0xdc, 0x4c, 0x5a, 0x6a, 0x65, 0x3e, 0x85, 0xd9, 0x0f, 0xfb, - 0x24, 0xd8, 0xb7, 0x08, 0xed, 0xbb, 0xec, 0x64, 0xad, 0x3c, 0x82, 0x6c, 0xc7, 0x61, 0x54, 0xb9, - 0x14, 0xbf, 0xcd, 0xaf, 0x0c, 0xc8, 0xdf, 0x73, 0xfb, 0x74, 0xb4, 0x39, 0xb9, 0x7a, 0x0b, 0x4a, - 0x09, 0xae, 0xa2, 0x69, 0xc8, 0xae, 0x3f, 0x59, 0xbf, 0x5b, 0x9e, 0x40, 0x73, 0x50, 0x78, 0x7a, - 0xf7, 0xce, 0xd6, 0x13, 0xab, 0x59, 0x6f, 0xac, 0xdf, 0xb6, 0xb6, 0xcb, 0x36, 0x2a, 0x43, 0x5e, - 0x89, 0xee, 0x3d, 0x7a, 0x72, 0x7b, 0xab, 0x4c, 0x56, 0xff, 0x28, 0x41, 0xe1, 0xb1, 0xc0, 0xba, - 0x49, 0x82, 0x5d, 0xa7, 0x4d, 0x50, 0x13, 0xca, 0xc9, 0x11, 0x1e, 0xba, 0xae, 0x25, 0x6e, 0xca, - 0xa4, 0xaf, 0x7a, 0x5c, 0x54, 0xcd, 0x09, 0xf4, 0x09, 0x14, 0xe3, 0xa3, 0x37, 0x74, 0x55, 0x6b, - 0x5e, 0x3b, 0x9f, 0x1b, 0x64, 0xbc, 0x09, 0x85, 0xd8, 0x20, 0x0d, 0x5d, 0xd1, 0xda, 0xd6, 0x0d, - 0xdb, 0xaa, 0x17, 0xb5, 0xaa, 0xd1, 0x49, 0x98, 0x44, 0x1f, 0x1f, 0xcc, 0xa4, 0xa0, 0xd7, 0x4e, - 0x6f, 0x06, 0xa1, 0xc7, 0x30, 0x77, 0x64, 0xca, 0x82, 0x6e, 0x68, 0xed, 0xa7, 0x4d, 0x63, 0x06, - 0xb9, 0xd8, 0x03, 0x74, 0x74, 0xd6, 0x85, 0x96, 0xf5, 0x19, 0x48, 0x1b, 0xd4, 0x55, 0x57, 0x86, - 0xd6, 0x0f, 0x03, 0xb7, 0x0b, 0x8b, 0xf7, 0x09, 0x8b, 0x0f, 0x48, 0x1c, 0xca, 0x9c, 0x36, 0x45, - 0xd7, 0xf4, 0xf4, 0xd2, 0x0e, 0x77, 0xaa, 0xd7, 0x87, 0x53, 0x0e, 0xfd, 0xba, 0x50, 0x8a, 0x8f, - 0x29, 0x68, 0x4a, 0xc6, 0xb4, 0x23, 0x93, 0xea, 0xb5, 0xa1, 0x74, 0x43, 0x6f, 0xcf, 0xa0, 0x94, - 0x98, 0x4c, 0xa4, 0x9d, 0x4e, 0x3b, 0xbf, 0x18, 0x94, 0xbd, 0x6d, 0x28, 0xc4, 0x06, 0x08, 0x29, - 0xf4, 0xd6, 0x0d, 0x19, 0x06, 0x99, 0x7e, 0x06, 0xf9, 0x68, 0x97, 0x8f, 0x6a, 0x69, 0x17, 0xe7, - 0x88, 0xe1, 0xa1, 0xee, 0xcd, 0xb6, 0xbc, 0x37, 0xe1, 0x66, 0x9a, 0xe2, 0x40, 0xd3, 0xb4, 0x0f, - 0x42, 0xfe, 0x69, 0x78, 0x6b, 0x22, 0xd6, 0xaf, 0x1f, 0x77, 0x6b, 0xfe, 0x6b, 0x6c, 0x28, 0x2c, - 0xdc, 0x27, 0x2c, 0xd6, 0x26, 0x2a, 0xea, 0xea, 0xa9, 0xa4, 0xed, 0x83, 0x53, 0xa8, 0xa4, 0x6f, - 0x3e, 0xcd, 0x09, 0xe4, 0x40, 0x31, 0xd6, 0x81, 0xd1, 0x94, 0x64, 0xeb, 0x9a, 0xc6, 0xea, 0xd5, - 0x61, 0x54, 0x43, 0x57, 0x1f, 0xc1, 0x6c, 0xa4, 0xe9, 0x42, 0x97, 0x8f, 0x61, 0x6c, 0xb4, 0x45, - 0x19, 0x14, 0xb6, 0x0e, 0x14, 0x62, 0xd5, 0x7b, 0x1a, 0x5b, 0x35, 0xdd, 0x4f, 0xca, 0x01, 0xb4, - 0xcd, 0x80, 0x39, 0x81, 0x36, 0x21, 0x27, 0x0b, 0x50, 0x64, 0xa6, 0x14, 0xff, 0x91, 0xfa, 0xbc, - 0xfa, 0xe6, 0xb1, 0x3a, 0xa1, 0xd1, 0x0d, 0xc8, 0xc9, 0xc2, 0x2b, 0xc5, 0x68, 0xac, 0x2a, 0xab, - 0x2e, 0x69, 0x75, 0x22, 0xd5, 0x87, 0x39, 0x81, 0x1a, 0x30, 0x25, 0xaa, 0x06, 0xa4, 0xbf, 0x32, - 0xd1, 0x8a, 0x62, 0x40, 0x6c, 0xeb, 0xf5, 0x8f, 0x6f, 0xed, 0x38, 0xac, 0xd3, 0x6f, 0xf1, 0x7f, - 0x56, 0x5e, 0x38, 0xae, 0xeb, 0xbc, 0x60, 0xa4, 0xdd, 0x59, 0x91, 0xbb, 0x6e, 0xd8, 0x0e, 0x65, - 0x81, 0xd3, 0xea, 0x33, 0x62, 0xaf, 0x1c, 0xd4, 0x1f, 0x2b, 0xc2, 0x94, 0xd2, 0xf0, 0x5b, 0xad, - 0x9c, 0x58, 0xdf, 0xfc, 0x37, 0x00, 0x00, 0xff, 0xff, 0x10, 0x02, 0x02, 0xfd, 0x06, 0x1c, 0x00, - 0x00, -} - -// Reference imports to suppress errors if they are not otherwise used. -var _ context.Context -var _ grpc.ClientConn - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion4 - -// MilvusServiceClient is the client API for MilvusService service. -// -// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. -type MilvusServiceClient interface { - CreateCollection(ctx context.Context, in *CreateCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) - DropCollection(ctx context.Context, in *DropCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) - HasCollection(ctx context.Context, in *HasCollectionRequest, opts ...grpc.CallOption) (*BoolResponse, error) - LoadCollection(ctx context.Context, in *LoadCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) - ReleaseCollection(ctx context.Context, in *ReleaseCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) - DescribeCollection(ctx context.Context, in *DescribeCollectionRequest, opts ...grpc.CallOption) (*DescribeCollectionResponse, error) - GetCollectionStatistics(ctx context.Context, in *CollectionStatsRequest, opts ...grpc.CallOption) (*CollectionStatsResponse, error) - ShowCollections(ctx context.Context, in *ShowCollectionRequest, opts ...grpc.CallOption) (*ShowCollectionResponse, error) - CreatePartition(ctx context.Context, in *CreatePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) - DropPartition(ctx context.Context, in *DropPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) - HasPartition(ctx context.Context, in *HasPartitionRequest, opts ...grpc.CallOption) (*BoolResponse, error) - LoadPartitions(ctx context.Context, in *LoadPartitonRequest, opts ...grpc.CallOption) (*commonpb.Status, error) - ReleasePartitions(ctx context.Context, in *ReleasePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) - GetPartitionStatistics(ctx context.Context, in *PartitionStatsRequest, opts ...grpc.CallOption) (*PartitionStatsResponse, error) - ShowPartitions(ctx context.Context, in *ShowPartitionRequest, opts ...grpc.CallOption) (*ShowPartitionResponse, error) - CreateIndex(ctx context.Context, in *CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) - DescribeIndex(ctx context.Context, in *DescribeIndexRequest, opts ...grpc.CallOption) (*DescribeIndexResponse, error) - Insert(ctx context.Context, in *InsertRequest, opts ...grpc.CallOption) (*InsertResponse, error) - Search(ctx context.Context, in *SearchRequest, opts ...grpc.CallOption) (*QueryResult, error) - Flush(ctx context.Context, in *FlushRequest, opts ...grpc.CallOption) (*commonpb.Status, error) -} - -type milvusServiceClient struct { - cc *grpc.ClientConn -} - -func NewMilvusServiceClient(cc *grpc.ClientConn) MilvusServiceClient { - return &milvusServiceClient{cc} -} - -func (c *milvusServiceClient) CreateCollection(ctx context.Context, in *CreateCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { - out := new(commonpb.Status) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/CreateCollection", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) DropCollection(ctx context.Context, in *DropCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { - out := new(commonpb.Status) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/DropCollection", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) HasCollection(ctx context.Context, in *HasCollectionRequest, opts ...grpc.CallOption) (*BoolResponse, error) { - out := new(BoolResponse) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/HasCollection", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) LoadCollection(ctx context.Context, in *LoadCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { - out := new(commonpb.Status) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/LoadCollection", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) ReleaseCollection(ctx context.Context, in *ReleaseCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { - out := new(commonpb.Status) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/ReleaseCollection", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) DescribeCollection(ctx context.Context, in *DescribeCollectionRequest, opts ...grpc.CallOption) (*DescribeCollectionResponse, error) { - out := new(DescribeCollectionResponse) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/DescribeCollection", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) GetCollectionStatistics(ctx context.Context, in *CollectionStatsRequest, opts ...grpc.CallOption) (*CollectionStatsResponse, error) { - out := new(CollectionStatsResponse) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/GetCollectionStatistics", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) ShowCollections(ctx context.Context, in *ShowCollectionRequest, opts ...grpc.CallOption) (*ShowCollectionResponse, error) { - out := new(ShowCollectionResponse) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/ShowCollections", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) CreatePartition(ctx context.Context, in *CreatePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { - out := new(commonpb.Status) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/CreatePartition", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) DropPartition(ctx context.Context, in *DropPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { - out := new(commonpb.Status) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/DropPartition", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) HasPartition(ctx context.Context, in *HasPartitionRequest, opts ...grpc.CallOption) (*BoolResponse, error) { - out := new(BoolResponse) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/HasPartition", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) LoadPartitions(ctx context.Context, in *LoadPartitonRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { - out := new(commonpb.Status) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/LoadPartitions", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) ReleasePartitions(ctx context.Context, in *ReleasePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { - out := new(commonpb.Status) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/ReleasePartitions", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) GetPartitionStatistics(ctx context.Context, in *PartitionStatsRequest, opts ...grpc.CallOption) (*PartitionStatsResponse, error) { - out := new(PartitionStatsResponse) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/GetPartitionStatistics", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) ShowPartitions(ctx context.Context, in *ShowPartitionRequest, opts ...grpc.CallOption) (*ShowPartitionResponse, error) { - out := new(ShowPartitionResponse) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/ShowPartitions", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) CreateIndex(ctx context.Context, in *CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { - out := new(commonpb.Status) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/CreateIndex", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) DescribeIndex(ctx context.Context, in *DescribeIndexRequest, opts ...grpc.CallOption) (*DescribeIndexResponse, error) { - out := new(DescribeIndexResponse) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/DescribeIndex", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) Insert(ctx context.Context, in *InsertRequest, opts ...grpc.CallOption) (*InsertResponse, error) { - out := new(InsertResponse) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/Insert", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) Search(ctx context.Context, in *SearchRequest, opts ...grpc.CallOption) (*QueryResult, error) { - out := new(QueryResult) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/Search", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *milvusServiceClient) Flush(ctx context.Context, in *FlushRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { - out := new(commonpb.Status) - err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/Flush", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -// MilvusServiceServer is the server API for MilvusService service. -type MilvusServiceServer interface { - CreateCollection(context.Context, *CreateCollectionRequest) (*commonpb.Status, error) - DropCollection(context.Context, *DropCollectionRequest) (*commonpb.Status, error) - HasCollection(context.Context, *HasCollectionRequest) (*BoolResponse, error) - LoadCollection(context.Context, *LoadCollectionRequest) (*commonpb.Status, error) - ReleaseCollection(context.Context, *ReleaseCollectionRequest) (*commonpb.Status, error) - DescribeCollection(context.Context, *DescribeCollectionRequest) (*DescribeCollectionResponse, error) - GetCollectionStatistics(context.Context, *CollectionStatsRequest) (*CollectionStatsResponse, error) - ShowCollections(context.Context, *ShowCollectionRequest) (*ShowCollectionResponse, error) - CreatePartition(context.Context, *CreatePartitionRequest) (*commonpb.Status, error) - DropPartition(context.Context, *DropPartitionRequest) (*commonpb.Status, error) - HasPartition(context.Context, *HasPartitionRequest) (*BoolResponse, error) - LoadPartitions(context.Context, *LoadPartitonRequest) (*commonpb.Status, error) - ReleasePartitions(context.Context, *ReleasePartitionRequest) (*commonpb.Status, error) - GetPartitionStatistics(context.Context, *PartitionStatsRequest) (*PartitionStatsResponse, error) - ShowPartitions(context.Context, *ShowPartitionRequest) (*ShowPartitionResponse, error) - CreateIndex(context.Context, *CreateIndexRequest) (*commonpb.Status, error) - DescribeIndex(context.Context, *DescribeIndexRequest) (*DescribeIndexResponse, error) - Insert(context.Context, *InsertRequest) (*InsertResponse, error) - Search(context.Context, *SearchRequest) (*QueryResult, error) - Flush(context.Context, *FlushRequest) (*commonpb.Status, error) -} - -// UnimplementedMilvusServiceServer can be embedded to have forward compatible implementations. -type UnimplementedMilvusServiceServer struct { -} - -func (*UnimplementedMilvusServiceServer) CreateCollection(ctx context.Context, req *CreateCollectionRequest) (*commonpb.Status, error) { - return nil, status.Errorf(codes.Unimplemented, "method CreateCollection not implemented") -} -func (*UnimplementedMilvusServiceServer) DropCollection(ctx context.Context, req *DropCollectionRequest) (*commonpb.Status, error) { - return nil, status.Errorf(codes.Unimplemented, "method DropCollection not implemented") -} -func (*UnimplementedMilvusServiceServer) HasCollection(ctx context.Context, req *HasCollectionRequest) (*BoolResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method HasCollection not implemented") -} -func (*UnimplementedMilvusServiceServer) LoadCollection(ctx context.Context, req *LoadCollectionRequest) (*commonpb.Status, error) { - return nil, status.Errorf(codes.Unimplemented, "method LoadCollection not implemented") -} -func (*UnimplementedMilvusServiceServer) ReleaseCollection(ctx context.Context, req *ReleaseCollectionRequest) (*commonpb.Status, error) { - return nil, status.Errorf(codes.Unimplemented, "method ReleaseCollection not implemented") -} -func (*UnimplementedMilvusServiceServer) DescribeCollection(ctx context.Context, req *DescribeCollectionRequest) (*DescribeCollectionResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method DescribeCollection not implemented") -} -func (*UnimplementedMilvusServiceServer) GetCollectionStatistics(ctx context.Context, req *CollectionStatsRequest) (*CollectionStatsResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method GetCollectionStatistics not implemented") -} -func (*UnimplementedMilvusServiceServer) ShowCollections(ctx context.Context, req *ShowCollectionRequest) (*ShowCollectionResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method ShowCollections not implemented") -} -func (*UnimplementedMilvusServiceServer) CreatePartition(ctx context.Context, req *CreatePartitionRequest) (*commonpb.Status, error) { - return nil, status.Errorf(codes.Unimplemented, "method CreatePartition not implemented") -} -func (*UnimplementedMilvusServiceServer) DropPartition(ctx context.Context, req *DropPartitionRequest) (*commonpb.Status, error) { - return nil, status.Errorf(codes.Unimplemented, "method DropPartition not implemented") -} -func (*UnimplementedMilvusServiceServer) HasPartition(ctx context.Context, req *HasPartitionRequest) (*BoolResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method HasPartition not implemented") -} -func (*UnimplementedMilvusServiceServer) LoadPartitions(ctx context.Context, req *LoadPartitonRequest) (*commonpb.Status, error) { - return nil, status.Errorf(codes.Unimplemented, "method LoadPartitions not implemented") -} -func (*UnimplementedMilvusServiceServer) ReleasePartitions(ctx context.Context, req *ReleasePartitionRequest) (*commonpb.Status, error) { - return nil, status.Errorf(codes.Unimplemented, "method ReleasePartitions not implemented") -} -func (*UnimplementedMilvusServiceServer) GetPartitionStatistics(ctx context.Context, req *PartitionStatsRequest) (*PartitionStatsResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method GetPartitionStatistics not implemented") -} -func (*UnimplementedMilvusServiceServer) ShowPartitions(ctx context.Context, req *ShowPartitionRequest) (*ShowPartitionResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented") -} -func (*UnimplementedMilvusServiceServer) CreateIndex(ctx context.Context, req *CreateIndexRequest) (*commonpb.Status, error) { - return nil, status.Errorf(codes.Unimplemented, "method CreateIndex not implemented") -} -func (*UnimplementedMilvusServiceServer) DescribeIndex(ctx context.Context, req *DescribeIndexRequest) (*DescribeIndexResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method DescribeIndex not implemented") -} -func (*UnimplementedMilvusServiceServer) Insert(ctx context.Context, req *InsertRequest) (*InsertResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method Insert not implemented") -} -func (*UnimplementedMilvusServiceServer) Search(ctx context.Context, req *SearchRequest) (*QueryResult, error) { - return nil, status.Errorf(codes.Unimplemented, "method Search not implemented") -} -func (*UnimplementedMilvusServiceServer) Flush(ctx context.Context, req *FlushRequest) (*commonpb.Status, error) { - return nil, status.Errorf(codes.Unimplemented, "method Flush not implemented") -} - -func RegisterMilvusServiceServer(s *grpc.Server, srv MilvusServiceServer) { - s.RegisterService(&_MilvusService_serviceDesc, srv) -} - -func _MilvusService_CreateCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CreateCollectionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).CreateCollection(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/CreateCollection", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).CreateCollection(ctx, req.(*CreateCollectionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_DropCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(DropCollectionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).DropCollection(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/DropCollection", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).DropCollection(ctx, req.(*DropCollectionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_HasCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(HasCollectionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).HasCollection(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/HasCollection", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).HasCollection(ctx, req.(*HasCollectionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_LoadCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(LoadCollectionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).LoadCollection(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/LoadCollection", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).LoadCollection(ctx, req.(*LoadCollectionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_ReleaseCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ReleaseCollectionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).ReleaseCollection(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/ReleaseCollection", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).ReleaseCollection(ctx, req.(*ReleaseCollectionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_DescribeCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(DescribeCollectionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).DescribeCollection(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/DescribeCollection", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).DescribeCollection(ctx, req.(*DescribeCollectionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_GetCollectionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CollectionStatsRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).GetCollectionStatistics(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/GetCollectionStatistics", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).GetCollectionStatistics(ctx, req.(*CollectionStatsRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_ShowCollections_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ShowCollectionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).ShowCollections(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/ShowCollections", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).ShowCollections(ctx, req.(*ShowCollectionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_CreatePartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CreatePartitionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).CreatePartition(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/CreatePartition", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).CreatePartition(ctx, req.(*CreatePartitionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_DropPartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(DropPartitionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).DropPartition(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/DropPartition", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).DropPartition(ctx, req.(*DropPartitionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_HasPartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(HasPartitionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).HasPartition(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/HasPartition", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).HasPartition(ctx, req.(*HasPartitionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_LoadPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(LoadPartitonRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).LoadPartitions(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/LoadPartitions", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).LoadPartitions(ctx, req.(*LoadPartitonRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_ReleasePartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ReleasePartitionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).ReleasePartitions(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/ReleasePartitions", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).ReleasePartitions(ctx, req.(*ReleasePartitionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_GetPartitionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(PartitionStatsRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).GetPartitionStatistics(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/GetPartitionStatistics", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).GetPartitionStatistics(ctx, req.(*PartitionStatsRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_ShowPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ShowPartitionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).ShowPartitions(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/ShowPartitions", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).ShowPartitions(ctx, req.(*ShowPartitionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_CreateIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CreateIndexRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).CreateIndex(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/CreateIndex", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).CreateIndex(ctx, req.(*CreateIndexRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_DescribeIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(DescribeIndexRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).DescribeIndex(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/DescribeIndex", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).DescribeIndex(ctx, req.(*DescribeIndexRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_Insert_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(InsertRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).Insert(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/Insert", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).Insert(ctx, req.(*InsertRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_Search_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(SearchRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).Search(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/Search", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).Search(ctx, req.(*SearchRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MilvusService_Flush_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(FlushRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MilvusServiceServer).Flush(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/milvus.proto.milvus.MilvusService/Flush", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MilvusServiceServer).Flush(ctx, req.(*FlushRequest)) - } - return interceptor(ctx, in, info, handler) -} - -var _MilvusService_serviceDesc = grpc.ServiceDesc{ - ServiceName: "milvus.proto.milvus.MilvusService", - HandlerType: (*MilvusServiceServer)(nil), - Methods: []grpc.MethodDesc{ - { - MethodName: "CreateCollection", - Handler: _MilvusService_CreateCollection_Handler, - }, - { - MethodName: "DropCollection", - Handler: _MilvusService_DropCollection_Handler, - }, - { - MethodName: "HasCollection", - Handler: _MilvusService_HasCollection_Handler, - }, - { - MethodName: "LoadCollection", - Handler: _MilvusService_LoadCollection_Handler, - }, - { - MethodName: "ReleaseCollection", - Handler: _MilvusService_ReleaseCollection_Handler, - }, - { - MethodName: "DescribeCollection", - Handler: _MilvusService_DescribeCollection_Handler, - }, - { - MethodName: "GetCollectionStatistics", - Handler: _MilvusService_GetCollectionStatistics_Handler, - }, - { - MethodName: "ShowCollections", - Handler: _MilvusService_ShowCollections_Handler, - }, - { - MethodName: "CreatePartition", - Handler: _MilvusService_CreatePartition_Handler, - }, - { - MethodName: "DropPartition", - Handler: _MilvusService_DropPartition_Handler, - }, - { - MethodName: "HasPartition", - Handler: _MilvusService_HasPartition_Handler, - }, - { - MethodName: "LoadPartitions", - Handler: _MilvusService_LoadPartitions_Handler, - }, - { - MethodName: "ReleasePartitions", - Handler: _MilvusService_ReleasePartitions_Handler, - }, - { - MethodName: "GetPartitionStatistics", - Handler: _MilvusService_GetPartitionStatistics_Handler, - }, - { - MethodName: "ShowPartitions", - Handler: _MilvusService_ShowPartitions_Handler, - }, - { - MethodName: "CreateIndex", - Handler: _MilvusService_CreateIndex_Handler, - }, - { - MethodName: "DescribeIndex", - Handler: _MilvusService_DescribeIndex_Handler, - }, - { - MethodName: "Insert", - Handler: _MilvusService_Insert_Handler, - }, - { - MethodName: "Search", - Handler: _MilvusService_Search_Handler, - }, - { - MethodName: "Flush", - Handler: _MilvusService_Flush_Handler, - }, - }, - Streams: []grpc.StreamDesc{}, - Metadata: "milvus.proto", + // 809 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x97, 0xdf, 0x6f, 0x2a, 0x45, + 0x14, 0xc7, 0x33, 0x85, 0xd2, 0x72, 0xf8, 0x51, 0xba, 0x2d, 0xb0, 0xad, 0x51, 0x71, 0x93, 0x5a, + 0x8c, 0x11, 0x92, 0xda, 0xc4, 0xf8, 0xd6, 0x50, 0xd4, 0x36, 0xd5, 0xa6, 0x6e, 0x8d, 0x0f, 0xbe, + 0x90, 0xd9, 0xdd, 0x91, 0xdd, 0xb8, 0xbb, 0xb3, 0xce, 0x0c, 0xd2, 0xf6, 0xd9, 0x47, 0x35, 0xfa, + 0x4f, 0xf8, 0xe2, 0x9b, 0xd6, 0xc4, 0x7f, 0xca, 0xdc, 0x3f, 0xe1, 0xde, 0xcc, 0xcc, 0x42, 0x81, + 0xcb, 0x03, 0xc9, 0x4d, 0x03, 0xbc, 0x31, 0xdf, 0x3d, 0x67, 0xe6, 0x73, 0xce, 0x19, 0xbe, 0x2c, + 0x50, 0x8c, 0x82, 0xf0, 0xa7, 0x01, 0x6f, 0x25, 0x8c, 0x0a, 0x6a, 0xec, 0x4d, 0xae, 0x5a, 0x7a, + 0x71, 0x58, 0x74, 0x69, 0x14, 0xd1, 0x58, 0x8b, 0x87, 0xe5, 0x20, 0x16, 0x84, 0xc5, 0x38, 0xd4, + 0x6b, 0xeb, 0x4f, 0x04, 0xf5, 0x73, 0x46, 0xb0, 0x20, 0xe7, 0x34, 0x0c, 0x89, 0x2b, 0x02, 0x1a, + 0xdb, 0xe4, 0xc7, 0x01, 0xe1, 0xc2, 0x38, 0x81, 0xac, 0x83, 0x39, 0x31, 0x51, 0x03, 0x35, 0x0b, + 0x27, 0xef, 0xb4, 0xa6, 0x76, 0x1f, 0xef, 0xf3, 0x15, 0xef, 0x77, 0x30, 0x27, 0xb6, 0x8a, 0x35, + 0xea, 0xb0, 0xe5, 0x39, 0xbd, 0x18, 0x47, 0xc4, 0xdc, 0x68, 0xa0, 0x66, 0xde, 0xce, 0x79, 0xce, + 0x35, 0x8e, 0x88, 0xf1, 0x3e, 0x94, 0xdd, 0xf1, 0x09, 0x52, 0x31, 0x33, 0xea, 0xf9, 0x8c, 0x6a, + 0xd4, 0x20, 0xc7, 0x5d, 0x9f, 0x44, 0xd8, 0xcc, 0x36, 0x50, 0xb3, 0x68, 0xa7, 0x2b, 0xeb, 0x17, + 0x04, 0xd5, 0x2e, 0xa3, 0xc9, 0x6a, 0x60, 0x4a, 0x9c, 0xfd, 0x0b, 0xcc, 0x9f, 0x99, 0xe6, 0x18, + 0x76, 0x9e, 0xce, 0xd5, 0x01, 0xf3, 0x71, 0xfe, 0x40, 0x70, 0xd0, 0x25, 0xdc, 0x65, 0x81, 0x43, + 0x56, 0x85, 0xe9, 0x14, 0x0e, 0xe7, 0x21, 0xf1, 0x84, 0xc6, 0x7c, 0x72, 0xce, 0xa8, 0x91, 0x99, + 0x98, 0xf3, 0xaf, 0x08, 0xaa, 0x5f, 0x52, 0xec, 0xad, 0x4a, 0x15, 0xbf, 0x23, 0x30, 0x6d, 0x12, + 0x12, 0xcc, 0x57, 0xa6, 0xb1, 0xbf, 0x21, 0xa8, 0x3d, 0xb1, 0xdc, 0x0a, 0x2c, 0xf8, 0x72, 0x81, + 0x6c, 0xa8, 0xbf, 0xc6, 0x93, 0x8e, 0xf9, 0x13, 0xd8, 0xe4, 0x52, 0x50, 0x53, 0x2e, 0x9c, 0xbc, + 0x37, 0x4d, 0x94, 0x5a, 0xd3, 0x15, 0xb9, 0xff, 0x16, 0x87, 0x03, 0x72, 0x83, 0x03, 0x66, 0xeb, + 0x78, 0xcb, 0x83, 0xea, 0xad, 0x4f, 0x87, 0xcf, 0xdb, 0x73, 0xeb, 0x1c, 0x6a, 0xb3, 0xa7, 0xa4, + 0xe0, 0x1f, 0x40, 0x65, 0xa6, 0x78, 0x5d, 0x43, 0xde, 0xde, 0x99, 0xae, 0x9e, 0x5b, 0xff, 0xca, + 0x79, 0x28, 0x0f, 0xbd, 0xc1, 0x4c, 0x04, 0x4b, 0xbf, 0x20, 0xc6, 0x11, 0x94, 0x93, 0x11, 0x89, + 0x8e, 0xcb, 0xaa, 0xb8, 0xd2, 0x58, 0x55, 0xc5, 0xff, 0x83, 0x60, 0x5f, 0x5a, 0xea, 0x7a, 0x51, + 0xff, 0x8d, 0x60, 0xef, 0x02, 0xf3, 0xf5, 0x82, 0x7e, 0x44, 0xb0, 0x27, 0x5d, 0x4d, 0x53, 0x2f, + 0x1b, 0xfa, 0x18, 0x76, 0xa6, 0xa1, 0xb9, 0x99, 0x55, 0x57, 0xbb, 0x3c, 0x45, 0xcd, 0xad, 0xff, + 0x10, 0xd4, 0x53, 0xf3, 0x5b, 0x91, 0x7e, 0x2f, 0x8c, 0xfe, 0x88, 0xa0, 0x3a, 0x66, 0x5e, 0xbe, + 0x47, 0x2e, 0x7a, 0x51, 0xbe, 0x86, 0xda, 0x2c, 0xf5, 0x9b, 0x3a, 0xa9, 0x7c, 0x55, 0x91, 0x26, + 0xb7, 0x1a, 0x13, 0xb4, 0xce, 0xb4, 0xb1, 0x4f, 0xd0, 0xa4, 0x05, 0xce, 0x19, 0x2d, 0x9a, 0x3b, + 0xda, 0x17, 0x08, 0x0c, 0xed, 0xb7, 0x97, 0xb1, 0x47, 0xee, 0x96, 0x3b, 0xd7, 0xb7, 0x01, 0xbe, + 0x0f, 0x48, 0xe8, 0x4d, 0xce, 0x34, 0xaf, 0x14, 0xf5, 0xb8, 0x0b, 0x45, 0x72, 0x27, 0x18, 0xee, + 0x25, 0x98, 0xe1, 0x88, 0x9b, 0x9b, 0x8b, 0x0e, 0xaf, 0xa0, 0xd2, 0x6e, 0x54, 0x96, 0xf5, 0x97, + 0x74, 0xea, 0xf4, 0x5d, 0x6a, 0xe5, 0x6b, 0xb6, 0x42, 0xa8, 0x28, 0x48, 0x4d, 0x9c, 0xc8, 0x34, + 0x99, 0x12, 0x48, 0x4d, 0xa7, 0x20, 0x9d, 0xa2, 0x14, 0xb5, 0xe3, 0xa7, 0x90, 0x4b, 0x1b, 0xb4, + 0xb1, 0x68, 0x83, 0xd2, 0x04, 0x2b, 0x82, 0xea, 0x4c, 0x6b, 0xd2, 0xfb, 0xf4, 0x0d, 0x18, 0xfa, + 0x48, 0xef, 0x89, 0x63, 0xf4, 0xed, 0x39, 0x6a, 0xcd, 0xf9, 0xab, 0xd4, 0x9a, 0xa5, 0xb6, 0x77, + 0x83, 0x19, 0x85, 0x5b, 0x2f, 0x11, 0x94, 0x2e, 0x63, 0x4e, 0x98, 0x58, 0x07, 0x3f, 0x31, 0x4e, + 0x61, 0x9b, 0xd1, 0x61, 0xcf, 0xc3, 0x02, 0xa7, 0x77, 0xef, 0x60, 0x6e, 0x6b, 0x3b, 0x21, 0x75, + 0xec, 0x2d, 0x46, 0x87, 0x5d, 0x2c, 0xb0, 0xf1, 0x16, 0xe4, 0x7d, 0xcc, 0xfd, 0xde, 0x0f, 0xe4, + 0x9e, 0x9b, 0xb9, 0x46, 0xa6, 0x59, 0xb2, 0xb7, 0xa5, 0x70, 0x45, 0xee, 0xb9, 0x75, 0x0d, 0xe5, + 0x51, 0x03, 0xd2, 0x4e, 0xbf, 0x0b, 0x05, 0x46, 0x87, 0x97, 0xdd, 0x9e, 0x43, 0xfa, 0x41, 0xac, + 0x1a, 0x91, 0xb1, 0x41, 0x49, 0x1d, 0xa9, 0xc8, 0xfd, 0x74, 0x00, 0x89, 0x3d, 0x55, 0x70, 0xc6, + 0xde, 0x56, 0xc2, 0x67, 0xb1, 0x67, 0xfd, 0x8f, 0xa0, 0x74, 0x4b, 0x30, 0x73, 0xfd, 0xf5, 0xf8, + 0x69, 0x31, 0x2a, 0x90, 0xf1, 0x78, 0x68, 0x6e, 0xaa, 0x5d, 0xe4, 0x47, 0xe3, 0x43, 0xd8, 0x4d, + 0x42, 0xec, 0x12, 0x9f, 0x86, 0x1e, 0x61, 0xbd, 0x3e, 0xa3, 0x83, 0x44, 0xf5, 0xad, 0x68, 0x57, + 0x26, 0x1e, 0x7c, 0x21, 0x75, 0xeb, 0x67, 0x04, 0xc5, 0xcf, 0xc3, 0x01, 0x5f, 0x6e, 0xb9, 0x9d, + 0xce, 0x77, 0x67, 0xfd, 0x40, 0xf8, 0x03, 0x47, 0x5e, 0x81, 0xf6, 0x43, 0x10, 0x86, 0xc1, 0x83, + 0x20, 0xae, 0xdf, 0xd6, 0xc7, 0x7f, 0xe4, 0x05, 0x5c, 0xb0, 0xc0, 0x19, 0x08, 0xe2, 0xb5, 0x47, + 0x10, 0x6d, 0xc5, 0x94, 0x46, 0x24, 0x8e, 0x93, 0x53, 0xeb, 0x8f, 0x5f, 0x05, 0x00, 0x00, 0xff, + 0xff, 0x3b, 0xfd, 0x8e, 0x53, 0x87, 0x10, 0x00, 0x00, } diff --git a/internal/proxynode/proxy.go b/internal/proxynode/proxy.go index c6026d884..fd91c38e0 100644 --- a/internal/proxynode/proxy.go +++ b/internal/proxynode/proxy.go @@ -12,7 +12,6 @@ import ( "time" "github.com/opentracing/opentracing-go" - "github.com/uber/jaeger-client-go" "github.com/uber/jaeger-client-go/config" "google.golang.org/grpc" @@ -72,11 +71,8 @@ func CreateProxy(ctx context.Context) (*Proxy, error) { Type: "const", Param: 1, }, - Reporter: &config.ReporterConfig{ - LogSpans: true, - }, } - p.tracer, p.closer, err = cfg.NewTracer(config.Logger(jaeger.StdLogger)) + p.tracer, p.closer, err = cfg.NewTracer() if err != nil { panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err)) } diff --git a/internal/proxyservice/interface.go b/internal/proxyservice/interface.go deleted file mode 100644 index b57290924..000000000 --- a/internal/proxyservice/interface.go +++ /dev/null @@ -1,16 +0,0 @@ -package proxyservice - -import ( - "github.com/zilliztech/milvus-distributed/internal/proto/proxypb" - "github.com/zilliztech/milvus-distributed/internal/util/typeutil" -) - -type ServiceBase = typeutil.Service - -type Interface interface { - ServiceBase - RegisterLink() (proxypb.RegisterLinkResponse, error) - RegisterNode(request proxypb.RegisterNodeRequest) (proxypb.RegisterNodeResponse, error) - // TODO: i'm sure it's not a best way to keep consistency, fix me - InvalidateCollectionMetaCache(request proxypb.InvalidateCollMetaCacheRequest) error -} diff --git a/internal/proxyservice/proxyservice.go b/internal/proxyservice/proxyservice.go deleted file mode 100644 index 99b3be15a..000000000 --- a/internal/proxyservice/proxyservice.go +++ /dev/null @@ -1,54 +0,0 @@ -package proxyservice - -import ( - "github.com/zilliztech/milvus-distributed/internal/proto/internalpb2" - "github.com/zilliztech/milvus-distributed/internal/proto/proxypb" -) - -type ProxyService struct { - // implement Service - - //nodeClients [] .Interface - // factory method - -} - -func (s ProxyService) Init() { - panic("implement me") -} - -func (s ProxyService) Start() { - panic("implement me") -} - -func (s ProxyService) Stop() { - panic("implement me") -} - -func (s ProxyService) GetServiceStates() (internalpb2.ServiceStates, error) { - panic("implement me") -} - -func (s ProxyService) GetTimeTickChannel() (string, error) { - panic("implement me") -} - -func (s ProxyService) GetStatisticsChannel() (string, error) { - panic("implement me") -} - -func (s ProxyService) RegisterLink() (proxypb.RegisterLinkResponse, error) { - panic("implement me") -} - -func (s ProxyService) RegisterNode(request proxypb.RegisterNodeRequest) (proxypb.RegisterNodeResponse, error) { - panic("implement me") -} - -func (s ProxyService) InvalidateCollectionMetaCache(request proxypb.InvalidateCollMetaCacheRequest) error { - panic("implement me") -} - -func NewProxyServiceImpl() Interface { - return &ProxyService{} -} diff --git a/internal/querynode/query_node.go b/internal/querynode/query_node.go index 38ad79bee..fa49db959 100644 --- a/internal/querynode/query_node.go +++ b/internal/querynode/query_node.go @@ -18,7 +18,6 @@ import ( "io" "github.com/opentracing/opentracing-go" - "github.com/uber/jaeger-client-go" "github.com/uber/jaeger-client-go/config" "google.golang.org/grpc" @@ -89,11 +88,8 @@ func newQueryNode(ctx context.Context, queryNodeID uint64) *QueryNode { Type: "const", Param: 1, }, - Reporter: &config.ReporterConfig{ - LogSpans: true, - }, } - q.tracer, q.closer, err = cfg.NewTracer(config.Logger(jaeger.StdLogger)) + q.tracer, q.closer, err = cfg.NewTracer() if err != nil { panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err)) } diff --git a/internal/timesync/time_sync_producer.go b/internal/timesync/time_sync_producer.go new file mode 100644 index 000000000..fdb0f20bf --- /dev/null +++ b/internal/timesync/time_sync_producer.go @@ -0,0 +1,141 @@ +package timesync + +import ( + "context" + "log" + + "github.com/zilliztech/milvus-distributed/internal/errors" + ms "github.com/zilliztech/milvus-distributed/internal/msgstream" + internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" +) + +type timeSyncMsgProducer struct { + //softTimeTickBarrier + proxyTtBarrier TimeTickBarrier + //hardTimeTickBarrier + writeNodeTtBarrier TimeTickBarrier + + ddSyncStream ms.MsgStream // insert & delete + dmSyncStream ms.MsgStream + k2sSyncStream ms.MsgStream + + ctx context.Context + cancel context.CancelFunc + + proxyWatchers []TimeTickWatcher + writeNodeWatchers []TimeTickWatcher +} + +func NewTimeSyncMsgProducer(ctx context.Context) (*timeSyncMsgProducer, error) { + ctx2, cancel := context.WithCancel(ctx) + return &timeSyncMsgProducer{ctx: ctx2, cancel: cancel}, nil +} + +func (syncMsgProducer *timeSyncMsgProducer) SetProxyTtBarrier(proxyTtBarrier TimeTickBarrier) { + syncMsgProducer.proxyTtBarrier = proxyTtBarrier +} + +func (syncMsgProducer *timeSyncMsgProducer) SetWriteNodeTtBarrier(writeNodeTtBarrier TimeTickBarrier) { + syncMsgProducer.writeNodeTtBarrier = writeNodeTtBarrier +} +func (syncMsgProducer *timeSyncMsgProducer) SetDDSyncStream(ddSync ms.MsgStream) { + syncMsgProducer.ddSyncStream = ddSync +} + +func (syncMsgProducer *timeSyncMsgProducer) SetDMSyncStream(dmSync ms.MsgStream) { + syncMsgProducer.dmSyncStream = dmSync +} + +func (syncMsgProducer *timeSyncMsgProducer) SetK2sSyncStream(k2sSync ms.MsgStream) { + syncMsgProducer.k2sSyncStream = k2sSync +} + +func (syncMsgProducer *timeSyncMsgProducer) WatchProxyTtBarrier(watcher TimeTickWatcher) { + syncMsgProducer.proxyWatchers = append(syncMsgProducer.proxyWatchers, watcher) +} + +func (syncMsgProducer *timeSyncMsgProducer) WatchWriteNodeTtBarrier(watcher TimeTickWatcher) { + syncMsgProducer.writeNodeWatchers = append(syncMsgProducer.writeNodeWatchers, watcher) +} + +func (syncMsgProducer *timeSyncMsgProducer) broadcastMsg(barrier TimeTickBarrier, streams []ms.MsgStream, watchers []TimeTickWatcher) error { + for { + select { + case <-syncMsgProducer.ctx.Done(): + { + log.Printf("broadcast context done, exit") + return errors.Errorf("broadcast done exit") + } + default: + timetick, err := barrier.GetTimeTick() + if err != nil { + log.Printf("broadcast get time tick error") + } + msgPack := ms.MsgPack{} + baseMsg := ms.BaseMsg{ + BeginTimestamp: timetick, + EndTimestamp: timetick, + HashValues: []uint32{0}, + } + timeTickResult := internalPb.TimeTickMsg{ + MsgType: internalPb.MsgType_kTimeTick, + PeerID: 0, + Timestamp: timetick, + } + timeTickMsg := &ms.TimeTickMsg{ + BaseMsg: baseMsg, + TimeTickMsg: timeTickResult, + } + msgPack.Msgs = append(msgPack.Msgs, timeTickMsg) + for _, stream := range streams { + err = stream.Broadcast(&msgPack) + } + + for _, watcher := range watchers { + watcher.Watch(timeTickMsg) + } + if err != nil { + return err + } + } + } +} + +func (syncMsgProducer *timeSyncMsgProducer) Start() error { + err := syncMsgProducer.proxyTtBarrier.Start() + if err != nil { + return err + } + + err = syncMsgProducer.writeNodeTtBarrier.Start() + if err != nil { + return err + } + + for _, watcher := range syncMsgProducer.proxyWatchers { + watcher.Start() + } + for _, watcher := range syncMsgProducer.writeNodeWatchers { + watcher.Start() + } + + go syncMsgProducer.broadcastMsg(syncMsgProducer.proxyTtBarrier, []ms.MsgStream{syncMsgProducer.dmSyncStream, syncMsgProducer.ddSyncStream}, syncMsgProducer.proxyWatchers) + go syncMsgProducer.broadcastMsg(syncMsgProducer.writeNodeTtBarrier, []ms.MsgStream{syncMsgProducer.k2sSyncStream}, syncMsgProducer.writeNodeWatchers) + + return nil +} + +func (syncMsgProducer *timeSyncMsgProducer) Close() { + syncMsgProducer.ddSyncStream.Close() + syncMsgProducer.dmSyncStream.Close() + syncMsgProducer.k2sSyncStream.Close() + syncMsgProducer.cancel() + syncMsgProducer.proxyTtBarrier.Close() + syncMsgProducer.writeNodeTtBarrier.Close() + for _, watcher := range syncMsgProducer.proxyWatchers { + watcher.Close() + } + for _, watcher := range syncMsgProducer.writeNodeWatchers { + watcher.Close() + } +} diff --git a/internal/timesync/timesync.go b/internal/timesync/timesync.go new file mode 100644 index 000000000..5cf8a50a9 --- /dev/null +++ b/internal/timesync/timesync.go @@ -0,0 +1,235 @@ +package timesync + +import ( + "context" + "log" + "math" + "sync/atomic" + + "github.com/zilliztech/milvus-distributed/internal/util/typeutil" + + "github.com/zilliztech/milvus-distributed/internal/errors" + ms "github.com/zilliztech/milvus-distributed/internal/msgstream" +) + +type ( + Timestamp = typeutil.Timestamp + UniqueID = typeutil.UniqueID + + TimeTickBarrier interface { + GetTimeTick() (Timestamp, error) + Start() error + Close() + } + + softTimeTickBarrier struct { + peer2LastTt map[UniqueID]Timestamp + minTtInterval Timestamp + lastTt int64 + outTt chan Timestamp + ttStream ms.MsgStream + ctx context.Context + cancel context.CancelFunc + } + + hardTimeTickBarrier struct { + peer2Tt map[UniqueID]Timestamp + outTt chan Timestamp + ttStream ms.MsgStream + ctx context.Context + cancel context.CancelFunc + } +) + +func (ttBarrier *softTimeTickBarrier) GetTimeTick() (Timestamp, error) { + select { + case <-ttBarrier.ctx.Done(): + return 0, errors.Errorf("[GetTimeTick] closed.") + case ts, ok := <-ttBarrier.outTt: + if !ok { + return 0, errors.Errorf("[GetTimeTick] closed.") + } + num := len(ttBarrier.outTt) + for i := 0; i < num; i++ { + ts, ok = <-ttBarrier.outTt + if !ok { + return 0, errors.Errorf("[GetTimeTick] closed.") + } + } + atomic.StoreInt64(&(ttBarrier.lastTt), int64(ts)) + return ts, ttBarrier.ctx.Err() + } +} + +func (ttBarrier *softTimeTickBarrier) Start() error { + go func() { + for { + select { + case <-ttBarrier.ctx.Done(): + log.Printf("[TtBarrierStart] %s\n", ttBarrier.ctx.Err()) + return + + case ttmsgs := <-ttBarrier.ttStream.Chan(): + if len(ttmsgs.Msgs) > 0 { + for _, timetickmsg := range ttmsgs.Msgs { + ttmsg := timetickmsg.(*ms.TimeTickMsg) + oldT, ok := ttBarrier.peer2LastTt[ttmsg.PeerID] + // log.Printf("[softTimeTickBarrier] peer(%d)=%d\n", ttmsg.PeerID, ttmsg.Timestamp) + + if !ok { + log.Printf("[softTimeTickBarrier] Warning: peerID %d not exist\n", ttmsg.PeerID) + continue + } + if ttmsg.Timestamp > oldT { + ttBarrier.peer2LastTt[ttmsg.PeerID] = ttmsg.Timestamp + + // get a legal Timestamp + ts := ttBarrier.minTimestamp() + lastTt := atomic.LoadInt64(&(ttBarrier.lastTt)) + if lastTt != 0 && ttBarrier.minTtInterval > ts-Timestamp(lastTt) { + continue + } + ttBarrier.outTt <- ts + } + } + } + } + } + }() + return nil +} + +func newSoftTimeTickBarrier(ctx context.Context, + ttStream *ms.MsgStream, + peerIds []UniqueID, + minTtInterval Timestamp) *softTimeTickBarrier { + + if len(peerIds) <= 0 { + log.Printf("[newSoftTimeTickBarrier] Error: peerIds is empty!\n") + return nil + } + + sttbarrier := softTimeTickBarrier{} + sttbarrier.minTtInterval = minTtInterval + sttbarrier.ttStream = *ttStream + sttbarrier.outTt = make(chan Timestamp, 1024) + sttbarrier.ctx, sttbarrier.cancel = context.WithCancel(ctx) + sttbarrier.peer2LastTt = make(map[UniqueID]Timestamp) + for _, id := range peerIds { + sttbarrier.peer2LastTt[id] = Timestamp(0) + } + if len(peerIds) != len(sttbarrier.peer2LastTt) { + log.Printf("[newSoftTimeTickBarrier] Warning: there are duplicate peerIds!\n") + } + + return &sttbarrier +} + +func (ttBarrier *softTimeTickBarrier) Close() { + ttBarrier.cancel() +} + +func (ttBarrier *softTimeTickBarrier) minTimestamp() Timestamp { + tempMin := Timestamp(math.MaxUint64) + for _, tt := range ttBarrier.peer2LastTt { + if tt < tempMin { + tempMin = tt + } + } + return tempMin +} + +func (ttBarrier *hardTimeTickBarrier) GetTimeTick() (Timestamp, error) { + select { + case <-ttBarrier.ctx.Done(): + return 0, errors.Errorf("[GetTimeTick] closed.") + case ts, ok := <-ttBarrier.outTt: + if !ok { + return 0, errors.Errorf("[GetTimeTick] closed.") + } + return ts, ttBarrier.ctx.Err() + } +} + +func (ttBarrier *hardTimeTickBarrier) Start() error { + go func() { + // Last timestamp synchronized + state := Timestamp(0) + for { + select { + case <-ttBarrier.ctx.Done(): + log.Printf("[TtBarrierStart] %s\n", ttBarrier.ctx.Err()) + return + + case ttmsgs := <-ttBarrier.ttStream.Chan(): + if len(ttmsgs.Msgs) > 0 { + for _, timetickmsg := range ttmsgs.Msgs { + + // Suppose ttmsg.Timestamp from stream is always larger than the previous one, + // that `ttmsg.Timestamp > oldT` + ttmsg := timetickmsg.(*ms.TimeTickMsg) + + oldT, ok := ttBarrier.peer2Tt[ttmsg.PeerID] + if !ok { + log.Printf("[hardTimeTickBarrier] Warning: peerID %d not exist\n", ttmsg.PeerID) + continue + } + + if oldT > state { + log.Printf("[hardTimeTickBarrier] Warning: peer(%d) timestamp(%d) ahead\n", + ttmsg.PeerID, ttmsg.Timestamp) + } + + ttBarrier.peer2Tt[ttmsg.PeerID] = ttmsg.Timestamp + + newState := ttBarrier.minTimestamp() + if newState > state { + ttBarrier.outTt <- newState + state = newState + } + } + } + } + } + }() + return nil +} + +func (ttBarrier *hardTimeTickBarrier) minTimestamp() Timestamp { + tempMin := Timestamp(math.MaxUint64) + for _, tt := range ttBarrier.peer2Tt { + if tt < tempMin { + tempMin = tt + } + } + return tempMin +} + +func newHardTimeTickBarrier(ctx context.Context, + ttStream *ms.MsgStream, + peerIds []UniqueID) *hardTimeTickBarrier { + + if len(peerIds) <= 0 { + log.Printf("[newSoftTimeTickBarrier] Error: peerIds is empty!") + return nil + } + + sttbarrier := hardTimeTickBarrier{} + sttbarrier.ttStream = *ttStream + sttbarrier.outTt = make(chan Timestamp, 1024) + sttbarrier.ctx, sttbarrier.cancel = context.WithCancel(ctx) + + sttbarrier.peer2Tt = make(map[UniqueID]Timestamp) + for _, id := range peerIds { + sttbarrier.peer2Tt[id] = Timestamp(0) + } + if len(peerIds) != len(sttbarrier.peer2Tt) { + log.Printf("[newSoftTimeTickBarrier] Warning: there are duplicate peerIds!") + } + + return &sttbarrier +} + +func (ttBarrier *hardTimeTickBarrier) Close() { + ttBarrier.cancel() +} diff --git a/internal/timesync/timetick_watcher.go b/internal/timesync/timetick_watcher.go new file mode 100644 index 000000000..27eb9e416 --- /dev/null +++ b/internal/timesync/timetick_watcher.go @@ -0,0 +1,11 @@ +package timesync + +import ( + ms "github.com/zilliztech/milvus-distributed/internal/msgstream" +) + +type TimeTickWatcher interface { + Watch(msg *ms.TimeTickMsg) + Start() + Close() +} diff --git a/internal/writenode/write_node.go b/internal/writenode/write_node.go index 5199148c9..280a3298c 100644 --- a/internal/writenode/write_node.go +++ b/internal/writenode/write_node.go @@ -6,7 +6,6 @@ import ( "io" "github.com/opentracing/opentracing-go" - "github.com/uber/jaeger-client-go" "github.com/uber/jaeger-client-go/config" ) @@ -52,12 +51,9 @@ func (node *WriteNode) Start() error { Type: "const", Param: 1, }, - Reporter: &config.ReporterConfig{ - LogSpans: true, - }, } var err error - node.tracer, node.closer, err = cfg.NewTracer(config.Logger(jaeger.StdLogger)) + node.tracer, node.closer, err = cfg.NewTracer() if err != nil { panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err)) } -- GitLab