DataNode scales flowgraph

Signed-off-by: yangxuan <xuan.yang@zilliz.com>
pull/5779/head
sunby 2021-05-25 15:35:37 +08:00 committed by zhenshan.cao
parent b9cf4c1095
commit c53afee616
19 changed files with 872 additions and 470 deletions

View File

@ -139,7 +139,6 @@ message VchannelPair {
PositionPair dml_position = 5;
}
message WatchDmChannelsRequest {
common.MsgBase base = 1;
repeated VchannelPair vchannels = 2;
@ -152,8 +151,10 @@ DataNode consists of multiple DataSyncService, each service controls one flowgra
// DataNode
type DataNode struct {
...
coll2Sync map[UniqueID]*dataSyncService
vchan2Sync map[string]*dataSyncService
vchan2FlushCh map[string]chan<- *flushMsg
...
replica Replica // TODO remove
}
// DataSyncService
@ -165,10 +166,34 @@ type dataSyncService struct {
idAllocator allocatorInterface
msFactory msgstream.Factory
collectionID UniqueID
segmentIDs []UniqueID
segmentIDs []UniqueID // getSegmentIDs() of Replica
}
```
DataNode Init -> Resigter to Etcd -> Discovery data service -> Discover master service -> IDLE
WatchDmChannels -> new dataSyncService -> HEALTH
```proto
message WatchDmChannelsRequest {
common.MsgBase base = 1;
repeated VchannelPair vchannels = 2;
}
```
`WatchDmChannels:`
1. If `DataNode.vchan2Sync` is empty, DataNode is in IDLE, `WatchDmChannels` will create new dataSyncService for every unique vchannel, then DataNode is in HEALTH.
2. If vchannel name of `VchannelPair` is not in `DataNode.vchan2Sync`, create a new dataSyncService.
3. If vchannel name of `VchannelPair` is in `DataNode.vchan2Sync`, ignore.
`newDataSyncService:`
```go
func newDataSyncService(ctx context.Context, flushChan <-chan *flushMsg, replica Replica,
alloc allocatorInterface, factory msgstream.Factory, vchanPair *datapb.VchannelPair) *dataSyncService
```
#### The boring design
• If collection:flowgraph = 1 : 1, datanode must have ability to scale flowgraph.

View File

@ -44,6 +44,7 @@ type Replica interface {
getSegmentByID(segmentID UniqueID) (*Segment, error)
bufferAutoFlushBinlogPaths(segmentID UniqueID, field2Path map[UniqueID]string) error
getBufferPaths(segID UniqueID) (map[UniqueID][]string, error)
getChannelName(segID UniqueID) (string, error)
}
// Segment is the data structure of segments in data node replica.
@ -85,6 +86,18 @@ func newReplica() Replica {
return replica
}
func (replica *CollectionSegmentReplica) getChannelName(segID UniqueID) (string, error) {
replica.mu.RLock()
defer replica.mu.RUnlock()
seg, ok := replica.segments[segID]
if !ok {
return "", fmt.Errorf("Cannot find segment, id = %v", segID)
}
return seg.channelName, nil
}
// bufferAutoFlushBinlogPaths buffers binlog paths generated by auto-flush
func (replica *CollectionSegmentReplica) bufferAutoFlushBinlogPaths(segID UniqueID, field2Path map[UniqueID]string) error {
replica.mu.RLock()

View File

@ -21,6 +21,7 @@ import (
"io"
"math/rand"
"strconv"
"sync"
"sync/atomic"
"time"
@ -46,18 +47,14 @@ const (
// services of data node.
//
// DataNode struct implements `types.Component`, `types.DataNode` interfaces.
// `dataSyncService` controls flowgraph in datanode.
// `metaService` initialize collections from master service when data node starts.
// `masterService` holds a grpc client of master service.
// `dataService` holds a grpc client of data service.
// `NodeID` is unique to each data node.
// `State` is current statement of this data node, indicating whether it's healthy.
//
// `NodeID` is unique to each data node.
//
// `State` is current statement of this data node, indicating whether it's healthy.
//
// `flushChan` transfer flush messages from data service to flowgraph of data node.
//
// `replica` holds replications of persistent data, including collections and segments.
// `vchan2SyncService` holds map of vchannlName and dataSyncService, so that datanode
// has ability to scale flowgraph
// `vchan2FlushCh` holds flush-signal channels for every flowgraph
type DataNode struct {
ctx context.Context
cancel context.CancelFunc
@ -66,17 +63,14 @@ type DataNode struct {
State atomic.Value // internalpb.StateCode_Initializing
watchDm chan struct{}
dataSyncService *dataSyncService
metaService *metaService
vchan2SyncService map[string]*dataSyncService
vchan2FlushCh map[string]chan<- *flushMsg
masterService types.MasterService
dataService types.DataService
session *sessionutil.Session
flushChan chan<- *flushMsg
replica Replica
closer io.Closer
msFactory msgstream.Factory
@ -92,12 +86,12 @@ func NewDataNode(ctx context.Context, factory msgstream.Factory) *DataNode {
Role: typeutil.DataNodeRole,
watchDm: make(chan struct{}, 1),
dataSyncService: nil,
metaService: nil,
masterService: nil,
dataService: nil,
replica: nil,
msFactory: factory,
masterService: nil,
dataService: nil,
msFactory: factory,
vchan2SyncService: make(map[string]*dataSyncService),
vchan2FlushCh: make(map[string]chan<- *flushMsg),
}
node.UpdateStateCode(internalpb.StateCode_Abnormal)
return node
@ -139,13 +133,14 @@ func (node *DataNode) Register() error {
// and address. Therefore, `SetDataServiceInterface()` must be called before this func.
// Registering return several channel names data node need.
//
// After registering, data node will wait until data service calls `WatchDmChannels`
// for `RPCConnectionTimeout` ms.
//
// At last, data node initializes its `dataSyncService` and `metaService`.
func (node *DataNode) Init() error {
ctx := context.Background()
node.session = sessionutil.NewSession(ctx, []string{Params.EtcdAddress})
node.session.Init(typeutil.DataNodeRole, Params.IP+":"+strconv.Itoa(Params.Port), false)
// TODO find DataService & MasterService
req := &datapb.RegisterNodeRequest{
Base: &commonpb.MsgBase{
SourceID: node.NodeID,
@ -187,35 +182,37 @@ func (node *DataNode) Init() error {
zap.Any("TimeTickChannelName", Params.TimeTickChannelName),
zap.Any("CompleteFlushChannelName", Params.TimeTickChannelName))
select {
case <-time.After(RPCConnectionTimeout):
return errors.New("Get DmChannels failed in 30 seconds")
case <-node.watchDm:
log.Debug("insert channel names set")
}
replica := newReplica()
var alloc allocatorInterface = newAllocator(node.masterService)
chanSize := 100
flushChan := make(chan *flushMsg, chanSize)
node.flushChan = flushChan
node.dataSyncService = newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory)
node.dataSyncService.init()
node.metaService = newMetaService(node.ctx, replica, node.masterService)
node.replica = replica
return nil
}
// Start `metaService` and `dataSyncService` and update state to HEALTHY
// NewDataSyncService adds a new dataSyncService to DataNode
func (node *DataNode) NewDataSyncService(vchanPair *datapb.VchannelPair) error {
if _, ok := node.vchan2SyncService[vchanPair.GetDmlVchannelName()]; ok {
return nil
}
replica := newReplica()
var alloc allocatorInterface = newAllocator(node.masterService)
flushChan := make(chan *flushMsg, 100)
dataSyncService := newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory, vchanPair)
metaService := newMetaService(node.ctx, replica, node.masterService)
node.vchan2SyncService[vchanPair.GetDmlVchannelName()] = dataSyncService
node.vchan2FlushCh[vchanPair.GetDmlVchannelName()] = flushChan
metaService.init()
go dataSyncService.start()
return nil
}
// Start will update state to HEALTHY
func (node *DataNode) Start() error {
node.metaService.init()
go node.dataSyncService.start()
node.UpdateStateCode(internalpb.StateCode_Healthy)
return nil
}
// UpdateStateCode update datanode's state code
func (node *DataNode) UpdateStateCode(code internalpb.StateCode) {
node.State.Store(code)
}
@ -227,18 +224,20 @@ func (node *DataNode) WatchDmChannels(ctx context.Context, in *datapb.WatchDmCha
}
switch {
case node.State.Load() != internalpb.StateCode_Initializing:
case node.State.Load() != internalpb.StateCode_Healthy:
status.Reason = fmt.Sprintf("DataNode %d not initializing!", node.NodeID)
return status, errors.New(status.GetReason())
case len(Params.InsertChannelNames) != 0:
status.Reason = fmt.Sprintf("DataNode %d has already set insert channels!", node.NodeID)
case len(in.GetVchannels()) == 0:
status.Reason = "Illegal request"
return status, errors.New(status.GetReason())
default:
Params.InsertChannelNames = in.GetChannelNames()
for _, chanPair := range in.GetVchannels() {
node.NewDataSyncService(chanPair)
}
status.ErrorCode = commonpb.ErrorCode_Success
node.watchDm <- struct{}{}
return status, nil
}
}
@ -257,32 +256,97 @@ func (node *DataNode) GetComponentStates(ctx context.Context) (*internalpb.Compo
return states, nil
}
func (node *DataNode) getChannelName(segID UniqueID) string {
for name, dataSync := range node.vchan2SyncService {
if dataSync.replica.hasSegment(segID) {
return name
}
}
return ""
}
// FlushSegments packs flush messages into flowgraph through flushChan.
func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) {
log.Debug("FlushSegments ...", zap.Int("num", len(req.SegmentIDs)))
ids := make([]UniqueID, 0)
ids = append(ids, req.SegmentIDs...)
flushmsg := &flushMsg{
msgID: req.Base.MsgID,
timestamp: req.Base.Timestamp,
segmentIDs: ids,
collectionID: req.CollectionID,
status := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
}
node.flushChan <- flushmsg
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
}, nil
for _, id := range req.SegmentIDs {
chanName := node.getChannelName(id)
if chanName == "" {
status.Reason = fmt.Sprintf("DataNode not find segment %d!", id)
return status, errors.New(status.GetReason())
}
flushCh, ok := node.vchan2FlushCh[chanName]
if !ok {
status.Reason = "DataNode abnormal!"
return status, errors.New(status.GetReason())
}
ddlFlushedCh := make(chan bool)
dmlFlushedCh := make(chan bool)
flushmsg := &flushMsg{
msgID: req.Base.MsgID,
timestamp: req.Base.Timestamp,
segmentID: id,
collectionID: req.CollectionID,
ddlFlushedCh: ddlFlushedCh,
dmlFlushedCh: dmlFlushedCh,
}
flushCh <- flushmsg
// GOOSE TODO get binlog paths.
waitReceive := func(wg *sync.WaitGroup, flushedCh <-chan bool, req *datapb.SaveBinlogPathsRequest) {
defer wg.Done()
select {
case <-time.After(300 * time.Second):
return
case isFlushed := <-flushedCh:
if isFlushed {
log.Debug("Yeah! It's safe to notify dataservice")
}
}
}
// TODO make a queue for this func
currentSegID := id
go func() {
log.Info("Waiting for flush completed", zap.Int64("segmentID", currentSegID))
req := &datapb.SaveBinlogPathsRequest{
Base: &commonpb.MsgBase{},
SegmentID: currentSegID,
CollectionID: req.CollectionID,
}
var wg sync.WaitGroup
wg.Add(1)
go waitReceive(&wg, ddlFlushedCh, req)
wg.Add(1)
go waitReceive(&wg, dmlFlushedCh, req)
wg.Wait()
// TODO
//status := node.dataService.SaveBinlogPaths(req)
log.Info("Flush Completed", zap.Int64("segmentID", currentSegID))
}()
}
status.ErrorCode = commonpb.ErrorCode_Success
return status, nil
}
func (node *DataNode) Stop() error {
node.cancel()
// close services
if node.dataSyncService != nil {
(*node.dataSyncService).close()
for _, syncService := range node.vchan2SyncService {
if syncService != nil {
(*syncService).close()
}
}
if node.closer != nil {

View File

@ -12,10 +12,16 @@
package datanode
import (
"math"
"os"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
@ -24,31 +30,56 @@ import (
func TestMain(t *testing.M) {
Params.Init()
refreshChannelNames()
code := t.Run()
os.Exit(code)
}
func TestDataNode(t *testing.T) {
node := newDataNodeMock()
node := newIDLEDataNodeMock()
node.Start()
t.Run("Test WatchDmChannels", func(t *testing.T) {
channelNames := Params.InsertChannelNames
req := &datapb.WatchDmChannelsRequest{
ChannelNames: channelNames,
node1 := newIDLEDataNodeMock()
node1.Start()
vchannels := []*datapb.VchannelPair{}
for _, ch := range Params.InsertChannelNames {
log.Debug("InsertChannels", zap.String("name", ch))
vpair := &datapb.VchannelPair{
CollectionID: 1,
DmlVchannelName: ch,
DdlVchannelName: Params.DDChannelNames[0],
DdlPosition: &datapb.PositionPair{},
DmlPosition: &datapb.PositionPair{},
}
vchannels = append(vchannels, vpair)
}
req := &datapb.WatchDmChannelsRequest{
Base: &commonpb.MsgBase{
MsgType: 0,
MsgID: 0,
Timestamp: 0,
SourceID: Params.NodeID,
},
Vchannels: vchannels,
}
_, err1 := node.WatchDmChannels(node.ctx, req)
assert.Error(t, err1)
node.UpdateStateCode(internalpb.StateCode_Initializing)
_, err2 := node.WatchDmChannels(node.ctx, req)
assert.Error(t, err2)
_, err := node1.WatchDmChannels(node.ctx, req)
assert.NoError(t, err)
assert.NotNil(t, node1.vchan2FlushCh)
assert.NotNil(t, node1.vchan2SyncService)
sync, ok := node1.vchan2SyncService[Params.InsertChannelNames[0]]
assert.True(t, ok)
assert.NotNil(t, sync)
assert.Equal(t, UniqueID(1), sync.collectionID)
assert.Equal(t, 2, len(node1.vchan2SyncService))
assert.Equal(t, len(node1.vchan2FlushCh), len(node1.vchan2SyncService))
Params.InsertChannelNames = []string{}
status, err3 := node.WatchDmChannels(node.ctx, req)
assert.NoError(t, err3)
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
_, err = node1.WatchDmChannels(node1.ctx, req)
assert.NoError(t, err)
assert.Equal(t, 2, len(node1.vchan2SyncService))
Params.InsertChannelNames = channelNames
<-node1.ctx.Done()
node1.Stop()
})
t.Run("Test GetComponentStates", func(t *testing.T) {
@ -57,16 +88,102 @@ func TestDataNode(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, stat.Status.ErrorCode)
})
t.Run("Test NewDataSyncService", func(t *testing.T) {
node2 := newIDLEDataNodeMock()
node2.Start()
dmChannelName := "fake-dm-channel-test-NewDataSyncService"
ddChannelName := "fake-dd-channel-test-NewDataSyncService"
vpair := &datapb.VchannelPair{
CollectionID: 1,
DmlVchannelName: dmChannelName,
DdlVchannelName: ddChannelName,
DdlPosition: &datapb.PositionPair{},
DmlPosition: &datapb.PositionPair{},
}
require.Equal(t, 0, len(node2.vchan2FlushCh))
require.Equal(t, 0, len(node2.vchan2SyncService))
err := node2.NewDataSyncService(vpair)
assert.NoError(t, err)
assert.Equal(t, 1, len(node2.vchan2FlushCh))
assert.Equal(t, 1, len(node2.vchan2SyncService))
err = node2.NewDataSyncService(vpair)
assert.NoError(t, err)
assert.Equal(t, 1, len(node2.vchan2FlushCh))
assert.Equal(t, 1, len(node2.vchan2SyncService))
<-node2.ctx.Done()
node2.Stop()
})
t.Run("Test FlushSegments", func(t *testing.T) {
dmChannelName := "fake-dm-channel-test-HEALTHDataNodeMock"
ddChannelName := "fake-dd-channel-test-HEALTHDataNodeMock"
node1 := newHEALTHDataNodeMock(dmChannelName, ddChannelName)
sync, ok := node1.vchan2SyncService[dmChannelName]
assert.True(t, ok)
sync.replica.addSegment(0, 1, 1, dmChannelName)
sync.replica.addSegment(1, 1, 1, dmChannelName)
req := &datapb.FlushSegmentsRequest{
Base: &commonpb.MsgBase{},
DbID: 0,
CollectionID: 1,
SegmentIDs: []int64{0, 1},
}
status, err := node.FlushSegments(node.ctx, req)
status, err := node1.FlushSegments(node.ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
timeTickMsgPack := msgstream.MsgPack{}
timeTickMsg := &msgstream.TimeTickMsg{
BaseMsg: msgstream.BaseMsg{
BeginTimestamp: Timestamp(0),
EndTimestamp: Timestamp(0),
HashValues: []uint32{0},
},
TimeTickMsg: internalpb.TimeTickMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_TimeTick,
MsgID: UniqueID(0),
Timestamp: math.MaxUint64,
SourceID: 0,
},
},
}
timeTickMsgPack.Msgs = append(timeTickMsgPack.Msgs, timeTickMsg)
// pulsar produce
msFactory := msgstream.NewPmsFactory()
m := map[string]interface{}{
"pulsarAddress": Params.PulsarAddress,
"receiveBufSize": 1024,
"pulsarBufSize": 1024}
err = msFactory.SetParams(m)
assert.NoError(t, err)
insertStream, _ := msFactory.NewMsgStream(node1.ctx)
insertStream.AsProducer([]string{dmChannelName})
ddStream, _ := msFactory.NewMsgStream(node1.ctx)
ddStream.AsProducer([]string{ddChannelName})
var insertMsgStream msgstream.MsgStream = insertStream
insertMsgStream.Start()
var ddMsgStream msgstream.MsgStream = ddStream
ddMsgStream.Start()
err = insertMsgStream.Broadcast(&timeTickMsgPack)
assert.NoError(t, err)
err = ddMsgStream.Broadcast(&timeTickMsgPack)
assert.NoError(t, err)
<-node1.ctx.Done()
node1.Stop()
})
t.Run("Test GetTimeTickChannel", func(t *testing.T) {

View File

@ -18,6 +18,7 @@ import (
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/internal/util/retry"
"go.etcd.io/etcd/clientv3"
@ -26,35 +27,44 @@ import (
)
type dataSyncService struct {
ctx context.Context
fg *flowgraph.TimeTickedFlowGraph
flushChan <-chan *flushMsg
replica Replica
idAllocator allocatorInterface
msFactory msgstream.Factory
ctx context.Context
fg *flowgraph.TimeTickedFlowGraph
flushChan <-chan *flushMsg
replica Replica
idAllocator allocatorInterface
msFactory msgstream.Factory
collectionID UniqueID
}
func newDataSyncService(ctx context.Context, flushChan <-chan *flushMsg,
replica Replica, alloc allocatorInterface, factory msgstream.Factory) *dataSyncService {
func newDataSyncService(ctx context.Context,
flushChan <-chan *flushMsg,
replica Replica,
alloc allocatorInterface,
factory msgstream.Factory,
vchanPair *datapb.VchannelPair) *dataSyncService {
service := &dataSyncService{
ctx: ctx,
fg: nil,
flushChan: flushChan,
replica: replica,
idAllocator: alloc,
msFactory: factory,
ctx: ctx,
fg: nil,
flushChan: flushChan,
replica: replica,
idAllocator: alloc,
msFactory: factory,
collectionID: vchanPair.GetCollectionID(),
}
service.initNodes(vchanPair)
return service
}
func (dsService *dataSyncService) init() {
if len(Params.InsertChannelNames) == 0 {
log.Error("InsertChannels not readly, init datasync service failed")
return
}
// func (dsService *dataSyncService) init() {
// if len(Params.InsertChannelNames) == 0 {
// log.Error("InsertChannels not readly, init datasync service failed")
// return
// }
dsService.initNodes()
}
// dsService.initNodes()
// }
func (dsService *dataSyncService) start() {
log.Debug("Data Sync Service Start Successfully")
@ -71,7 +81,7 @@ func (dsService *dataSyncService) close() {
}
}
func (dsService *dataSyncService) initNodes() {
func (dsService *dataSyncService) initNodes(vchanPair *datapb.VchannelPair) {
// TODO: add delete pipeline support
var kvClient *clientv3.Client
var err error
@ -96,14 +106,16 @@ func (dsService *dataSyncService) initNodes() {
m := map[string]interface{}{
"PulsarAddress": Params.PulsarAddress,
"ReceiveBufSize": 1024,
"PulsarBufSize": 1024}
"PulsarBufSize": 1024,
}
err = dsService.msFactory.SetParams(m)
if err != nil {
panic(err)
}
var dmStreamNode Node = newDmInputNode(dsService.ctx, dsService.msFactory)
var ddStreamNode Node = newDDInputNode(dsService.ctx, dsService.msFactory)
var dmStreamNode Node = newDmInputNode(dsService.ctx, dsService.msFactory, vchanPair.GetDmlVchannelName(), vchanPair.GetDmlPosition())
var ddStreamNode Node = newDDInputNode(dsService.ctx, dsService.msFactory, vchanPair.GetDdlVchannelName(), vchanPair.GetDdlPosition())
var filterDmNode Node = newFilteredDmNode()
var ddNode Node = newDDNode(dsService.ctx, mt, dsService.flushChan, dsService.replica, dsService.idAllocator)

View File

@ -21,23 +21,17 @@ import (
"github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
)
// NOTE: start pulsar before test
func TestDataSyncService_Start(t *testing.T) {
const ctxTimeInMillisecond = 2000
const closeWithDeadline = true
var ctx context.Context
if closeWithDeadline {
var cancel context.CancelFunc
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel = context.WithDeadline(context.Background(), d)
defer cancel()
} else {
ctx = context.Background()
}
delay := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), delay)
defer cancel()
// init data node
pulsarURL := Params.PulsarAddress
@ -45,19 +39,37 @@ func TestDataSyncService_Start(t *testing.T) {
Factory := &MetaFactory{}
collMeta := Factory.CollectionMetaFactory(UniqueID(0), "coll1")
chanSize := 100
flushChan := make(chan *flushMsg, chanSize)
flushChan := make(chan *flushMsg, 100)
replica := newReplica()
allocFactory := AllocatorFactory{}
allocFactory := NewAllocatorFactory(1)
msFactory := msgstream.NewPmsFactory()
m := map[string]interface{}{
"pulsarAddress": pulsarURL,
"receiveBufSize": 1024,
"pulsarBufSize": 1024}
err := msFactory.SetParams(m)
sync := newDataSyncService(ctx, flushChan, replica, &allocFactory, msFactory)
insertChannelName := "data_sync_service_test_dml"
ddlChannelName := "data_sync_service_test_ddl"
Params.FlushInsertBufferSize = 1
vchanPair := &datapb.VchannelPair{
CollectionID: collMeta.GetID(),
DmlVchannelName: insertChannelName,
DdlVchannelName: ddlChannelName,
DmlPosition: &datapb.PositionPair{
StartPosition: &internalpb.MsgPosition{},
EndPosition: &internalpb.MsgPosition{},
},
DdlPosition: &datapb.PositionPair{
StartPosition: &internalpb.MsgPosition{},
EndPosition: &internalpb.MsgPosition{},
},
}
sync := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchanPair)
sync.replica.addCollection(collMeta.ID, collMeta.Schema)
sync.init()
go sync.start()
timeRange := TimeRange{
@ -65,12 +77,18 @@ func TestDataSyncService_Start(t *testing.T) {
timestampMax: math.MaxUint64,
}
dataFactory := NewDataFactory()
insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2)
insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2, insertChannelName)
msgPack := msgstream.MsgPack{
BeginTs: timeRange.timestampMin,
EndTs: timeRange.timestampMax,
Msgs: insertMessages,
StartPositions: []*internalpb.MsgPosition{{
ChannelName: insertChannelName,
}},
EndPositions: []*internalpb.MsgPosition{{
ChannelName: insertChannelName,
}},
}
// generate timeTick
@ -94,15 +112,12 @@ func TestDataSyncService_Start(t *testing.T) {
timeTickMsgPack.Msgs = append(timeTickMsgPack.Msgs, timeTickMsg)
// pulsar produce
insertChannels := Params.InsertChannelNames
ddChannels := Params.DDChannelNames
assert.NoError(t, err)
insertStream, _ := msFactory.NewMsgStream(ctx)
insertStream.AsProducer(insertChannels)
insertStream.AsProducer([]string{insertChannelName})
ddStream, _ := msFactory.NewMsgStream(ctx)
ddStream.AsProducer(ddChannels)
ddStream.AsProducer([]string{ddlChannelName})
var insertMsgStream msgstream.MsgStream = insertStream
insertMsgStream.Start()
@ -119,7 +134,6 @@ func TestDataSyncService_Start(t *testing.T) {
assert.NoError(t, err)
// dataSync
Params.FlushInsertBufferSize = 1
<-sync.ctx.Done()
sync.close()

View File

@ -109,7 +109,7 @@ func (ddNode *ddNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
timestampMin: msMsg.TimestampMin(),
timestampMax: msMsg.TimestampMax(),
},
flushMessages: make([]*flushMsg, 0),
// flushMessages: make([]*flushMsg, 0),
gcRecord: &gcRecord{
collections: make([]UniqueID, 0),
},
@ -141,34 +141,29 @@ func (ddNode *ddNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
select {
case fmsg := <-ddNode.inFlushCh:
log.Debug(". receive flush message ...")
localSegs := make([]UniqueID, 0, len(fmsg.segmentIDs))
for _, segID := range fmsg.segmentIDs {
if ddNode.replica.hasSegment(segID) {
localSegs = append(localSegs, segID)
seg, _ := ddNode.replica.getSegmentByID(segID)
collID := seg.collectionID
if ddNode.ddBuffer.size(collID) > 0 {
log.Debug(".. ddl buffer not empty, flushing ...")
ddNode.flushMap.Store(collID, ddNode.ddBuffer.ddData[collID])
delete(ddNode.ddBuffer.ddData, collID)
binlogMetaCh := make(chan *datapb.DDLBinlogMeta)
go flush(collID, ddNode.flushMap, ddNode.kv, ddNode.idAllocator, binlogMetaCh)
go ddNode.flushComplete(binlogMetaCh, collID)
}
}
}
if len(localSegs) <= 0 {
segID := fmsg.segmentID
if !ddNode.replica.hasSegment(segID) {
log.Debug(".. Segment not exist in this datanode, skip flushing ...")
break
}
seg, _ := ddNode.replica.getSegmentByID(segID)
collID := seg.collectionID
if ddNode.ddBuffer.size(collID) > 0 {
log.Debug(".. ddl buffer not empty, flushing ...")
ddNode.flushMap.Store(collID, ddNode.ddBuffer.ddData[collID])
delete(ddNode.ddBuffer.ddData, collID)
binlogMetaCh := make(chan *datapb.DDLBinlogMeta)
go flush(collID, ddNode.flushMap, ddNode.kv, ddNode.idAllocator, binlogMetaCh)
go ddNode.flushComplete(binlogMetaCh, collID, fmsg.ddlFlushedCh)
} else {
// GOOSE TODO newest position
fmsg.ddlFlushedCh <- true
}
log.Debug(".. notifying insertbuffer ...")
fmsg.segmentIDs = localSegs
ddNode.ddMsg.flushMessages = append(ddNode.ddMsg.flushMessages, fmsg)
ddNode.ddMsg.flushMessage = fmsg
default:
}
@ -181,7 +176,7 @@ func (ddNode *ddNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
return []Msg{res}
}
func (ddNode *ddNode) flushComplete(binlogMetaCh <-chan *datapb.DDLBinlogMeta, collID UniqueID) {
func (ddNode *ddNode) flushComplete(binlogMetaCh <-chan *datapb.DDLBinlogMeta, collID UniqueID, ddlFlushedCh chan<- bool) {
binlogMeta := <-binlogMetaCh
if binlogMeta == nil {
return
@ -192,6 +187,10 @@ func (ddNode *ddNode) flushComplete(binlogMetaCh <-chan *datapb.DDLBinlogMeta, c
if err != nil {
log.Error("Save binlog meta to etcd Wrong", zap.Error(err))
}
ddlFlushedCh <- true
// TODO remove above
// ddlFlushCh <- binlogMetaCh
}
/*

View File

@ -151,7 +151,7 @@ func TestFlowGraphDDNode_Operate(t *testing.T) {
inFlushCh <- &flushMsg{
msgID: 5,
timestamp: 5,
segmentIDs: []UniqueID{1},
segmentID: UniqueID(1),
collectionID: collID,
}

View File

@ -66,7 +66,7 @@ func (fdmNode *filterDmNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
var iMsg = insertMsg{
insertMessages: make([]*msgstream.InsertMsg, 0),
flushMessages: make([]*flushMsg, 0),
// flushMessages: make([]*flushMsg, 0),
timeRange: TimeRange{
timestampMin: msgStreamMsg.TimestampMin(),
timestampMax: msgStreamMsg.TimestampMax(),
@ -75,7 +75,8 @@ func (fdmNode *filterDmNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
endPositions: make([]*internalpb.MsgPosition, 0),
}
iMsg.flushMessages = append(iMsg.flushMessages, ddMsg.flushMessages...)
// iMsg.flushMessages = append(iMsg.flushMessages, ddMsg.flushMessages...)
iMsg.flushMessage = ddMsg.flushMessage
for _, msg := range msgStreamMsg.TsMessages() {
switch msg.Type() {

View File

@ -93,6 +93,7 @@ func (ib *insertBuffer) size(segmentID UniqueID) int32 {
}
func (ib *insertBuffer) full(segmentID UniqueID) bool {
log.Debug("Segment size", zap.Any("segment", segmentID), zap.Int32("size", ib.size(segmentID)), zap.Int32("maxsize", ib.maxSize))
return ib.size(segmentID) >= ib.maxSize
}
@ -493,55 +494,6 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
}
}
// iMsg is Flush() msg from dataservice
// 1. insertBuffer(not empty) -> binLogs -> minIO/S3
for _, msg := range iMsg.flushMessages {
for _, currentSegID := range msg.segmentIDs {
log.Debug(". Receiving flush message", zap.Int64("segmentID", currentSegID))
// finishCh := make(chan bool)
finishCh := make(chan map[UniqueID]string)
go ibNode.completeFlush(currentSegID, finishCh)
if ibNode.insertBuffer.size(currentSegID) <= 0 {
log.Debug(".. Buffer empty ...")
finishCh <- make(map[UniqueID]string)
continue
}
log.Debug(".. Buffer not empty, flushing ..")
ibNode.flushMap.Store(currentSegID, ibNode.insertBuffer.insertData[currentSegID])
delete(ibNode.insertBuffer.insertData, currentSegID)
clearFn := func() {
finishCh <- nil
log.Debug(".. Clearing flush Buffer ..")
ibNode.flushMap.Delete(currentSegID)
}
seg, err := ibNode.replica.getSegmentByID(currentSegID)
if err != nil {
log.Error("Flush failed .. cannot get segment ..", zap.Error(err))
clearFn()
continue
}
collSch, err := ibNode.getCollectionSchemaByID(seg.collectionID)
if err != nil {
log.Error("Flush failed .. cannot get collection schema ..", zap.Error(err))
clearFn()
continue
}
collMeta := &etcdpb.CollectionMeta{
Schema: collSch,
ID: seg.collectionID,
}
go flushSegment(collMeta, currentSegID, seg.partitionID, seg.collectionID,
&ibNode.flushMap, ibNode.minIOKV, finishCh, ibNode.idAllocator)
}
}
for _, segToFlush := range segToUpdate {
// If full, auto flush
if ibNode.insertBuffer.full(segToFlush) {
@ -570,6 +522,57 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
}
}
// iMsg is Flush() msg from dataservice
// 1. insertBuffer(not empty) -> binLogs -> minIO/S3
// for _, msg := range iMsg.flushMessages {
// for _, currentSegID := range msg.segmentIDs {
if iMsg.flushMessage != nil && ibNode.replica.hasSegment(iMsg.flushMessage.segmentID) {
currentSegID := iMsg.flushMessage.segmentID
log.Debug(". Receiving flush message", zap.Int64("segmentID", currentSegID))
finishCh := make(chan map[UniqueID]string)
go ibNode.completeFlush(currentSegID, finishCh, iMsg.flushMessage.dmlFlushedCh)
if ibNode.insertBuffer.size(currentSegID) <= 0 {
log.Debug(".. Buffer empty ...")
finishCh <- make(map[UniqueID]string)
} else {
log.Debug(".. Buffer not empty, flushing ..")
ibNode.flushMap.Store(currentSegID, ibNode.insertBuffer.insertData[currentSegID])
delete(ibNode.insertBuffer.insertData, currentSegID)
clearFn := func() {
finishCh <- nil
log.Debug(".. Clearing flush Buffer ..")
ibNode.flushMap.Delete(currentSegID)
}
var collMeta *etcdpb.CollectionMeta
var collSch *schemapb.CollectionSchema
seg, err := ibNode.replica.getSegmentByID(currentSegID)
if err != nil {
log.Error("Flush failed .. cannot get segment ..", zap.Error(err))
clearFn()
// TODO add error handling
}
collSch, err = ibNode.getCollectionSchemaByID(seg.collectionID)
if err != nil {
log.Error("Flush failed .. cannot get collection schema ..", zap.Error(err))
clearFn()
// TODO add error handling
}
collMeta = &etcdpb.CollectionMeta{
Schema: collSch,
ID: seg.collectionID,
}
go flushSegment(collMeta, currentSegID, seg.partitionID, seg.collectionID,
&ibNode.flushMap, ibNode.minIOKV, finishCh, ibNode.idAllocator)
}
}
if err := ibNode.writeHardTimeTick(iMsg.timeRange.timestampMax); err != nil {
log.Error("send hard time tick into pulsar channel failed", zap.Error(err))
}
@ -685,14 +688,17 @@ func (ibNode *insertBufferNode) bufferAutoFlushPaths(wait <-chan map[UniqueID]st
return ibNode.replica.bufferAutoFlushBinlogPaths(segID, field2Path)
}
func (ibNode *insertBufferNode) completeFlush(segID UniqueID, wait <-chan map[UniqueID]string) {
func (ibNode *insertBufferNode) completeFlush(segID UniqueID, wait <-chan map[UniqueID]string, dmlFlushedCh chan<- bool) {
field2Path := <-wait
if field2Path == nil {
return
}
dmlFlushedCh <- true
// TODO Call DataService RPC SaveBinlogPaths
// TODO GetBufferedAutoFlushBinlogPaths
ibNode.replica.bufferAutoFlushBinlogPaths(segID, field2Path)
bufferField2Paths, err := ibNode.replica.getBufferPaths(segID)
if err != nil {

View File

@ -70,12 +70,19 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
assert.Nil(t, err)
iBNode := newInsertBufferNode(ctx, newBinlogMeta(), replica, msFactory, NewAllocatorFactory())
inMsg := genInsertMsg()
ddlFlushedCh := make(chan bool)
dmlFlushedCh := make(chan bool)
inMsg := genInsertMsg(ddlFlushedCh, dmlFlushedCh)
var iMsg flowgraph.Msg = &inMsg
iBNode.Operate([]flowgraph.Msg{iMsg})
isflushed := <-dmlFlushedCh
assert.True(t, isflushed)
}
func genInsertMsg() insertMsg {
func genInsertMsg(ddlFlushedCh, dmlFlushedCh chan<- bool) insertMsg {
timeRange := TimeRange{
timestampMin: 0,
@ -92,7 +99,7 @@ func genInsertMsg() insertMsg {
var iMsg = &insertMsg{
insertMessages: make([]*msgstream.InsertMsg, 0),
flushMessages: make([]*flushMsg, 0),
// flushMessages: make([]*flushMsg, 0),
timeRange: TimeRange{
timestampMin: timeRange.timestampMin,
timestampMax: timeRange.timestampMax,
@ -104,14 +111,15 @@ func genInsertMsg() insertMsg {
dataFactory := NewDataFactory()
iMsg.insertMessages = append(iMsg.insertMessages, dataFactory.GetMsgStreamInsertMsgs(2)...)
fmsg := &flushMsg{
iMsg.flushMessage = &flushMsg{
msgID: 1,
timestamp: 2000,
segmentIDs: []UniqueID{1},
segmentID: UniqueID(1),
collectionID: UniqueID(1),
ddlFlushedCh: ddlFlushedCh,
dmlFlushedCh: dmlFlushedCh,
}
iMsg.flushMessages = append(iMsg.flushMessages, fmsg)
return *iMsg
}

View File

@ -30,7 +30,7 @@ type key2SegMsg struct {
type ddMsg struct {
collectionRecords map[UniqueID][]*metaOperateRecord
partitionRecords map[UniqueID][]*metaOperateRecord
flushMessages []*flushMsg
flushMessage *flushMsg
gcRecord *gcRecord
timeRange TimeRange
}
@ -42,7 +42,7 @@ type metaOperateRecord struct {
type insertMsg struct {
insertMessages []*msgstream.InsertMsg
flushMessages []*flushMsg
flushMessage *flushMsg
gcRecord *gcRecord
timeRange TimeRange
startPositions []*internalpb.MsgPosition
@ -66,8 +66,10 @@ type gcRecord struct {
type flushMsg struct {
msgID UniqueID
timestamp Timestamp
segmentIDs []UniqueID
segmentID UniqueID
collectionID UniqueID
ddlFlushedCh chan<- bool
dmlFlushedCh chan<- bool
}
func (ksMsg *key2SegMsg) TimeTick() Timestamp {

View File

@ -13,38 +13,40 @@ package datanode
import (
"context"
"strings"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/flowgraph"
)
func newDmInputNode(ctx context.Context, factory msgstream.Factory) *flowgraph.InputNode {
func newDmInputNode(ctx context.Context, factory msgstream.Factory, vchannelName string, vchannelPos *datapb.PositionPair) *flowgraph.InputNode {
// TODO use position pair in Seek
maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism
consumeChannels := Params.InsertChannelNames
// consumeChannels := Params.InsertChannelNames
consumeSubName := Params.MsgChannelSubName
insertStream, _ := factory.NewTtMsgStream(ctx)
insertStream.AsConsumer(consumeChannels, consumeSubName)
log.Debug("datanode AsConsumer: " + strings.Join(consumeChannels, ", ") + " : " + consumeSubName)
insertStream.AsConsumer([]string{vchannelName}, consumeSubName)
log.Debug("datanode AsConsumer: " + vchannelName + " : " + consumeSubName)
var stream msgstream.MsgStream = insertStream
node := flowgraph.NewInputNode(&stream, "dmInputNode", maxQueueLength, maxParallelism)
return node
}
func newDDInputNode(ctx context.Context, factory msgstream.Factory) *flowgraph.InputNode {
func newDDInputNode(ctx context.Context, factory msgstream.Factory, vchannelName string, vchannelPos *datapb.PositionPair) *flowgraph.InputNode {
// TODO use position pair in Seek
maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism
consumeSubName := Params.MsgChannelSubName
tmpStream, _ := factory.NewTtMsgStream(ctx)
tmpStream.AsConsumer(Params.DDChannelNames, consumeSubName)
log.Debug("datanode AsConsumer: " + strings.Join(Params.DDChannelNames, ", ") + " : " + consumeSubName)
tmpStream.AsConsumer([]string{vchannelName}, consumeSubName)
log.Debug("datanode AsConsumer: " + vchannelName + " : " + consumeSubName)
var stream msgstream.MsgStream = tmpStream
node := flowgraph.NewInputNode(&stream, "ddInputNode", maxQueueLength, maxParallelism)

View File

@ -32,6 +32,7 @@ import (
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/masterpb"
@ -42,7 +43,7 @@ import (
const ctxTimeInMillisecond = 5000
const debug = false
func newDataNodeMock() *DataNode {
func newIDLEDataNodeMock() *DataNode {
var ctx context.Context
if debug {
@ -59,27 +60,59 @@ func newDataNodeMock() *DataNode {
msFactory := msgstream.NewPmsFactory()
node := NewDataNode(ctx, msFactory)
replica := newReplica()
ms := &MasterServiceFactory{
ID: 0,
collectionID: 1,
collectionName: "collection-1",
}
node.SetMasterServiceInterface(ms)
ds := &DataServiceFactory{}
node.SetDataServiceInterface(ds)
var alloc allocatorInterface = NewAllocatorFactory(100)
return node
}
chanSize := 100
flushChan := make(chan *flushMsg, chanSize)
node.flushChan = flushChan
node.dataSyncService = newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory)
node.dataSyncService.init()
node.metaService = newMetaService(node.ctx, replica, node.masterService)
node.replica = replica
func newHEALTHDataNodeMock(dmChannelName, ddChannelName string) *DataNode {
var ctx context.Context
if debug {
ctx = context.Background()
} else {
var cancel context.CancelFunc
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel = context.WithDeadline(context.Background(), d)
go func() {
<-ctx.Done()
cancel()
}()
}
msFactory := msgstream.NewPmsFactory()
node := NewDataNode(ctx, msFactory)
ms := &MasterServiceFactory{
ID: 0,
collectionID: 1,
collectionName: "collection-1",
}
node.SetMasterServiceInterface(ms)
ds := &DataServiceFactory{}
node.SetDataServiceInterface(ds)
vpair := &datapb.VchannelPair{
CollectionID: 1,
DmlVchannelName: dmChannelName,
DdlVchannelName: ddChannelName,
DdlPosition: &datapb.PositionPair{},
DmlPosition: &datapb.PositionPair{},
}
_ = node.NewDataSyncService(vpair)
return node
}
@ -160,6 +193,28 @@ type DataServiceFactory struct {
types.DataService
}
func (ds *DataServiceFactory) RegisterNode(ctx context.Context, req *datapb.RegisterNodeRequest) (*datapb.RegisterNodeResponse, error) {
ret := &datapb.RegisterNodeResponse{Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success}}
ret.InitParams = &internalpb.InitParams{
NodeID: Params.NodeID,
StartParams: []*commonpb.KeyValuePair{
{Key: "DDChannelName", Value: "fake-dd-channel-name"},
{Key: "SegmentStatisticsChannelName", Value: "fake-segment-statistics-channel-name"},
{Key: "TimeTickChannelName", Value: "fake-time-tick-channel-name"},
{Key: "CompleteFlushChannelName", Value: "fake-complete-flush-name"},
},
}
return ret, nil
}
func (ds *DataServiceFactory) WatchDmChannels(ctx context.Context, req *datapb.FlushRequest) (*commonpb.Status, error) {
ret := &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}
return ret, nil
}
func (mf *MetaFactory) CollectionMetaFactory(collectionID UniqueID, collectionName string) *etcdpb.CollectionMeta {
sch := schemapb.CollectionSchema{
Name: collectionName,
@ -377,7 +432,7 @@ func GenRowData() (rawData []byte) {
return
}
func (df *DataFactory) GenMsgStreamInsertMsg(idx int) *msgstream.InsertMsg {
func (df *DataFactory) GenMsgStreamInsertMsg(idx int, chanName string) *msgstream.InsertMsg {
var msg = &msgstream.InsertMsg{
BaseMsg: msgstream.BaseMsg{
HashValues: []uint32{uint32(idx)},
@ -385,14 +440,14 @@ func (df *DataFactory) GenMsgStreamInsertMsg(idx int) *msgstream.InsertMsg {
InsertRequest: internalpb.InsertRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Insert,
MsgID: 0, // GOOSE TODO
MsgID: 0,
Timestamp: Timestamp(idx + 1000),
SourceID: 0,
},
CollectionName: "col1", // GOOSE TODO
CollectionName: "col1",
PartitionName: "default",
SegmentID: 1, // GOOSE TODO
ChannelID: "0", // GOOSE TODO
SegmentID: 1,
ChannelID: chanName,
Timestamps: []Timestamp{Timestamp(idx + 1000)},
RowIDs: []UniqueID{UniqueID(idx)},
RowData: []*commonpb.Blob{{Value: df.rawData}},
@ -401,9 +456,9 @@ func (df *DataFactory) GenMsgStreamInsertMsg(idx int) *msgstream.InsertMsg {
return msg
}
func (df *DataFactory) GetMsgStreamTsInsertMsgs(n int) (inMsgs []msgstream.TsMsg) {
func (df *DataFactory) GetMsgStreamTsInsertMsgs(n int, chanName string) (inMsgs []msgstream.TsMsg) {
for i := 0; i < n; i++ {
var msg = df.GenMsgStreamInsertMsg(i)
var msg = df.GenMsgStreamInsertMsg(i, chanName)
var tsMsg msgstream.TsMsg = msg
inMsgs = append(inMsgs, tsMsg)
}
@ -412,7 +467,7 @@ func (df *DataFactory) GetMsgStreamTsInsertMsgs(n int) (inMsgs []msgstream.TsMsg
func (df *DataFactory) GetMsgStreamInsertMsgs(n int) (inMsgs []*msgstream.InsertMsg) {
for i := 0; i < n; i++ {
var msg = df.GenMsgStreamInsertMsg(i)
var msg = df.GenMsgStreamInsertMsg(i, "")
inMsgs = append(inMsgs, msg)
}
return
@ -436,7 +491,7 @@ func NewAllocatorFactory(id ...UniqueID) *AllocatorFactory {
func (alloc *AllocatorFactory) allocID() (UniqueID, error) {
alloc.Lock()
defer alloc.Unlock()
return alloc.r.Int63n(1000000), nil
return alloc.r.Int63n(10000), nil
}
func (alloc *AllocatorFactory) genKey(isalloc bool, ids ...UniqueID) (key string, err error) {

View File

@ -107,7 +107,7 @@ func (c *dataNodeCluster) WatchInsertChannels(channels []string) {
Timestamp: 0, // todo
SourceID: Params.NodeID,
},
ChannelNames: group,
// ChannelNames: group, // TODO
})
if err = VerifyResponse(resp, err); err != nil {
log.Error("watch dm channels error", zap.Stringer("dataNode", c.nodes[i]), zap.Error(err))

View File

@ -116,7 +116,7 @@ func (c *cluster) watch(nodes []*datapb.DataNodeInfo) []*datapb.DataNodeInfo {
Base: &commonpb.MsgBase{
SourceID: Params.NodeID,
},
ChannelNames: uncompletes,
// ChannelNames: uncompletes, // TODO
}
resp, err := cli.WatchDmChannels(context.Background(), req)
if err != nil {

View File

@ -77,7 +77,7 @@ func (s *Server) RegisterNode(ctx context.Context, req *datapb.RegisterNodeReque
Timestamp: 0,
SourceID: Params.NodeID,
},
ChannelNames: s.insertChannels,
// ChannelNames: s.insertChannels, // TODO
})
if err = VerifyResponse(resp, err); err != nil {

View File

@ -169,9 +169,17 @@ message GetPartitionStatisticsResponse {
message GetSegmentInfoChannelRequest {
}
message VchannelPair {
int64 collectionID = 1;
string dml_vchannel_name = 2;
string ddl_vchannel_name = 3;
PositionPair ddl_position = 4;
PositionPair dml_position = 5;
}
message WatchDmChannelsRequest {
common.MsgBase base = 1;
repeated string channel_names = 2;
repeated VchannelPair vchannels = 2;
}
message FlushSegmentsRequest {
@ -232,10 +240,10 @@ message SegmentInfo {
uint64 last_expire_time = 10;
}
message ID2PathList {
int64 ID = 1;
repeated string Paths = 2;
}
message ID2PathList {
int64 ID = 1;
repeated string Paths = 2;
}
message PositionPair {
internal.MsgPosition start_position = 1;
@ -252,6 +260,12 @@ message SaveBinlogPathsRequest {
PositionPair ddl_position =7;
}
message DataNodeTtMsg {
common.MsgBase base =1;
string channel_name = 2;
uint64 timestamp = 3;
}
message CheckPoint {
int64 segmentID = 1;
internal.MsgPosition position = 2;
@ -304,9 +318,5 @@ message GetRecoveryInfoRequest {
int64 partitionID = 3;
}
message DataNodeTtMsg {
common.MsgBase base =1;
string channel_name = 2;
uint64 timestamp = 3;
}

View File

@ -1223,9 +1223,80 @@ func (m *GetSegmentInfoChannelRequest) XXX_DiscardUnknown() {
var xxx_messageInfo_GetSegmentInfoChannelRequest proto.InternalMessageInfo
type VchannelPair struct {
CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
DmlVchannelName string `protobuf:"bytes,2,opt,name=dml_vchannel_name,json=dmlVchannelName,proto3" json:"dml_vchannel_name,omitempty"`
DdlVchannelName string `protobuf:"bytes,3,opt,name=ddl_vchannel_name,json=ddlVchannelName,proto3" json:"ddl_vchannel_name,omitempty"`
DdlPosition *PositionPair `protobuf:"bytes,4,opt,name=ddl_position,json=ddlPosition,proto3" json:"ddl_position,omitempty"`
DmlPosition *PositionPair `protobuf:"bytes,5,opt,name=dml_position,json=dmlPosition,proto3" json:"dml_position,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *VchannelPair) Reset() { *m = VchannelPair{} }
func (m *VchannelPair) String() string { return proto.CompactTextString(m) }
func (*VchannelPair) ProtoMessage() {}
func (*VchannelPair) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{22}
}
func (m *VchannelPair) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_VchannelPair.Unmarshal(m, b)
}
func (m *VchannelPair) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_VchannelPair.Marshal(b, m, deterministic)
}
func (m *VchannelPair) XXX_Merge(src proto.Message) {
xxx_messageInfo_VchannelPair.Merge(m, src)
}
func (m *VchannelPair) XXX_Size() int {
return xxx_messageInfo_VchannelPair.Size(m)
}
func (m *VchannelPair) XXX_DiscardUnknown() {
xxx_messageInfo_VchannelPair.DiscardUnknown(m)
}
var xxx_messageInfo_VchannelPair proto.InternalMessageInfo
func (m *VchannelPair) GetCollectionID() int64 {
if m != nil {
return m.CollectionID
}
return 0
}
func (m *VchannelPair) GetDmlVchannelName() string {
if m != nil {
return m.DmlVchannelName
}
return ""
}
func (m *VchannelPair) GetDdlVchannelName() string {
if m != nil {
return m.DdlVchannelName
}
return ""
}
func (m *VchannelPair) GetDdlPosition() *PositionPair {
if m != nil {
return m.DdlPosition
}
return nil
}
func (m *VchannelPair) GetDmlPosition() *PositionPair {
if m != nil {
return m.DmlPosition
}
return nil
}
type WatchDmChannelsRequest struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
ChannelNames []string `protobuf:"bytes,2,rep,name=channel_names,json=channelNames,proto3" json:"channel_names,omitempty"`
Vchannels []*VchannelPair `protobuf:"bytes,2,rep,name=vchannels,proto3" json:"vchannels,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -1235,7 +1306,7 @@ func (m *WatchDmChannelsRequest) Reset() { *m = WatchDmChannelsRequest{}
func (m *WatchDmChannelsRequest) String() string { return proto.CompactTextString(m) }
func (*WatchDmChannelsRequest) ProtoMessage() {}
func (*WatchDmChannelsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{22}
return fileDescriptor_3385cd32ad6cfe64, []int{23}
}
func (m *WatchDmChannelsRequest) XXX_Unmarshal(b []byte) error {
@ -1263,9 +1334,9 @@ func (m *WatchDmChannelsRequest) GetBase() *commonpb.MsgBase {
return nil
}
func (m *WatchDmChannelsRequest) GetChannelNames() []string {
func (m *WatchDmChannelsRequest) GetVchannels() []*VchannelPair {
if m != nil {
return m.ChannelNames
return m.Vchannels
}
return nil
}
@ -1284,7 +1355,7 @@ func (m *FlushSegmentsRequest) Reset() { *m = FlushSegmentsRequest{} }
func (m *FlushSegmentsRequest) String() string { return proto.CompactTextString(m) }
func (*FlushSegmentsRequest) ProtoMessage() {}
func (*FlushSegmentsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{23}
return fileDescriptor_3385cd32ad6cfe64, []int{24}
}
func (m *FlushSegmentsRequest) XXX_Unmarshal(b []byte) error {
@ -1345,7 +1416,7 @@ func (m *SegmentMsg) Reset() { *m = SegmentMsg{} }
func (m *SegmentMsg) String() string { return proto.CompactTextString(m) }
func (*SegmentMsg) ProtoMessage() {}
func (*SegmentMsg) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{24}
return fileDescriptor_3385cd32ad6cfe64, []int{25}
}
func (m *SegmentMsg) XXX_Unmarshal(b []byte) error {
@ -1393,7 +1464,7 @@ func (m *SegmentFieldBinlogMeta) Reset() { *m = SegmentFieldBinlogMeta{}
func (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) }
func (*SegmentFieldBinlogMeta) ProtoMessage() {}
func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{25}
return fileDescriptor_3385cd32ad6cfe64, []int{26}
}
func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error {
@ -1441,7 +1512,7 @@ func (m *DDLBinlogMeta) Reset() { *m = DDLBinlogMeta{} }
func (m *DDLBinlogMeta) String() string { return proto.CompactTextString(m) }
func (*DDLBinlogMeta) ProtoMessage() {}
func (*DDLBinlogMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{26}
return fileDescriptor_3385cd32ad6cfe64, []int{27}
}
func (m *DDLBinlogMeta) XXX_Unmarshal(b []byte) error {
@ -1488,7 +1559,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{27}
return fileDescriptor_3385cd32ad6cfe64, []int{28}
}
func (m *FieldFlushMeta) XXX_Unmarshal(b []byte) error {
@ -1536,7 +1607,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{28}
return fileDescriptor_3385cd32ad6cfe64, []int{29}
}
func (m *SegmentFlushMeta) XXX_Unmarshal(b []byte) error {
@ -1590,7 +1661,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{29}
return fileDescriptor_3385cd32ad6cfe64, []int{30}
}
func (m *DDLFlushMeta) XXX_Unmarshal(b []byte) error {
@ -1638,7 +1709,7 @@ func (m *CollectionInfo) Reset() { *m = CollectionInfo{} }
func (m *CollectionInfo) String() string { return proto.CompactTextString(m) }
func (*CollectionInfo) ProtoMessage() {}
func (*CollectionInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{30}
return fileDescriptor_3385cd32ad6cfe64, []int{31}
}
func (m *CollectionInfo) XXX_Unmarshal(b []byte) error {
@ -1700,7 +1771,7 @@ func (m *SegmentInfo) Reset() { *m = SegmentInfo{} }
func (m *SegmentInfo) String() string { return proto.CompactTextString(m) }
func (*SegmentInfo) ProtoMessage() {}
func (*SegmentInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{31}
return fileDescriptor_3385cd32ad6cfe64, []int{32}
}
func (m *SegmentInfo) XXX_Unmarshal(b []byte) error {
@ -1803,7 +1874,7 @@ func (m *ID2PathList) Reset() { *m = ID2PathList{} }
func (m *ID2PathList) String() string { return proto.CompactTextString(m) }
func (*ID2PathList) ProtoMessage() {}
func (*ID2PathList) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{32}
return fileDescriptor_3385cd32ad6cfe64, []int{33}
}
func (m *ID2PathList) XXX_Unmarshal(b []byte) error {
@ -1850,7 +1921,7 @@ func (m *PositionPair) Reset() { *m = PositionPair{} }
func (m *PositionPair) String() string { return proto.CompactTextString(m) }
func (*PositionPair) ProtoMessage() {}
func (*PositionPair) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{33}
return fileDescriptor_3385cd32ad6cfe64, []int{34}
}
func (m *PositionPair) XXX_Unmarshal(b []byte) error {
@ -1902,7 +1973,7 @@ func (m *SaveBinlogPathsRequest) Reset() { *m = SaveBinlogPathsRequest{}
func (m *SaveBinlogPathsRequest) String() string { return proto.CompactTextString(m) }
func (*SaveBinlogPathsRequest) ProtoMessage() {}
func (*SaveBinlogPathsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{34}
return fileDescriptor_3385cd32ad6cfe64, []int{35}
}
func (m *SaveBinlogPathsRequest) XXX_Unmarshal(b []byte) error {
@ -1972,6 +2043,61 @@ func (m *SaveBinlogPathsRequest) GetDdlPosition() *PositionPair {
return nil
}
type DataNodeTtMsg struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
ChannelName string `protobuf:"bytes,2,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *DataNodeTtMsg) Reset() { *m = DataNodeTtMsg{} }
func (m *DataNodeTtMsg) String() string { return proto.CompactTextString(m) }
func (*DataNodeTtMsg) ProtoMessage() {}
func (*DataNodeTtMsg) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{36}
}
func (m *DataNodeTtMsg) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DataNodeTtMsg.Unmarshal(m, b)
}
func (m *DataNodeTtMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_DataNodeTtMsg.Marshal(b, m, deterministic)
}
func (m *DataNodeTtMsg) XXX_Merge(src proto.Message) {
xxx_messageInfo_DataNodeTtMsg.Merge(m, src)
}
func (m *DataNodeTtMsg) XXX_Size() int {
return xxx_messageInfo_DataNodeTtMsg.Size(m)
}
func (m *DataNodeTtMsg) XXX_DiscardUnknown() {
xxx_messageInfo_DataNodeTtMsg.DiscardUnknown(m)
}
var xxx_messageInfo_DataNodeTtMsg proto.InternalMessageInfo
func (m *DataNodeTtMsg) GetBase() *commonpb.MsgBase {
if m != nil {
return m.Base
}
return nil
}
func (m *DataNodeTtMsg) GetChannelName() string {
if m != nil {
return m.ChannelName
}
return ""
}
func (m *DataNodeTtMsg) GetTimestamp() uint64 {
if m != nil {
return m.Timestamp
}
return 0
}
type CheckPoint struct {
SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
Position *internalpb.MsgPosition `protobuf:"bytes,2,opt,name=position,proto3" json:"position,omitempty"`
@ -1985,7 +2111,7 @@ func (m *CheckPoint) Reset() { *m = CheckPoint{} }
func (m *CheckPoint) String() string { return proto.CompactTextString(m) }
func (*CheckPoint) ProtoMessage() {}
func (*CheckPoint) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{35}
return fileDescriptor_3385cd32ad6cfe64, []int{37}
}
func (m *CheckPoint) XXX_Unmarshal(b []byte) error {
@ -2042,7 +2168,7 @@ func (m *VchannelInfo) Reset() { *m = VchannelInfo{} }
func (m *VchannelInfo) String() string { return proto.CompactTextString(m) }
func (*VchannelInfo) ProtoMessage() {}
func (*VchannelInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{36}
return fileDescriptor_3385cd32ad6cfe64, []int{38}
}
func (m *VchannelInfo) XXX_Unmarshal(b []byte) error {
@ -2110,7 +2236,7 @@ func (m *SegmentBinlogs) Reset() { *m = SegmentBinlogs{} }
func (m *SegmentBinlogs) String() string { return proto.CompactTextString(m) }
func (*SegmentBinlogs) ProtoMessage() {}
func (*SegmentBinlogs) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{37}
return fileDescriptor_3385cd32ad6cfe64, []int{39}
}
func (m *SegmentBinlogs) XXX_Unmarshal(b []byte) error {
@ -2157,7 +2283,7 @@ func (m *FieldBinlog) Reset() { *m = FieldBinlog{} }
func (m *FieldBinlog) String() string { return proto.CompactTextString(m) }
func (*FieldBinlog) ProtoMessage() {}
func (*FieldBinlog) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{38}
return fileDescriptor_3385cd32ad6cfe64, []int{40}
}
func (m *FieldBinlog) XXX_Unmarshal(b []byte) error {
@ -2204,7 +2330,7 @@ func (m *ChannelStatus) Reset() { *m = ChannelStatus{} }
func (m *ChannelStatus) String() string { return proto.CompactTextString(m) }
func (*ChannelStatus) ProtoMessage() {}
func (*ChannelStatus) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{39}
return fileDescriptor_3385cd32ad6cfe64, []int{41}
}
func (m *ChannelStatus) XXX_Unmarshal(b []byte) error {
@ -2252,7 +2378,7 @@ func (m *DataNodeInfo) Reset() { *m = DataNodeInfo{} }
func (m *DataNodeInfo) String() string { return proto.CompactTextString(m) }
func (*DataNodeInfo) ProtoMessage() {}
func (*DataNodeInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{40}
return fileDescriptor_3385cd32ad6cfe64, []int{42}
}
func (m *DataNodeInfo) XXX_Unmarshal(b []byte) error {
@ -2307,7 +2433,7 @@ func (m *GetRecoveryInfoResponse) Reset() { *m = GetRecoveryInfoResponse
func (m *GetRecoveryInfoResponse) String() string { return proto.CompactTextString(m) }
func (*GetRecoveryInfoResponse) ProtoMessage() {}
func (*GetRecoveryInfoResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{41}
return fileDescriptor_3385cd32ad6cfe64, []int{43}
}
func (m *GetRecoveryInfoResponse) XXX_Unmarshal(b []byte) error {
@ -2362,7 +2488,7 @@ func (m *GetRecoveryInfoRequest) Reset() { *m = GetRecoveryInfoRequest{}
func (m *GetRecoveryInfoRequest) String() string { return proto.CompactTextString(m) }
func (*GetRecoveryInfoRequest) ProtoMessage() {}
func (*GetRecoveryInfoRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{42}
return fileDescriptor_3385cd32ad6cfe64, []int{44}
}
func (m *GetRecoveryInfoRequest) XXX_Unmarshal(b []byte) error {
@ -2404,61 +2530,6 @@ func (m *GetRecoveryInfoRequest) GetPartitionID() int64 {
return 0
}
type DataNodeTtMsg struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
ChannelName string `protobuf:"bytes,2,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *DataNodeTtMsg) Reset() { *m = DataNodeTtMsg{} }
func (m *DataNodeTtMsg) String() string { return proto.CompactTextString(m) }
func (*DataNodeTtMsg) ProtoMessage() {}
func (*DataNodeTtMsg) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{43}
}
func (m *DataNodeTtMsg) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DataNodeTtMsg.Unmarshal(m, b)
}
func (m *DataNodeTtMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_DataNodeTtMsg.Marshal(b, m, deterministic)
}
func (m *DataNodeTtMsg) XXX_Merge(src proto.Message) {
xxx_messageInfo_DataNodeTtMsg.Merge(m, src)
}
func (m *DataNodeTtMsg) XXX_Size() int {
return xxx_messageInfo_DataNodeTtMsg.Size(m)
}
func (m *DataNodeTtMsg) XXX_DiscardUnknown() {
xxx_messageInfo_DataNodeTtMsg.DiscardUnknown(m)
}
var xxx_messageInfo_DataNodeTtMsg proto.InternalMessageInfo
func (m *DataNodeTtMsg) GetBase() *commonpb.MsgBase {
if m != nil {
return m.Base
}
return nil
}
func (m *DataNodeTtMsg) GetChannelName() string {
if m != nil {
return m.ChannelName
}
return ""
}
func (m *DataNodeTtMsg) GetTimestamp() uint64 {
if m != nil {
return m.Timestamp
}
return 0
}
func init() {
proto.RegisterEnum("milvus.proto.data.ChannelWatchState", ChannelWatchState_name, ChannelWatchState_value)
proto.RegisterType((*RegisterNodeRequest)(nil), "milvus.proto.data.RegisterNodeRequest")
@ -2483,6 +2554,7 @@ func init() {
proto.RegisterType((*GetPartitionStatisticsRequest)(nil), "milvus.proto.data.GetPartitionStatisticsRequest")
proto.RegisterType((*GetPartitionStatisticsResponse)(nil), "milvus.proto.data.GetPartitionStatisticsResponse")
proto.RegisterType((*GetSegmentInfoChannelRequest)(nil), "milvus.proto.data.GetSegmentInfoChannelRequest")
proto.RegisterType((*VchannelPair)(nil), "milvus.proto.data.VchannelPair")
proto.RegisterType((*WatchDmChannelsRequest)(nil), "milvus.proto.data.WatchDmChannelsRequest")
proto.RegisterType((*FlushSegmentsRequest)(nil), "milvus.proto.data.FlushSegmentsRequest")
proto.RegisterType((*SegmentMsg)(nil), "milvus.proto.data.SegmentMsg")
@ -2496,6 +2568,7 @@ func init() {
proto.RegisterType((*ID2PathList)(nil), "milvus.proto.data.ID2PathList")
proto.RegisterType((*PositionPair)(nil), "milvus.proto.data.PositionPair")
proto.RegisterType((*SaveBinlogPathsRequest)(nil), "milvus.proto.data.SaveBinlogPathsRequest")
proto.RegisterType((*DataNodeTtMsg)(nil), "milvus.proto.data.DataNodeTtMsg")
proto.RegisterType((*CheckPoint)(nil), "milvus.proto.data.CheckPoint")
proto.RegisterType((*VchannelInfo)(nil), "milvus.proto.data.VchannelInfo")
proto.RegisterType((*SegmentBinlogs)(nil), "milvus.proto.data.SegmentBinlogs")
@ -2504,145 +2577,146 @@ func init() {
proto.RegisterType((*DataNodeInfo)(nil), "milvus.proto.data.DataNodeInfo")
proto.RegisterType((*GetRecoveryInfoResponse)(nil), "milvus.proto.data.GetRecoveryInfoResponse")
proto.RegisterType((*GetRecoveryInfoRequest)(nil), "milvus.proto.data.GetRecoveryInfoRequest")
proto.RegisterType((*DataNodeTtMsg)(nil), "milvus.proto.data.DataNodeTtMsg")
}
func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad6cfe64) }
var fileDescriptor_3385cd32ad6cfe64 = []byte{
// 2101 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0x5b, 0x6f, 0x1b, 0xc7,
0x15, 0xd6, 0x92, 0xba, 0xf1, 0x70, 0x49, 0x49, 0x63, 0x55, 0x61, 0x69, 0x5b, 0x96, 0x37, 0x89,
0xa3, 0x38, 0xa8, 0x14, 0xd3, 0x45, 0x6f, 0xee, 0x05, 0x96, 0x19, 0xab, 0x44, 0x2d, 0x57, 0x1d,
0x39, 0x09, 0xd0, 0xa0, 0x20, 0x56, 0xdc, 0x11, 0xb5, 0x15, 0x77, 0x97, 0xd9, 0x59, 0xca, 0xf2,
0x93, 0x83, 0xb4, 0x0f, 0x45, 0x51, 0xd4, 0xed, 0x3f, 0x68, 0x0b, 0x14, 0x28, 0xd0, 0x3e, 0xf4,
0xb9, 0xef, 0xfd, 0x0b, 0xfd, 0x3d, 0xc5, 0x5c, 0xf6, 0x3e, 0x24, 0xd7, 0x94, 0x0d, 0xbd, 0x71,
0x86, 0xe7, 0x36, 0xe7, 0xf2, 0xcd, 0x99, 0xb3, 0x80, 0x2c, 0x33, 0x30, 0xbb, 0x94, 0xf8, 0xe7,
0x76, 0x8f, 0xec, 0x0c, 0x7d, 0x2f, 0xf0, 0xd0, 0x9a, 0x63, 0x0f, 0xce, 0x47, 0x54, 0xac, 0x76,
0x18, 0x41, 0x53, 0xef, 0x79, 0x8e, 0xe3, 0xb9, 0x62, 0xab, 0x59, 0xb7, 0xdd, 0x80, 0xf8, 0xae,
0x39, 0x90, 0x6b, 0x3d, 0xc9, 0xd0, 0xd4, 0x69, 0xef, 0x94, 0x38, 0xa6, 0x58, 0x19, 0x2f, 0xe1,
0x1a, 0x26, 0x7d, 0x9b, 0x06, 0xc4, 0x7f, 0xea, 0x59, 0x04, 0x93, 0x2f, 0x47, 0x84, 0x06, 0xe8,
0x63, 0x98, 0x3f, 0x36, 0x29, 0x69, 0x68, 0x5b, 0xda, 0x76, 0xb5, 0x75, 0x63, 0x27, 0xa5, 0x52,
0x2a, 0x3b, 0xa0, 0xfd, 0x3d, 0x93, 0x12, 0xcc, 0x29, 0xd1, 0x77, 0x60, 0xc9, 0xb4, 0x2c, 0x9f,
0x50, 0xda, 0x28, 0x4d, 0x60, 0x7a, 0x28, 0x68, 0x70, 0x48, 0x6c, 0xbc, 0xd2, 0x60, 0x3d, 0x6d,
0x01, 0x1d, 0x7a, 0x2e, 0x25, 0x68, 0x0f, 0xaa, 0xb6, 0x6b, 0x07, 0xdd, 0xa1, 0xe9, 0x9b, 0x0e,
0x95, 0x96, 0xdc, 0x4e, 0x0b, 0x8d, 0x0e, 0xda, 0x71, 0xed, 0xe0, 0x90, 0x13, 0x62, 0xb0, 0xa3,
0xdf, 0xe8, 0x3e, 0x2c, 0xd2, 0xc0, 0x0c, 0x46, 0xa1, 0x4d, 0xd7, 0x95, 0x36, 0x1d, 0x71, 0x12,
0x2c, 0x49, 0x8d, 0x0b, 0xd0, 0x1f, 0x0f, 0x46, 0xf4, 0x74, 0x76, 0x5f, 0x20, 0x98, 0xb7, 0x8e,
0x3b, 0x6d, 0xae, 0xb4, 0x8c, 0xf9, 0x6f, 0x64, 0x80, 0xde, 0xf3, 0x06, 0x03, 0xd2, 0x0b, 0x6c,
0xcf, 0xed, 0xb4, 0x1b, 0xf3, 0xfc, 0xbf, 0xd4, 0x9e, 0xf1, 0x67, 0x0d, 0x56, 0x8f, 0x48, 0xdf,
0x21, 0x6e, 0xd0, 0x69, 0x87, 0xea, 0xd7, 0x61, 0xa1, 0xe7, 0x8d, 0xdc, 0x80, 0xeb, 0xaf, 0x61,
0xb1, 0x40, 0xb7, 0x41, 0xef, 0x9d, 0x9a, 0xae, 0x4b, 0x06, 0x5d, 0xd7, 0x74, 0x08, 0x57, 0x55,
0xc1, 0x55, 0xb9, 0xf7, 0xd4, 0x74, 0x48, 0x4e, 0x63, 0x39, 0xaf, 0x11, 0x6d, 0x41, 0x75, 0x68,
0xfa, 0x81, 0x9d, 0x32, 0x2a, 0xb9, 0x65, 0xfc, 0x45, 0x83, 0x8d, 0x87, 0x94, 0xda, 0x7d, 0x37,
0x67, 0xd9, 0x06, 0x2c, 0xba, 0x9e, 0x45, 0x3a, 0x6d, 0x6e, 0x5a, 0x19, 0xcb, 0x15, 0xba, 0x0e,
0x95, 0x21, 0x21, 0x7e, 0xd7, 0xf7, 0x06, 0xa1, 0x61, 0xcb, 0x6c, 0x03, 0x7b, 0x03, 0x82, 0x7e,
0x01, 0x6b, 0x34, 0x23, 0x88, 0x36, 0xca, 0x5b, 0xe5, 0xed, 0x6a, 0xeb, 0xdd, 0x9d, 0x5c, 0x66,
0xef, 0x64, 0x95, 0xe2, 0x3c, 0xb7, 0xf1, 0x55, 0x09, 0xae, 0x45, 0x74, 0xc2, 0x56, 0xf6, 0x9b,
0x79, 0x8e, 0x92, 0x7e, 0x64, 0x9e, 0x58, 0x14, 0xf1, 0x5c, 0xe4, 0xf2, 0x72, 0xd2, 0xe5, 0x05,
0x22, 0x98, 0xf5, 0xe7, 0x42, 0xce, 0x9f, 0xe8, 0x16, 0x54, 0xc9, 0xc5, 0xd0, 0xf6, 0x49, 0x37,
0xb0, 0x1d, 0xd2, 0x58, 0xdc, 0xd2, 0xb6, 0xe7, 0x31, 0x88, 0xad, 0x67, 0xb6, 0x43, 0x12, 0x39,
0xbb, 0x54, 0x3c, 0x67, 0xff, 0xa6, 0xc1, 0x3b, 0xb9, 0x28, 0xc9, 0x42, 0xc2, 0xb0, 0xca, 0x4f,
0x1e, 0x7b, 0x86, 0x55, 0x13, 0x73, 0xf8, 0x9d, 0x49, 0x0e, 0x8f, 0xc9, 0x71, 0x8e, 0x7f, 0xb6,
0xc2, 0xfa, 0xab, 0x06, 0xd7, 0x8e, 0x4e, 0xbd, 0xe7, 0x52, 0x05, 0x9d, 0xbd, 0xc0, 0xb2, 0xa1,
0x28, 0x4d, 0x0f, 0x45, 0x39, 0x1f, 0x8a, 0xb0, 0x4c, 0xe7, 0xe3, 0x32, 0x35, 0xce, 0x60, 0x3d,
0x6d, 0xa2, 0x74, 0xe2, 0x26, 0x40, 0x94, 0x78, 0xc2, 0x7d, 0x65, 0x9c, 0xd8, 0x99, 0xcd, 0x21,
0x67, 0xf0, 0xce, 0x3e, 0x09, 0xa4, 0x2e, 0xf6, 0x1f, 0xb9, 0x84, 0x4f, 0xd2, 0x16, 0x96, 0xb2,
0x16, 0x1a, 0xff, 0x2e, 0x45, 0xe0, 0xc2, 0x55, 0x75, 0xdc, 0x13, 0x0f, 0xdd, 0x80, 0x4a, 0x44,
0x22, 0xcb, 0x24, 0xde, 0x40, 0xdf, 0x85, 0x05, 0x66, 0xa9, 0xa8, 0x91, 0x7a, 0x16, 0x7c, 0xc3,
0x33, 0x25, 0x64, 0x62, 0x41, 0x8f, 0x3a, 0x50, 0xa7, 0x81, 0xe9, 0x07, 0xdd, 0xa1, 0x47, 0xb9,
0xb7, 0xb9, 0xfb, 0xab, 0x2d, 0x63, 0x0c, 0x7c, 0x1f, 0xd0, 0xfe, 0xa1, 0xa4, 0xc4, 0x35, 0xce,
0x19, 0x2e, 0xd1, 0x27, 0xa0, 0x13, 0xd7, 0x8a, 0x05, 0xcd, 0x17, 0x16, 0x54, 0x25, 0xae, 0x15,
0x89, 0x89, 0xe3, 0xb3, 0x50, 0x3c, 0x3e, 0x7f, 0xd0, 0xa0, 0x91, 0x0f, 0x90, 0xcc, 0x88, 0x58,
0xa2, 0x56, 0x58, 0x22, 0x7a, 0x20, 0x98, 0x88, 0x08, 0xd0, 0x44, 0xc8, 0x8b, 0x82, 0x84, 0x25,
0x8b, 0x61, 0xc3, 0x37, 0x62, 0x6b, 0xf8, 0x3f, 0x6f, 0x2d, 0x59, 0x7e, 0xa3, 0xc1, 0x46, 0x56,
0xd7, 0x65, 0xce, 0xfd, 0x6d, 0x58, 0xb0, 0xdd, 0x13, 0x2f, 0x3c, 0xf6, 0xe6, 0x04, 0xe0, 0x61,
0xba, 0x04, 0xb1, 0xe1, 0xc0, 0xf5, 0x7d, 0x12, 0x74, 0x5c, 0x4a, 0xfc, 0x60, 0xcf, 0x76, 0x07,
0x5e, 0xff, 0xd0, 0x0c, 0x4e, 0x2f, 0x51, 0x23, 0xa9, 0x74, 0x2f, 0x65, 0xd2, 0xdd, 0xf8, 0x87,
0x06, 0x37, 0xd4, 0xfa, 0xe4, 0xd1, 0x9b, 0xb0, 0x7c, 0x62, 0x93, 0x81, 0x15, 0x43, 0x40, 0xb4,
0x66, 0xb5, 0x32, 0x64, 0xc4, 0xf2, 0x84, 0xe3, 0x1a, 0x95, 0xa3, 0xc0, 0xb7, 0xdd, 0xfe, 0x13,
0x9b, 0x06, 0x58, 0xd0, 0x27, 0xfc, 0x59, 0x2e, 0x9e, 0x99, 0xbf, 0x15, 0x99, 0x29, 0x4c, 0x7d,
0x24, 0xae, 0x2e, 0xfa, 0x76, 0x1b, 0x16, 0x45, 0xfb, 0x60, 0xfc, 0x5e, 0x83, 0xcd, 0x7d, 0x12,
0x3c, 0x8a, 0xf6, 0x98, 0x99, 0x36, 0x0d, 0xec, 0xde, 0x15, 0x18, 0xf3, 0x4a, 0x83, 0x5b, 0x63,
0x8d, 0x91, 0x11, 0x94, 0x88, 0x16, 0x5e, 0x80, 0x6a, 0x44, 0xfb, 0x19, 0x79, 0xf1, 0x99, 0x39,
0x18, 0x91, 0x43, 0xd3, 0xf6, 0x05, 0xa2, 0xcd, 0x88, 0xef, 0xff, 0xd4, 0xe0, 0xe6, 0x3e, 0x61,
0xcd, 0xa8, 0xb8, 0x73, 0xae, 0xd0, 0x3b, 0x05, 0x3a, 0xbd, 0x3f, 0x8a, 0x60, 0x2a, 0xad, 0xbd,
0x12, 0xf7, 0x6d, 0xf2, 0x72, 0x4c, 0xe0, 0x82, 0x4c, 0x74, 0xe9, 0x3c, 0xc3, 0x83, 0x8d, 0xcf,
0xcd, 0xa0, 0x77, 0xda, 0x76, 0x2e, 0x5f, 0x01, 0xef, 0x42, 0x2d, 0xd9, 0x15, 0x8a, 0x32, 0xae,
0x60, 0x3d, 0xd1, 0x16, 0x52, 0xd6, 0x0b, 0xaf, 0xf3, 0xa7, 0xc1, 0xe5, 0x3b, 0x98, 0x59, 0xc3,
0x98, 0x06, 0xee, 0xf9, 0x1c, 0x70, 0x5f, 0x00, 0x48, 0xe3, 0x0e, 0x68, 0x7f, 0x06, 0xbb, 0xbe,
0x07, 0x4b, 0x52, 0x9a, 0x8c, 0xd4, 0x34, 0xa8, 0x0e, 0xc9, 0x8d, 0x23, 0xd8, 0x90, 0xfb, 0x8f,
0x19, 0x26, 0x0a, 0xfc, 0x3c, 0x20, 0x81, 0x89, 0x1a, 0xb0, 0x24, 0x61, 0x52, 0xb6, 0x18, 0xe1,
0x92, 0x35, 0xc3, 0xc7, 0x9c, 0xae, 0xcb, 0xb0, 0x50, 0xb6, 0xe2, 0x70, 0x1c, 0x41, 0xaf, 0xf1,
0x2b, 0xa8, 0xb5, 0xdb, 0x4f, 0x12, 0xb2, 0xee, 0xc0, 0x8a, 0x65, 0x0d, 0xba, 0x49, 0x2e, 0x8d,
0x73, 0xd5, 0x2c, 0x6b, 0x10, 0x63, 0x36, 0x7a, 0x0f, 0xea, 0x01, 0xed, 0xe6, 0x85, 0xeb, 0x01,
0x8d, 0xa9, 0x8c, 0x03, 0xa8, 0x73, 0x63, 0x79, 0x50, 0xa7, 0xd8, 0x7a, 0x1b, 0xf4, 0x84, 0xb8,
0x30, 0x41, 0xaa, 0xb1, 0xb1, 0x94, 0xc1, 0x61, 0xd8, 0x62, 0xc5, 0x12, 0x27, 0xb7, 0x58, 0x37,
0x01, 0x6c, 0xda, 0x3d, 0x61, 0xd4, 0xc4, 0xe2, 0x36, 0x2e, 0xe3, 0x8a, 0x4d, 0x1f, 0x8b, 0x0d,
0xf4, 0x7d, 0x58, 0xe4, 0xfa, 0x59, 0xdb, 0xa2, 0xa8, 0x38, 0x1e, 0x8d, 0xf4, 0x09, 0xb0, 0x64,
0x30, 0x3e, 0x05, 0xbd, 0xdd, 0x7e, 0x12, 0xdb, 0x91, 0xcd, 0x2e, 0x4d, 0x91, 0x5d, 0x05, 0xce,
0xf8, 0x12, 0xea, 0x31, 0xc2, 0xf2, 0x1e, 0xb2, 0x0e, 0xa5, 0x48, 0x5c, 0xa9, 0xd3, 0x46, 0x3f,
0x82, 0x45, 0x31, 0x62, 0x90, 0x19, 0xf4, 0x7e, 0xda, 0x66, 0x39, 0x7e, 0x48, 0xc0, 0x34, 0xdf,
0xc0, 0x92, 0x89, 0x65, 0x78, 0x84, 0x4a, 0xe2, 0x65, 0x58, 0xc6, 0x89, 0x1d, 0xe3, 0x3f, 0x65,
0xa8, 0x26, 0x12, 0x30, 0xa7, 0xfe, 0xcd, 0xbc, 0x0d, 0xde, 0x87, 0xba, 0xcd, 0x2f, 0xd7, 0xae,
0x44, 0x00, 0x8e, 0x98, 0x15, 0x5c, 0xb3, 0x93, 0x57, 0x2e, 0xfa, 0x26, 0x2c, 0xbb, 0x23, 0xa7,
0xeb, 0x7b, 0xcf, 0xa9, 0x7c, 0xec, 0x2d, 0xb9, 0x23, 0x07, 0x7b, 0xcf, 0x69, 0xdc, 0x3c, 0x2f,
0x5e, 0xba, 0x79, 0x5e, 0x7a, 0x53, 0xcd, 0xf3, 0xf2, 0x6c, 0xcd, 0xf3, 0x26, 0x54, 0x1d, 0xf3,
0x82, 0x9d, 0xb2, 0xeb, 0x8e, 0x9c, 0x46, 0x45, 0x24, 0xb1, 0x63, 0x5e, 0x60, 0xef, 0xf9, 0xd3,
0x91, 0x83, 0xb6, 0x61, 0x75, 0x60, 0xd2, 0xa0, 0x9b, 0x7c, 0xd8, 0x02, 0x7f, 0xd8, 0xd6, 0xd9,
0xfe, 0x27, 0xd1, 0xe3, 0xd6, 0xb8, 0x0f, 0xd5, 0x4e, 0xbb, 0xc5, 0x32, 0x89, 0xb5, 0x40, 0xb9,
0xd8, 0xad, 0xc3, 0xc2, 0x61, 0x22, 0xf1, 0xc4, 0x82, 0xc1, 0xae, 0x1e, 0xda, 0xc2, 0x2e, 0x15,
0x85, 0x87, 0xb4, 0x37, 0xe5, 0xa1, 0xd2, 0x4c, 0x1e, 0x32, 0xfe, 0x55, 0x86, 0x8d, 0x23, 0xf3,
0x9c, 0xbc, 0xfd, 0x2e, 0xb5, 0xd0, 0x2d, 0xf1, 0x04, 0xd6, 0x38, 0x0a, 0xb4, 0x12, 0xf6, 0xc8,
0x97, 0x93, 0x0a, 0xcf, 0x13, 0x21, 0xc1, 0x79, 0x46, 0xf4, 0x53, 0xa8, 0xa7, 0xc0, 0x35, 0x04,
0xa3, 0x2d, 0x85, 0xa8, 0x14, 0x5a, 0xe3, 0x0c, 0x1f, 0xda, 0x03, 0xdd, 0x72, 0x06, 0xb1, 0xb7,
0x17, 0xb9, 0x49, 0xb7, 0x14, 0x72, 0x92, 0xf1, 0xc6, 0x55, 0xcb, 0x19, 0x44, 0x11, 0x63, 0x32,
0xac, 0x41, 0xb6, 0x38, 0x0a, 0xc8, 0xb0, 0x22, 0x19, 0x0c, 0xa8, 0xe1, 0xd1, 0x29, 0xe9, 0x9d,
0x1d, 0x7a, 0xb6, 0x1b, 0x4c, 0x81, 0xe8, 0x1f, 0xc3, 0xf2, 0x0c, 0xe9, 0x11, 0xf1, 0xb0, 0xea,
0x61, 0x18, 0xe1, 0x9d, 0x08, 0x98, 0x10, 0xf1, 0xaa, 0xb8, 0x23, 0xe7, 0xe7, 0x27, 0x0c, 0x28,
0x8c, 0xdf, 0x95, 0x40, 0xff, 0x4c, 0xa2, 0x0c, 0x47, 0xb4, 0x22, 0x48, 0xbd, 0x05, 0xc9, 0x89,
0x95, 0x6a, 0x88, 0xb5, 0x0f, 0x35, 0x4a, 0xc8, 0xd9, 0x2c, 0x4f, 0x70, 0x9d, 0x31, 0x46, 0x0e,
0xff, 0x09, 0x53, 0x15, 0xfa, 0x4a, 0xf4, 0x1c, 0xd5, 0xd6, 0x4d, 0x85, 0xbf, 0x63, 0x8f, 0xe2,
0x24, 0x07, 0xda, 0x86, 0x15, 0x79, 0xc1, 0x85, 0x7d, 0x13, 0x4f, 0xa0, 0x32, 0xce, 0x6e, 0x1b,
0x3e, 0xd4, 0xe5, 0x6f, 0x91, 0x35, 0x74, 0x4a, 0x68, 0xf6, 0x40, 0x3f, 0x89, 0x9b, 0x8d, 0x49,
0xaf, 0xcb, 0x44, 0x4f, 0x82, 0x53, 0x3c, 0xc6, 0x43, 0xa8, 0x26, 0xfe, 0x9c, 0xd0, 0x00, 0x34,
0x60, 0xe9, 0x38, 0xa1, 0xa7, 0x82, 0xc3, 0xa5, 0xd1, 0x85, 0x9a, 0xbc, 0x10, 0x44, 0x07, 0xcb,
0xba, 0x3b, 0x3e, 0x5b, 0x14, 0xad, 0x09, 0xff, 0x8d, 0x7e, 0x90, 0x1e, 0xa6, 0xbc, 0xa7, 0x74,
0x20, 0x17, 0xc2, 0x9b, 0xda, 0xe4, 0x95, 0x60, 0x7c, 0xa5, 0x81, 0xde, 0x36, 0x03, 0xf3, 0xa9,
0x67, 0x89, 0xb9, 0x4d, 0x23, 0x9e, 0xb6, 0x0b, 0x1d, 0xe1, 0x92, 0xfd, 0x73, 0x4e, 0x7c, 0x1a,
0x26, 0x6b, 0x19, 0x87, 0x4b, 0xf4, 0x43, 0x58, 0x96, 0xf9, 0x11, 0x0e, 0x5c, 0xb7, 0xc6, 0xdb,
0x20, 0x5b, 0xf1, 0x88, 0xc3, 0xf8, 0xaf, 0xc6, 0x87, 0x55, 0x98, 0xf4, 0xbc, 0x73, 0xe2, 0xbf,
0x48, 0x8d, 0x04, 0x5e, 0x1f, 0xe2, 0x1e, 0x24, 0x6c, 0x11, 0x41, 0x53, 0x15, 0x70, 0xb2, 0x2a,
0x62, 0x53, 0xd0, 0x83, 0x38, 0x10, 0xe5, 0xb1, 0x5d, 0x51, 0x3a, 0x8f, 0xe2, 0x58, 0xfd, 0x49,
0x4c, 0x36, 0xd2, 0xe7, 0xb8, 0xd2, 0x39, 0xa4, 0xf1, 0xb5, 0x06, 0xb5, 0x30, 0xba, 0xcf, 0x66,
0xec, 0xdb, 0x0b, 0x4c, 0xb5, 0x6f, 0x40, 0x85, 0xdd, 0xcc, 0x34, 0x30, 0x9d, 0x21, 0x37, 0x63,
0x1e, 0xc7, 0x1b, 0x77, 0xef, 0xc1, 0x5a, 0x2e, 0xfd, 0x50, 0x1d, 0xe0, 0x53, 0xb7, 0xe7, 0x39,
0xc3, 0x01, 0x09, 0xc8, 0xea, 0x1c, 0xd2, 0x61, 0xf9, 0x51, 0xb8, 0xd2, 0x5a, 0xff, 0xab, 0x41,
0x95, 0xd9, 0x7d, 0x24, 0x3e, 0x4f, 0xa1, 0x21, 0x20, 0xfe, 0xfe, 0x76, 0x86, 0x9e, 0x1b, 0xcd,
0xcb, 0xd0, 0xc7, 0x63, 0x00, 0x27, 0x4f, 0x2a, 0xe3, 0xd0, 0xbc, 0x33, 0x86, 0x23, 0x43, 0x6e,
0xcc, 0x21, 0x87, 0x6b, 0x64, 0x9d, 0xc5, 0x33, 0xbb, 0x77, 0x16, 0x36, 0x65, 0x13, 0x34, 0x66,
0x48, 0x43, 0x8d, 0x99, 0x31, 0x9c, 0x5c, 0x88, 0x59, 0x4d, 0x98, 0xe5, 0xc6, 0x1c, 0xfa, 0x12,
0xd6, 0xd9, 0x83, 0x34, 0x7a, 0x17, 0x87, 0x0a, 0x5b, 0xe3, 0x15, 0xe6, 0x88, 0x5f, 0x53, 0xa5,
0x09, 0x7a, 0xf2, 0xeb, 0x18, 0x52, 0x8d, 0xec, 0x15, 0x1f, 0xf0, 0x9a, 0x1f, 0x4c, 0xa5, 0x8b,
0x54, 0xec, 0xc3, 0x02, 0x7f, 0x25, 0x20, 0x55, 0x09, 0x26, 0xbf, 0x84, 0x35, 0x27, 0xbd, 0xda,
0x8d, 0x39, 0xf4, 0x6b, 0x58, 0xc9, 0x7c, 0x83, 0x40, 0x1f, 0x2a, 0x44, 0xaa, 0xbf, 0x26, 0x35,
0xef, 0x16, 0x21, 0x4d, 0xfa, 0x25, 0x39, 0xa7, 0x57, 0xfa, 0x45, 0xf1, 0xad, 0x41, 0xe9, 0x17,
0xd5, 0xc0, 0xdf, 0x98, 0x43, 0x7d, 0xa8, 0xa7, 0xc7, 0x0f, 0x68, 0x5b, 0xc1, 0xac, 0x9c, 0xc8,
0x36, 0x3f, 0x2c, 0x40, 0x19, 0x29, 0x72, 0x60, 0x35, 0x3b, 0x65, 0x46, 0x77, 0x27, 0x0a, 0x48,
0xd7, 0xcb, 0x47, 0x85, 0x68, 0x23, 0x75, 0x2f, 0x78, 0x16, 0xe7, 0xa6, 0x9c, 0x68, 0x47, 0x2d,
0x66, 0xdc, 0xf8, 0xb5, 0xb9, 0x5b, 0x98, 0x3e, 0x52, 0x4d, 0x60, 0x2d, 0x37, 0xb5, 0x44, 0x1f,
0x4d, 0x92, 0x93, 0x99, 0xec, 0x34, 0xa7, 0xcf, 0x55, 0x8d, 0x39, 0xf4, 0xb5, 0xb8, 0xab, 0x54,
0x93, 0x40, 0x74, 0x4f, 0xad, 0x6d, 0xc2, 0x08, 0xb3, 0xd9, 0x7a, 0x1d, 0x96, 0xe8, 0xac, 0x2f,
0xf9, 0x3d, 0xa3, 0x98, 0xa6, 0x65, 0xf1, 0x29, 0x94, 0x37, 0x7e, 0x4c, 0xd8, 0xbc, 0xf7, 0x1a,
0x1c, 0x91, 0x01, 0x5e, 0xf6, 0x73, 0x41, 0x08, 0x57, 0xbb, 0x53, 0x93, 0x73, 0x36, 0xac, 0xfa,
0x02, 0x56, 0x32, 0x6f, 0x20, 0x65, 0xfd, 0xab, 0xdf, 0x49, 0x05, 0xc0, 0x25, 0x73, 0x6d, 0xa3,
0x31, 0x45, 0xa6, 0xb8, 0xda, 0x9b, 0x77, 0x8b, 0x90, 0x86, 0x07, 0x69, 0xfd, 0xbd, 0x0c, 0xcb,
0xe1, 0x85, 0x7c, 0x05, 0xb7, 0xda, 0x15, 0x5c, 0x33, 0x5f, 0xc0, 0x4a, 0x66, 0x94, 0xaa, 0xf4,
0xae, 0x7a, 0xdc, 0x3a, 0x2d, 0x74, 0x9f, 0x43, 0x2d, 0x35, 0x35, 0x45, 0x1f, 0x8c, 0xbb, 0x68,
0xb2, 0x68, 0x3d, 0x59, 0xf0, 0xde, 0xfd, 0x5f, 0xde, 0xeb, 0xdb, 0xc1, 0xe9, 0xe8, 0x98, 0xfd,
0xb3, 0x2b, 0x48, 0xbf, 0x65, 0x7b, 0xf2, 0xd7, 0x6e, 0xe8, 0xa0, 0x5d, 0xce, 0xbd, 0xcb, 0xd4,
0x0c, 0x8f, 0x8f, 0x17, 0xf9, 0xea, 0xfe, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0xa8, 0x93, 0x0b,
0x43, 0x5b, 0x23, 0x00, 0x00,
// 2144 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x19, 0x5b, 0x6f, 0x1b, 0x59,
0x39, 0x63, 0xe7, 0x62, 0x7f, 0xbe, 0x24, 0x39, 0x1b, 0xb2, 0xc6, 0x6d, 0xd3, 0x74, 0xd8, 0xed,
0x66, 0xb3, 0x22, 0xd9, 0xba, 0x88, 0x5b, 0x59, 0x50, 0x53, 0x6f, 0x83, 0x45, 0x53, 0xc2, 0x49,
0xb7, 0x2b, 0xb1, 0x42, 0xd6, 0xc4, 0x73, 0xe2, 0x0c, 0xf1, 0xcc, 0x78, 0xe7, 0x8c, 0xd3, 0xf4,
0xa9, 0xab, 0x85, 0x07, 0x40, 0x88, 0x05, 0x7e, 0x01, 0x20, 0x21, 0x21, 0xc1, 0x03, 0xcf, 0xbc,
0xf3, 0x17, 0xf8, 0x3d, 0xe8, 0x5c, 0xe6, 0x7e, 0x6c, 0x4f, 0x9c, 0x56, 0x79, 0x9b, 0x73, 0xe6,
0xbb, 0x9d, 0xef, 0x7e, 0xbe, 0x03, 0xc8, 0x34, 0x7c, 0xa3, 0x4b, 0x89, 0x77, 0x6e, 0xf5, 0xc8,
0xce, 0xd0, 0x73, 0x7d, 0x17, 0xad, 0xda, 0xd6, 0xe0, 0x7c, 0x44, 0xc5, 0x6a, 0x87, 0x01, 0x34,
0xab, 0x3d, 0xd7, 0xb6, 0x5d, 0x47, 0x6c, 0x35, 0xeb, 0x96, 0xe3, 0x13, 0xcf, 0x31, 0x06, 0x72,
0x5d, 0x8d, 0x23, 0x34, 0xab, 0xb4, 0x77, 0x4a, 0x6c, 0x43, 0xac, 0xf4, 0x57, 0xf0, 0x16, 0x26,
0x7d, 0x8b, 0xfa, 0xc4, 0x7b, 0xea, 0x9a, 0x04, 0x93, 0xcf, 0x47, 0x84, 0xfa, 0xe8, 0x43, 0x98,
0x3f, 0x36, 0x28, 0x69, 0x68, 0x9b, 0xda, 0x56, 0xa5, 0x75, 0x73, 0x27, 0xc1, 0x52, 0x32, 0x3b,
0xa0, 0xfd, 0x3d, 0x83, 0x12, 0xcc, 0x21, 0xd1, 0xb7, 0x61, 0xc9, 0x30, 0x4d, 0x8f, 0x50, 0xda,
0x28, 0x4c, 0x40, 0x7a, 0x28, 0x60, 0x70, 0x00, 0xac, 0x7f, 0xa5, 0xc1, 0x5a, 0x52, 0x02, 0x3a,
0x74, 0x1d, 0x4a, 0xd0, 0x1e, 0x54, 0x2c, 0xc7, 0xf2, 0xbb, 0x43, 0xc3, 0x33, 0x6c, 0x2a, 0x25,
0xb9, 0x93, 0x24, 0x1a, 0x1e, 0xb4, 0xe3, 0x58, 0xfe, 0x21, 0x07, 0xc4, 0x60, 0x85, 0xdf, 0xe8,
0x3e, 0x2c, 0x52, 0xdf, 0xf0, 0x47, 0x81, 0x4c, 0x37, 0x94, 0x32, 0x1d, 0x71, 0x10, 0x2c, 0x41,
0xf5, 0x0b, 0xa8, 0x3e, 0x1e, 0x8c, 0xe8, 0xe9, 0xec, 0xba, 0x40, 0x30, 0x6f, 0x1e, 0x77, 0xda,
0x9c, 0x69, 0x11, 0xf3, 0x6f, 0xa4, 0x43, 0xb5, 0xe7, 0x0e, 0x06, 0xa4, 0xe7, 0x5b, 0xae, 0xd3,
0x69, 0x37, 0xe6, 0xf9, 0xbf, 0xc4, 0x9e, 0xfe, 0x27, 0x0d, 0x56, 0x8e, 0x48, 0xdf, 0x26, 0x8e,
0xdf, 0x69, 0x07, 0xec, 0xd7, 0x60, 0xa1, 0xe7, 0x8e, 0x1c, 0x9f, 0xf3, 0xaf, 0x61, 0xb1, 0x40,
0x77, 0xa0, 0xda, 0x3b, 0x35, 0x1c, 0x87, 0x0c, 0xba, 0x8e, 0x61, 0x13, 0xce, 0xaa, 0x8c, 0x2b,
0x72, 0xef, 0xa9, 0x61, 0x93, 0x0c, 0xc7, 0x62, 0x96, 0x23, 0xda, 0x84, 0xca, 0xd0, 0xf0, 0x7c,
0x2b, 0x21, 0x54, 0x7c, 0x4b, 0xff, 0x8b, 0x06, 0xeb, 0x0f, 0x29, 0xb5, 0xfa, 0x4e, 0x46, 0xb2,
0x75, 0x58, 0x74, 0x5c, 0x93, 0x74, 0xda, 0x5c, 0xb4, 0x22, 0x96, 0x2b, 0x74, 0x03, 0xca, 0x43,
0x42, 0xbc, 0xae, 0xe7, 0x0e, 0x02, 0xc1, 0x4a, 0x6c, 0x03, 0xbb, 0x03, 0x82, 0x7e, 0x06, 0xab,
0x34, 0x45, 0x88, 0x36, 0x8a, 0x9b, 0xc5, 0xad, 0x4a, 0xeb, 0x1b, 0x3b, 0x19, 0xcf, 0xde, 0x49,
0x33, 0xc5, 0x59, 0x6c, 0xfd, 0x8b, 0x02, 0xbc, 0x15, 0xc2, 0x09, 0x59, 0xd9, 0x37, 0xd3, 0x1c,
0x25, 0xfd, 0x50, 0x3c, 0xb1, 0xc8, 0xa3, 0xb9, 0x50, 0xe5, 0xc5, 0xb8, 0xca, 0x73, 0x58, 0x30,
0xad, 0xcf, 0x85, 0x8c, 0x3e, 0xd1, 0x6d, 0xa8, 0x90, 0x8b, 0xa1, 0xe5, 0x91, 0xae, 0x6f, 0xd9,
0xa4, 0xb1, 0xb8, 0xa9, 0x6d, 0xcd, 0x63, 0x10, 0x5b, 0xcf, 0x2c, 0x9b, 0xc4, 0x7c, 0x76, 0x29,
0xbf, 0xcf, 0xfe, 0x4d, 0x83, 0xb7, 0x33, 0x56, 0x92, 0x81, 0x84, 0x61, 0x85, 0x9f, 0x3c, 0xd2,
0x0c, 0x8b, 0x26, 0xa6, 0xf0, 0xbb, 0x93, 0x14, 0x1e, 0x81, 0xe3, 0x0c, 0xfe, 0x6c, 0x81, 0xf5,
0x57, 0x0d, 0xde, 0x3a, 0x3a, 0x75, 0x5f, 0x48, 0x16, 0x74, 0xf6, 0x00, 0x4b, 0x9b, 0xa2, 0x30,
0xdd, 0x14, 0xc5, 0xac, 0x29, 0x82, 0x30, 0x9d, 0x8f, 0xc2, 0x54, 0x3f, 0x83, 0xb5, 0xa4, 0x88,
0x52, 0x89, 0x1b, 0x00, 0xa1, 0xe3, 0x09, 0xf5, 0x15, 0x71, 0x6c, 0x67, 0x36, 0x85, 0x9c, 0xc1,
0xdb, 0xfb, 0xc4, 0x97, 0xbc, 0xd8, 0x3f, 0x72, 0x05, 0x9d, 0x24, 0x25, 0x2c, 0xa4, 0x25, 0xd4,
0xff, 0x5d, 0x08, 0x93, 0x0b, 0x67, 0xd5, 0x71, 0x4e, 0x5c, 0x74, 0x13, 0xca, 0x21, 0x88, 0x0c,
0x93, 0x68, 0x03, 0x7d, 0x07, 0x16, 0x98, 0xa4, 0x22, 0x46, 0xea, 0xe9, 0xe4, 0x1b, 0x9c, 0x29,
0x46, 0x13, 0x0b, 0x78, 0xd4, 0x81, 0x3a, 0xf5, 0x0d, 0xcf, 0xef, 0x0e, 0x5d, 0xca, 0xb5, 0xcd,
0xd5, 0x5f, 0x69, 0xe9, 0x63, 0xd2, 0xf7, 0x01, 0xed, 0x1f, 0x4a, 0x48, 0x5c, 0xe3, 0x98, 0xc1,
0x12, 0x7d, 0x0c, 0x55, 0xe2, 0x98, 0x11, 0xa1, 0xf9, 0xdc, 0x84, 0x2a, 0xc4, 0x31, 0x43, 0x32,
0x91, 0x7d, 0x16, 0xf2, 0xdb, 0xe7, 0xf7, 0x1a, 0x34, 0xb2, 0x06, 0x92, 0x1e, 0x11, 0x51, 0xd4,
0x72, 0x53, 0x44, 0x0f, 0x04, 0x12, 0x11, 0x06, 0x9a, 0x98, 0xf2, 0x42, 0x23, 0x61, 0x89, 0xa2,
0x5b, 0xf0, 0xb5, 0x48, 0x1a, 0xfe, 0xe7, 0x8d, 0x39, 0xcb, 0xaf, 0x34, 0x58, 0x4f, 0xf3, 0xba,
0xca, 0xb9, 0xbf, 0x05, 0x0b, 0x96, 0x73, 0xe2, 0x06, 0xc7, 0xde, 0x98, 0x90, 0x78, 0x18, 0x2f,
0x01, 0xac, 0xdb, 0x70, 0x63, 0x9f, 0xf8, 0x1d, 0x87, 0x12, 0xcf, 0xdf, 0xb3, 0x9c, 0x81, 0xdb,
0x3f, 0x34, 0xfc, 0xd3, 0x2b, 0xc4, 0x48, 0xc2, 0xdd, 0x0b, 0x29, 0x77, 0xd7, 0xff, 0xa1, 0xc1,
0x4d, 0x35, 0x3f, 0x79, 0xf4, 0x26, 0x94, 0x4e, 0x2c, 0x32, 0x30, 0xa3, 0x14, 0x10, 0xae, 0x59,
0xac, 0x0c, 0x19, 0xb0, 0x3c, 0xe1, 0xb8, 0x46, 0xe5, 0xc8, 0xf7, 0x2c, 0xa7, 0xff, 0xc4, 0xa2,
0x3e, 0x16, 0xf0, 0x31, 0x7d, 0x16, 0xf3, 0x7b, 0xe6, 0xaf, 0x85, 0x67, 0x0a, 0x51, 0x1f, 0x89,
0xd2, 0x45, 0xdf, 0x6c, 0xc3, 0xa2, 0x68, 0x1f, 0xf4, 0xdf, 0x69, 0xb0, 0xb1, 0x4f, 0xfc, 0x47,
0xe1, 0x1e, 0x13, 0xd3, 0xa2, 0xbe, 0xd5, 0xbb, 0x06, 0x61, 0xbe, 0xd2, 0xe0, 0xf6, 0x58, 0x61,
0xa4, 0x05, 0x65, 0x46, 0x0b, 0x0a, 0xa0, 0x3a, 0xa3, 0xfd, 0x84, 0xbc, 0x7c, 0x6e, 0x0c, 0x46,
0xe4, 0xd0, 0xb0, 0x3c, 0x91, 0xd1, 0x66, 0xcc, 0xef, 0xff, 0xd4, 0xe0, 0xd6, 0x3e, 0x61, 0xcd,
0xa8, 0xa8, 0x39, 0xd7, 0xa8, 0x9d, 0x1c, 0x9d, 0xde, 0x1f, 0x84, 0x31, 0x95, 0xd2, 0x5e, 0x8b,
0xfa, 0x36, 0x78, 0x38, 0xc6, 0xf2, 0x82, 0x74, 0x74, 0xa9, 0x3c, 0xfd, 0xcf, 0x05, 0xa8, 0x3e,
0x97, 0x7d, 0x1b, 0x63, 0x96, 0xd1, 0x83, 0xa6, 0xd0, 0xc3, 0x36, 0xac, 0x9a, 0xf6, 0xa0, 0x7b,
0xae, 0xe8, 0x01, 0x97, 0x4d, 0x7b, 0xf0, 0x3c, 0xde, 0x07, 0x32, 0x58, 0x33, 0x0d, 0x5b, 0x94,
0xb0, 0x66, 0x12, 0x76, 0x0f, 0xaa, 0x0c, 0x36, 0x55, 0xa7, 0x6e, 0x2b, 0x12, 0x5d, 0x50, 0x93,
0xb8, 0x7e, 0x2a, 0xa6, 0x39, 0x08, 0x8b, 0x14, 0xa3, 0x61, 0xc7, 0x68, 0x2c, 0xe4, 0xa5, 0x61,
0x87, 0x34, 0xf4, 0xdf, 0x6a, 0xb0, 0xfe, 0xa9, 0xe1, 0xf7, 0x4e, 0xdb, 0xf6, 0xd5, 0xf3, 0xc2,
0x47, 0x50, 0x0e, 0x0e, 0x1f, 0x24, 0x36, 0x95, 0x34, 0x71, 0x23, 0xe0, 0x08, 0x83, 0xdd, 0x1d,
0xd6, 0xf8, 0x55, 0xea, 0xea, 0x1d, 0xdf, 0xac, 0x6e, 0x9f, 0x2c, 0x74, 0xf3, 0x99, 0x42, 0x77,
0x01, 0x20, 0x85, 0x3b, 0xa0, 0xfd, 0x19, 0xe4, 0xfa, 0x2e, 0x2c, 0x49, 0x6a, 0xd2, 0xb3, 0xa7,
0x95, 0xb6, 0x00, 0x5c, 0x3f, 0x82, 0x75, 0xb9, 0xff, 0x98, 0xd5, 0x10, 0x51, 0x6f, 0x0e, 0x88,
0x6f, 0xa0, 0x06, 0x2c, 0xc9, 0xb2, 0x22, 0x3d, 0x38, 0x58, 0xb2, 0xcb, 0xc3, 0x31, 0x87, 0xeb,
0xb2, 0xda, 0x21, 0xdd, 0x16, 0x8e, 0xc3, 0x52, 0xa5, 0xff, 0x02, 0x6a, 0xed, 0xf6, 0x93, 0x18,
0xad, 0xbb, 0xc0, 0x3c, 0xb5, 0x1b, 0xc7, 0xd2, 0x38, 0x56, 0xcd, 0x34, 0x07, 0x51, 0x8d, 0x43,
0xef, 0x40, 0xdd, 0xa7, 0xdd, 0x2c, 0xf1, 0xaa, 0x4f, 0x23, 0x28, 0xfd, 0x00, 0xea, 0x5c, 0x58,
0x6e, 0xd4, 0x29, 0xb2, 0xde, 0x81, 0x6a, 0x8c, 0x9c, 0x70, 0x9f, 0x32, 0xae, 0x44, 0xc2, 0x52,
0x56, 0x3e, 0x82, 0x96, 0x34, 0xa2, 0x38, 0xb9, 0x25, 0xbd, 0x05, 0x60, 0xd1, 0xee, 0x09, 0x83,
0x26, 0x26, 0x97, 0xb1, 0x84, 0xcb, 0x16, 0x7d, 0x2c, 0x36, 0xd0, 0xf7, 0x60, 0x91, 0xf3, 0x67,
0x6d, 0x9e, 0x22, 0x43, 0x71, 0x6b, 0x24, 0x4f, 0x80, 0x25, 0x82, 0xfe, 0x09, 0x54, 0xdb, 0xed,
0x27, 0x91, 0x1c, 0x79, 0x92, 0x49, 0x8e, 0x33, 0xbe, 0x82, 0x7a, 0x54, 0x91, 0x78, 0xcf, 0x5d,
0x87, 0x42, 0x48, 0xae, 0xd0, 0x69, 0xa3, 0x8f, 0x60, 0x51, 0x8c, 0x64, 0xa4, 0x07, 0xbd, 0x9b,
0x94, 0x59, 0x8e, 0x6b, 0x62, 0x65, 0x8d, 0x6f, 0x60, 0x89, 0xc4, 0x3c, 0x3c, 0xcc, 0xe2, 0xe2,
0x26, 0x5d, 0xc4, 0xb1, 0x1d, 0xfd, 0x3f, 0x45, 0xa8, 0xc4, 0x1c, 0x30, 0xc3, 0xfe, 0xf5, 0xdc,
0xa5, 0xde, 0x85, 0xba, 0xc5, 0x9b, 0x91, 0xae, 0x8c, 0x7e, 0x9e, 0x00, 0xcb, 0xb8, 0x66, 0xc5,
0x5b, 0x14, 0xf4, 0x75, 0x28, 0x39, 0x23, 0xbb, 0xeb, 0xb9, 0x2f, 0xa8, 0xbc, 0x1c, 0x2f, 0x39,
0x23, 0x1b, 0xbb, 0x2f, 0x68, 0x74, 0xd9, 0x58, 0xbc, 0xf2, 0x65, 0x63, 0xe9, 0x75, 0x5d, 0x36,
0x4a, 0xb3, 0x5d, 0x36, 0x36, 0xa0, 0x62, 0x1b, 0x17, 0xec, 0x94, 0x5d, 0x67, 0x64, 0x37, 0xca,
0xc2, 0x89, 0x6d, 0xe3, 0x02, 0xbb, 0x2f, 0x9e, 0x8e, 0x6c, 0xb4, 0x05, 0x2b, 0x03, 0x83, 0xfa,
0xdd, 0xf8, 0x20, 0x00, 0xf8, 0x20, 0xa0, 0xce, 0xf6, 0x3f, 0x0e, 0x87, 0x01, 0xfa, 0x7d, 0xa8,
0x74, 0xda, 0x2d, 0xe6, 0x49, 0xac, 0x65, 0xcc, 0xd8, 0x6e, 0x0d, 0x16, 0x0e, 0x63, 0x8e, 0x27,
0x16, 0x2c, 0xed, 0x56, 0xe3, 0x05, 0x42, 0xa1, 0x21, 0xed, 0x75, 0x69, 0xa8, 0x30, 0x93, 0x86,
0xf4, 0x7f, 0x15, 0x61, 0xfd, 0xc8, 0x38, 0x27, 0x6f, 0xbe, 0xab, 0xcf, 0x55, 0x25, 0x9e, 0xc0,
0x2a, 0xcf, 0x02, 0xad, 0x98, 0x3c, 0xb2, 0x82, 0xab, 0xf2, 0x79, 0xcc, 0x24, 0x38, 0x8b, 0x88,
0x7e, 0x0c, 0xf5, 0x44, 0x72, 0x0d, 0x92, 0xd1, 0xa6, 0x82, 0x54, 0x22, 0x5b, 0xe3, 0x14, 0x5e,
0xa6, 0x21, 0x58, 0xbc, 0x7c, 0x43, 0x90, 0x69, 0x4c, 0x96, 0x2e, 0xdf, 0x98, 0xe8, 0x5f, 0x6a,
0x50, 0x6b, 0x1b, 0xbe, 0xf1, 0xd4, 0x35, 0xc9, 0xb3, 0x19, 0x2b, 0x65, 0x8e, 0xb9, 0xdb, 0x4d,
0x28, 0xb3, 0x58, 0xa0, 0xbe, 0x61, 0x0f, 0xb9, 0x9d, 0xe6, 0x71, 0xb4, 0xc1, 0xaa, 0x05, 0x3c,
0x3a, 0x25, 0xbd, 0xb3, 0x43, 0xd7, 0x72, 0xfc, 0x29, 0x75, 0xe2, 0x87, 0x50, 0x9a, 0xc1, 0x47,
0x43, 0x1c, 0x16, 0xc2, 0x2c, 0x51, 0xb9, 0x27, 0x22, 0x57, 0x09, 0xa7, 0x29, 0x3b, 0x23, 0xfb,
0xa7, 0x27, 0x2c, 0x5b, 0xe9, 0xbf, 0x89, 0xf5, 0x9e, 0x3c, 0xad, 0xe6, 0x29, 0x17, 0x9b, 0x10,
0x3f, 0xae, 0x4a, 0x03, 0xfb, 0x50, 0xa3, 0x84, 0x9c, 0xcd, 0x32, 0x37, 0xa9, 0x32, 0xc4, 0xd0,
0xea, 0x3f, 0x62, 0xac, 0x02, 0x5d, 0x89, 0xc6, 0xa7, 0xd2, 0xba, 0xa5, 0x30, 0x7a, 0xa4, 0x51,
0x1c, 0xc7, 0x40, 0x5b, 0xb0, 0x2c, 0xab, 0x6c, 0xd0, 0xbc, 0x71, 0x2f, 0x2e, 0xe2, 0xf4, 0xb6,
0xee, 0x41, 0x5d, 0x7e, 0x0b, 0xd7, 0xa5, 0x53, 0x4c, 0xb3, 0x07, 0xd5, 0x93, 0xa8, 0xe3, 0x99,
0x34, 0x12, 0x88, 0x35, 0x46, 0x38, 0x81, 0xa3, 0x3f, 0x84, 0x4a, 0xec, 0xe7, 0x84, 0x2e, 0xa4,
0x01, 0x4b, 0xc7, 0x31, 0x3e, 0x65, 0x1c, 0x2c, 0xf5, 0x2e, 0xd4, 0x64, 0x55, 0x12, 0xd7, 0x0e,
0xd6, 0x62, 0x72, 0xc7, 0x14, 0xfd, 0x11, 0xff, 0x46, 0xdf, 0x4f, 0x4e, 0xc0, 0xde, 0x51, 0x2a,
0x90, 0x13, 0xe1, 0x3d, 0x77, 0xbc, 0x2e, 0xe9, 0x5f, 0x68, 0x50, 0x0d, 0x82, 0x86, 0xbb, 0x48,
0x23, 0x7a, 0x22, 0x11, 0x3c, 0x82, 0x25, 0xfb, 0x73, 0x4e, 0x3c, 0x1a, 0x38, 0x6b, 0x11, 0x07,
0x4b, 0xf4, 0x03, 0x28, 0x85, 0x0d, 0x78, 0x71, 0x6c, 0x16, 0x49, 0x1c, 0x04, 0x87, 0x18, 0xfa,
0x7f, 0x35, 0x3e, 0x61, 0xc4, 0xa4, 0xe7, 0x9e, 0x13, 0xef, 0x65, 0x62, 0x8e, 0x73, 0xf9, 0x08,
0x7e, 0x10, 0x93, 0x65, 0xfa, 0x65, 0x80, 0x33, 0x0b, 0x11, 0xd0, 0x83, 0xc8, 0x10, 0xc5, 0xb1,
0xad, 0x59, 0xd2, 0x8f, 0x22, 0x5b, 0xfd, 0x51, 0x8c, 0xa3, 0x92, 0xe7, 0xb8, 0xd6, 0xe1, 0xf1,
0xf6, 0x3d, 0x58, 0xcd, 0x58, 0x1e, 0xd5, 0x01, 0x3e, 0x71, 0x7a, 0xae, 0x3d, 0x1c, 0x10, 0x9f,
0xac, 0xcc, 0xa1, 0x2a, 0x94, 0x1e, 0x05, 0x2b, 0xad, 0xf5, 0xbf, 0x1a, 0x54, 0x98, 0x43, 0x1c,
0x89, 0xe7, 0x3c, 0x34, 0x04, 0xc4, 0xe7, 0x15, 0xf6, 0xd0, 0x75, 0xc2, 0xf9, 0x22, 0xfa, 0x70,
0x4c, 0xac, 0x67, 0x41, 0xa5, 0x0a, 0x9a, 0x77, 0xc7, 0x60, 0xa4, 0xc0, 0xf5, 0x39, 0x64, 0x73,
0x8e, 0xac, 0xb3, 0x78, 0x66, 0xf5, 0xce, 0x82, 0xa6, 0x6c, 0x02, 0xc7, 0x14, 0x68, 0xc0, 0x31,
0x35, 0xb6, 0x94, 0x0b, 0x31, 0xdb, 0x0a, 0x1c, 0x4c, 0x9f, 0x43, 0x9f, 0xc3, 0x1a, 0xbb, 0xc0,
0x87, 0x73, 0x84, 0x80, 0x61, 0x6b, 0x3c, 0xc3, 0x0c, 0xf0, 0x25, 0x59, 0x1a, 0x50, 0x8d, 0xbf,
0x26, 0x22, 0xd5, 0x13, 0x87, 0xe2, 0xc1, 0xb3, 0xf9, 0xde, 0x54, 0xb8, 0x90, 0xc5, 0x3e, 0x2c,
0xf0, 0x5b, 0x02, 0x52, 0x79, 0x7f, 0xfc, 0xe5, 0xb0, 0x39, 0x69, 0xca, 0xa1, 0xcf, 0xa1, 0x5f,
0xc2, 0x72, 0xea, 0xcd, 0x06, 0xbd, 0xaf, 0x20, 0xa9, 0x7e, 0x7d, 0x6b, 0x6e, 0xe7, 0x01, 0x8d,
0xeb, 0x25, 0xfe, 0xae, 0xa1, 0xd4, 0x8b, 0xe2, 0x6d, 0x46, 0xa9, 0x17, 0xd5, 0x03, 0x89, 0x3e,
0x87, 0xfa, 0x50, 0x4f, 0x8e, 0x6b, 0xd0, 0x96, 0x02, 0x59, 0x39, 0xc1, 0x6e, 0xbe, 0x9f, 0x03,
0x32, 0x64, 0x64, 0xc3, 0x4a, 0x7a, 0x2a, 0x8f, 0xb6, 0x27, 0x12, 0x48, 0xc6, 0xcb, 0x07, 0xb9,
0x60, 0x43, 0x76, 0x2f, 0xb9, 0x17, 0x67, 0xa6, 0xc2, 0x68, 0x47, 0x4d, 0x66, 0xdc, 0xb8, 0xba,
0xb9, 0x9b, 0x1b, 0x3e, 0x64, 0x4d, 0x60, 0x35, 0x33, 0xe5, 0x45, 0x1f, 0x4c, 0xa2, 0x93, 0x9a,
0xf9, 0x34, 0xa7, 0xcf, 0xa1, 0xf5, 0x39, 0xf4, 0xa5, 0x28, 0x13, 0xaa, 0xc9, 0x29, 0xba, 0xa7,
0xe6, 0x36, 0x61, 0xe4, 0xdb, 0x6c, 0x5d, 0x06, 0x25, 0x3c, 0xeb, 0x2b, 0x9e, 0xe2, 0x15, 0xd3,
0xc7, 0x74, 0x7e, 0x0a, 0xe8, 0x8d, 0x1f, 0xab, 0x36, 0xef, 0x5d, 0x02, 0x23, 0x14, 0xc0, 0x4d,
0x3f, 0xaf, 0x04, 0xe9, 0x6a, 0x77, 0xaa, 0x73, 0xce, 0x96, 0xab, 0x3e, 0x83, 0xe5, 0xd4, 0x1d,
0x48, 0x19, 0xff, 0xea, 0x7b, 0x52, 0x8e, 0xe4, 0x92, 0xaa, 0x98, 0x68, 0x4c, 0x90, 0x29, 0xaa,
0x6a, 0x73, 0x3b, 0x0f, 0x68, 0x70, 0x90, 0xd6, 0xdf, 0x8b, 0x50, 0x0a, 0x3a, 0x9d, 0x6b, 0xa8,
0x6a, 0xd7, 0x50, 0x66, 0x3e, 0x83, 0xe5, 0xd4, 0x90, 0x55, 0xa9, 0x5d, 0xf5, 0x20, 0x76, 0x9a,
0xe9, 0x3e, 0x85, 0x5a, 0x62, 0x6a, 0x8a, 0xde, 0x1b, 0x57, 0x68, 0xd2, 0xd9, 0x7a, 0x32, 0xe1,
0xbd, 0xfb, 0x3f, 0xbf, 0xd7, 0xb7, 0xfc, 0xd3, 0xd1, 0x31, 0xfb, 0xb3, 0x2b, 0x40, 0xbf, 0x69,
0xb9, 0xf2, 0x6b, 0x37, 0x50, 0xd0, 0x2e, 0xc7, 0xde, 0x65, 0x6c, 0x86, 0xc7, 0xc7, 0x8b, 0x7c,
0x75, 0xff, 0xff, 0x01, 0x00, 0x00, 0xff, 0xff, 0xfa, 0xfd, 0xa2, 0x92, 0x8b, 0x24, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.