Add client and grpc service

Signed-off-by: sunby <bingyi.sun@zilliz.com>
This commit is contained in:
sunby 2021-01-22 11:07:07 +08:00 committed by yefu.chen
parent c83a91eb1e
commit ef79b8d8c7
20 changed files with 1548 additions and 1163 deletions

View File

@ -187,55 +187,64 @@ func NewUnmarshalDispatcher() *UnmarshalDispatcher
RocksMQ is a RocksDB-based messaging/streaming library.
```GO
// All the following UniqueIDs are 64-bit integer, which is combined with timestamp and increasing number
```go
type ProducerMessage struct {
payload []byte
}
```
```go
type ConsumerMessage struct {
msgID UniqueID
msgID MessageID
payload []byte
}
```
type IDAllocator interface {
Alloc(count uint32) (UniqueID, UniqueID, error)
AllocOne() (UniqueID, error)
UpdateID() error
```GO
type Channel struct {
beginOffset MessageID
endOffset MessageID
}
type ConsumerGroupContext struct {
currentOffset MessageID
}
// Every collection has its RocksMQ
type RocksMQ struct {
store *gorocksdb.DB
kv kv.Base
idAllocator IDAllocator
produceMu sync.Mutex
consumeMu sync.Mutex
channels map[string]Channel
cgCtxs map[string]ConsumerGroupContext
mu sync.Mutex
}
func (rmq *RocksMQ) CreateChannel(channelName string) error
func (rmq *RocksMQ) DestroyChannel(channelName string) error
func (rmq *RocksMQ) CreateConsumerGroup(groupName string) error
func (rmq *RocksMQ) DestroyConsumerGroup(groupName string) error
func (rmq *RocksMQ) Produce(channelName string, messages []ProducerMessage) error
func (rmq *RocksMQ) Consume(groupName string, channelName string, n int) ([]ConsumerMessage, error)
func (rmq *RocksMQ) Seek(groupName string, channelName string, msgID MessageID) error
func NewRocksMQ(name string, idAllocator IDAllocator) (*RocksMQ, error)
func (rmq *RocksMQ) CreateChannel(channelName string) error // create channel, add record in meta-store
func (rmq *RocksMQ) DestroyChannel(channelName string) error // drop channel, delete record in meta-store
func (rmq *RocksMQ) CreateConsumerGroup(groupName string) error // create consumer group, add record in meta-store
func (rmq *RocksMQ) DestroyConsumerGroup(groupName string) error // drop consumer group, delete record in meta-store
func (rmq *RocksMQ) Produce(channelName string, messages []ProducerMessage) error // produce a batch of message, insert into rocksdb
func (rmq *RocksMQ) Consume(groupName string, channelName string, n int) ([]ConsumerMessage, error) // comsume up to n messages, modify current_id in Etcd
func (rmq *RocksMQ) Seek(groupName string, channelName string, msgID MessageID) error // modify current_id in Etcd
```
##### A.4.1 Meta (stored in Etcd)
```go
// channel meta
"$(channel_name)/begin_id", UniqueID
"$(channel_name)/end_id", UniqueID
* channel meta
// consumer group meta
"$(group_name)/$(channel_name)/current_id", UniqueID
```go
"$(channel_name)/begin_id", MessageID
"$(channel_name)/end_id", MessageID
```
* consumer group meta
```go
"$(group_name)/$(channel_name)/current_id", MessageID
```

View File

@ -1,15 +1,21 @@
package dataservice
import "fmt"
import (
"fmt"
"strconv"
"sync"
)
type (
channelRange []string
insertChannelMapper struct {
channelRanges []channelRange
channelGroup []string
insertChannelManager struct {
mu sync.RWMutex
count int
channelGroups map[UniqueID][]channelGroup // collection id to channel ranges
}
)
func (cr channelRange) Contains(channelName string) bool {
func (cr channelGroup) Contains(channelName string) bool {
for _, name := range cr {
if name == channelName {
return true
@ -18,30 +24,68 @@ func (cr channelRange) Contains(channelName string) bool {
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)
func newInsertChannelManager() *insertChannelManager {
return &insertChannelManager{
count: 0,
channelGroups: make(map[UniqueID][]channelGroup),
}
return mapper
}
func (mapper *insertChannelMapper) GetChannelRange(channelName string) (channelRange, error) {
for _, cr := range mapper.channelRanges {
func (cm *insertChannelManager) AllocChannels(collectionID UniqueID, groupNum int) ([]channelGroup, error) {
cm.mu.Lock()
defer cm.mu.Unlock()
if _, ok := cm.channelGroups[collectionID]; ok {
return nil, fmt.Errorf("channel group of collection %d already exist", collectionID)
}
channels := Params.InsertChannelNumPerCollection
m, n := channels/int64(groupNum), channels%int64(groupNum)
cg := make([]channelGroup, 0)
var i, j int64 = 0, 0
for i < channels {
var group []string
if j < n {
group = make([]string, m+1)
} else {
group = make([]string, m)
}
for k := 0; k < len(group); k++ {
group = append(group, Params.InsertChannelPrefixName+strconv.Itoa(cm.count))
cm.count++
}
i += int64(len(group))
j++
cg = append(cg, group)
}
return cg, nil
}
func (cm *insertChannelManager) GetChannelGroup(collectionID UniqueID, channelName string) (channelGroup, error) {
cm.mu.RLock()
defer cm.mu.RUnlock()
_, ok := cm.channelGroups[collectionID]
if !ok {
return nil, fmt.Errorf("can not find collection %d", collectionID)
}
for _, cr := range cm.channelGroups[collectionID] {
if cr.Contains(channelName) {
return cr, nil
}
}
return nil, fmt.Errorf("channel name %s not found", channelName)
}
func (cm *insertChannelManager) ContainsCollection(collectionID UniqueID) (bool, []string) {
cm.mu.RLock()
defer cm.mu.RUnlock()
_, ok := cm.channelGroups[collectionID]
if !ok {
return false, nil
}
ret := make([]string, 0)
for _, cr := range cm.channelGroups[collectionID] {
for _, c := range cr {
ret = append(ret, c)
}
}
return true, ret
}

View File

@ -1,129 +0,0 @@
package dataservice
import (
"context"
"log"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
)
type ddHandler struct {
meta *meta
segAllocator segmentAllocator
}
func newDDHandler(meta *meta, segAllocator segmentAllocator) *ddHandler {
return &ddHandler{
meta: meta,
segAllocator: segAllocator,
}
}
func (handler *ddHandler) Start(ctx context.Context, inputStream ms.MsgStream) {
for {
select {
case msgPack := <-inputStream.Chan():
for _, msg := range msgPack.Msgs {
switch msg.Type() {
case commonpb.MsgType_kCreateCollection:
createCollectionMsg, ok := msg.(*ms.CreateCollectionMsg)
if !ok {
log.Println("message with type MsgType_kCreateCollection can not be cast to CreateCollectionMsg")
continue
}
if err := handler.handleCreateCollection(&createCollectionMsg.CreateCollectionRequest); err != nil {
log.Printf("handle create collection error: %s", err.Error())
}
case commonpb.MsgType_kDropCollection:
dropCollectionMsg, ok := msg.(*ms.DropCollectionMsg)
if !ok {
log.Println("message with type MsgType_kDropCollection can not be cast to DropCollectionMsg")
continue
}
if err := handler.handleDropCollection(&dropCollectionMsg.DropCollectionRequest); err != nil {
log.Printf("handle drop collection error: %s", err.Error())
}
case commonpb.MsgType_kCreatePartition:
createPartitionMsg, ok := msg.(*ms.CreatePartitionMsg)
if !ok {
log.Println("message with type MsgType_kCreatePartition can not be cast to CreatePartitionMsg")
continue
}
if err := handler.handleCreatePartition(&createPartitionMsg.CreatePartitionRequest); err != nil {
log.Printf("handle create partition error: %s", err.Error())
}
case commonpb.MsgType_kDropPartition:
dropPartitionMsg, ok := msg.(*ms.DropPartitionMsg)
if !ok {
log.Println("message with type MsgType_kDropPartition can not be cast to DropPartitionMsg")
continue
}
if err := handler.handleDropPartition(&dropPartitionMsg.DropPartitionRequest); err != nil {
log.Printf("handle drop partition error: %s", err.Error())
}
default:
log.Printf("invalid message type %s", msg.Type())
}
}
case <-ctx.Done():
log.Println("dd handler is shut down.")
break
}
}
}
func (handler *ddHandler) handleCreateCollection(req *internalpb2.CreateCollectionRequest) error {
var schema schemapb.CollectionSchema
if err := proto.UnmarshalMerge(req.Schema, &schema); err != nil {
return err
}
info := &collectionInfo{
ID: req.CollectionID,
Schema: &schema,
}
return handler.meta.AddCollection(info)
}
func (handler *ddHandler) handleDropCollection(req *internalpb2.DropCollectionRequest) error {
if err := handler.meta.DropCollection(req.CollectionID); err != nil {
return err
}
segmentIDs := handler.meta.GetSegmentsByCollectionID(req.CollectionID)
for _, id := range segmentIDs {
if err := handler.meta.DropSegment(id); err != nil {
return err
}
handler.segAllocator.DropSegment(id)
}
return nil
}
func (handler *ddHandler) handleCreatePartition(req *internalpb2.CreatePartitionRequest) error {
return handler.meta.AddPartition(req.CollectionID, req.PartitionID)
}
func (handler *ddHandler) handleDropPartition(req *internalpb2.DropPartitionRequest) error {
if err := handler.meta.DropPartition(req.CollectionID, req.PartitionID); err != nil {
return err
}
ids := handler.meta.GetSegmentsByPartitionID(req.PartitionID)
for _, id := range ids {
if err := handler.meta.DropSegment(id); err != nil {
return err
}
handler.segAllocator.DropSegment(id)
}
return nil
}

View File

@ -1,63 +0,0 @@
package dataservice
import (
"fmt"
"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 *Server) RegisterNode(context.Context, *datapb.RegisterNodeRequest) (*datapb.RegisterNodeResponse, error) {
return nil, nil
}
func (ds *Server) Flush(context.Context, *datapb.FlushRequest) (*commonpb.Status, error) {
return nil, nil
}
func (ds *Server) AssignSegmentID(ctx context.Context, request *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error) {
resp := &datapb.AssignSegIDResponse{
SegIDAssignments: make([]*datapb.SegIDAssignment, 0),
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
},
}
task := &allocateTask{
baseTask: baseTask{
sch: ds.scheduler,
meta: ds.meta,
cv: make(chan error),
},
req: request,
resp: resp,
segAllocator: ds.segAllocator,
insertCMapper: ds.insertCMapper,
}
if err := ds.scheduler.Enqueue(task); err != nil {
resp.Status.Reason = fmt.Sprintf("enqueue error: %s", err.Error())
return resp, nil
}
if err := task.WaitToFinish(ctx); err != nil {
resp.Status.Reason = fmt.Sprintf("wait to finish error: %s", err.Error())
return resp, nil
}
return resp, nil
}
func (ds *Server) ShowSegments(context.Context, *datapb.ShowSegmentRequest) (*datapb.ShowSegmentResponse, error) {
return nil, nil
}
func (ds *Server) GetSegmentStates(context.Context, *datapb.SegmentStatesRequest) (*datapb.SegmentStatesResponse, error) {
return nil, nil
}
func (ds *Server) GetInsertBinlogPaths(context.Context, *datapb.InsertBinlogPathRequest) (*datapb.InsertBinlogPathsResponse, error) {
return nil, nil
}
func (ds *Server) GetInsertChannels(context.Context, *datapb.InsertChannelRequest) (*internalpb2.StringList, error) {
return nil, nil
}

View File

@ -16,8 +16,14 @@ import (
)
type (
UniqueID = typeutil.UniqueID
Timestamp = typeutil.Timestamp
UniqueID = typeutil.UniqueID
Timestamp = typeutil.Timestamp
errSegmentNotFound struct {
segmentID UniqueID
}
errCollectionNotFound struct {
collectionID UniqueID
}
collectionInfo struct {
ID UniqueID
Schema *schemapb.CollectionSchema
@ -33,7 +39,23 @@ type (
}
)
func newMetaTable(kv kv.TxnBase, allocator allocator) (*meta, error) {
func newErrSegmentNotFound(segmentID UniqueID) errSegmentNotFound {
return errSegmentNotFound{segmentID: segmentID}
}
func (err errSegmentNotFound) Error() string {
return fmt.Sprintf("segment %d not found", err.segmentID)
}
func newErrCollectionNotFound(collectionID UniqueID) errCollectionNotFound {
return errCollectionNotFound{collectionID: collectionID}
}
func (err errCollectionNotFound) Error() string {
return fmt.Sprintf("collection %d not found", err.collectionID)
}
func newMeta(kv kv.TxnBase, allocator allocator) (*meta, error) {
mt := &meta{
client: kv,
collID2Info: make(map[UniqueID]*collectionInfo),
@ -80,7 +102,7 @@ func (meta *meta) DropCollection(collID UniqueID) error {
defer meta.ddLock.Unlock()
if _, ok := meta.collID2Info[collID]; !ok {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
return newErrCollectionNotFound(collID)
}
delete(meta.collID2Info, collID)
return nil
@ -98,7 +120,7 @@ func (meta *meta) GetCollection(collectionID UniqueID) (*collectionInfo, error)
collectionInfo, ok := meta.collID2Info[collectionID]
if !ok {
return nil, fmt.Errorf("collection %d not found", collectionID)
return nil, newErrCollectionNotFound(collectionID)
}
return collectionInfo, nil
}
@ -119,9 +141,10 @@ func (meta *meta) BuildSegment(collectionID UniqueID, partitionID UniqueID, chan
PartitionID: partitionID,
InsertChannels: channelRange,
OpenTime: ts,
CloseTime: 0,
SealedTime: 0,
NumRows: 0,
MemSize: 0,
State: datapb.SegmentState_SegmentGrowing,
}, nil
}
@ -156,7 +179,7 @@ func (meta *meta) DropSegment(segmentID UniqueID) error {
meta.ddLock.Unlock()
if _, ok := meta.segID2Info[segmentID]; !ok {
return fmt.Errorf("segment %d not found", segmentID)
return newErrSegmentNotFound(segmentID)
}
delete(meta.segID2Info, segmentID)
return nil
@ -168,23 +191,52 @@ func (meta *meta) GetSegment(segID UniqueID) (*datapb.SegmentInfo, error) {
segmentInfo, ok := meta.segID2Info[segID]
if !ok {
return nil, errors.Errorf("GetSegmentByID:can't find segment id = %d", segID)
return nil, newErrSegmentNotFound(segID)
}
return segmentInfo, nil
}
func (meta *meta) CloseSegment(segID UniqueID, closeTs Timestamp) error {
func (meta *meta) SealSegment(segID UniqueID) error {
meta.ddLock.Lock()
defer meta.ddLock.Unlock()
segInfo, ok := meta.segID2Info[segID]
if !ok {
return errors.Errorf("DropSegment:can't find segment id = " + strconv.FormatInt(segID, 10))
return newErrSegmentNotFound(segID)
}
segInfo.CloseTime = closeTs
ts, err := meta.allocator.allocTimestamp()
if err != nil {
return err
}
segInfo.SealedTime = ts
segInfo.State = datapb.SegmentState_SegmentSealed
err := meta.saveSegmentInfo(segInfo)
err = meta.saveSegmentInfo(segInfo)
if err != nil {
_ = meta.reloadFromKV()
return err
}
return nil
}
func (meta *meta) FlushSegment(segID UniqueID) error {
meta.ddLock.Lock()
defer meta.ddLock.Unlock()
segInfo, ok := meta.segID2Info[segID]
if !ok {
return newErrSegmentNotFound(segID)
}
ts, err := meta.allocator.allocTimestamp()
if err != nil {
return err
}
segInfo.FlushedTime = ts
segInfo.State = datapb.SegmentState_SegmentFlushed
err = meta.saveSegmentInfo(segInfo)
if err != nil {
_ = meta.reloadFromKV()
return err
@ -205,13 +257,13 @@ func (meta *meta) GetSegmentsByCollectionID(collectionID UniqueID) []UniqueID {
return ret
}
func (meta *meta) GetSegmentsByPartitionID(partitionID UniqueID) []UniqueID {
func (meta *meta) GetSegmentsByCollectionAndPartitionID(collectionID, partitionID UniqueID) []UniqueID {
meta.ddLock.RLock()
defer meta.ddLock.RUnlock()
ret := make([]UniqueID, 0)
for _, info := range meta.segID2Info {
if info.PartitionID == partitionID {
if info.CollectionID == collectionID && info.PartitionID == partitionID {
ret = append(ret, info.SegmentID)
}
}
@ -223,7 +275,7 @@ func (meta *meta) AddPartition(collectionID UniqueID, partitionID UniqueID) erro
defer meta.ddLock.Unlock()
coll, ok := meta.collID2Info[collectionID]
if !ok {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collectionID, 10))
return newErrCollectionNotFound(collectionID)
}
for _, t := range coll.partitions {
@ -241,7 +293,7 @@ func (meta *meta) DropPartition(collID UniqueID, partitionID UniqueID) error {
collection, ok := meta.collID2Info[collID]
if !ok {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
return newErrCollectionNotFound(collID)
}
idx := -1

View File

@ -1,13 +1,7 @@
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 {
@ -16,42 +10,22 @@ type ParamTable struct {
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
EtcdAddress string
MetaRootPath string
KvRootPath string
PulsarAddress string
// 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
InsertChannelPrefixName string
InsertChannelNumPerCollection int64
StatisticsChannelName string
TimeTickChannelName string
DataNodeNum int64
}
var Params ParamTable
@ -72,35 +46,11 @@ func (p *ParamTable) Init() {
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() {
@ -131,14 +81,6 @@ func (p *ParamTable) initPulsarAddress() {
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 {
@ -162,24 +104,6 @@ func (p *ParamTable) initKvRootPath() {
}
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")
}
@ -191,180 +115,3 @@ func (p *ParamTable) initSegmentSizeFactor() {
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}
}

View File

@ -1,89 +0,0 @@
package dataservice
import (
"context"
"log"
"github.com/zilliztech/milvus-distributed/internal/errors"
ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
)
//type ddRequestScheduler interface {}
//type ddReqFIFOScheduler struct {}
type ddRequestScheduler struct {
ctx context.Context
cancel context.CancelFunc
globalIDAllocator func() (UniqueID, error)
reqQueue chan task
scheduleTimeStamp Timestamp
ddMsgStream ms.MsgStream
}
func NewDDRequestScheduler(ctx context.Context) *ddRequestScheduler {
const channelSize = 1024
ctx2, cancel := context.WithCancel(ctx)
rs := ddRequestScheduler{
ctx: ctx2,
cancel: cancel,
reqQueue: make(chan task, channelSize),
}
return &rs
}
func (rs *ddRequestScheduler) Enqueue(task task) error {
rs.reqQueue <- task
return nil
}
func (rs *ddRequestScheduler) SetIDAllocator(allocGlobalID func() (UniqueID, error)) {
rs.globalIDAllocator = allocGlobalID
}
func (rs *ddRequestScheduler) SetDDMsgStream(ddStream ms.MsgStream) {
rs.ddMsgStream = ddStream
}
func (rs *ddRequestScheduler) scheduleLoop() {
for {
select {
case task := <-rs.reqQueue:
err := rs.schedule(task)
if err != nil {
log.Println(err)
}
case <-rs.ctx.Done():
log.Print("server is closed, exit task execution loop")
return
}
}
}
func (rs *ddRequestScheduler) schedule(t task) error {
timeStamp, err := t.Ts()
if err != nil {
log.Println(err)
return err
}
if timeStamp < rs.scheduleTimeStamp {
t.Notify(errors.Errorf("input timestamp = %d, schduler timestamp = %d", timeStamp, rs.scheduleTimeStamp))
} else {
rs.scheduleTimeStamp = timeStamp
err = t.Execute()
t.Notify(err)
}
return nil
}
func (rs *ddRequestScheduler) Start() error {
go rs.scheduleLoop()
return nil
}
func (rs *ddRequestScheduler) Close() {
rs.cancel()
}

View File

@ -26,13 +26,13 @@ func (err errRemainInSufficient) Error() string {
// 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(collectionID UniqueID, partitionID UniqueID, segmentID UniqueID, cRange channelRange) error
OpenSegment(collectionID UniqueID, partitionID UniqueID, segmentID UniqueID, cRange channelGroup) 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)
SealSegment(segmentID UniqueID) error
// DropSegment drop the segment from allocator.
DropSegment(segmentID UniqueID)
// ExpireAllocations check all allocations' expire time and remove the expired allocation.
@ -50,7 +50,7 @@ type (
sealed bool
lastExpireTime Timestamp
allocations []*allocation
cRange channelRange
cRange channelGroup
}
allocation struct {
rowNums int
@ -79,7 +79,7 @@ func newSegmentAssigner(metaTable *meta, allocator allocator) (*segmentAllocator
return segmentAllocator, nil
}
func (allocator *segmentAllocatorImpl) OpenSegment(collectionID UniqueID, partitionID UniqueID, segmentID UniqueID, cRange channelRange) error {
func (allocator *segmentAllocatorImpl) OpenSegment(collectionID UniqueID, partitionID UniqueID, segmentID UniqueID, cRange channelGroup) error {
if _, ok := allocator.segments[segmentID]; ok {
return fmt.Errorf("segment %d already exist", segmentID)
}
@ -99,7 +99,8 @@ func (allocator *segmentAllocatorImpl) OpenSegment(collectionID UniqueID, partit
return nil
}
func (allocator *segmentAllocatorImpl) AllocSegment(collectionID UniqueID, partitionID UniqueID, channelName string, requestRows int) (segID UniqueID, retCount int, expireTime Timestamp, err error) {
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()
@ -195,14 +196,18 @@ func (allocator *segmentAllocatorImpl) checkSegmentSealed(segStatus *segmentStat
return float64(segMeta.NumRows) >= allocator.segmentThresholdFactor*float64(segStatus.total), nil
}
func (allocator *segmentAllocatorImpl) SealSegment(segmentID UniqueID) {
func (allocator *segmentAllocatorImpl) SealSegment(segmentID UniqueID) error {
allocator.mu.Lock()
defer allocator.mu.Unlock()
status, ok := allocator.segments[segmentID]
if !ok {
return
return nil
}
if err := allocator.mt.SealSegment(segmentID); err != nil {
return err
}
status.sealed = true
return nil
}
func (allocator *segmentAllocatorImpl) DropSegment(segmentID UniqueID) {
@ -223,7 +228,6 @@ func (allocator *segmentAllocatorImpl) ExpireAllocations(timeTick Timestamp) err
i--
}
}
return nil
}

View File

@ -1,8 +1,338 @@
package dataservice
type Server struct {
segAllocator segmentAllocator
meta *meta
insertCMapper insertChannelMapper
scheduler *ddRequestScheduler
import (
"context"
"fmt"
"log"
"sync"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/timesync"
etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd"
"go.etcd.io/etcd/clientv3"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
)
type DataService interface {
typeutil.Service
RegisterNode(req *datapb.RegisterNodeRequest) (*datapb.RegisterNodeResponse, error)
Flush(req *datapb.FlushRequest) (*commonpb.Status, error)
AssignSegmentID(req *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error)
ShowSegments(req *datapb.ShowSegmentRequest) (*datapb.ShowSegmentResponse, error)
GetSegmentStates(req *datapb.SegmentStatesRequest) (*datapb.SegmentStatesResponse, error)
GetInsertBinlogPaths(req *datapb.InsertBinlogPathRequest) (*datapb.InsertBinlogPathsResponse, error)
GetInsertChannels(req *datapb.InsertChannelRequest) (*internalpb2.StringList, error)
GetCollectionStatistics(req *datapb.CollectionStatsRequest) (*datapb.CollectionStatsResponse, error)
GetPartitionStatistics(req *datapb.PartitionStatsRequest) (*datapb.PartitionStatsResponse, error)
GetComponentStates() (*internalpb2.ComponentStates, error)
GetTimeTickChannel() (*milvuspb.StringResponse, error)
GetStatisticsChannel() (*milvuspb.StringResponse, error)
}
type (
datanode struct {
nodeID int64
address struct {
ip string
port int64
}
// todo add client
}
Server struct {
ctx context.Context
state internalpb2.StateCode
client *etcdkv.EtcdKV
meta *meta
segAllocator segmentAllocator
statsHandler *statsHandler
insertChannelMgr *insertChannelManager
allocator allocator
msgProducer *timesync.MsgProducer
nodeIDCounter int64
nodes []*datanode
registerFinishCh chan struct{}
registerMu sync.RWMutex
}
)
func CreateServer(ctx context.Context) (*Server, error) {
return &Server{
ctx: ctx,
state: internalpb2.StateCode_INITIALIZING,
insertChannelMgr: newInsertChannelManager(),
nodeIDCounter: 0,
nodes: make([]*datanode, 0),
registerFinishCh: make(chan struct{}),
}, nil
}
func (s *Server) Init() error {
Params.Init()
s.allocator = newAllocatorImpl()
if err := s.initMeta(); err != nil {
return err
}
s.statsHandler = newStatsHandler(s.meta)
segAllocator, err := newSegmentAssigner(s.meta, s.allocator)
if err != nil {
return err
}
s.segAllocator = segAllocator
if err = s.initMsgProducer(); err != nil {
return err
}
return nil
}
func (s *Server) initMeta() error {
etcdClient, err := clientv3.New(clientv3.Config{Endpoints: []string{Params.EtcdAddress}})
if err != nil {
return err
}
etcdKV := etcdkv.NewEtcdKV(etcdClient, Params.MetaRootPath)
s.client = etcdKV
s.meta, err = newMeta(etcdKV, s.allocator)
if err != nil {
return err
}
return nil
}
func (s *Server) initMsgProducer() error {
// todo ttstream and peerids
timeTickBarrier := timesync.NewHardTimeTickBarrier(nil, nil)
// todo add watchers
producer, err := timesync.NewTimeSyncMsgProducer(timeTickBarrier)
if err != nil {
return err
}
s.msgProducer = producer
return nil
}
func (s *Server) Start() error {
s.waitDataNodeRegister()
// todo add load meta from master
s.msgProducer.Start(s.ctx)
return nil
}
func (s *Server) waitDataNodeRegister() {
<-s.registerFinishCh
}
func (s *Server) Stop() error {
s.msgProducer.Close()
return nil
}
func (s *Server) GetComponentStates() (*internalpb2.ComponentStates, error) {
// todo foreach datanode, call GetServiceStates
return nil, nil
}
func (s *Server) GetTimeTickChannel() (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
},
Value: Params.TimeTickChannelName,
}, nil
}
func (s *Server) GetStatisticsChannel() (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
},
Value: Params.StatisticsChannelName,
}, nil
}
func (s *Server) RegisterNode(req *datapb.RegisterNodeRequest) (*datapb.RegisterNodeResponse, error) {
s.registerMu.Lock()
defer s.registerMu.Unlock()
resp := &datapb.RegisterNodeResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
},
}
if !s.checkDataNodeNotExist(req.Address.Ip, req.Address.Port) {
resp.Status.Reason = fmt.Sprintf("data node with address %s exist", req.Address.String())
return resp, nil
}
s.nodeIDCounter++
s.nodes = append(s.nodes, &datanode{
nodeID: s.nodeIDCounter,
address: struct {
ip string
port int64
}{ip: req.Address.Ip, port: req.Address.Port},
})
if s.nodeIDCounter == Params.DataNodeNum {
close(s.registerFinishCh)
}
resp.Status.ErrorCode = commonpb.ErrorCode_SUCCESS
// add init params
return resp, nil
}
func (s *Server) checkDataNodeNotExist(ip string, port int64) bool {
for _, node := range s.nodes {
if node.address.ip == ip || node.address.port == port {
return false
}
}
return true
}
func (s *Server) Flush(req *datapb.FlushRequest) (*commonpb.Status, error) {
// todo call datanode flush
return nil, nil
}
func (s *Server) AssignSegmentID(req *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error) {
resp := &datapb.AssignSegIDResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
},
SegIDAssignments: make([]*datapb.SegIDAssignment, 0),
}
for _, r := range req.SegIDRequests {
result := &datapb.SegIDAssignment{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
},
}
segmentID, retCount, expireTs, err := s.segAllocator.AllocSegment(r.CollectionID, r.PartitionID, r.ChannelName, int(r.Count))
if err != nil {
if _, ok := err.(errRemainInSufficient); !ok {
result.Status.Reason = fmt.Sprintf("allocation of Collection %d, Partition %d, Channel %s, Count %d error: %s",
r.CollectionID, r.PartitionID, r.ChannelName, r.Count, err.Error())
resp.SegIDAssignments = append(resp.SegIDAssignments, result)
continue
}
log.Printf("no enough space for allocation of Collection %d, Partition %d, Channel %s, Count %d",
r.CollectionID, r.PartitionID, r.ChannelName, r.Count)
if err = s.openNewSegment(r.CollectionID, r.PartitionID, r.ChannelName); err != nil {
result.Status.Reason = fmt.Sprintf("open new segment of Collection %d, Partition %d, Channel %s, Count %d error: %s",
r.CollectionID, r.PartitionID, r.ChannelName, r.Count, err.Error())
resp.SegIDAssignments = append(resp.SegIDAssignments, result)
continue
}
segmentID, retCount, expireTs, err = s.segAllocator.AllocSegment(r.CollectionID, r.PartitionID, r.ChannelName, int(r.Count))
if err != nil {
result.Status.Reason = fmt.Sprintf("retry allocation of Collection %d, Partition %d, Channel %s, Count %d error: %s",
r.CollectionID, r.PartitionID, r.ChannelName, r.Count, err.Error())
resp.SegIDAssignments = append(resp.SegIDAssignments, result)
continue
}
}
result.Status.ErrorCode = commonpb.ErrorCode_SUCCESS
result.CollectionID = r.CollectionID
result.SegID = segmentID
result.PartitionID = r.PartitionID
result.Count = uint32(retCount)
result.ExpireTime = expireTs
result.ChannelName = r.ChannelName
resp.SegIDAssignments = append(resp.SegIDAssignments, result)
}
return resp, nil
}
func (s *Server) openNewSegment(collectionID UniqueID, partitionID UniqueID, channelName string) error {
group, err := s.insertChannelMgr.GetChannelGroup(collectionID, channelName)
if err != nil {
return err
}
segmentInfo, err := s.meta.BuildSegment(collectionID, partitionID, group)
if err != nil {
return err
}
if err = s.meta.AddSegment(segmentInfo); err != nil {
return err
}
if err = s.segAllocator.OpenSegment(collectionID, partitionID, segmentInfo.SegmentID, segmentInfo.InsertChannels); err != nil {
return err
}
return nil
}
func (s *Server) ShowSegments(req *datapb.ShowSegmentRequest) (*datapb.ShowSegmentResponse, error) {
ids := s.meta.GetSegmentsByCollectionAndPartitionID(req.CollectionID, req.PartitionID)
return &datapb.ShowSegmentResponse{SegmentIDs: ids}, nil
}
func (s *Server) GetSegmentStates(req *datapb.SegmentStatesRequest) (*datapb.SegmentStatesResponse, error) {
resp := &datapb.SegmentStatesResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
},
}
segmentInfo, err := s.meta.GetSegment(req.SegmentID)
if err != nil {
resp.Status.Reason = "get segment states error: " + err.Error()
return resp, nil
}
resp.State = segmentInfo.State
resp.CreateTime = segmentInfo.OpenTime
resp.SealedTime = segmentInfo.SealedTime
resp.FlushedTime = segmentInfo.FlushedTime
// TODO start/end positions
return resp, nil
}
func (s *Server) GetInsertBinlogPaths(req *datapb.InsertBinlogPathRequest) (*datapb.InsertBinlogPathsResponse, error) {
panic("implement me")
}
func (s *Server) GetInsertChannels(req *datapb.InsertChannelRequest) (*internalpb2.StringList, error) {
resp := &internalpb2.StringList{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
},
}
contains, ret := s.insertChannelMgr.ContainsCollection(req.CollectionID)
if contains {
resp.Values = ret
return resp, nil
}
channelGroups, err := s.insertChannelMgr.AllocChannels(req.CollectionID, len(s.nodes))
if err != nil {
resp.Status.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR
resp.Status.Reason = err.Error()
return resp, nil
}
channels := make([]string, Params.InsertChannelNumPerCollection)
for _, group := range channelGroups {
for _, c := range group {
channels = append(channels, c)
}
}
// todo datanode watch dm channels
resp.Values = channels
return resp, nil
}
func (s *Server) GetCollectionStatistics(req *datapb.CollectionStatsRequest) (*datapb.CollectionStatsResponse, error) {
// todo implement
return nil, nil
}
func (s *Server) GetPartitionStatistics(req *datapb.PartitionStatsRequest) (*datapb.PartitionStatsResponse, error) {
// todo implement
return nil, nil
}

View File

@ -1,126 +0,0 @@
package dataservice
import (
"context"
"fmt"
"log"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/errors"
)
// TODO: get timestamp from timestampOracle
type task interface {
Type() commonpb.MsgType
Ts() (Timestamp, error)
Execute() error
WaitToFinish(ctx context.Context) error
Notify(err error)
}
type baseTask struct {
sch *ddRequestScheduler
meta *meta
cv chan error
}
func (bt *baseTask) Notify(err error) {
bt.cv <- err
}
func (bt *baseTask) WaitToFinish(ctx context.Context) error {
for {
select {
case <-ctx.Done():
return errors.Errorf("context done")
case err, ok := <-bt.cv:
if !ok {
return errors.Errorf("notify chan closed")
}
return err
}
}
}
type allocateTask struct {
baseTask
req *datapb.AssignSegIDRequest
resp *datapb.AssignSegIDResponse
segAllocator segmentAllocator
insertCMapper insertChannelMapper
}
func (task *allocateTask) Type() commonpb.MsgType {
return commonpb.MsgType_kAllocateSegment
}
func (task *allocateTask) Ts() (Timestamp, error) {
return task.req.Timestamp, nil
}
func (task *allocateTask) Execute() error {
for _, req := range task.req.SegIDRequests {
result := &datapb.SegIDAssignment{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
},
}
segmentID, retCount, expireTs, err := task.segAllocator.AllocSegment(req.CollectionID, req.PartitionID, req.ChannelName, int(req.Count))
if err != nil {
if _, ok := err.(errRemainInSufficient); !ok {
result.Status.Reason = fmt.Sprintf("allocation of Collection %d, Partition %d, Channel %s, Count %d error: %s",
req.CollectionID, req.PartitionID, req.ChannelName, req.Count, err.Error())
task.resp.SegIDAssignments = append(task.resp.SegIDAssignments, result)
continue
}
log.Printf("no enough space for allocation of Collection %d, Partition %d, Channel %s, Count %d",
req.CollectionID, req.PartitionID, req.ChannelName, req.Count)
if err = task.openNewSegment(req.CollectionID, req.PartitionID, req.ChannelName); err != nil {
result.Status.Reason = fmt.Sprintf("open new segment of Collection %d, Partition %d, Channel %s, Count %d error: %s",
req.CollectionID, req.PartitionID, req.ChannelName, req.Count, err.Error())
task.resp.SegIDAssignments = append(task.resp.SegIDAssignments, result)
continue
}
segmentID, retCount, expireTs, err = task.segAllocator.AllocSegment(req.CollectionID, req.PartitionID, req.ChannelName, int(req.Count))
if err != nil {
result.Status.Reason = fmt.Sprintf("retry allocation of Collection %d, Partition %d, Channel %s, Count %d error: %s",
req.CollectionID, req.PartitionID, req.ChannelName, req.Count, err.Error())
task.resp.SegIDAssignments = append(task.resp.SegIDAssignments, result)
continue
}
}
result.Status.ErrorCode = commonpb.ErrorCode_SUCCESS
result.CollectionID = req.CollectionID
result.SegID = segmentID
result.PartitionID = req.PartitionID
result.Count = uint32(retCount)
result.ExpireTime = expireTs
result.ChannelName = req.ChannelName
task.resp.SegIDAssignments = append(task.resp.SegIDAssignments, result)
}
return nil
}
func (task *allocateTask) openNewSegment(collectionID UniqueID, partitionID UniqueID, channelName string) error {
cRange, err := task.insertCMapper.GetChannelRange(channelName)
if err != nil {
return err
}
segmentInfo, err := task.meta.BuildSegment(collectionID, partitionID, cRange)
if err != nil {
return err
}
if err = task.meta.AddSegment(segmentInfo); err != nil {
return err
}
if err = task.segAllocator.OpenSegment(collectionID, partitionID, segmentInfo.SegmentID, segmentInfo.InsertChannels); err != nil {
return err
}
return nil
}

View File

@ -10,45 +10,27 @@ import (
type (
proxyTimeTickWatcher struct {
allocator segmentAllocator
msgQueue chan *msgstream.TimeTickMsg
ctx context.Context
cancelFunc context.CancelFunc
allocator segmentAllocator
msgQueue chan *msgstream.TimeTickMsg
}
dataNodeTimeTickWatcher struct {
allocator segmentAllocator
msgQueue chan *msgstream.TimeTickMsg
ctx context.Context
cancelFunc context.CancelFunc
allocator segmentAllocator
msgQueue chan *msgstream.TimeTickMsg
}
)
func newProxyTimeTickWatcher(ctx context.Context, allocator segmentAllocator) *proxyTimeTickWatcher {
cancel, cancelFunc := context.WithCancel(ctx)
func newProxyTimeTickWatcher(allocator segmentAllocator) *proxyTimeTickWatcher {
return &proxyTimeTickWatcher{
allocator: allocator,
msgQueue: make(chan *msgstream.TimeTickMsg, 1),
ctx: cancel,
cancelFunc: cancelFunc,
allocator: allocator,
msgQueue: make(chan *msgstream.TimeTickMsg, 1),
}
}
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() {
func (watcher *proxyTimeTickWatcher) StartBackgroundLoop(ctx context.Context) {
for {
select {
case <-watcher.ctx.Done():
case <-ctx.Done():
log.Println("proxy time tick watcher clsoed")
return
case msg := <-watcher.msgQueue:
if err := watcher.allocator.ExpireAllocations(msg.Base.Timestamp); err != nil {
@ -58,13 +40,14 @@ func (watcher *proxyTimeTickWatcher) handleProxyTimeTickMsg() {
}
}
func newDataNodeTimeTickWatcher(ctx context.Context, allocator segmentAllocator) *dataNodeTimeTickWatcher {
cancel, cancelFunc := context.WithCancel(ctx)
func (watcher *proxyTimeTickWatcher) Watch(msg *msgstream.TimeTickMsg) {
watcher.msgQueue <- msg
}
func newDataNodeTimeTickWatcher(allocator segmentAllocator) *dataNodeTimeTickWatcher {
return &dataNodeTimeTickWatcher{
allocator: allocator,
msgQueue: make(chan *msgstream.TimeTickMsg, 1),
ctx: cancel,
cancelFunc: cancelFunc,
allocator: allocator,
msgQueue: make(chan *msgstream.TimeTickMsg, 1),
}
}
@ -72,18 +55,11 @@ 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() {
func (watcher *dataNodeTimeTickWatcher) StartBackgroundLoop(ctx context.Context) {
for {
select {
case <-watcher.ctx.Done():
case <-ctx.Done():
log.Println("data node time tick watcher clsoed")
return
case msg := <-watcher.msgQueue:
segments, err := watcher.allocator.GetSealedSegments()

View File

@ -0,0 +1,64 @@
package dataservice
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
)
type Client struct {
grpcClient datapb.DataServiceClient
}
func (c *Client) GetComponentStates() (*internalpb2.ComponentStates, error) {
return c.grpcClient.GetComponentStates(context.Background(), nil)
}
func (c *Client) GetTimeTickChannel() (*milvuspb.StringResponse, error) {
return c.grpcClient.GetTimeTickChannel(context.Background(), nil)
}
func (c *Client) GetStatisticsChannel() (*milvuspb.StringResponse, error) {
return c.grpcClient.GetStatisticsChannel(context.Background(), nil)
}
func (c *Client) RegisterNode(req *datapb.RegisterNodeRequest) (*datapb.RegisterNodeResponse, error) {
return c.grpcClient.RegisterNode(context.Background(), req)
}
func (c *Client) Flush(req *datapb.FlushRequest) (*commonpb.Status, error) {
return c.grpcClient.Flush(context.Background(), req)
}
func (c *Client) AssignSegmentID(req *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error) {
return c.grpcClient.AssignSegmentID(context.Background(), req)
}
func (c *Client) ShowSegments(req *datapb.ShowSegmentRequest) (*datapb.ShowSegmentResponse, error) {
return c.grpcClient.ShowSegments(context.Background(), req)
}
func (c *Client) GetSegmentStates(req *datapb.SegmentStatesRequest) (*datapb.SegmentStatesResponse, error) {
return c.grpcClient.GetSegmentStates(context.Background(), req)
}
func (c *Client) GetInsertBinlogPaths(req *datapb.InsertBinlogPathRequest) (*datapb.InsertBinlogPathsResponse, error) {
return c.grpcClient.GetInsertBinlogPaths(context.Background(), req)
}
func (c *Client) GetInsertChannels(req *datapb.InsertChannelRequest) (*internalpb2.StringList, error) {
return c.grpcClient.GetInsertChannels(context.Background(), req)
}
func (c *Client) GetCollectionStatistics(req *datapb.CollectionStatsRequest) (*datapb.CollectionStatsResponse, error) {
return c.grpcClient.GetCollectionStatistics(context.Background(), req)
}
func (c *Client) GetPartitionStatistics(req *datapb.PartitionStatsRequest) (*datapb.PartitionStatsResponse, error) {
return c.grpcClient.GetPartitionStatistics(context.Background(), req)
}

View File

@ -0,0 +1,65 @@
package dataservice
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/dataservice"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
)
type Service struct {
server *dataservice.Server
}
func (s *Service) RegisterNode(ctx context.Context, request *datapb.RegisterNodeRequest) (*datapb.RegisterNodeResponse, error) {
return s.server.RegisterNode(request)
}
func (s *Service) Flush(ctx context.Context, request *datapb.FlushRequest) (*commonpb.Status, error) {
return s.server.Flush(request)
}
func (s *Service) AssignSegmentID(ctx context.Context, request *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error) {
return s.server.AssignSegmentID(request)
}
func (s *Service) ShowSegments(ctx context.Context, request *datapb.ShowSegmentRequest) (*datapb.ShowSegmentResponse, error) {
return s.server.ShowSegments(request)
}
func (s *Service) GetSegmentStates(ctx context.Context, request *datapb.SegmentStatesRequest) (*datapb.SegmentStatesResponse, error) {
return s.server.GetSegmentStates(request)
}
func (s *Service) GetInsertBinlogPaths(ctx context.Context, request *datapb.InsertBinlogPathRequest) (*datapb.InsertBinlogPathsResponse, error) {
return s.server.GetInsertBinlogPaths(request)
}
func (s *Service) GetInsertChannels(ctx context.Context, request *datapb.InsertChannelRequest) (*internalpb2.StringList, error) {
return s.server.GetInsertChannels(request)
}
func (s *Service) GetCollectionStatistics(ctx context.Context, request *datapb.CollectionStatsRequest) (*datapb.CollectionStatsResponse, error) {
return s.server.GetCollectionStatistics(request)
}
func (s *Service) GetPartitionStatistics(ctx context.Context, request *datapb.PartitionStatsRequest) (*datapb.PartitionStatsResponse, error) {
return s.server.GetPartitionStatistics(request)
}
func (s *Service) GetComponentStates(ctx context.Context, empty *commonpb.Empty) (*internalpb2.ComponentStates, error) {
return s.server.GetComponentStates()
}
func (s *Service) GetTimeTickChannel(ctx context.Context, empty *commonpb.Empty) (*milvuspb.StringResponse, error) {
return s.server.GetTimeTickChannel()
}
func (s *Service) GetStatisticsChannel(ctx context.Context, empty *commonpb.Empty) (*milvuspb.StringResponse, error) {
return s.server.GetStatisticsChannel()
}

View File

@ -6,6 +6,7 @@ option go_package = "github.com/zilliztech/milvus-distributed/internal/proto/dat
import "common.proto";
import "internal.proto";
import "milvus.proto";
message RegisterNodeRequest {
@ -15,6 +16,7 @@ message RegisterNodeRequest {
message RegisterNodeResponse {
internal.InitParams init_params = 1;
common.Status status = 2;
}
@ -30,8 +32,7 @@ message SegIDRequest {
message AssignSegIDRequest {
int64 nodeID = 1;
string peer_role = 2;
uint64 timestamp = 3;
repeated SegIDRequest segIDRequests = 4;
repeated SegIDRequest segIDRequests = 3;
}
@ -66,6 +67,7 @@ message ShowSegmentRequest {
message ShowSegmentResponse {
repeated int64 segmentIDs = 1;
common.Status status = 2;
}
enum SegmentState {
@ -73,6 +75,7 @@ enum SegmentState {
SegmentNotExist = 1;
SegmentGrowing = 2;
SegmentSealed = 3;
SegmentFlushed = 4;
}
@ -85,8 +88,10 @@ message SegmentStatesResponse {
SegmentState state = 1;
uint64 create_time = 2;
uint64 sealed_time = 3;
repeated internal.MsgPosition start_positions = 4;
repeated internal.MsgPosition end_positions = 5;
uint64 flushed_time = 4;
repeated internal.MsgPosition start_positions = 5;
repeated internal.MsgPosition end_positions = 6;
common.Status status = 7;
}
message InsertBinlogPathRequest {
@ -97,6 +102,7 @@ message InsertBinlogPathRequest {
message InsertBinlogPathsResponse {
repeated int64 fieldIDs = 1;
repeated internal.StringList paths = 2;
common.Status status = 3;
}
@ -126,9 +132,11 @@ message SegmentInfo {
int64 partitionID=3;
repeated string insert_channels = 4;
uint64 open_time=5;
uint64 close_time=6;
int64 num_rows=7;
int64 mem_size=8;
uint64 sealed_time = 6;
uint64 flushed_time = 7;
int64 num_rows=8;
int64 mem_size=9;
SegmentState state = 10;
}
message SegmentMsg{
@ -136,6 +144,29 @@ message SegmentMsg{
SegmentInfo segment = 2;
}
message CollectionStatsRequest{
common.MsgBase base = 1;
string db_name = 2;
string collection_name = 3;
}
message CollectionStatsResponse {
repeated common.KeyValuePair stats = 1;
common.Status status = 2;
}
message PartitionStatsRequest{
common.MsgBase base = 1;
string db_name = 2;
string collection_name = 3;
string partition_name = 4;
}
message PartitionStatsResponse {
repeated common.KeyValuePair stats = 1;
common.Status status = 2;
}
message FieldFlushMeta {
int64 fieldID = 1;
repeated string binlog_paths = 2;
@ -164,5 +195,10 @@ service DataService {
rpc GetInsertBinlogPaths(InsertBinlogPathRequest) returns (InsertBinlogPathsResponse) {}
rpc GetInsertChannels(InsertChannelRequest) returns (internal.StringList) {}
rpc GetCollectionStatistics(CollectionStatsRequest) returns (CollectionStatsResponse) {}
rpc GetPartitionStatistics(PartitionStatsRequest) returns (PartitionStatsResponse) {}
rpc GetComponentStates(common.Empty) returns (internal.ComponentStates) {}
rpc GetTimeTickChannel(common.Empty) returns(milvus.StringResponse) {}
rpc GetStatisticsChannel(common.Empty) returns(milvus.StringResponse){}
}

View File

@ -9,6 +9,7 @@ import (
proto "github.com/golang/protobuf/proto"
commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
internalpb2 "github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
milvuspb "github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
grpc "google.golang.org/grpc"
codes "google.golang.org/grpc/codes"
status "google.golang.org/grpc/status"
@ -33,6 +34,7 @@ const (
SegmentState_SegmentNotExist SegmentState = 1
SegmentState_SegmentGrowing SegmentState = 2
SegmentState_SegmentSealed SegmentState = 3
SegmentState_SegmentFlushed SegmentState = 4
)
var SegmentState_name = map[int32]string{
@ -40,6 +42,7 @@ var SegmentState_name = map[int32]string{
1: "SegmentNotExist",
2: "SegmentGrowing",
3: "SegmentSealed",
4: "SegmentFlushed",
}
var SegmentState_value = map[string]int32{
@ -47,6 +50,7 @@ var SegmentState_value = map[string]int32{
"SegmentNotExist": 1,
"SegmentGrowing": 2,
"SegmentSealed": 3,
"SegmentFlushed": 4,
}
func (x SegmentState) String() string {
@ -106,6 +110,7 @@ func (m *RegisterNodeRequest) GetAddress() *commonpb.Address {
type RegisterNodeResponse struct {
InitParams *internalpb2.InitParams `protobuf:"bytes,1,opt,name=init_params,json=initParams,proto3" json:"init_params,omitempty"`
Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -143,6 +148,13 @@ func (m *RegisterNodeResponse) GetInitParams() *internalpb2.InitParams {
return nil
}
func (m *RegisterNodeResponse) GetStatus() *commonpb.Status {
if m != nil {
return m.Status
}
return nil
}
type SegIDRequest struct {
Count uint32 `protobuf:"varint,1,opt,name=count,proto3" json:"count,omitempty"`
ChannelName string `protobuf:"bytes,2,opt,name=channelName,proto3" json:"channelName,omitempty"`
@ -225,8 +237,7 @@ func (m *SegIDRequest) GetPartitionName() string {
type AssignSegIDRequest struct {
NodeID int64 `protobuf:"varint,1,opt,name=nodeID,proto3" json:"nodeID,omitempty"`
PeerRole string `protobuf:"bytes,2,opt,name=peer_role,json=peerRole,proto3" json:"peer_role,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
SegIDRequests []*SegIDRequest `protobuf:"bytes,4,rep,name=segIDRequests,proto3" json:"segIDRequests,omitempty"`
SegIDRequests []*SegIDRequest `protobuf:"bytes,3,rep,name=segIDRequests,proto3" json:"segIDRequests,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -271,13 +282,6 @@ func (m *AssignSegIDRequest) GetPeerRole() string {
return ""
}
func (m *AssignSegIDRequest) GetTimestamp() uint64 {
if m != nil {
return m.Timestamp
}
return 0
}
func (m *AssignSegIDRequest) GetSegIDRequests() []*SegIDRequest {
if m != nil {
return m.SegIDRequests
@ -546,10 +550,11 @@ func (m *ShowSegmentRequest) GetPartitionID() int64 {
}
type ShowSegmentResponse struct {
SegmentIDs []int64 `protobuf:"varint,1,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
SegmentIDs []int64 `protobuf:"varint,1,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,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:"-"`
}
func (m *ShowSegmentResponse) Reset() { *m = ShowSegmentResponse{} }
@ -584,6 +589,13 @@ func (m *ShowSegmentResponse) GetSegmentIDs() []int64 {
return nil
}
func (m *ShowSegmentResponse) GetStatus() *commonpb.Status {
if m != nil {
return m.Status
}
return nil
}
type SegmentStatesRequest struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
@ -635,8 +647,10 @@ type SegmentStatesResponse struct {
State SegmentState `protobuf:"varint,1,opt,name=state,proto3,enum=milvus.proto.data.SegmentState" json:"state,omitempty"`
CreateTime uint64 `protobuf:"varint,2,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"`
SealedTime uint64 `protobuf:"varint,3,opt,name=sealed_time,json=sealedTime,proto3" json:"sealed_time,omitempty"`
StartPositions []*internalpb2.MsgPosition `protobuf:"bytes,4,rep,name=start_positions,json=startPositions,proto3" json:"start_positions,omitempty"`
EndPositions []*internalpb2.MsgPosition `protobuf:"bytes,5,rep,name=end_positions,json=endPositions,proto3" json:"end_positions,omitempty"`
FlushedTime uint64 `protobuf:"varint,4,opt,name=flushed_time,json=flushedTime,proto3" json:"flushed_time,omitempty"`
StartPositions []*internalpb2.MsgPosition `protobuf:"bytes,5,rep,name=start_positions,json=startPositions,proto3" json:"start_positions,omitempty"`
EndPositions []*internalpb2.MsgPosition `protobuf:"bytes,6,rep,name=end_positions,json=endPositions,proto3" json:"end_positions,omitempty"`
Status *commonpb.Status `protobuf:"bytes,7,opt,name=status,proto3" json:"status,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -688,6 +702,13 @@ func (m *SegmentStatesResponse) GetSealedTime() uint64 {
return 0
}
func (m *SegmentStatesResponse) GetFlushedTime() uint64 {
if m != nil {
return m.FlushedTime
}
return 0
}
func (m *SegmentStatesResponse) GetStartPositions() []*internalpb2.MsgPosition {
if m != nil {
return m.StartPositions
@ -702,6 +723,13 @@ func (m *SegmentStatesResponse) GetEndPositions() []*internalpb2.MsgPosition {
return nil
}
func (m *SegmentStatesResponse) GetStatus() *commonpb.Status {
if m != nil {
return m.Status
}
return nil
}
type InsertBinlogPathRequest struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
@ -752,6 +780,7 @@ func (m *InsertBinlogPathRequest) GetSegmentID() int64 {
type InsertBinlogPathsResponse struct {
FieldIDs []int64 `protobuf:"varint,1,rep,packed,name=fieldIDs,proto3" json:"fieldIDs,omitempty"`
Paths []*internalpb2.StringList `protobuf:"bytes,2,rep,name=paths,proto3" json:"paths,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:"-"`
@ -796,6 +825,13 @@ func (m *InsertBinlogPathsResponse) GetPaths() []*internalpb2.StringList {
return nil
}
func (m *InsertBinlogPathsResponse) GetStatus() *commonpb.Status {
if m != nil {
return m.Status
}
return nil
}
type InsertChannelRequest struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"`
@ -962,17 +998,19 @@ func (m *FlushSegRequest) GetSegmentIDs() []int64 {
}
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:"-"`
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"`
SealedTime uint64 `protobuf:"varint,6,opt,name=sealed_time,json=sealedTime,proto3" json:"sealed_time,omitempty"`
FlushedTime uint64 `protobuf:"varint,7,opt,name=flushed_time,json=flushedTime,proto3" json:"flushed_time,omitempty"`
NumRows int64 `protobuf:"varint,8,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"`
MemSize int64 `protobuf:"varint,9,opt,name=mem_size,json=memSize,proto3" json:"mem_size,omitempty"`
State SegmentState `protobuf:"varint,10,opt,name=state,proto3,enum=milvus.proto.data.SegmentState" json:"state,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *SegmentInfo) Reset() { *m = SegmentInfo{} }
@ -1035,9 +1073,16 @@ func (m *SegmentInfo) GetOpenTime() uint64 {
return 0
}
func (m *SegmentInfo) GetCloseTime() uint64 {
func (m *SegmentInfo) GetSealedTime() uint64 {
if m != nil {
return m.CloseTime
return m.SealedTime
}
return 0
}
func (m *SegmentInfo) GetFlushedTime() uint64 {
if m != nil {
return m.FlushedTime
}
return 0
}
@ -1056,6 +1101,13 @@ func (m *SegmentInfo) GetMemSize() int64 {
return 0
}
func (m *SegmentInfo) GetState() SegmentState {
if m != nil {
return m.State
}
return SegmentState_SegmentNone
}
type SegmentMsg struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
Segment *SegmentInfo `protobuf:"bytes,2,opt,name=segment,proto3" json:"segment,omitempty"`
@ -1103,6 +1155,218 @@ func (m *SegmentMsg) GetSegment() *SegmentInfo {
return nil
}
type CollectionStatsRequest struct {
Base *commonpb.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"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
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_3385cd32ad6cfe64, []int{18}
}
func (m *CollectionStatsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CollectionStatsRequest.Unmarshal(m, b)
}
func (m *CollectionStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_CollectionStatsRequest.Marshal(b, m, deterministic)
}
func (m *CollectionStatsRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_CollectionStatsRequest.Merge(m, src)
}
func (m *CollectionStatsRequest) XXX_Size() int {
return xxx_messageInfo_CollectionStatsRequest.Size(m)
}
func (m *CollectionStatsRequest) XXX_DiscardUnknown() {
xxx_messageInfo_CollectionStatsRequest.DiscardUnknown(m)
}
var xxx_messageInfo_CollectionStatsRequest proto.InternalMessageInfo
func (m *CollectionStatsRequest) GetBase() *commonpb.MsgBase {
if m != nil {
return m.Base
}
return nil
}
func (m *CollectionStatsRequest) GetDbName() string {
if m != nil {
return m.DbName
}
return ""
}
func (m *CollectionStatsRequest) GetCollectionName() string {
if m != nil {
return m.CollectionName
}
return ""
}
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:"-"`
}
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_3385cd32ad6cfe64, []int{19}
}
func (m *CollectionStatsResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CollectionStatsResponse.Unmarshal(m, b)
}
func (m *CollectionStatsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_CollectionStatsResponse.Marshal(b, m, deterministic)
}
func (m *CollectionStatsResponse) XXX_Merge(src proto.Message) {
xxx_messageInfo_CollectionStatsResponse.Merge(m, src)
}
func (m *CollectionStatsResponse) XXX_Size() int {
return xxx_messageInfo_CollectionStatsResponse.Size(m)
}
func (m *CollectionStatsResponse) XXX_DiscardUnknown() {
xxx_messageInfo_CollectionStatsResponse.DiscardUnknown(m)
}
var xxx_messageInfo_CollectionStatsResponse proto.InternalMessageInfo
func (m *CollectionStatsResponse) GetStats() []*commonpb.KeyValuePair {
if m != nil {
return m.Stats
}
return nil
}
func (m *CollectionStatsResponse) GetStatus() *commonpb.Status {
if m != nil {
return m.Status
}
return nil
}
type PartitionStatsRequest struct {
Base *commonpb.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"`
PartitionName string `protobuf:"bytes,4,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
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_3385cd32ad6cfe64, []int{20}
}
func (m *PartitionStatsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PartitionStatsRequest.Unmarshal(m, b)
}
func (m *PartitionStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_PartitionStatsRequest.Marshal(b, m, deterministic)
}
func (m *PartitionStatsRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_PartitionStatsRequest.Merge(m, src)
}
func (m *PartitionStatsRequest) XXX_Size() int {
return xxx_messageInfo_PartitionStatsRequest.Size(m)
}
func (m *PartitionStatsRequest) XXX_DiscardUnknown() {
xxx_messageInfo_PartitionStatsRequest.DiscardUnknown(m)
}
var xxx_messageInfo_PartitionStatsRequest proto.InternalMessageInfo
func (m *PartitionStatsRequest) GetBase() *commonpb.MsgBase {
if m != nil {
return m.Base
}
return nil
}
func (m *PartitionStatsRequest) GetDbName() string {
if m != nil {
return m.DbName
}
return ""
}
func (m *PartitionStatsRequest) GetCollectionName() string {
if m != nil {
return m.CollectionName
}
return ""
}
func (m *PartitionStatsRequest) GetPartitionName() string {
if m != nil {
return m.PartitionName
}
return ""
}
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:"-"`
}
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_3385cd32ad6cfe64, []int{21}
}
func (m *PartitionStatsResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PartitionStatsResponse.Unmarshal(m, b)
}
func (m *PartitionStatsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_PartitionStatsResponse.Marshal(b, m, deterministic)
}
func (m *PartitionStatsResponse) XXX_Merge(src proto.Message) {
xxx_messageInfo_PartitionStatsResponse.Merge(m, src)
}
func (m *PartitionStatsResponse) XXX_Size() int {
return xxx_messageInfo_PartitionStatsResponse.Size(m)
}
func (m *PartitionStatsResponse) XXX_DiscardUnknown() {
xxx_messageInfo_PartitionStatsResponse.DiscardUnknown(m)
}
var xxx_messageInfo_PartitionStatsResponse proto.InternalMessageInfo
func (m *PartitionStatsResponse) GetStats() []*commonpb.KeyValuePair {
if m != nil {
return m.Stats
}
return nil
}
func (m *PartitionStatsResponse) GetStatus() *commonpb.Status {
if m != nil {
return m.Status
}
return nil
}
type FieldFlushMeta struct {
FieldID int64 `protobuf:"varint,1,opt,name=fieldID,proto3" json:"fieldID,omitempty"`
BinlogPaths []string `protobuf:"bytes,2,rep,name=binlog_paths,json=binlogPaths,proto3" json:"binlog_paths,omitempty"`
@ -1115,7 +1379,7 @@ func (m *FieldFlushMeta) Reset() { *m = FieldFlushMeta{} }
func (m *FieldFlushMeta) String() string { return proto.CompactTextString(m) }
func (*FieldFlushMeta) ProtoMessage() {}
func (*FieldFlushMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{18}
return fileDescriptor_3385cd32ad6cfe64, []int{22}
}
func (m *FieldFlushMeta) XXX_Unmarshal(b []byte) error {
@ -1163,7 +1427,7 @@ func (m *SegmentFlushMeta) Reset() { *m = SegmentFlushMeta{} }
func (m *SegmentFlushMeta) String() string { return proto.CompactTextString(m) }
func (*SegmentFlushMeta) ProtoMessage() {}
func (*SegmentFlushMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{19}
return fileDescriptor_3385cd32ad6cfe64, []int{23}
}
func (m *SegmentFlushMeta) XXX_Unmarshal(b []byte) error {
@ -1217,7 +1481,7 @@ func (m *DDLFlushMeta) Reset() { *m = DDLFlushMeta{} }
func (m *DDLFlushMeta) String() string { return proto.CompactTextString(m) }
func (*DDLFlushMeta) ProtoMessage() {}
func (*DDLFlushMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{20}
return fileDescriptor_3385cd32ad6cfe64, []int{24}
}
func (m *DDLFlushMeta) XXX_Unmarshal(b []byte) error {
@ -1272,6 +1536,10 @@ func init() {
proto.RegisterType((*FlushSegRequest)(nil), "milvus.proto.data.FlushSegRequest")
proto.RegisterType((*SegmentInfo)(nil), "milvus.proto.data.SegmentInfo")
proto.RegisterType((*SegmentMsg)(nil), "milvus.proto.data.SegmentMsg")
proto.RegisterType((*CollectionStatsRequest)(nil), "milvus.proto.data.CollectionStatsRequest")
proto.RegisterType((*CollectionStatsResponse)(nil), "milvus.proto.data.CollectionStatsResponse")
proto.RegisterType((*PartitionStatsRequest)(nil), "milvus.proto.data.PartitionStatsRequest")
proto.RegisterType((*PartitionStatsResponse)(nil), "milvus.proto.data.PartitionStatsResponse")
proto.RegisterType((*FieldFlushMeta)(nil), "milvus.proto.data.FieldFlushMeta")
proto.RegisterType((*SegmentFlushMeta)(nil), "milvus.proto.data.SegmentFlushMeta")
proto.RegisterType((*DDLFlushMeta)(nil), "milvus.proto.data.DDLFlushMeta")
@ -1280,87 +1548,101 @@ func init() {
func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad6cfe64) }
var fileDescriptor_3385cd32ad6cfe64 = []byte{
// 1267 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x57, 0xdb, 0x6e, 0x1b, 0x45,
0x18, 0xce, 0xfa, 0x90, 0xd8, 0xbf, 0x1d, 0xdb, 0x9d, 0xa4, 0xe0, 0xba, 0xa7, 0x74, 0xa4, 0x36,
0x51, 0x05, 0x09, 0x6a, 0x55, 0x0e, 0x37, 0x88, 0x06, 0xb7, 0x91, 0x45, 0x13, 0x55, 0x63, 0x10,
0x52, 0xb9, 0xb0, 0xd6, 0xde, 0xc9, 0x7a, 0xa4, 0xdd, 0xd9, 0x65, 0x67, 0xdc, 0x44, 0xb9, 0x81,
0x6b, 0xd4, 0x27, 0x80, 0x7b, 0x2e, 0x79, 0x04, 0x78, 0x05, 0x1e, 0x09, 0xed, 0xcc, 0x78, 0xbd,
0x6b, 0xaf, 0xe3, 0x28, 0x50, 0xee, 0x3c, 0xbf, 0xbf, 0xf9, 0x0f, 0xf3, 0xfd, 0xa7, 0x05, 0xe4,
0xd8, 0xd2, 0x1e, 0x08, 0x1a, 0xbd, 0x65, 0x23, 0xba, 0x1f, 0x46, 0x81, 0x0c, 0xd0, 0x0d, 0x9f,
0x79, 0x6f, 0x27, 0x42, 0x9f, 0xf6, 0x63, 0x40, 0xa7, 0x3e, 0x0a, 0x7c, 0x3f, 0xe0, 0x5a, 0xd4,
0x69, 0x30, 0x2e, 0x69, 0xc4, 0x6d, 0x4f, 0x9f, 0xf1, 0x4f, 0xb0, 0x45, 0xa8, 0xcb, 0x84, 0xa4,
0xd1, 0x49, 0xe0, 0x50, 0x42, 0x7f, 0x9c, 0x50, 0x21, 0xd1, 0x27, 0x50, 0x1a, 0xda, 0x82, 0xb6,
0xad, 0x1d, 0x6b, 0xaf, 0xf6, 0xe4, 0xce, 0x7e, 0x46, 0xad, 0x51, 0x78, 0x2c, 0xdc, 0x43, 0x5b,
0x50, 0xa2, 0x90, 0xe8, 0x53, 0xd8, 0xb0, 0x1d, 0x27, 0xa2, 0x42, 0xb4, 0x0b, 0x97, 0x5c, 0x7a,
0xae, 0x31, 0x64, 0x0a, 0xc6, 0x6f, 0x60, 0x3b, 0xeb, 0x80, 0x08, 0x03, 0x2e, 0x28, 0x3a, 0x84,
0x1a, 0xe3, 0x4c, 0x0e, 0x42, 0x3b, 0xb2, 0x7d, 0x61, 0x1c, 0x79, 0x90, 0xd5, 0x99, 0xc4, 0xd2,
0xe3, 0x4c, 0xbe, 0x56, 0x40, 0x02, 0x2c, 0xf9, 0x8d, 0xff, 0xb6, 0xa0, 0xde, 0xa7, 0x6e, 0xaf,
0x3b, 0x0d, 0x6b, 0x1b, 0xca, 0xa3, 0x60, 0xc2, 0xa5, 0x52, 0xb7, 0x49, 0xf4, 0x01, 0xed, 0x40,
0x6d, 0x34, 0xb6, 0x39, 0xa7, 0xde, 0x89, 0xed, 0x53, 0xe5, 0x7e, 0x95, 0xa4, 0x45, 0x08, 0x43,
0x7d, 0x14, 0x78, 0x1e, 0x1d, 0x49, 0x16, 0xf0, 0x5e, 0xb7, 0x5d, 0xdc, 0xb1, 0xf6, 0x8a, 0x24,
0x23, 0x8b, 0xb5, 0x84, 0x76, 0x24, 0x99, 0x81, 0x94, 0x14, 0x24, 0x2d, 0x42, 0xb7, 0xa1, 0x1a,
0xdf, 0x18, 0xf0, 0xd8, 0x4a, 0x59, 0x59, 0xa9, 0xc4, 0x02, 0x65, 0xe2, 0x21, 0x34, 0x12, 0xac,
0x46, 0xac, 0x2b, 0xc4, 0x66, 0x22, 0x8d, 0x61, 0xf8, 0x0f, 0x0b, 0xd0, 0x73, 0x21, 0x98, 0xcb,
0x33, 0x81, 0x7d, 0x00, 0xeb, 0x3c, 0x70, 0x68, 0xaf, 0xab, 0x22, 0x2b, 0x12, 0x73, 0x8a, 0x4d,
0x86, 0x94, 0x46, 0x83, 0x28, 0xf0, 0xa6, 0x81, 0x55, 0x62, 0x01, 0x09, 0x3c, 0x8a, 0xee, 0x40,
0x55, 0x32, 0x9f, 0x0a, 0x69, 0xfb, 0xa1, 0x0a, 0xa9, 0x44, 0x66, 0x02, 0xf4, 0x02, 0x36, 0x45,
0xca, 0x84, 0x68, 0x97, 0x76, 0x8a, 0x7b, 0xb5, 0x27, 0xf7, 0xf7, 0x17, 0x52, 0x6c, 0x3f, 0xed,
0x0a, 0xc9, 0xde, 0xc2, 0x7f, 0x15, 0xa0, 0xa9, 0xfe, 0xd7, 0x5e, 0xfb, 0x94, 0x2b, 0x1a, 0x14,
0xc8, 0x38, 0xab, 0x0f, 0x57, 0xa0, 0x21, 0xa1, 0xaf, 0x98, 0xa6, 0x6f, 0x9e, 0x9c, 0xd2, 0x6a,
0x72, 0xca, 0x8b, 0xe4, 0xdc, 0x87, 0x1a, 0x3d, 0x0f, 0x59, 0x44, 0x07, 0xf1, 0x13, 0xa8, 0xc7,
0x2f, 0x11, 0xd0, 0xa2, 0x6f, 0x99, 0x4f, 0xd1, 0x53, 0x58, 0x17, 0xd2, 0x96, 0x13, 0xd1, 0xde,
0x50, 0xb9, 0x78, 0x3b, 0x37, 0xbf, 0xfb, 0x0a, 0x42, 0x0c, 0x34, 0x4b, 0x79, 0x65, 0x25, 0xe5,
0xd5, 0x3c, 0xca, 0x7f, 0xb5, 0x60, 0x2b, 0x43, 0xb9, 0xa9, 0x90, 0x13, 0x68, 0x89, 0xec, 0xc3,
0xc6, 0x65, 0x12, 0x73, 0x84, 0x97, 0x71, 0x34, 0x83, 0x92, 0x85, 0xbb, 0xa9, 0x00, 0x0b, 0x57,
0x0e, 0x10, 0x9f, 0x43, 0xfd, 0xa5, 0x37, 0x11, 0xe3, 0xeb, 0x37, 0x0e, 0x04, 0x25, 0x67, 0xd8,
0xeb, 0x2a, 0xa3, 0x45, 0xa2, 0x7e, 0x5f, 0x85, 0x52, 0xfc, 0xce, 0x02, 0xd4, 0x1f, 0x07, 0x67,
0x7d, 0xea, 0xaa, 0x80, 0xae, 0xed, 0xc0, 0xbc, 0xb1, 0xc2, 0xea, 0xfc, 0x29, 0x2e, 0xe4, 0x0f,
0x7e, 0x06, 0x5b, 0x19, 0x6f, 0x0c, 0x49, 0xf7, 0x00, 0x84, 0x16, 0xf5, 0xba, 0x9a, 0x9e, 0x22,
0x49, 0x49, 0xf0, 0x29, 0x6c, 0x9b, 0x2b, 0xf1, 0xc3, 0x52, 0x71, 0xfd, 0x30, 0xee, 0x40, 0x35,
0xd1, 0x6b, 0x62, 0x98, 0x09, 0xf0, 0xef, 0x05, 0xb8, 0x39, 0x67, 0xc8, 0x78, 0xf8, 0x0c, 0xca,
0x31, 0x97, 0xda, 0x54, 0x63, 0x59, 0x7d, 0x27, 0x17, 0x89, 0x46, 0xc7, 0xf5, 0x32, 0x8a, 0xa8,
0x2d, 0x4d, 0xbd, 0x14, 0x74, 0xbd, 0x68, 0x91, 0xaa, 0x97, 0xfb, 0x50, 0x13, 0xd4, 0xf6, 0xa8,
0xa3, 0x01, 0xba, 0xbf, 0x80, 0x16, 0x29, 0xc0, 0x37, 0xd0, 0x14, 0xd2, 0x8e, 0xe4, 0x20, 0x0c,
0x84, 0x7a, 0xc5, 0x69, 0x8b, 0xc1, 0x4b, 0xba, 0xfc, 0xb1, 0x70, 0x5f, 0x1b, 0x28, 0x69, 0xa8,
0xab, 0xd3, 0xa3, 0x40, 0x47, 0xb0, 0x49, 0xb9, 0x93, 0x52, 0x55, 0xbe, 0xb2, 0xaa, 0x3a, 0xe5,
0x4e, 0xa2, 0x08, 0x33, 0xf8, 0xb0, 0xc7, 0x05, 0x8d, 0xe4, 0x21, 0xe3, 0x5e, 0xe0, 0xbe, 0xb6,
0xe5, 0xf8, 0x7d, 0x71, 0x12, 0xc2, 0xad, 0x79, 0x53, 0x33, 0x5a, 0x3a, 0x50, 0x39, 0x65, 0xd4,
0x73, 0x66, 0x69, 0x93, 0x9c, 0xd1, 0x67, 0x50, 0x0e, 0x63, 0x70, 0xbb, 0xa0, 0x82, 0x5c, 0x36,
0x15, 0xfb, 0x32, 0x62, 0xdc, 0x7d, 0xc5, 0x84, 0x24, 0x1a, 0x8f, 0x7f, 0xb6, 0x60, 0x5b, 0x9b,
0xfc, 0x5a, 0xb7, 0xd5, 0xf7, 0x5b, 0xb6, 0x39, 0x63, 0x12, 0xfb, 0x70, 0xf3, 0x7b, 0x5b, 0x8e,
0xc6, 0x5d, 0xff, 0x5f, 0xbb, 0x10, 0x9b, 0x9b, 0x4d, 0x07, 0xfd, 0x1a, 0x55, 0x92, 0x91, 0xe1,
0xdf, 0x2c, 0x68, 0xaa, 0x06, 0xd5, 0xa7, 0xee, 0xff, 0x1e, 0xec, 0x5c, 0xf5, 0x97, 0x16, 0xaa,
0xff, 0x5d, 0x01, 0x6a, 0xa6, 0xb8, 0x7a, 0xfc, 0x34, 0xc8, 0xe6, 0x8b, 0x35, 0x97, 0x2f, 0xff,
0x4d, 0xa3, 0x42, 0xbb, 0xd0, 0x64, 0x2a, 0x05, 0x06, 0xe6, 0xa1, 0xb4, 0x63, 0x55, 0xd2, 0x60,
0xe9, 0xcc, 0x50, 0xb3, 0x2b, 0x08, 0x29, 0xd7, 0xe5, 0x5b, 0x56, 0xe5, 0x5b, 0x89, 0x05, 0xaa,
0x78, 0xef, 0x02, 0x8c, 0xbc, 0x40, 0x64, 0xa6, 0x65, 0x55, 0x49, 0xd4, 0xdf, 0xb7, 0xa0, 0xc2,
0x27, 0xfe, 0x20, 0x0a, 0xce, 0xf4, 0xb8, 0x2c, 0x92, 0x0d, 0x3e, 0xf1, 0x49, 0x70, 0x26, 0xe2,
0xbf, 0x7c, 0xea, 0x0f, 0x04, 0xbb, 0xd0, 0x13, 0xb1, 0x48, 0x36, 0x7c, 0xea, 0xf7, 0xd9, 0x05,
0xc5, 0xe7, 0x00, 0xe6, 0x35, 0x8e, 0x85, 0x7b, 0x0d, 0x9a, 0x3e, 0x87, 0x0d, 0xf3, 0x5a, 0x66,
0x84, 0xdd, 0x5b, 0xde, 0xcc, 0xe2, 0xf7, 0x26, 0x53, 0x38, 0x3e, 0x86, 0xc6, 0xcb, 0xb8, 0xba,
0x54, 0xaa, 0x1c, 0x53, 0x69, 0xa3, 0x36, 0x6c, 0x98, 0x7a, 0x33, 0x44, 0x4c, 0x8f, 0xe8, 0x01,
0xd4, 0x87, 0xaa, 0x60, 0x07, 0xb3, 0x22, 0xac, 0x92, 0xda, 0x70, 0x56, 0xc4, 0xf8, 0x17, 0x0b,
0x5a, 0xc6, 0xce, 0x4c, 0xe3, 0xe5, 0xe4, 0xde, 0x05, 0x60, 0x62, 0x70, 0x1a, 0xa3, 0xa9, 0xa3,
0xdc, 0xaf, 0x90, 0x2a, 0x13, 0x2f, 0xb5, 0x00, 0x7d, 0x01, 0xeb, 0xca, 0xfe, 0xb4, 0xb1, 0x3d,
0xc8, 0x89, 0x2c, 0x1b, 0x01, 0x31, 0x17, 0xf0, 0x77, 0x50, 0xef, 0x76, 0x5f, 0xcd, 0xfc, 0x98,
0x4f, 0x23, 0x2b, 0x27, 0x8d, 0x56, 0xc7, 0xf8, 0xf8, 0x07, 0xb5, 0x5b, 0x27, 0x73, 0x01, 0x35,
0x93, 0x54, 0x3e, 0x09, 0x38, 0x6d, 0xad, 0xa1, 0x2d, 0xb5, 0xf8, 0x69, 0x81, 0x7c, 0x71, 0xce,
0x84, 0x6c, 0x59, 0x08, 0x41, 0xc3, 0x08, 0x8f, 0xa2, 0xe0, 0x8c, 0x71, 0xb7, 0x55, 0x40, 0x37,
0x60, 0x73, 0xaa, 0x49, 0x4d, 0x87, 0x56, 0xf1, 0xc9, 0x9f, 0x65, 0xa8, 0x75, 0x6d, 0x69, 0xf7,
0xf5, 0xd7, 0x0d, 0xb2, 0xa1, 0x9e, 0xfe, 0x4a, 0x40, 0x8f, 0x72, 0xc2, 0xcf, 0xf9, 0x8e, 0xe9,
0xec, 0xae, 0xc4, 0xe9, 0x76, 0x8b, 0xd7, 0xd0, 0x11, 0x94, 0xd5, 0x1b, 0xa1, 0xbc, 0x09, 0x98,
0xde, 0x71, 0x3a, 0x97, 0x2d, 0x46, 0x78, 0x0d, 0x0d, 0xa1, 0x99, 0xac, 0x6b, 0x86, 0xdc, 0x87,
0x39, 0x2a, 0x17, 0xb7, 0xf8, 0xce, 0xa3, 0x55, 0xb0, 0xc4, 0xd9, 0x01, 0xd4, 0x53, 0xdb, 0x86,
0xc8, 0x35, 0xb0, 0xb8, 0x1c, 0xe5, 0x1a, 0xc8, 0xd9, 0x5a, 0xf0, 0x1a, 0x72, 0xa1, 0x75, 0x44,
0x65, 0x66, 0x63, 0x40, 0xbb, 0x2b, 0x56, 0x83, 0xe9, 0xf2, 0xd2, 0xd9, 0x5b, 0x0d, 0x4c, 0x0c,
0x45, 0xb0, 0x7d, 0x44, 0xe5, 0xc2, 0x1c, 0x44, 0x8f, 0x73, 0x74, 0x2c, 0x19, 0xcc, 0x9d, 0x8f,
0xae, 0x80, 0x4d, 0xdb, 0xb4, 0xe1, 0x46, 0x62, 0x33, 0x69, 0x77, 0xbb, 0x4b, 0x95, 0x64, 0x07,
0x55, 0x67, 0xf5, 0xb8, 0xc5, 0x6b, 0x87, 0x5f, 0xbd, 0xf9, 0xd2, 0x65, 0x72, 0x3c, 0x19, 0xc6,
0xe9, 0x71, 0x70, 0xc1, 0x3c, 0x8f, 0x5d, 0x48, 0x3a, 0x1a, 0x1f, 0xe8, 0xbb, 0x1f, 0x3b, 0x4c,
0xc8, 0x88, 0x0d, 0x27, 0x92, 0x3a, 0x07, 0x53, 0x0d, 0x07, 0x4a, 0xe1, 0x41, 0x6c, 0x39, 0x1c,
0x0e, 0xd7, 0xd5, 0xe9, 0xe9, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xd9, 0xa6, 0xbb, 0x19, 0xe7,
0x0f, 0x00, 0x00,
// 1496 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x58, 0xdd, 0x6e, 0x1b, 0xc5,
0x1e, 0xcf, 0x7a, 0xfd, 0x11, 0xff, 0xed, 0x38, 0xee, 0x24, 0x4d, 0x52, 0xb7, 0xa7, 0x4d, 0xf6,
0xa8, 0x4d, 0x5a, 0x9d, 0x93, 0x1c, 0xb5, 0x3a, 0x14, 0x6e, 0x10, 0x4d, 0xdd, 0x46, 0x56, 0x9b,
0x28, 0x1a, 0x17, 0x2a, 0x7a, 0x63, 0xad, 0xbd, 0x13, 0x67, 0x60, 0x3f, 0xcc, 0xce, 0xb8, 0x49,
0x73, 0x03, 0x57, 0x20, 0x21, 0x24, 0xb8, 0x86, 0x6b, 0x5e, 0x00, 0x1e, 0x80, 0x57, 0xe0, 0x31,
0x78, 0x0a, 0x84, 0x76, 0x66, 0xf6, 0xcb, 0xde, 0xc4, 0xc6, 0x6d, 0xe9, 0x9d, 0xe7, 0xbf, 0xbf,
0xf9, 0x7f, 0x7f, 0x8d, 0x01, 0x59, 0x26, 0x37, 0x3b, 0x8c, 0xf8, 0x2f, 0x69, 0x8f, 0x6c, 0x0f,
0x7c, 0x8f, 0x7b, 0xe8, 0x92, 0x43, 0xed, 0x97, 0x43, 0x26, 0x4f, 0xdb, 0x01, 0xa0, 0x51, 0xed,
0x79, 0x8e, 0xe3, 0xb9, 0x92, 0xd4, 0xa8, 0x51, 0x97, 0x13, 0xdf, 0x35, 0x6d, 0x75, 0xae, 0x26,
0x2f, 0x18, 0x5f, 0xc2, 0x12, 0x26, 0x7d, 0xca, 0x38, 0xf1, 0x0f, 0x3c, 0x8b, 0x60, 0xf2, 0xc5,
0x90, 0x30, 0x8e, 0xfe, 0x07, 0xf9, 0xae, 0xc9, 0xc8, 0x9a, 0xb6, 0xae, 0x6d, 0x55, 0xee, 0x5e,
0xdb, 0x4e, 0x09, 0x51, 0xec, 0xf7, 0x59, 0x7f, 0xd7, 0x64, 0x04, 0x0b, 0x24, 0x7a, 0x0f, 0x4a,
0xa6, 0x65, 0xf9, 0x84, 0xb1, 0xb5, 0xdc, 0x05, 0x97, 0x1e, 0x48, 0x0c, 0x0e, 0xc1, 0xc6, 0xf7,
0x1a, 0x2c, 0xa7, 0x35, 0x60, 0x03, 0xcf, 0x65, 0x04, 0xed, 0x42, 0x85, 0xba, 0x94, 0x77, 0x06,
0xa6, 0x6f, 0x3a, 0x4c, 0x69, 0xb2, 0x91, 0x66, 0x1a, 0x99, 0xd6, 0x72, 0x29, 0x3f, 0x14, 0x40,
0x0c, 0x34, 0xfa, 0x8d, 0xee, 0x41, 0x91, 0x71, 0x93, 0x0f, 0x43, 0x9d, 0xae, 0x66, 0xea, 0xd4,
0x16, 0x10, 0xac, 0xa0, 0xc6, 0xef, 0x1a, 0x54, 0xdb, 0xa4, 0xdf, 0x6a, 0x86, 0xce, 0x58, 0x86,
0x42, 0xcf, 0x1b, 0xba, 0x5c, 0xe8, 0xb0, 0x80, 0xe5, 0x01, 0xad, 0x43, 0xa5, 0x77, 0x6c, 0xba,
0x2e, 0xb1, 0x0f, 0x4c, 0x87, 0x08, 0x01, 0x65, 0x9c, 0x24, 0x21, 0x03, 0xaa, 0x3d, 0xcf, 0xb6,
0x49, 0x8f, 0x53, 0xcf, 0x6d, 0x35, 0xd7, 0xf4, 0x75, 0x6d, 0x4b, 0xc7, 0x29, 0x5a, 0xc0, 0x65,
0x60, 0xfa, 0x9c, 0x2a, 0x48, 0x5e, 0x40, 0x92, 0x24, 0x74, 0x15, 0xca, 0xc1, 0x8d, 0x8e, 0x1b,
0x48, 0x29, 0x08, 0x29, 0xf3, 0x01, 0x41, 0x88, 0xb8, 0x09, 0xb5, 0x08, 0x2b, 0x11, 0x45, 0x81,
0x58, 0x88, 0xa8, 0x01, 0xcc, 0xf8, 0x41, 0x03, 0xf4, 0x80, 0x31, 0xda, 0x77, 0x53, 0x86, 0xad,
0x40, 0xd1, 0xf5, 0x2c, 0xd2, 0x6a, 0x0a, 0xcb, 0x74, 0xac, 0x4e, 0x81, 0xc8, 0x01, 0x21, 0x7e,
0xc7, 0xf7, 0xec, 0xd0, 0xb0, 0xf9, 0x80, 0x80, 0x3d, 0x9b, 0xa0, 0x47, 0xb0, 0xc0, 0x12, 0x4c,
0xd8, 0x9a, 0xbe, 0xae, 0x6f, 0x55, 0xee, 0xde, 0xd8, 0x1e, 0x4b, 0xc4, 0xed, 0xa4, 0x30, 0x9c,
0xbe, 0x65, 0xfc, 0x96, 0x83, 0x45, 0xf1, 0x5d, 0xea, 0xe5, 0x10, 0x57, 0x38, 0x5a, 0x80, 0x94,
0x3a, 0xf2, 0x30, 0x85, 0xa3, 0xa3, 0x00, 0xe9, 0xc9, 0x00, 0x8d, 0xba, 0x3f, 0x3f, 0xd9, 0xfd,
0x85, 0x71, 0xf7, 0xdf, 0x80, 0x0a, 0x39, 0x1d, 0x50, 0x9f, 0x74, 0x38, 0x55, 0xee, 0xcd, 0x63,
0x90, 0xa4, 0x67, 0xd4, 0x21, 0x89, 0x1c, 0x2b, 0x4d, 0x9d, 0x63, 0xe9, 0xa0, 0xce, 0x4f, 0x0c,
0x6a, 0x39, 0x2b, 0xa8, 0x3f, 0x6a, 0xb0, 0x94, 0x0a, 0xaa, 0x2a, 0x9c, 0x03, 0xa8, 0xb3, 0xb4,
0x63, 0x83, 0xea, 0x09, 0x62, 0x64, 0x9c, 0x17, 0xa3, 0x18, 0x8a, 0xc7, 0xee, 0xce, 0x56, 0x44,
0xa7, 0x50, 0x7d, 0x6c, 0x0f, 0xd9, 0xf1, 0xec, 0x0d, 0x05, 0x41, 0xde, 0xea, 0xb6, 0x9a, 0x42,
0xa8, 0x8e, 0xc5, 0xef, 0x69, 0x42, 0x6a, 0x7c, 0xa7, 0x01, 0x6a, 0x1f, 0x7b, 0x27, 0x6d, 0xd2,
0x17, 0x06, 0xcd, 0xac, 0xc0, 0xa8, 0xb0, 0xdc, 0xe4, 0xfc, 0xd1, 0xc7, 0xf2, 0xc7, 0xf8, 0x0c,
0x96, 0x52, 0xda, 0xa8, 0x20, 0x5d, 0x07, 0x60, 0x92, 0xd4, 0x6a, 0xca, 0xf0, 0xe8, 0x38, 0x41,
0x99, 0xcd, 0xe9, 0x47, 0xb0, 0xac, 0xe4, 0x04, 0x1f, 0x08, 0x9b, 0xdd, 0xf6, 0x6b, 0x50, 0x8e,
0x94, 0x51, 0x86, 0xc7, 0x04, 0xe3, 0xcf, 0x1c, 0x5c, 0x1e, 0x11, 0xa4, 0xcc, 0xfa, 0x3f, 0x14,
0x02, 0x5d, 0xa4, 0xa8, 0xda, 0x79, 0x4d, 0x21, 0xba, 0x88, 0x25, 0x3a, 0x28, 0xb2, 0x9e, 0x4f,
0x4c, 0xae, 0x8a, 0x2c, 0x27, 0x8b, 0x4c, 0x92, 0x44, 0x91, 0xdd, 0x80, 0x0a, 0x23, 0xa6, 0x4d,
0x2c, 0x09, 0xd0, 0x25, 0x40, 0x92, 0x04, 0x60, 0x03, 0xaa, 0x47, 0x41, 0xbe, 0x85, 0x88, 0xbc,
0x40, 0x54, 0x14, 0x4d, 0x40, 0x9e, 0xc0, 0x22, 0xe3, 0xa6, 0xcf, 0x3b, 0x03, 0x8f, 0x89, 0xe8,
0xb0, 0xb5, 0x42, 0x56, 0x59, 0x44, 0x43, 0x65, 0x9f, 0xf5, 0x0f, 0x15, 0x14, 0xd7, 0xc4, 0xd5,
0xf0, 0xc8, 0xd0, 0x1e, 0x2c, 0x10, 0xd7, 0x4a, 0xb0, 0x2a, 0x4e, 0xcd, 0xaa, 0x4a, 0x5c, 0x2b,
0x66, 0x34, 0x4b, 0xfb, 0x30, 0x28, 0xac, 0xb6, 0x5c, 0x46, 0x7c, 0xbe, 0x4b, 0x5d, 0xdb, 0xeb,
0x1f, 0x9a, 0xfc, 0xf8, 0x6d, 0xc5, 0xfa, 0x67, 0x0d, 0xae, 0x8c, 0xca, 0x8a, 0xe3, 0xdd, 0x80,
0xf9, 0x23, 0x4a, 0x6c, 0x2b, 0x4e, 0xe2, 0xe8, 0x8c, 0xee, 0x43, 0x61, 0x10, 0x80, 0xd7, 0x72,
0xc2, 0x35, 0xe7, 0x8d, 0xee, 0x36, 0xf7, 0xa9, 0xdb, 0x7f, 0x4a, 0x19, 0xc7, 0x12, 0x9f, 0x70,
0x89, 0x3e, 0xbd, 0x4b, 0xbe, 0xd2, 0x60, 0x59, 0xea, 0xf9, 0x50, 0x4e, 0x86, 0xb7, 0xdb, 0x79,
0x32, 0x66, 0xb9, 0xe1, 0xc0, 0xe5, 0xe7, 0x26, 0xef, 0x1d, 0x37, 0x9d, 0xd7, 0x56, 0x21, 0x10,
0x17, 0x0f, 0x38, 0xe9, 0xc2, 0x32, 0x4e, 0xd1, 0x8c, 0x9f, 0x34, 0x58, 0x14, 0x3d, 0xb6, 0x4d,
0xfa, 0xff, 0xb8, 0xb1, 0x23, 0x0d, 0x2c, 0x3f, 0xda, 0xc0, 0x8c, 0x3f, 0x72, 0x50, 0x51, 0xa5,
0xde, 0x72, 0x8f, 0xbc, 0x74, 0x96, 0x69, 0x23, 0x59, 0xf6, 0x66, 0x7a, 0x2d, 0xda, 0x84, 0x45,
0x2a, 0x52, 0xa0, 0xa3, 0x1c, 0x25, 0x15, 0x2b, 0xe3, 0x1a, 0x4d, 0x66, 0x86, 0x18, 0xbf, 0xde,
0x80, 0xb8, 0xb2, 0x55, 0x14, 0x44, 0xab, 0x98, 0x0f, 0x08, 0x59, 0xbd, 0xa6, 0x38, 0xb1, 0xd7,
0x94, 0xc6, 0x7b, 0xcd, 0x15, 0x98, 0x77, 0x87, 0x4e, 0xc7, 0xf7, 0x4e, 0x98, 0x18, 0xef, 0x3a,
0x2e, 0xb9, 0x43, 0x07, 0x7b, 0x27, 0x2c, 0xf8, 0xe4, 0x10, 0xa7, 0xc3, 0xe8, 0x99, 0x9c, 0xeb,
0x3a, 0x2e, 0x39, 0xc4, 0x69, 0xd3, 0xb3, 0x44, 0xf7, 0x84, 0xbf, 0xd3, 0x3d, 0x8d, 0x53, 0x00,
0x45, 0xde, 0x67, 0xfd, 0x19, 0x52, 0xe0, 0x7d, 0x28, 0xa9, 0x48, 0xa8, 0x61, 0x73, 0xfd, 0x7c,
0xc1, 0x41, 0x2c, 0x71, 0x08, 0x0f, 0x66, 0xed, 0xca, 0xc3, 0x28, 0x46, 0x81, 0x52, 0xaf, 0x31,
0x73, 0x56, 0xa1, 0x64, 0x75, 0xe5, 0xbe, 0x23, 0x77, 0xbc, 0xa2, 0xd5, 0x15, 0xfb, 0xd0, 0x26,
0x2c, 0xc6, 0x89, 0x20, 0x01, 0xba, 0x00, 0xd4, 0x62, 0xb2, 0xd8, 0x88, 0xbe, 0xd1, 0x60, 0x75,
0x4c, 0x1d, 0xd5, 0xa9, 0xee, 0x4b, 0xdf, 0x86, 0xab, 0xd0, 0x46, 0xa6, 0x42, 0x4f, 0xc8, 0xab,
0x4f, 0x4c, 0x7b, 0x48, 0x0e, 0x4d, 0xea, 0x4b, 0xef, 0xce, 0x38, 0x89, 0x7f, 0xd1, 0xe0, 0xf2,
0x61, 0x98, 0x99, 0xef, 0xda, 0x2f, 0x19, 0x0b, 0x65, 0x3e, 0x6b, 0xa1, 0xfc, 0x5a, 0x83, 0x95,
0x51, 0xa5, 0xdf, 0x89, 0xf7, 0xf6, 0xa1, 0xf6, 0x38, 0x98, 0x22, 0xa2, 0xbb, 0xed, 0x13, 0x6e,
0xa2, 0x35, 0x28, 0xa9, 0xb9, 0xa2, 0x7a, 0x47, 0x78, 0x0c, 0x8a, 0xb1, 0x2b, 0x06, 0x53, 0x27,
0x1e, 0x36, 0x65, 0x5c, 0xe9, 0xc6, 0xc3, 0xca, 0xf8, 0x56, 0x83, 0xba, 0x4a, 0xdf, 0x98, 0xe3,
0xc5, 0xfd, 0xe8, 0x5f, 0x00, 0x94, 0x75, 0x54, 0x45, 0x0b, 0xd5, 0xe7, 0x71, 0x99, 0xb2, 0xc7,
0x92, 0x80, 0x3e, 0x80, 0xa2, 0x90, 0x1f, 0x6e, 0x10, 0x1b, 0x19, 0x05, 0x93, 0xb6, 0x00, 0xab,
0x0b, 0xc6, 0xc7, 0x50, 0x6d, 0x36, 0x9f, 0xc6, 0x7a, 0x8c, 0x76, 0x3e, 0x2d, 0xa3, 0xf3, 0x4d,
0xb6, 0xf1, 0x8e, 0x27, 0xde, 0xac, 0x51, 0x6b, 0x40, 0x8b, 0x51, 0xf7, 0x3d, 0xf0, 0x5c, 0x52,
0x9f, 0x43, 0x4b, 0xe2, 0xb9, 0x25, 0x09, 0xfc, 0xd1, 0x29, 0x65, 0xbc, 0xae, 0x21, 0x04, 0x35,
0x45, 0xdc, 0xf3, 0xbd, 0x13, 0xea, 0xf6, 0xeb, 0x39, 0x74, 0x09, 0x16, 0x42, 0x4e, 0xa2, 0xe5,
0xd5, 0xf5, 0x04, 0x4c, 0x39, 0xa0, 0x9e, 0xbf, 0xfb, 0x6b, 0x19, 0x2a, 0x4d, 0x93, 0x9b, 0x6d,
0xf9, 0x6f, 0x04, 0x32, 0xa1, 0x9a, 0x7c, 0xc6, 0xa3, 0x5b, 0x19, 0x2e, 0xc9, 0xf8, 0xa7, 0xa1,
0xb1, 0x39, 0x11, 0x27, 0x53, 0xd0, 0x98, 0x43, 0x7b, 0x50, 0x10, 0xf2, 0x51, 0x56, 0x63, 0x4c,
0xbe, 0x36, 0x1a, 0x17, 0x65, 0x99, 0x31, 0x87, 0xba, 0xb0, 0x18, 0x3d, 0x9c, 0x54, 0xc0, 0x6f,
0x66, 0xb0, 0x1c, 0x7f, 0x31, 0x37, 0x6e, 0x4d, 0x82, 0x45, 0xca, 0x76, 0xa0, 0x9a, 0xd8, 0xfb,
0x59, 0xa6, 0x80, 0xf1, 0x67, 0x4a, 0xa6, 0x80, 0x8c, 0xf7, 0x83, 0x31, 0x87, 0xfa, 0x50, 0xdf,
0x23, 0x3c, 0xb5, 0x86, 0xa3, 0xcd, 0x09, 0x13, 0x23, 0xec, 0x42, 0x8d, 0xad, 0xc9, 0xc0, 0x48,
0x90, 0x0f, 0xcb, 0x7b, 0x84, 0x8f, 0xed, 0x80, 0xe8, 0x4e, 0x06, 0x8f, 0x73, 0xb6, 0xd2, 0xc6,
0x7f, 0xa6, 0xc0, 0x26, 0x65, 0x9a, 0x70, 0x29, 0x92, 0x19, 0x4d, 0xed, 0xcd, 0x73, 0x99, 0xa4,
0xf7, 0xad, 0xc6, 0xe4, 0x55, 0x53, 0x98, 0xb5, 0xba, 0x47, 0x78, 0x7a, 0x5c, 0x50, 0xc6, 0x69,
0x8f, 0xa1, 0xdb, 0x19, 0x82, 0xb2, 0xc7, 0x5c, 0xe3, 0xce, 0x34, 0xd0, 0xc8, 0x2c, 0x0f, 0x56,
0xf6, 0x08, 0x4f, 0xf5, 0x58, 0x25, 0x32, 0x2b, 0x20, 0x99, 0x03, 0xa4, 0x71, 0x7b, 0x0a, 0x64,
0x24, 0xf0, 0x05, 0x20, 0x61, 0xa4, 0x33, 0xf0, 0xdc, 0x38, 0x4d, 0x1a, 0x99, 0xe5, 0xf1, 0xc8,
0x19, 0xf0, 0x57, 0xa3, 0x09, 0x18, 0xf9, 0x6e, 0x84, 0x87, 0x31, 0x87, 0x9e, 0x0b, 0xde, 0xc1,
0xba, 0xf3, 0x8c, 0xf6, 0x3e, 0x57, 0x21, 0xb8, 0x90, 0xf7, 0xbf, 0xd3, 0xdf, 0xd4, 0x41, 0x46,
0x25, 0xa1, 0xf4, 0xa7, 0x22, 0xe1, 0x62, 0xe7, 0xbc, 0x39, 0xd6, 0xbb, 0x1f, 0xbd, 0xf8, 0xb0,
0x4f, 0xf9, 0xf1, 0xb0, 0x1b, 0xdc, 0xde, 0x39, 0xa3, 0xb6, 0x4d, 0xcf, 0x38, 0xe9, 0x1d, 0xef,
0xc8, 0x0b, 0xff, 0xb5, 0x28, 0xe3, 0x3e, 0xed, 0x0e, 0x39, 0xb1, 0x76, 0x42, 0xd3, 0x77, 0x04,
0xcb, 0x9d, 0xc0, 0xd1, 0x83, 0x6e, 0xb7, 0x28, 0x4e, 0xf7, 0xfe, 0x0a, 0x00, 0x00, 0xff, 0xff,
0x6d, 0x7e, 0x0b, 0xe1, 0x8c, 0x15, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
@ -1382,6 +1664,11 @@ type DataServiceClient interface {
GetSegmentStates(ctx context.Context, in *SegmentStatesRequest, opts ...grpc.CallOption) (*SegmentStatesResponse, error)
GetInsertBinlogPaths(ctx context.Context, in *InsertBinlogPathRequest, opts ...grpc.CallOption) (*InsertBinlogPathsResponse, error)
GetInsertChannels(ctx context.Context, in *InsertChannelRequest, opts ...grpc.CallOption) (*internalpb2.StringList, error)
GetCollectionStatistics(ctx context.Context, in *CollectionStatsRequest, opts ...grpc.CallOption) (*CollectionStatsResponse, error)
GetPartitionStatistics(ctx context.Context, in *PartitionStatsRequest, opts ...grpc.CallOption) (*PartitionStatsResponse, error)
GetComponentStates(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*internalpb2.ComponentStates, error)
GetTimeTickChannel(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
GetStatisticsChannel(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
}
type dataServiceClient struct {
@ -1455,6 +1742,51 @@ func (c *dataServiceClient) GetInsertChannels(ctx context.Context, in *InsertCha
return out, nil
}
func (c *dataServiceClient) GetCollectionStatistics(ctx context.Context, in *CollectionStatsRequest, opts ...grpc.CallOption) (*CollectionStatsResponse, error) {
out := new(CollectionStatsResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetCollectionStatistics", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *dataServiceClient) GetPartitionStatistics(ctx context.Context, in *PartitionStatsRequest, opts ...grpc.CallOption) (*PartitionStatsResponse, error) {
out := new(PartitionStatsResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetPartitionStatistics", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *dataServiceClient) GetComponentStates(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*internalpb2.ComponentStates, error) {
out := new(internalpb2.ComponentStates)
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetComponentStates", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *dataServiceClient) GetTimeTickChannel(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
out := new(milvuspb.StringResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetTimeTickChannel", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *dataServiceClient) GetStatisticsChannel(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
out := new(milvuspb.StringResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetStatisticsChannel", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
// DataServiceServer is the server API for DataService service.
type DataServiceServer interface {
RegisterNode(context.Context, *RegisterNodeRequest) (*RegisterNodeResponse, error)
@ -1464,6 +1796,11 @@ type DataServiceServer interface {
GetSegmentStates(context.Context, *SegmentStatesRequest) (*SegmentStatesResponse, error)
GetInsertBinlogPaths(context.Context, *InsertBinlogPathRequest) (*InsertBinlogPathsResponse, error)
GetInsertChannels(context.Context, *InsertChannelRequest) (*internalpb2.StringList, error)
GetCollectionStatistics(context.Context, *CollectionStatsRequest) (*CollectionStatsResponse, error)
GetPartitionStatistics(context.Context, *PartitionStatsRequest) (*PartitionStatsResponse, error)
GetComponentStates(context.Context, *commonpb.Empty) (*internalpb2.ComponentStates, error)
GetTimeTickChannel(context.Context, *commonpb.Empty) (*milvuspb.StringResponse, error)
GetStatisticsChannel(context.Context, *commonpb.Empty) (*milvuspb.StringResponse, error)
}
// UnimplementedDataServiceServer can be embedded to have forward compatible implementations.
@ -1491,6 +1828,21 @@ func (*UnimplementedDataServiceServer) GetInsertBinlogPaths(ctx context.Context,
func (*UnimplementedDataServiceServer) GetInsertChannels(ctx context.Context, req *InsertChannelRequest) (*internalpb2.StringList, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetInsertChannels not implemented")
}
func (*UnimplementedDataServiceServer) GetCollectionStatistics(ctx context.Context, req *CollectionStatsRequest) (*CollectionStatsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetCollectionStatistics not implemented")
}
func (*UnimplementedDataServiceServer) GetPartitionStatistics(ctx context.Context, req *PartitionStatsRequest) (*PartitionStatsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetPartitionStatistics not implemented")
}
func (*UnimplementedDataServiceServer) GetComponentStates(ctx context.Context, req *commonpb.Empty) (*internalpb2.ComponentStates, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetComponentStates not implemented")
}
func (*UnimplementedDataServiceServer) GetTimeTickChannel(ctx context.Context, req *commonpb.Empty) (*milvuspb.StringResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetTimeTickChannel not implemented")
}
func (*UnimplementedDataServiceServer) GetStatisticsChannel(ctx context.Context, req *commonpb.Empty) (*milvuspb.StringResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetStatisticsChannel not implemented")
}
func RegisterDataServiceServer(s *grpc.Server, srv DataServiceServer) {
s.RegisterService(&_DataService_serviceDesc, srv)
@ -1622,6 +1974,96 @@ func _DataService_GetInsertChannels_Handler(srv interface{}, ctx context.Context
return interceptor(ctx, in, info, handler)
}
func _DataService_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.(DataServiceServer).GetCollectionStatistics(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.data.DataService/GetCollectionStatistics",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(DataServiceServer).GetCollectionStatistics(ctx, req.(*CollectionStatsRequest))
}
return interceptor(ctx, in, info, handler)
}
func _DataService_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.(DataServiceServer).GetPartitionStatistics(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.data.DataService/GetPartitionStatistics",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(DataServiceServer).GetPartitionStatistics(ctx, req.(*PartitionStatsRequest))
}
return interceptor(ctx, in, info, handler)
}
func _DataService_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(commonpb.Empty)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(DataServiceServer).GetComponentStates(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.data.DataService/GetComponentStates",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(DataServiceServer).GetComponentStates(ctx, req.(*commonpb.Empty))
}
return interceptor(ctx, in, info, handler)
}
func _DataService_GetTimeTickChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(commonpb.Empty)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(DataServiceServer).GetTimeTickChannel(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.data.DataService/GetTimeTickChannel",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(DataServiceServer).GetTimeTickChannel(ctx, req.(*commonpb.Empty))
}
return interceptor(ctx, in, info, handler)
}
func _DataService_GetStatisticsChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(commonpb.Empty)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(DataServiceServer).GetStatisticsChannel(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.data.DataService/GetStatisticsChannel",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(DataServiceServer).GetStatisticsChannel(ctx, req.(*commonpb.Empty))
}
return interceptor(ctx, in, info, handler)
}
var _DataService_serviceDesc = grpc.ServiceDesc{
ServiceName: "milvus.proto.data.DataService",
HandlerType: (*DataServiceServer)(nil),
@ -1654,6 +2096,26 @@ var _DataService_serviceDesc = grpc.ServiceDesc{
MethodName: "GetInsertChannels",
Handler: _DataService_GetInsertChannels_Handler,
},
{
MethodName: "GetCollectionStatistics",
Handler: _DataService_GetCollectionStatistics_Handler,
},
{
MethodName: "GetPartitionStatistics",
Handler: _DataService_GetPartitionStatistics_Handler,
},
{
MethodName: "GetComponentStates",
Handler: _DataService_GetComponentStates_Handler,
},
{
MethodName: "GetTimeTickChannel",
Handler: _DataService_GetTimeTickChannel_Handler,
},
{
MethodName: "GetStatisticsChannel",
Handler: _DataService_GetStatisticsChannel_Handler,
},
},
Streams: []grpc.StreamDesc{},
Metadata: "data_service.proto",

View File

@ -21,6 +21,7 @@ message ComponentInfo {
message ComponentStates {
ComponentInfo state = 1;
repeated ComponentInfo subcomponent_states = 2;
common.Status status = 3;
}
message NodeInfo {
@ -34,7 +35,8 @@ message InitParams {
}
message StringList {
repeated string values = 2;
repeated string values = 1;
common.Status status = 2;
}
message TimeTickMsg {

View File

@ -115,6 +115,7 @@ func (m *ComponentInfo) GetExtraInfo() []*commonpb.KeyValuePair {
type ComponentStates struct {
State *ComponentInfo `protobuf:"bytes,1,opt,name=state,proto3" json:"state,omitempty"`
SubcomponentStates []*ComponentInfo `protobuf:"bytes,2,rep,name=subcomponent_states,json=subcomponentStates,proto3" json:"subcomponent_states,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:"-"`
@ -159,6 +160,13 @@ func (m *ComponentStates) GetSubcomponentStates() []*ComponentInfo {
return nil
}
func (m *ComponentStates) GetStatus() *commonpb.Status {
if m != nil {
return m.Status
}
return nil
}
type NodeInfo struct {
Address *commonpb.Address `protobuf:"bytes,1,opt,name=address,proto3" json:"address,omitempty"`
Role string `protobuf:"bytes,2,opt,name=role,proto3" json:"role,omitempty"`
@ -254,10 +262,11 @@ func (m *InitParams) GetStartParams() []*commonpb.KeyValuePair {
}
type StringList struct {
Values []string `protobuf:"bytes,2,rep,name=values,proto3" json:"values,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
Values []string `protobuf:"bytes,1,rep,name=values,proto3" json:"values,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:"-"`
}
func (m *StringList) Reset() { *m = StringList{} }
@ -292,6 +301,13 @@ func (m *StringList) GetValues() []string {
return nil
}
func (m *StringList) GetStatus() *commonpb.Status {
if m != nil {
return m.Status
}
return nil
}
type TimeTickMsg struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@ -1601,89 +1617,90 @@ func init() {
func init() { proto.RegisterFile("internal.proto", fileDescriptor_41f4a519b878ee3b) }
var fileDescriptor_41f4a519b878ee3b = []byte{
// 1340 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x58, 0xcf, 0x6f, 0xdc, 0x44,
0x18, 0xc5, 0xbb, 0x9b, 0xfd, 0xf1, 0xed, 0x26, 0x4d, 0x4d, 0x7f, 0xb8, 0x10, 0xe8, 0xd6, 0x14,
0x58, 0x40, 0x24, 0x55, 0x8a, 0x10, 0xe2, 0xd2, 0x26, 0xd9, 0x96, 0x5a, 0x4d, 0x42, 0x98, 0x4d,
0x2b, 0xd1, 0x8b, 0x35, 0x6b, 0x4f, 0x76, 0x87, 0xfa, 0xc7, 0x76, 0x66, 0xdc, 0x74, 0x7b, 0xe6,
0x86, 0xe0, 0x80, 0xc4, 0x91, 0x0b, 0xe2, 0xef, 0x00, 0x89, 0x13, 0x12, 0x57, 0xc4, 0x89, 0xbf,
0x84, 0x13, 0x9a, 0x19, 0xdb, 0xbb, 0x9b, 0x38, 0x21, 0x0d, 0x20, 0x54, 0x89, 0xdb, 0xce, 0xf3,
0xe7, 0xcf, 0xdf, 0x7b, 0xf3, 0xfc, 0x6c, 0x2f, 0x2c, 0xd0, 0x48, 0x10, 0x16, 0xe1, 0x60, 0x79,
0xc4, 0x62, 0x11, 0x9b, 0xe7, 0x43, 0x1a, 0x3c, 0x4e, 0xb8, 0x5e, 0x2d, 0x67, 0x07, 0x5f, 0x6a,
0x79, 0x71, 0x18, 0xc6, 0x91, 0x86, 0xed, 0x1f, 0x0c, 0x98, 0xdf, 0x88, 0xc3, 0x51, 0x1c, 0x91,
0x48, 0x38, 0xd1, 0x5e, 0x6c, 0x5e, 0x80, 0x6a, 0x14, 0xfb, 0xc4, 0xe9, 0x5a, 0x46, 0xdb, 0xe8,
0x94, 0x51, 0xba, 0x32, 0x4d, 0xa8, 0xb0, 0x38, 0x20, 0x56, 0xa9, 0x6d, 0x74, 0x1a, 0x48, 0xfd,
0x36, 0x6f, 0x00, 0x70, 0x81, 0x05, 0x71, 0xbd, 0xd8, 0x27, 0x56, 0xb9, 0x6d, 0x74, 0x16, 0x56,
0xdb, 0xcb, 0x85, 0xd7, 0x5d, 0xee, 0xc9, 0xc2, 0x8d, 0xd8, 0x27, 0xa8, 0xc1, 0xb3, 0x9f, 0xe6,
0x4d, 0x00, 0xf2, 0x44, 0x30, 0xec, 0xd2, 0x68, 0x2f, 0xb6, 0x2a, 0xed, 0x72, 0xa7, 0xb9, 0x7a,
0x65, 0xb6, 0x41, 0x3a, 0xee, 0x5d, 0x32, 0xbe, 0x8f, 0x83, 0x84, 0xec, 0x60, 0xca, 0x50, 0x43,
0x9d, 0x24, 0xc7, 0xb5, 0xbf, 0x37, 0xe0, 0x4c, 0x4e, 0x40, 0x5d, 0x83, 0x9b, 0x1f, 0xc2, 0x9c,
0xba, 0x84, 0x62, 0xd0, 0x5c, 0xbd, 0x7a, 0xc4, 0x44, 0x33, 0xbc, 0x91, 0x3e, 0xc5, 0xbc, 0x07,
0x2f, 0xf2, 0xa4, 0xef, 0x65, 0x87, 0x5c, 0x85, 0x72, 0xab, 0xa4, 0x46, 0x3b, 0x59, 0x27, 0x73,
0xba, 0x81, 0x1e, 0xc9, 0xbe, 0x0f, 0xf5, 0x6d, 0xa9, 0xa3, 0x54, 0xf8, 0x7d, 0xa8, 0x61, 0xdf,
0x67, 0x84, 0xf3, 0x74, 0xc0, 0xa5, 0x42, 0xc6, 0x6b, 0xba, 0x06, 0x65, 0xc5, 0x45, 0x3b, 0x60,
0x7f, 0x06, 0xe0, 0x44, 0x54, 0xec, 0x60, 0x86, 0x43, 0x7e, 0xe4, 0xde, 0x75, 0xa1, 0xc5, 0x05,
0x66, 0xc2, 0x1d, 0xa9, 0xba, 0x94, 0xcd, 0x09, 0x84, 0x6e, 0xaa, 0xd3, 0x74, 0x77, 0xfb, 0x2a,
0x40, 0x4f, 0x30, 0x1a, 0x0d, 0x36, 0x29, 0x17, 0xf2, 0x5a, 0x8f, 0x65, 0x9d, 0xee, 0xd6, 0x40,
0xe9, 0xca, 0xbe, 0x01, 0xcd, 0x5d, 0x1a, 0x92, 0x5d, 0xea, 0x3d, 0xdc, 0xe2, 0x03, 0xf3, 0x1a,
0x54, 0xfa, 0x98, 0x93, 0x63, 0x99, 0x6e, 0xf1, 0xc1, 0x3a, 0xe6, 0x04, 0xa9, 0x4a, 0xfb, 0x77,
0x03, 0x2e, 0x6e, 0x30, 0xa2, 0x2c, 0x12, 0x04, 0xc4, 0x13, 0x34, 0x8e, 0x10, 0x79, 0x94, 0x10,
0x2e, 0x9e, 0xbd, 0x9b, 0x79, 0x11, 0x6a, 0x7e, 0xdf, 0x8d, 0x70, 0x98, 0xe9, 0x56, 0xf5, 0xfb,
0xdb, 0x38, 0x24, 0xe6, 0x1b, 0xb0, 0xe0, 0xe5, 0xfd, 0x25, 0xa2, 0xfc, 0xdb, 0x40, 0x07, 0x50,
0xa9, 0xba, 0xdf, 0x77, 0xba, 0x56, 0x45, 0x29, 0xaa, 0x7e, 0x9b, 0x36, 0xb4, 0x26, 0x55, 0x4e,
0xd7, 0x9a, 0x53, 0xc7, 0x66, 0x30, 0xa9, 0x0f, 0xf7, 0x86, 0x24, 0xc4, 0x56, 0xb5, 0x6d, 0x74,
0x5a, 0x28, 0x5d, 0xd9, 0x3f, 0x19, 0x70, 0xbe, 0xcb, 0xe2, 0xd1, 0xf3, 0x4c, 0xce, 0xfe, 0xb2,
0x04, 0x17, 0xf4, 0x1e, 0xed, 0x60, 0x26, 0xe8, 0xbf, 0xc4, 0xe2, 0x4d, 0x38, 0x33, 0xb9, 0xaa,
0x2e, 0x28, 0xa6, 0xf1, 0x3a, 0x2c, 0x8c, 0xb2, 0x39, 0x74, 0x5d, 0x45, 0xd5, 0xcd, 0xe7, 0xe8,
0x0c, 0xdb, 0xb9, 0x63, 0xd8, 0x56, 0x0b, 0xb6, 0xb2, 0x0d, 0xcd, 0xbc, 0x91, 0xd3, 0xb5, 0x6a,
0xaa, 0x64, 0x1a, 0xb2, 0xbf, 0x28, 0xc1, 0x39, 0xb9, 0xa9, 0xff, 0xab, 0x21, 0xd5, 0xf8, 0xb1,
0x04, 0xa6, 0x76, 0x87, 0x13, 0xf9, 0xe4, 0xc9, 0x7f, 0xa9, 0xc5, 0x2b, 0x00, 0x7b, 0x94, 0x04,
0xfe, 0xb4, 0x0e, 0x0d, 0x85, 0xfc, 0x2d, 0x0d, 0x2c, 0xa8, 0xa9, 0x26, 0x39, 0xff, 0x6c, 0x29,
0xa3, 0x56, 0x3f, 0xd1, 0xd2, 0xa8, 0xad, 0x9f, 0x38, 0x6a, 0xd5, 0x69, 0x69, 0xd4, 0xfe, 0x5a,
0x82, 0x79, 0x27, 0xe2, 0x84, 0x89, 0xe7, 0xc0, 0x48, 0x4b, 0xd0, 0xe0, 0x64, 0x10, 0xca, 0xc7,
0x5f, 0xa6, 0xe4, 0x04, 0x90, 0x47, 0xbd, 0x21, 0x8e, 0x22, 0x12, 0xa4, 0x5a, 0x36, 0xd0, 0x04,
0x30, 0x5f, 0x05, 0x10, 0x34, 0x24, 0x5c, 0xe0, 0x70, 0xc4, 0xad, 0x5a, 0xbb, 0xdc, 0xa9, 0xa0,
0x29, 0x44, 0xa6, 0x28, 0x8b, 0xf7, 0x9d, 0xae, 0x16, 0xb2, 0x8c, 0xd2, 0x95, 0xf9, 0x1e, 0xd4,
0x59, 0xbc, 0xef, 0xfa, 0x58, 0x60, 0xab, 0xa1, 0x24, 0xbe, 0x54, 0x28, 0xc9, 0x7a, 0x10, 0xf7,
0x51, 0x8d, 0xc5, 0xfb, 0x5d, 0x2c, 0xb0, 0xfd, 0xad, 0x01, 0xf3, 0x3d, 0x82, 0x99, 0x37, 0x3c,
0xbd, 0xac, 0x6f, 0xc1, 0x22, 0x23, 0x3c, 0x09, 0x84, 0x3b, 0xa1, 0xa5, 0xf5, 0x3d, 0xa3, 0xf1,
0x8d, 0x9c, 0xdc, 0x0a, 0xcc, 0x3d, 0x4a, 0x08, 0x1b, 0x2b, 0x79, 0x8f, 0x9d, 0x50, 0xd7, 0xd9,
0xbf, 0x4d, 0xcd, 0x27, 0x5b, 0xf1, 0x53, 0xcc, 0x77, 0x1d, 0xaa, 0xf2, 0x9d, 0x25, 0xe1, 0x6a,
0xaa, 0xe6, 0xea, 0xcb, 0x85, 0xe7, 0xf4, 0x54, 0x09, 0x4a, 0x4b, 0x0b, 0x49, 0x95, 0x8b, 0x49,
0x5d, 0x86, 0x66, 0x48, 0x04, 0xa3, 0x9e, 0x2b, 0xc6, 0xa3, 0xcc, 0x11, 0xa0, 0xa1, 0xdd, 0xf1,
0x48, 0xdd, 0x53, 0x43, 0x2a, 0xb8, 0x35, 0xd7, 0x2e, 0x77, 0x5a, 0x48, 0xfd, 0xb6, 0x7f, 0x31,
0x60, 0xbe, 0x4b, 0x02, 0x22, 0xc8, 0xe9, 0x85, 0x2f, 0xb0, 0x6d, 0xa9, 0xd0, 0xb6, 0x33, 0x8e,
0x2b, 0x1f, 0xef, 0xb8, 0xca, 0x21, 0xc7, 0x5d, 0x81, 0xd6, 0x88, 0xd1, 0x10, 0xb3, 0xb1, 0xfb,
0x90, 0x8c, 0x35, 0x0d, 0x99, 0x6f, 0x1a, 0xbb, 0x4b, 0xc6, 0xdc, 0xfe, 0xce, 0x80, 0xfa, 0xed,
0x20, 0xe1, 0xc3, 0x53, 0xbd, 0xe0, 0xcc, 0xde, 0x2f, 0xa5, 0x83, 0xf7, 0xcb, 0xc1, 0xf8, 0x29,
0x17, 0xc4, 0x8f, 0x0d, 0xad, 0xfc, 0x16, 0xdc, 0xc5, 0x83, 0x74, 0x13, 0x66, 0x30, 0xfb, 0x0f,
0x03, 0x1a, 0x9b, 0x31, 0xf6, 0x55, 0x04, 0xff, 0xe3, 0x53, 0x2e, 0xc1, 0x24, 0x45, 0x33, 0x8d,
0x27, 0xb1, 0x3a, 0x15, 0x8f, 0x95, 0xd9, 0x78, 0xbc, 0x0c, 0x4d, 0x2a, 0x07, 0x72, 0x47, 0x58,
0x0c, 0xb5, 0xb8, 0x0d, 0x04, 0x0a, 0xda, 0x91, 0x88, 0xcc, 0xcf, 0xac, 0x40, 0xe5, 0x67, 0xf5,
0xc4, 0xf9, 0x99, 0x36, 0x51, 0xf9, 0xc9, 0xc0, 0xea, 0xe9, 0x59, 0xa5, 0xd1, 0x29, 0x17, 0xd4,
0xe3, 0xf7, 0x46, 0xbe, 0xfa, 0x3a, 0x58, 0x82, 0x46, 0x2f, 0x27, 0xa6, 0xdf, 0x93, 0x27, 0x80,
0xb4, 0xc7, 0x16, 0x09, 0x63, 0x36, 0xee, 0xd1, 0xa7, 0x24, 0xe5, 0x3d, 0x85, 0x48, 0x6a, 0xdb,
0x49, 0x88, 0xe2, 0x7d, 0x9e, 0xee, 0x4c, 0xb6, 0xb4, 0xbf, 0x36, 0xe0, 0xec, 0xa1, 0x8b, 0x9e,
0x42, 0xf8, 0xbb, 0x50, 0xef, 0x91, 0x81, 0x6c, 0x91, 0xbd, 0xa8, 0xaf, 0x1c, 0xf5, 0x49, 0x75,
0x04, 0x45, 0x94, 0x37, 0xb0, 0x3f, 0x37, 0xe4, 0x07, 0x82, 0x4f, 0x9e, 0xa8, 0xe5, 0x21, 0x75,
0x8d, 0xd3, 0xa8, 0x6b, 0x5e, 0x83, 0x73, 0x51, 0x12, 0xba, 0x8c, 0x04, 0x58, 0x10, 0xdf, 0x4d,
0x5d, 0xc1, 0x53, 0xb5, 0xcc, 0x28, 0x09, 0x91, 0x3e, 0x94, 0x0e, 0xc8, 0xed, 0xaf, 0x0c, 0x80,
0xdb, 0xd2, 0x02, 0x7a, 0x8c, 0x83, 0x1e, 0x37, 0x8e, 0x7f, 0xc4, 0x96, 0x66, 0x3d, 0xb4, 0x9e,
0x79, 0x88, 0x2b, 0x8d, 0xca, 0x45, 0x1c, 0x72, 0x8d, 0x26, 0xe4, 0x53, 0x9b, 0x69, 0x5d, 0xbe,
0x31, 0xa0, 0x35, 0x25, 0x1f, 0x9f, 0xb5, 0xbb, 0x71, 0xd0, 0xee, 0x2a, 0xf4, 0xa4, 0x07, 0x5c,
0x3e, 0x65, 0x8b, 0x70, 0x62, 0x8b, 0x4b, 0x50, 0x57, 0x92, 0x4c, 0xf9, 0x22, 0xd2, 0xbe, 0x30,
0xdf, 0x81, 0xb3, 0x8c, 0x78, 0x24, 0x12, 0xc1, 0xd8, 0x0d, 0x63, 0x9f, 0xee, 0x51, 0xe2, 0xab,
0xdb, 0xa2, 0x8e, 0x16, 0xb3, 0x03, 0x5b, 0x29, 0x6e, 0xff, 0x6c, 0xc0, 0xc2, 0x27, 0xf2, 0x59,
0x20, 0xbf, 0x16, 0xf5, 0x64, 0xcf, 0xee, 0xa0, 0x9b, 0x8a, 0x4b, 0x2a, 0x8f, 0xb6, 0xd0, 0x6b,
0x7f, 0x6d, 0x21, 0x8e, 0xea, 0x3c, 0xb5, 0x8d, 0x94, 0x58, 0xbf, 0x36, 0x9d, 0x44, 0xe2, 0xc9,
0xc6, 0x22, 0xfd, 0xb2, 0xa5, 0x25, 0xf6, 0xa1, 0xb9, 0xc5, 0x07, 0x3b, 0x31, 0x57, 0x91, 0x24,
0x73, 0x35, 0x0d, 0x61, 0x9d, 0xdd, 0x86, 0x0a, 0x8d, 0x66, 0x8a, 0xa9, 0xd8, 0x38, 0x07, 0x73,
0x21, 0x1f, 0xe4, 0xcf, 0x53, 0xbd, 0x90, 0x3b, 0x93, 0xc7, 0xb3, 0xd2, 0xb6, 0x82, 0x26, 0xc0,
0xdb, 0x1f, 0x40, 0x23, 0xff, 0x67, 0xc1, 0x5c, 0x84, 0x96, 0xb3, 0xed, 0xec, 0x3a, 0x6b, 0x9b,
0xce, 0x03, 0x67, 0xfb, 0xa3, 0xc5, 0x17, 0xcc, 0x26, 0xd4, 0xee, 0xdc, 0x5a, 0xdb, 0xdc, 0xbd,
0xf3, 0xe9, 0xa2, 0x61, 0xb6, 0xa0, 0xbe, 0xb6, 0xbe, 0xfd, 0x31, 0xda, 0x5a, 0xdb, 0x5c, 0x2c,
0xad, 0xdf, 0x7a, 0xb0, 0x31, 0xa0, 0x62, 0x98, 0xf4, 0xa5, 0x88, 0x2b, 0x4f, 0x69, 0x10, 0xd0,
0xa7, 0x82, 0x78, 0xc3, 0x15, 0xcd, 0xf2, 0x5d, 0x9f, 0x72, 0xc1, 0x68, 0x3f, 0x11, 0xc4, 0x5f,
0xc9, 0xb8, 0xae, 0x28, 0xea, 0xf9, 0x72, 0xd4, 0x5f, 0xed, 0x57, 0x15, 0x74, 0xfd, 0xcf, 0x00,
0x00, 0x00, 0xff, 0xff, 0xff, 0xf9, 0xa9, 0x8f, 0x7f, 0x11, 0x00, 0x00,
// 1353 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x58, 0x4d, 0x6f, 0x1b, 0x55,
0x17, 0x7e, 0xc7, 0x76, 0xfc, 0x71, 0xec, 0xa4, 0xe9, 0xbc, 0xfd, 0x98, 0xbe, 0x6f, 0xa0, 0xee,
0xf0, 0x65, 0x40, 0x24, 0x55, 0x8a, 0x10, 0x62, 0xd3, 0x26, 0x71, 0x4b, 0x47, 0x4d, 0x42, 0xb8,
0x4e, 0x2b, 0xb5, 0x9b, 0xd1, 0xb5, 0xe7, 0xc6, 0xbe, 0x74, 0x3e, 0xdc, 0x7b, 0xaf, 0x9b, 0xba,
0x6b, 0x76, 0x08, 0x16, 0x48, 0x2c, 0xd9, 0xf0, 0x43, 0x40, 0x62, 0x85, 0xc4, 0x16, 0x21, 0x21,
0xf1, 0x4b, 0x58, 0xa1, 0xfb, 0x31, 0x63, 0x3b, 0x9d, 0x84, 0xd4, 0x80, 0x50, 0x25, 0x76, 0x73,
0xcf, 0x9c, 0x39, 0x73, 0x9e, 0xe7, 0x3c, 0xf3, 0xdc, 0x6b, 0xc3, 0x12, 0x8d, 0x05, 0x61, 0x31,
0x0e, 0x57, 0x87, 0x2c, 0x11, 0x89, 0x7d, 0x3e, 0xa2, 0xe1, 0xe3, 0x11, 0xd7, 0xab, 0xd5, 0xf4,
0xe6, 0xff, 0x1a, 0xbd, 0x24, 0x8a, 0x92, 0x58, 0x87, 0xdd, 0x6f, 0x2d, 0x58, 0xdc, 0x4a, 0xa2,
0x61, 0x12, 0x93, 0x58, 0x78, 0xf1, 0x41, 0x62, 0x5f, 0x80, 0x72, 0x9c, 0x04, 0xc4, 0x6b, 0x3b,
0x56, 0xd3, 0x6a, 0x15, 0x91, 0x59, 0xd9, 0x36, 0x94, 0x58, 0x12, 0x12, 0xa7, 0xd0, 0xb4, 0x5a,
0x35, 0xa4, 0xae, 0xed, 0xeb, 0x00, 0x5c, 0x60, 0x41, 0xfc, 0x5e, 0x12, 0x10, 0xa7, 0xd8, 0xb4,
0x5a, 0x4b, 0xeb, 0xcd, 0xd5, 0xdc, 0xf7, 0xae, 0x76, 0x64, 0xe2, 0x56, 0x12, 0x10, 0x54, 0xe3,
0xe9, 0xa5, 0x7d, 0x03, 0x80, 0x3c, 0x11, 0x0c, 0xfb, 0x34, 0x3e, 0x48, 0x9c, 0x52, 0xb3, 0xd8,
0xaa, 0xaf, 0x5f, 0x99, 0x2d, 0x60, 0xda, 0xbd, 0x43, 0xc6, 0xf7, 0x70, 0x38, 0x22, 0x7b, 0x98,
0x32, 0x54, 0x53, 0x0f, 0xc9, 0x76, 0xdd, 0x5f, 0x2c, 0x38, 0x93, 0x01, 0x50, 0xef, 0xe0, 0xf6,
0x07, 0xb0, 0xa0, 0x5e, 0xa1, 0x10, 0xd4, 0xd7, 0x5f, 0x3d, 0xa6, 0xa3, 0x19, 0xdc, 0x48, 0x3f,
0x62, 0xdf, 0x85, 0xff, 0xf2, 0x51, 0xb7, 0x97, 0xde, 0xf2, 0x55, 0x94, 0x3b, 0x05, 0xd5, 0xda,
0xe9, 0x2a, 0xd9, 0xd3, 0x05, 0x4c, 0x4b, 0xd7, 0xa0, 0x2c, 0x2b, 0x8d, 0xb8, 0x62, 0xa9, 0xbe,
0xfe, 0xff, 0x5c, 0x90, 0x1d, 0x95, 0x82, 0x4c, 0xaa, 0x7b, 0x0f, 0xaa, 0xbb, 0x92, 0x7c, 0x39,
0x96, 0xf7, 0xa0, 0x82, 0x83, 0x80, 0x11, 0xce, 0x0d, 0xaa, 0x95, 0xdc, 0x0a, 0x1b, 0x3a, 0x07,
0xa5, 0xc9, 0x79, 0x63, 0x73, 0x3f, 0x01, 0xf0, 0x62, 0x2a, 0xf6, 0x30, 0xc3, 0x11, 0x3f, 0x76,
0xe0, 0x6d, 0x68, 0x70, 0x81, 0x99, 0xf0, 0x87, 0x2a, 0xcf, 0x50, 0x70, 0x8a, 0xe9, 0xd4, 0xd5,
0x63, 0xba, 0xba, 0x7b, 0x1f, 0xa0, 0x23, 0x18, 0x8d, 0xfb, 0xdb, 0x94, 0x0b, 0xf9, 0xae, 0xc7,
0x32, 0x4f, 0x82, 0x28, 0xb6, 0x6a, 0xc8, 0xac, 0xa6, 0xe8, 0x29, 0x9c, 0x9e, 0x9e, 0xeb, 0x50,
0xdf, 0xa7, 0x11, 0xd9, 0xa7, 0xbd, 0x87, 0x3b, 0xbc, 0x6f, 0x5f, 0x85, 0x52, 0x17, 0x73, 0x72,
0x22, 0x3d, 0x3b, 0xbc, 0xbf, 0x89, 0x39, 0x41, 0x2a, 0xd3, 0xfd, 0xd5, 0x82, 0x8b, 0x5b, 0x8c,
0x28, 0x31, 0x86, 0x21, 0xe9, 0x09, 0x9a, 0xc4, 0x88, 0x3c, 0x1a, 0x11, 0x2e, 0x9e, 0xbf, 0x9a,
0x7d, 0x11, 0x2a, 0x41, 0xd7, 0x8f, 0x71, 0x94, 0x92, 0x5d, 0x0e, 0xba, 0xbb, 0x38, 0x22, 0xf6,
0xeb, 0xb0, 0xd4, 0xcb, 0xea, 0xcb, 0x88, 0xd2, 0x40, 0x0d, 0x1d, 0x89, 0xca, 0x51, 0x05, 0x5d,
0xaf, 0xed, 0x94, 0xd4, 0x18, 0xd4, 0xb5, 0xed, 0x42, 0x63, 0x92, 0xe5, 0xb5, 0x9d, 0x05, 0x75,
0x6f, 0x26, 0x26, 0x49, 0xe5, 0xbd, 0x01, 0x89, 0xb0, 0x53, 0x6e, 0x5a, 0xad, 0x06, 0x32, 0x2b,
0xf7, 0x7b, 0x0b, 0xce, 0xb7, 0x59, 0x32, 0x7c, 0x91, 0xc1, 0xb9, 0x9f, 0x17, 0xe0, 0x82, 0x9e,
0xd1, 0x1e, 0x66, 0x82, 0xfe, 0x4d, 0x28, 0xde, 0x80, 0x33, 0x93, 0xb7, 0xea, 0x84, 0x7c, 0x18,
0xaf, 0xc1, 0xd2, 0x30, 0xed, 0x43, 0xe7, 0x95, 0x54, 0xde, 0x62, 0x16, 0x9d, 0x41, 0xbb, 0x70,
0x02, 0xda, 0x72, 0xce, 0x28, 0x9b, 0x50, 0xcf, 0x0a, 0x79, 0x6d, 0xa7, 0xa2, 0x52, 0xa6, 0x43,
0xee, 0x67, 0x05, 0x38, 0x27, 0x87, 0xfa, 0x2f, 0x1b, 0x92, 0x8d, 0xef, 0x0a, 0x60, 0x6b, 0x75,
0x78, 0x71, 0x40, 0x9e, 0xfc, 0x93, 0x5c, 0xbc, 0x04, 0x70, 0x40, 0x49, 0x18, 0x4c, 0xf3, 0x50,
0x53, 0x91, 0x3f, 0xc5, 0x81, 0x03, 0x15, 0x55, 0x24, 0xc3, 0x9f, 0x2e, 0xa5, 0x3f, 0xeb, 0xbd,
0xd3, 0xf8, 0x73, 0xf5, 0xd4, 0xfe, 0xac, 0x1e, 0x33, 0xfe, 0xfc, 0x53, 0x01, 0x16, 0xbd, 0x98,
0x13, 0x26, 0x5e, 0x00, 0x21, 0xad, 0x40, 0x8d, 0x93, 0x7e, 0x24, 0x37, 0xda, 0x94, 0xc9, 0x49,
0x40, 0xde, 0xed, 0x0d, 0x70, 0x1c, 0x93, 0xd0, 0x70, 0x59, 0x43, 0x93, 0x80, 0xfd, 0x32, 0x80,
0xa0, 0x11, 0xe1, 0x02, 0x47, 0x43, 0xee, 0x54, 0x9a, 0xc5, 0x56, 0x09, 0x4d, 0x45, 0xa4, 0x8b,
0xb2, 0xe4, 0xd0, 0x6b, 0x6b, 0x22, 0x8b, 0xc8, 0xac, 0xec, 0x77, 0xa1, 0xca, 0x92, 0x43, 0x3f,
0xc0, 0x02, 0x3b, 0x35, 0x45, 0xf1, 0xa5, 0x5c, 0x4a, 0x36, 0xc3, 0xa4, 0x8b, 0x2a, 0x2c, 0x39,
0x6c, 0x63, 0x81, 0xdd, 0xaf, 0x2d, 0x58, 0xec, 0x10, 0xcc, 0x7a, 0x83, 0xf9, 0x69, 0x7d, 0x13,
0x96, 0x19, 0xe1, 0xa3, 0x50, 0xf8, 0x13, 0x58, 0x9a, 0xdf, 0x33, 0x3a, 0xbe, 0x95, 0x81, 0x5b,
0x83, 0x85, 0x47, 0x23, 0xc2, 0xc6, 0xe6, 0x74, 0x71, 0x42, 0x87, 0x3a, 0xcf, 0xfd, 0x79, 0xaa,
0x3f, 0x59, 0x8a, 0xcf, 0xd1, 0xdf, 0x3c, 0x9b, 0x76, 0x2e, 0xa8, 0x62, 0x3e, 0xa8, 0xcb, 0x50,
0x8f, 0x88, 0x60, 0xb4, 0xe7, 0x8b, 0xf1, 0x30, 0x55, 0x04, 0xe8, 0xd0, 0xfe, 0x78, 0xa8, 0xbe,
0xa9, 0x01, 0x15, 0xdc, 0x59, 0x68, 0x16, 0x5b, 0x0d, 0xa4, 0xae, 0xdd, 0x1f, 0x2d, 0x58, 0x6c,
0x93, 0x90, 0x08, 0x32, 0x3f, 0xf1, 0x39, 0xb2, 0x2d, 0xe4, 0xca, 0x76, 0x46, 0x71, 0xc5, 0x93,
0x15, 0x57, 0x7a, 0x46, 0x71, 0x57, 0xa0, 0x31, 0x64, 0x34, 0xc2, 0x6c, 0xec, 0x3f, 0x24, 0x63,
0x0d, 0x43, 0xfa, 0x9b, 0x8e, 0xdd, 0x21, 0x63, 0xee, 0x7e, 0x63, 0x41, 0xf5, 0x56, 0x38, 0xe2,
0x83, 0xb9, 0x0e, 0x38, 0xb3, 0xdf, 0x4b, 0xe1, 0xe8, 0xf7, 0x72, 0xd4, 0x7e, 0x8a, 0x39, 0xf6,
0xe3, 0x42, 0x23, 0xfb, 0x04, 0xf7, 0x71, 0xdf, 0x0c, 0x61, 0x26, 0xe6, 0xfe, 0x66, 0x41, 0x6d,
0x3b, 0xc1, 0x81, 0xb2, 0xe0, 0xbf, 0xbc, 0xcb, 0x15, 0x98, 0xb8, 0x68, 0xca, 0xf1, 0xc4, 0x56,
0xa7, 0xec, 0xb1, 0x34, 0x6b, 0x8f, 0x97, 0xa1, 0x4e, 0x65, 0x43, 0xfe, 0x10, 0x8b, 0x81, 0x26,
0xb7, 0x86, 0x40, 0x85, 0xf6, 0x64, 0x44, 0xfa, 0x67, 0x9a, 0xa0, 0xfc, 0xb3, 0x7c, 0x6a, 0xff,
0x34, 0x45, 0x94, 0x7f, 0x32, 0x70, 0x3a, 0xba, 0x57, 0x29, 0x74, 0xca, 0x05, 0xed, 0xf1, 0xbb,
0xc3, 0x40, 0x1d, 0xfa, 0x57, 0xa0, 0xd6, 0xc9, 0x80, 0xe9, 0xc3, 0xf5, 0x24, 0x20, 0xe5, 0xb1,
0x43, 0xa2, 0x84, 0x8d, 0x3b, 0xf4, 0x29, 0x31, 0xb8, 0xa7, 0x22, 0x12, 0xda, 0xee, 0x28, 0x42,
0xc9, 0x21, 0x37, 0x93, 0x49, 0x97, 0xee, 0x97, 0x16, 0x9c, 0x7d, 0xe6, 0xa5, 0x73, 0x10, 0x7f,
0x07, 0xaa, 0x1d, 0xd2, 0x97, 0x25, 0xd2, 0xd3, 0xfd, 0xda, 0x71, 0x3f, 0xde, 0x8e, 0x81, 0x88,
0xb2, 0x02, 0xee, 0xa7, 0x96, 0xfc, 0x55, 0x11, 0x90, 0x27, 0x6a, 0xf9, 0x0c, 0xbb, 0xd6, 0x3c,
0xec, 0xda, 0x57, 0xe1, 0x5c, 0x3c, 0x8a, 0x7c, 0x46, 0x42, 0x2c, 0x48, 0xe0, 0x1b, 0x55, 0x70,
0xc3, 0x96, 0x1d, 0x8f, 0x22, 0xa4, 0x6f, 0x99, 0x06, 0xb9, 0xfb, 0x85, 0x05, 0x70, 0x4b, 0x4a,
0x40, 0xb7, 0x71, 0x54, 0xe3, 0xd6, 0xc9, 0x5b, 0x6c, 0x61, 0x56, 0x43, 0x9b, 0xa9, 0x86, 0xb8,
0xe2, 0xa8, 0x98, 0x87, 0x21, 0xe3, 0x68, 0x02, 0xde, 0xc8, 0x4c, 0xf3, 0xf2, 0x95, 0x05, 0x8d,
0x29, 0xfa, 0xf8, 0xac, 0xdc, 0xad, 0xa3, 0x72, 0x57, 0xa6, 0x27, 0x35, 0xe0, 0xf3, 0x29, 0x59,
0x44, 0x13, 0x59, 0x5c, 0x82, 0xaa, 0xa2, 0x64, 0x4a, 0x17, 0xb1, 0xd6, 0x85, 0xfd, 0x36, 0x9c,
0x65, 0xa4, 0x47, 0x62, 0x11, 0x8e, 0xfd, 0x28, 0x09, 0xe8, 0x01, 0x25, 0x81, 0xfa, 0x2c, 0xaa,
0x68, 0x39, 0xbd, 0xb1, 0x63, 0xe2, 0xee, 0x0f, 0x16, 0x2c, 0x7d, 0x2c, 0xf7, 0x02, 0xf9, 0x13,
0x53, 0x77, 0xf6, 0xfc, 0x0a, 0xba, 0xa1, 0xb0, 0x18, 0x7a, 0xb4, 0x84, 0x5e, 0xf9, 0x63, 0x09,
0x71, 0x54, 0xe5, 0x46, 0x36, 0x92, 0x62, 0x7d, 0x6c, 0x3a, 0x0d, 0xc5, 0x93, 0xc1, 0x22, 0x7d,
0xd8, 0xd2, 0x14, 0x07, 0x50, 0xdf, 0xe1, 0xfd, 0xbd, 0x84, 0x2b, 0x4b, 0x92, 0xbe, 0x6a, 0x4c,
0x58, 0x7b, 0xb7, 0xa5, 0x4c, 0xa3, 0x6e, 0x62, 0xca, 0x36, 0xce, 0xc1, 0x42, 0xc4, 0xfb, 0xd9,
0x7e, 0xaa, 0x17, 0x72, 0x32, 0x99, 0x3d, 0x2b, 0x6e, 0x4b, 0x68, 0x12, 0x78, 0xeb, 0x7d, 0xa8,
0x65, 0xff, 0x61, 0xd8, 0xcb, 0xd0, 0xf0, 0x76, 0xbd, 0x7d, 0x6f, 0x63, 0xdb, 0x7b, 0xe0, 0xed,
0x7e, 0xb8, 0xfc, 0x1f, 0xbb, 0x0e, 0x95, 0xdb, 0x37, 0x37, 0xb6, 0xf7, 0x6f, 0xdf, 0x5f, 0xb6,
0xec, 0x06, 0x54, 0x37, 0x36, 0x77, 0x3f, 0x42, 0x3b, 0x1b, 0xdb, 0xcb, 0x85, 0xcd, 0x9b, 0x0f,
0xb6, 0xfa, 0x54, 0x0c, 0x46, 0x5d, 0x49, 0xe2, 0xda, 0x53, 0x1a, 0x86, 0xf4, 0xa9, 0x20, 0xbd,
0xc1, 0x9a, 0x46, 0xf9, 0x4e, 0x40, 0xb9, 0x60, 0xb4, 0x3b, 0x12, 0x24, 0x58, 0x4b, 0xb1, 0xae,
0x29, 0xe8, 0xd9, 0x72, 0xd8, 0x5d, 0xef, 0x96, 0x55, 0xe8, 0xda, 0xef, 0x01, 0x00, 0x00, 0xff,
0xff, 0x61, 0xa6, 0x06, 0xaf, 0xe9, 0x11, 0x00, 0x00,
}

View File

@ -3,6 +3,7 @@ package timesync
import (
"context"
"log"
"sync"
ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
@ -10,77 +11,77 @@ import (
)
type MsgProducer struct {
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
ttBarrier TimeTickBarrier
ctx context.Context
cancel context.CancelFunc
watchers []TimeTickWatcher
watchers []TimeTickWatcher
}
func NewTimeSyncMsgProducer(ctx context.Context, ttBarrier TimeTickBarrier, watchers ...TimeTickWatcher) (*MsgProducer, error) {
childCtx, cancelFunc := context.WithCancel(ctx)
func NewTimeSyncMsgProducer(ttBarrier TimeTickBarrier, watchers ...TimeTickWatcher) (*MsgProducer, error) {
return &MsgProducer{
ctx: childCtx,
cancel: cancelFunc,
ttBarrier: ttBarrier,
watchers: watchers,
}, nil
}
func (producer *MsgProducer) broadcastMsg() {
defer producer.wg.Done()
for {
select {
case <-producer.ctx.Done():
log.Printf("broadcast context done, exit")
return
default:
tt, err := producer.ttBarrier.GetTimeTick()
if err != nil {
log.Printf("broadcast get time tick error")
}
baseMsg := ms.BaseMsg{
BeginTimestamp: tt,
EndTimestamp: tt,
HashValues: []uint32{0},
}
timeTickResult := internalpb2.TimeTickMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kTimeTick,
MsgID: 0,
Timestamp: tt,
SourceID: 0,
},
}
timeTickMsg := &ms.TimeTickMsg{
BaseMsg: baseMsg,
TimeTickMsg: timeTickResult,
}
for _, watcher := range producer.watchers {
watcher.Watch(timeTickMsg)
}
}
tt, err := producer.ttBarrier.GetTimeTick()
if err != nil {
log.Printf("broadcast get time tick error")
}
baseMsg := ms.BaseMsg{
BeginTimestamp: tt,
EndTimestamp: tt,
HashValues: []uint32{0},
}
timeTickResult := internalpb2.TimeTickMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kTimeTick,
MsgID: 0,
Timestamp: tt,
SourceID: 0,
},
}
timeTickMsg := &ms.TimeTickMsg{
BaseMsg: baseMsg,
TimeTickMsg: timeTickResult,
}
for _, watcher := range producer.watchers {
watcher.Watch(timeTickMsg)
}
}
}
func (producer *MsgProducer) Start() error {
err := producer.ttBarrier.Start()
if err != nil {
return err
}
func (producer *MsgProducer) Start(ctx context.Context) {
producer.ctx, producer.cancel = context.WithCancel(ctx)
producer.wg.Add(2 + len(producer.watchers))
go producer.startTTBarrier()
for _, watcher := range producer.watchers {
watcher.Start()
go producer.startWatcher(watcher)
}
go producer.broadcastMsg()
}
return nil
func (producer *MsgProducer) startTTBarrier() {
defer producer.wg.Done()
producer.ttBarrier.StartBackgroundLoop(producer.ctx)
}
func (producer *MsgProducer) startWatcher(watcher TimeTickWatcher) {
defer producer.wg.Done()
watcher.StartBackgroundLoop(producer.ctx)
}
func (producer *MsgProducer) Close() {
producer.cancel()
producer.ttBarrier.Close()
for _, watcher := range producer.watchers {
watcher.Close()
}
producer.wg.Wait()
}

View File

@ -18,8 +18,7 @@ type (
TimeTickBarrier interface {
GetTimeTick() (Timestamp, error)
Start() error
Close()
StartBackgroundLoop(ctx context.Context)
}
softTimeTickBarrier struct {
@ -29,7 +28,6 @@ type (
outTt chan Timestamp
ttStream ms.MsgStream
ctx context.Context
cancel context.CancelFunc
}
hardTimeTickBarrier struct {
@ -37,10 +35,30 @@ type (
outTt chan Timestamp
ttStream ms.MsgStream
ctx context.Context
cancel context.CancelFunc
}
)
func NewSoftTimeTickBarrier(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.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) GetTimeTick() (Timestamp, error) {
select {
case <-ttBarrier.ctx.Done():
@ -61,72 +79,39 @@ func (ttBarrier *softTimeTickBarrier) GetTimeTick() (Timestamp, error) {
}
}
func (ttBarrier *softTimeTickBarrier) Start() error {
go func() {
for {
select {
case <-ttBarrier.ctx.Done():
log.Printf("[TtBarrierStart] %s\n", ttBarrier.ctx.Err())
return
func (ttBarrier *softTimeTickBarrier) StartBackgroundLoop(ctx context.Context) {
ttBarrier.ctx = ctx
for {
select {
case <-ctx.Done():
log.Printf("[TtBarrierStart] %s\n", 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.Base.SourceID]
// log.Printf("[softTimeTickBarrier] peer(%d)=%d\n", ttmsg.PeerID, ttmsg.Timestamp)
case ttmsgs := <-ttBarrier.ttStream.Chan():
if len(ttmsgs.Msgs) > 0 {
for _, timetickmsg := range ttmsgs.Msgs {
ttmsg := timetickmsg.(*ms.TimeTickMsg)
oldT, ok := ttBarrier.peer2LastTt[ttmsg.Base.SourceID]
// log.Printf("[softTimeTickBarrier] peer(%d)=%d\n", ttmsg.PeerID, ttmsg.Timestamp)
if !ok {
log.Printf("[softTimeTickBarrier] Warning: peerID %d not exist\n", ttmsg.Base.SourceID)
continue
}
if ttmsg.Base.Timestamp > oldT {
ttBarrier.peer2LastTt[ttmsg.Base.SourceID] = ttmsg.Base.Timestamp
if !ok {
log.Printf("[softTimeTickBarrier] Warning: peerID %d not exist\n", ttmsg.Base.SourceID)
// get a legal Timestamp
ts := ttBarrier.minTimestamp()
lastTt := atomic.LoadInt64(&(ttBarrier.lastTt))
if lastTt != 0 && ttBarrier.minTtInterval > ts-Timestamp(lastTt) {
continue
}
if ttmsg.Base.Timestamp > oldT {
ttBarrier.peer2LastTt[ttmsg.Base.SourceID] = ttmsg.Base.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
}
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 {
@ -151,48 +136,45 @@ func (ttBarrier *hardTimeTickBarrier) GetTimeTick() (Timestamp, error) {
}
}
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
func (ttBarrier *hardTimeTickBarrier) StartBackgroundLoop(ctx context.Context) {
ttBarrier.ctx = ctx
// Last timestamp synchronized
state := Timestamp(0)
for {
select {
case <-ctx.Done():
log.Printf("[TtBarrierStart] %s\n", ctx.Err())
return
case ttmsgs := <-ttBarrier.ttStream.Chan():
if len(ttmsgs.Msgs) > 0 {
for _, timetickmsg := range ttmsgs.Msgs {
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)
// 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.Base.SourceID]
if !ok {
log.Printf("[hardTimeTickBarrier] Warning: peerID %d not exist\n", ttmsg.Base.SourceID)
continue
}
oldT, ok := ttBarrier.peer2Tt[ttmsg.Base.SourceID]
if !ok {
log.Printf("[hardTimeTickBarrier] Warning: peerID %d not exist\n", ttmsg.Base.SourceID)
continue
}
if oldT > state {
log.Printf("[hardTimeTickBarrier] Warning: peer(%d) timestamp(%d) ahead\n",
ttmsg.Base.SourceID, ttmsg.Base.Timestamp)
}
if oldT > state {
log.Printf("[hardTimeTickBarrier] Warning: peer(%d) timestamp(%d) ahead\n",
ttmsg.Base.SourceID, ttmsg.Base.Timestamp)
}
ttBarrier.peer2Tt[ttmsg.Base.SourceID] = ttmsg.Base.Timestamp
ttBarrier.peer2Tt[ttmsg.Base.SourceID] = ttmsg.Base.Timestamp
newState := ttBarrier.minTimestamp()
if newState > state {
ttBarrier.outTt <- newState
state = newState
}
newState := ttBarrier.minTimestamp()
if newState > state {
ttBarrier.outTt <- newState
state = newState
}
}
}
}
}()
return nil
}
}
func (ttBarrier *hardTimeTickBarrier) minTimestamp() Timestamp {
@ -205,10 +187,7 @@ func (ttBarrier *hardTimeTickBarrier) minTimestamp() Timestamp {
return tempMin
}
func newHardTimeTickBarrier(ctx context.Context,
ttStream *ms.MsgStream,
peerIds []UniqueID) *hardTimeTickBarrier {
func NewHardTimeTickBarrier(ttStream *ms.MsgStream, peerIds []UniqueID) *hardTimeTickBarrier {
if len(peerIds) <= 0 {
log.Printf("[newSoftTimeTickBarrier] Error: peerIds is empty!")
return nil
@ -217,7 +196,6 @@ func newHardTimeTickBarrier(ctx context.Context,
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 {
@ -229,7 +207,3 @@ func newHardTimeTickBarrier(ctx context.Context,
return &sttbarrier
}
func (ttBarrier *hardTimeTickBarrier) Close() {
ttBarrier.cancel()
}

View File

@ -1,6 +1,7 @@
package timesync
import (
"context"
"log"
ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
@ -8,26 +9,34 @@ import (
type TimeTickWatcher interface {
Watch(msg *ms.TimeTickMsg)
Start()
Close()
StartBackgroundLoop(ctx context.Context)
}
type MsgTimeTickWatcher struct {
streams []ms.MsgStream
streams []ms.MsgStream
msgQueue chan *ms.TimeTickMsg
}
func (watcher *MsgTimeTickWatcher) Watch(msg *ms.TimeTickMsg) {
msgPack := &ms.MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, msg)
for _, stream := range watcher.streams {
if err := stream.Broadcast(msgPack); err != nil {
log.Printf("stream broadcast failed %s", err.Error())
}
}
watcher.msgQueue <- msg
}
func (watcher *MsgTimeTickWatcher) Start() {
func (watcher *MsgTimeTickWatcher) StartBackgroundLoop(ctx context.Context) {
for {
select {
case <-ctx.Done():
log.Println("msg time tick watcher closed")
return
case msg := <-watcher.msgQueue:
msgPack := &ms.MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, msg)
for _, stream := range watcher.streams {
if err := stream.Broadcast(msgPack); err != nil {
log.Printf("stream broadcast failed %s", err.Error())
}
}
}
}
}
func (watcher *MsgTimeTickWatcher) Close() {