Skip to content
Snippets Groups Projects
Commit c53afee6 authored by Bingyi Sun 【孙秉义】's avatar Bingyi Sun 【孙秉义】 Committed by zhenshan.cao
Browse files

DataNode scales flowgraph


Signed-off-by: default avataryangxuan <xuan.yang@zilliz.com>
parent b9cf4c10
No related branches found
No related tags found
No related merge requests found
Showing
with 861 additions and 459 deletions
...@@ -139,7 +139,6 @@ message VchannelPair { ...@@ -139,7 +139,6 @@ message VchannelPair {
PositionPair dml_position = 5; PositionPair dml_position = 5;
} }
message WatchDmChannelsRequest { message WatchDmChannelsRequest {
common.MsgBase base = 1; common.MsgBase base = 1;
repeated VchannelPair vchannels = 2; repeated VchannelPair vchannels = 2;
...@@ -152,8 +151,10 @@ DataNode consists of multiple DataSyncService, each service controls one flowgra ...@@ -152,8 +151,10 @@ DataNode consists of multiple DataSyncService, each service controls one flowgra
// DataNode // DataNode
type DataNode struct { type DataNode struct {
... ...
coll2Sync map[UniqueID]*dataSyncService vchan2Sync map[string]*dataSyncService
vchan2FlushCh map[string]chan<- *flushMsg
... ...
replica Replica // TODO remove
} }
// DataSyncService // DataSyncService
...@@ -165,9 +166,33 @@ type dataSyncService struct { ...@@ -165,9 +166,33 @@ type dataSyncService struct {
idAllocator allocatorInterface idAllocator allocatorInterface
msFactory msgstream.Factory msFactory msgstream.Factory
collectionID UniqueID 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 #### The boring design
......
...@@ -44,6 +44,7 @@ type Replica interface { ...@@ -44,6 +44,7 @@ type Replica interface {
getSegmentByID(segmentID UniqueID) (*Segment, error) getSegmentByID(segmentID UniqueID) (*Segment, error)
bufferAutoFlushBinlogPaths(segmentID UniqueID, field2Path map[UniqueID]string) error bufferAutoFlushBinlogPaths(segmentID UniqueID, field2Path map[UniqueID]string) error
getBufferPaths(segID UniqueID) (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. // Segment is the data structure of segments in data node replica.
...@@ -85,6 +86,18 @@ func newReplica() Replica { ...@@ -85,6 +86,18 @@ func newReplica() Replica {
return 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 // bufferAutoFlushBinlogPaths buffers binlog paths generated by auto-flush
func (replica *CollectionSegmentReplica) bufferAutoFlushBinlogPaths(segID UniqueID, field2Path map[UniqueID]string) error { func (replica *CollectionSegmentReplica) bufferAutoFlushBinlogPaths(segID UniqueID, field2Path map[UniqueID]string) error {
replica.mu.RLock() replica.mu.RLock()
......
...@@ -21,6 +21,7 @@ import ( ...@@ -21,6 +21,7 @@ import (
"io" "io"
"math/rand" "math/rand"
"strconv" "strconv"
"sync"
"sync/atomic" "sync/atomic"
"time" "time"
...@@ -46,18 +47,14 @@ const ( ...@@ -46,18 +47,14 @@ const (
// services of data node. // services of data node.
// //
// DataNode struct implements `types.Component`, `types.DataNode` interfaces. // 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. // `masterService` holds a grpc client of master service.
// `dataService` holds a grpc client of data 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. // `vchan2SyncService` holds map of vchannlName and dataSyncService, so that datanode
// // has ability to scale flowgraph
// `State` is current statement of this data node, indicating whether it's healthy. // `vchan2FlushCh` holds flush-signal channels for every flowgraph
//
// `flushChan` transfer flush messages from data service to flowgraph of data node.
//
// `replica` holds replications of persistent data, including collections and segments.
type DataNode struct { type DataNode struct {
ctx context.Context ctx context.Context
cancel context.CancelFunc cancel context.CancelFunc
...@@ -66,17 +63,14 @@ type DataNode struct { ...@@ -66,17 +63,14 @@ type DataNode struct {
State atomic.Value // internalpb.StateCode_Initializing State atomic.Value // internalpb.StateCode_Initializing
watchDm chan struct{} watchDm chan struct{}
dataSyncService *dataSyncService vchan2SyncService map[string]*dataSyncService
metaService *metaService vchan2FlushCh map[string]chan<- *flushMsg
masterService types.MasterService masterService types.MasterService
dataService types.DataService dataService types.DataService
session *sessionutil.Session session *sessionutil.Session
flushChan chan<- *flushMsg
replica Replica
closer io.Closer closer io.Closer
msFactory msgstream.Factory msFactory msgstream.Factory
...@@ -92,12 +86,12 @@ func NewDataNode(ctx context.Context, factory msgstream.Factory) *DataNode { ...@@ -92,12 +86,12 @@ func NewDataNode(ctx context.Context, factory msgstream.Factory) *DataNode {
Role: typeutil.DataNodeRole, Role: typeutil.DataNodeRole,
watchDm: make(chan struct{}, 1), watchDm: make(chan struct{}, 1),
dataSyncService: nil, masterService: nil,
metaService: nil, dataService: nil,
masterService: nil, msFactory: factory,
dataService: nil,
replica: nil, vchan2SyncService: make(map[string]*dataSyncService),
msFactory: factory, vchan2FlushCh: make(map[string]chan<- *flushMsg),
} }
node.UpdateStateCode(internalpb.StateCode_Abnormal) node.UpdateStateCode(internalpb.StateCode_Abnormal)
return node return node
...@@ -139,13 +133,14 @@ func (node *DataNode) Register() error { ...@@ -139,13 +133,14 @@ func (node *DataNode) Register() error {
// and address. Therefore, `SetDataServiceInterface()` must be called before this func. // and address. Therefore, `SetDataServiceInterface()` must be called before this func.
// Registering return several channel names data node need. // 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`. // At last, data node initializes its `dataSyncService` and `metaService`.
func (node *DataNode) Init() error { func (node *DataNode) Init() error {
ctx := context.Background() 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{ req := &datapb.RegisterNodeRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
SourceID: node.NodeID, SourceID: node.NodeID,
...@@ -187,35 +182,37 @@ func (node *DataNode) Init() error { ...@@ -187,35 +182,37 @@ func (node *DataNode) Init() error {
zap.Any("TimeTickChannelName", Params.TimeTickChannelName), zap.Any("TimeTickChannelName", Params.TimeTickChannelName),
zap.Any("CompleteFlushChannelName", Params.TimeTickChannelName)) zap.Any("CompleteFlushChannelName", Params.TimeTickChannelName))
select { return nil
case <-time.After(RPCConnectionTimeout): }
return errors.New("Get DmChannels failed in 30 seconds")
case <-node.watchDm: // NewDataSyncService adds a new dataSyncService to DataNode
log.Debug("insert channel names set") func (node *DataNode) NewDataSyncService(vchanPair *datapb.VchannelPair) error {
if _, ok := node.vchan2SyncService[vchanPair.GetDmlVchannelName()]; ok {
return nil
} }
replica := newReplica() replica := newReplica()
var alloc allocatorInterface = newAllocator(node.masterService) var alloc allocatorInterface = newAllocator(node.masterService)
chanSize := 100 flushChan := make(chan *flushMsg, 100)
flushChan := make(chan *flushMsg, chanSize) dataSyncService := newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory, vchanPair)
node.flushChan = flushChan metaService := newMetaService(node.ctx, replica, node.masterService)
node.dataSyncService = newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory) node.vchan2SyncService[vchanPair.GetDmlVchannelName()] = dataSyncService
node.dataSyncService.init() node.vchan2FlushCh[vchanPair.GetDmlVchannelName()] = flushChan
node.metaService = newMetaService(node.ctx, replica, node.masterService)
node.replica = replica metaService.init()
go dataSyncService.start()
return nil return nil
} }
// Start `metaService` and `dataSyncService` and update state to HEALTHY // Start will update state to HEALTHY
func (node *DataNode) Start() error { func (node *DataNode) Start() error {
node.metaService.init()
go node.dataSyncService.start()
node.UpdateStateCode(internalpb.StateCode_Healthy) node.UpdateStateCode(internalpb.StateCode_Healthy)
return nil return nil
} }
// UpdateStateCode update datanode's state code
func (node *DataNode) UpdateStateCode(code internalpb.StateCode) { func (node *DataNode) UpdateStateCode(code internalpb.StateCode) {
node.State.Store(code) node.State.Store(code)
} }
...@@ -227,18 +224,20 @@ func (node *DataNode) WatchDmChannels(ctx context.Context, in *datapb.WatchDmCha ...@@ -227,18 +224,20 @@ func (node *DataNode) WatchDmChannels(ctx context.Context, in *datapb.WatchDmCha
} }
switch { 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) status.Reason = fmt.Sprintf("DataNode %d not initializing!", node.NodeID)
return status, errors.New(status.GetReason()) return status, errors.New(status.GetReason())
case len(Params.InsertChannelNames) != 0: case len(in.GetVchannels()) == 0:
status.Reason = fmt.Sprintf("DataNode %d has already set insert channels!", node.NodeID) status.Reason = "Illegal request"
return status, errors.New(status.GetReason()) return status, errors.New(status.GetReason())
default: default:
Params.InsertChannelNames = in.GetChannelNames() for _, chanPair := range in.GetVchannels() {
node.NewDataSyncService(chanPair)
}
status.ErrorCode = commonpb.ErrorCode_Success status.ErrorCode = commonpb.ErrorCode_Success
node.watchDm <- struct{}{}
return status, nil return status, nil
} }
} }
...@@ -257,32 +256,97 @@ func (node *DataNode) GetComponentStates(ctx context.Context) (*internalpb.Compo ...@@ -257,32 +256,97 @@ func (node *DataNode) GetComponentStates(ctx context.Context) (*internalpb.Compo
return states, nil 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. // FlushSegments packs flush messages into flowgraph through flushChan.
func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) { func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) {
log.Debug("FlushSegments ...", zap.Int("num", len(req.SegmentIDs))) log.Debug("FlushSegments ...", zap.Int("num", len(req.SegmentIDs)))
ids := make([]UniqueID, 0) status := &commonpb.Status{
ids = append(ids, req.SegmentIDs...) ErrorCode: commonpb.ErrorCode_UnexpectedError,
flushmsg := &flushMsg{
msgID: req.Base.MsgID,
timestamp: req.Base.Timestamp,
segmentIDs: ids,
collectionID: req.CollectionID,
} }
node.flushChan <- flushmsg for _, id := range req.SegmentIDs {
return &commonpb.Status{ chanName := node.getChannelName(id)
ErrorCode: commonpb.ErrorCode_Success, if chanName == "" {
Reason: "", status.Reason = fmt.Sprintf("DataNode not find segment %d!", id)
}, nil 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 { func (node *DataNode) Stop() error {
node.cancel() node.cancel()
// close services // close services
if node.dataSyncService != nil { for _, syncService := range node.vchan2SyncService {
(*node.dataSyncService).close() if syncService != nil {
(*syncService).close()
}
} }
if node.closer != nil { if node.closer != nil {
......
...@@ -12,10 +12,16 @@ ...@@ -12,10 +12,16 @@
package datanode package datanode
import ( import (
"math"
"os"
"testing" "testing"
"github.com/stretchr/testify/assert" "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/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
...@@ -24,31 +30,56 @@ import ( ...@@ -24,31 +30,56 @@ import (
func TestMain(t *testing.M) { func TestMain(t *testing.M) {
Params.Init() Params.Init()
refreshChannelNames() refreshChannelNames()
code := t.Run()
os.Exit(code)
} }
func TestDataNode(t *testing.T) { func TestDataNode(t *testing.T) {
node := newDataNodeMock() node := newIDLEDataNodeMock()
node.Start() node.Start()
t.Run("Test WatchDmChannels", func(t *testing.T) { t.Run("Test WatchDmChannels", func(t *testing.T) {
channelNames := Params.InsertChannelNames 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{ req := &datapb.WatchDmChannelsRequest{
ChannelNames: channelNames, 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) _, err := node1.WatchDmChannels(node.ctx, req)
_, err2 := node.WatchDmChannels(node.ctx, req) assert.NoError(t, err)
assert.Error(t, err2) assert.NotNil(t, node1.vchan2FlushCh)
assert.NotNil(t, node1.vchan2SyncService)
Params.InsertChannelNames = []string{} sync, ok := node1.vchan2SyncService[Params.InsertChannelNames[0]]
status, err3 := node.WatchDmChannels(node.ctx, req) assert.True(t, ok)
assert.NoError(t, err3) assert.NotNil(t, sync)
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) assert.Equal(t, UniqueID(1), sync.collectionID)
assert.Equal(t, 2, len(node1.vchan2SyncService))
assert.Equal(t, len(node1.vchan2FlushCh), len(node1.vchan2SyncService))
_, 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) { t.Run("Test GetComponentStates", func(t *testing.T) {
...@@ -57,16 +88,102 @@ func TestDataNode(t *testing.T) { ...@@ -57,16 +88,102 @@ func TestDataNode(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, stat.Status.ErrorCode) 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) { 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{ req := &datapb.FlushSegmentsRequest{
Base: &commonpb.MsgBase{}, Base: &commonpb.MsgBase{},
DbID: 0, DbID: 0,
CollectionID: 1, CollectionID: 1,
SegmentIDs: []int64{0, 1}, SegmentIDs: []int64{0, 1},
} }
status, err := node.FlushSegments(node.ctx, req) status, err := node1.FlushSegments(node.ctx, req)
assert.NoError(t, err) assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) 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) { t.Run("Test GetTimeTickChannel", func(t *testing.T) {
......
...@@ -18,6 +18,7 @@ import ( ...@@ -18,6 +18,7 @@ import (
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/msgstream" "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/flowgraph"
"github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/retry"
"go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3"
...@@ -26,35 +27,44 @@ import ( ...@@ -26,35 +27,44 @@ import (
) )
type dataSyncService struct { type dataSyncService struct {
ctx context.Context ctx context.Context
fg *flowgraph.TimeTickedFlowGraph fg *flowgraph.TimeTickedFlowGraph
flushChan <-chan *flushMsg flushChan <-chan *flushMsg
replica Replica replica Replica
idAllocator allocatorInterface idAllocator allocatorInterface
msFactory msgstream.Factory msFactory msgstream.Factory
collectionID UniqueID
} }
func newDataSyncService(ctx context.Context, flushChan <-chan *flushMsg, func newDataSyncService(ctx context.Context,
replica Replica, alloc allocatorInterface, factory msgstream.Factory) *dataSyncService { flushChan <-chan *flushMsg,
replica Replica,
alloc allocatorInterface,
factory msgstream.Factory,
vchanPair *datapb.VchannelPair) *dataSyncService {
service := &dataSyncService{ service := &dataSyncService{
ctx: ctx, ctx: ctx,
fg: nil, fg: nil,
flushChan: flushChan, flushChan: flushChan,
replica: replica, replica: replica,
idAllocator: alloc, idAllocator: alloc,
msFactory: factory, msFactory: factory,
collectionID: vchanPair.GetCollectionID(),
} }
service.initNodes(vchanPair)
return service return service
} }
func (dsService *dataSyncService) init() { // func (dsService *dataSyncService) init() {
if len(Params.InsertChannelNames) == 0 { // if len(Params.InsertChannelNames) == 0 {
log.Error("InsertChannels not readly, init datasync service failed") // log.Error("InsertChannels not readly, init datasync service failed")
return // return
} // }
dsService.initNodes() // dsService.initNodes()
} // }
func (dsService *dataSyncService) start() { func (dsService *dataSyncService) start() {
log.Debug("Data Sync Service Start Successfully") log.Debug("Data Sync Service Start Successfully")
...@@ -71,7 +81,7 @@ func (dsService *dataSyncService) close() { ...@@ -71,7 +81,7 @@ func (dsService *dataSyncService) close() {
} }
} }
func (dsService *dataSyncService) initNodes() { func (dsService *dataSyncService) initNodes(vchanPair *datapb.VchannelPair) {
// TODO: add delete pipeline support // TODO: add delete pipeline support
var kvClient *clientv3.Client var kvClient *clientv3.Client
var err error var err error
...@@ -96,14 +106,16 @@ func (dsService *dataSyncService) initNodes() { ...@@ -96,14 +106,16 @@ func (dsService *dataSyncService) initNodes() {
m := map[string]interface{}{ m := map[string]interface{}{
"PulsarAddress": Params.PulsarAddress, "PulsarAddress": Params.PulsarAddress,
"ReceiveBufSize": 1024, "ReceiveBufSize": 1024,
"PulsarBufSize": 1024} "PulsarBufSize": 1024,
}
err = dsService.msFactory.SetParams(m) err = dsService.msFactory.SetParams(m)
if err != nil { if err != nil {
panic(err) panic(err)
} }
var dmStreamNode Node = newDmInputNode(dsService.ctx, dsService.msFactory) var dmStreamNode Node = newDmInputNode(dsService.ctx, dsService.msFactory, vchanPair.GetDmlVchannelName(), vchanPair.GetDmlPosition())
var ddStreamNode Node = newDDInputNode(dsService.ctx, dsService.msFactory) var ddStreamNode Node = newDDInputNode(dsService.ctx, dsService.msFactory, vchanPair.GetDdlVchannelName(), vchanPair.GetDdlPosition())
var filterDmNode Node = newFilteredDmNode() var filterDmNode Node = newFilteredDmNode()
var ddNode Node = newDDNode(dsService.ctx, mt, dsService.flushChan, dsService.replica, dsService.idAllocator) var ddNode Node = newDDNode(dsService.ctx, mt, dsService.flushChan, dsService.replica, dsService.idAllocator)
......
...@@ -21,23 +21,17 @@ import ( ...@@ -21,23 +21,17 @@ import (
"github.com/milvus-io/milvus/internal/msgstream" "github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
) )
// NOTE: start pulsar before test // NOTE: start pulsar before test
func TestDataSyncService_Start(t *testing.T) { func TestDataSyncService_Start(t *testing.T) {
const ctxTimeInMillisecond = 2000 const ctxTimeInMillisecond = 2000
const closeWithDeadline = true
var ctx context.Context delay := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), delay)
if closeWithDeadline { defer cancel()
var cancel context.CancelFunc
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel = context.WithDeadline(context.Background(), d)
defer cancel()
} else {
ctx = context.Background()
}
// init data node // init data node
pulsarURL := Params.PulsarAddress pulsarURL := Params.PulsarAddress
...@@ -45,19 +39,37 @@ func TestDataSyncService_Start(t *testing.T) { ...@@ -45,19 +39,37 @@ func TestDataSyncService_Start(t *testing.T) {
Factory := &MetaFactory{} Factory := &MetaFactory{}
collMeta := Factory.CollectionMetaFactory(UniqueID(0), "coll1") collMeta := Factory.CollectionMetaFactory(UniqueID(0), "coll1")
chanSize := 100 flushChan := make(chan *flushMsg, 100)
flushChan := make(chan *flushMsg, chanSize)
replica := newReplica() replica := newReplica()
allocFactory := AllocatorFactory{} allocFactory := NewAllocatorFactory(1)
msFactory := msgstream.NewPmsFactory() msFactory := msgstream.NewPmsFactory()
m := map[string]interface{}{ m := map[string]interface{}{
"pulsarAddress": pulsarURL, "pulsarAddress": pulsarURL,
"receiveBufSize": 1024, "receiveBufSize": 1024,
"pulsarBufSize": 1024} "pulsarBufSize": 1024}
err := msFactory.SetParams(m) 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.replica.addCollection(collMeta.ID, collMeta.Schema)
sync.init()
go sync.start() go sync.start()
timeRange := TimeRange{ timeRange := TimeRange{
...@@ -65,12 +77,18 @@ func TestDataSyncService_Start(t *testing.T) { ...@@ -65,12 +77,18 @@ func TestDataSyncService_Start(t *testing.T) {
timestampMax: math.MaxUint64, timestampMax: math.MaxUint64,
} }
dataFactory := NewDataFactory() dataFactory := NewDataFactory()
insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2) insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2, insertChannelName)
msgPack := msgstream.MsgPack{ msgPack := msgstream.MsgPack{
BeginTs: timeRange.timestampMin, BeginTs: timeRange.timestampMin,
EndTs: timeRange.timestampMax, EndTs: timeRange.timestampMax,
Msgs: insertMessages, Msgs: insertMessages,
StartPositions: []*internalpb.MsgPosition{{
ChannelName: insertChannelName,
}},
EndPositions: []*internalpb.MsgPosition{{
ChannelName: insertChannelName,
}},
} }
// generate timeTick // generate timeTick
...@@ -94,15 +112,12 @@ func TestDataSyncService_Start(t *testing.T) { ...@@ -94,15 +112,12 @@ func TestDataSyncService_Start(t *testing.T) {
timeTickMsgPack.Msgs = append(timeTickMsgPack.Msgs, timeTickMsg) timeTickMsgPack.Msgs = append(timeTickMsgPack.Msgs, timeTickMsg)
// pulsar produce // pulsar produce
insertChannels := Params.InsertChannelNames
ddChannels := Params.DDChannelNames
assert.NoError(t, err) assert.NoError(t, err)
insertStream, _ := msFactory.NewMsgStream(ctx) insertStream, _ := msFactory.NewMsgStream(ctx)
insertStream.AsProducer(insertChannels) insertStream.AsProducer([]string{insertChannelName})
ddStream, _ := msFactory.NewMsgStream(ctx) ddStream, _ := msFactory.NewMsgStream(ctx)
ddStream.AsProducer(ddChannels) ddStream.AsProducer([]string{ddlChannelName})
var insertMsgStream msgstream.MsgStream = insertStream var insertMsgStream msgstream.MsgStream = insertStream
insertMsgStream.Start() insertMsgStream.Start()
...@@ -119,7 +134,6 @@ func TestDataSyncService_Start(t *testing.T) { ...@@ -119,7 +134,6 @@ func TestDataSyncService_Start(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
// dataSync // dataSync
Params.FlushInsertBufferSize = 1
<-sync.ctx.Done() <-sync.ctx.Done()
sync.close() sync.close()
......
...@@ -109,7 +109,7 @@ func (ddNode *ddNode) Operate(in []flowgraph.Msg) []flowgraph.Msg { ...@@ -109,7 +109,7 @@ func (ddNode *ddNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
timestampMin: msMsg.TimestampMin(), timestampMin: msMsg.TimestampMin(),
timestampMax: msMsg.TimestampMax(), timestampMax: msMsg.TimestampMax(),
}, },
flushMessages: make([]*flushMsg, 0), // flushMessages: make([]*flushMsg, 0),
gcRecord: &gcRecord{ gcRecord: &gcRecord{
collections: make([]UniqueID, 0), collections: make([]UniqueID, 0),
}, },
...@@ -141,34 +141,29 @@ func (ddNode *ddNode) Operate(in []flowgraph.Msg) []flowgraph.Msg { ...@@ -141,34 +141,29 @@ func (ddNode *ddNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
select { select {
case fmsg := <-ddNode.inFlushCh: case fmsg := <-ddNode.inFlushCh:
log.Debug(". receive flush message ...") log.Debug(". receive flush message ...")
localSegs := make([]UniqueID, 0, len(fmsg.segmentIDs)) segID := fmsg.segmentID
for _, segID := range fmsg.segmentIDs { if !ddNode.replica.hasSegment(segID) {
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 {
log.Debug(".. Segment not exist in this datanode, skip flushing ...") log.Debug(".. Segment not exist in this datanode, skip flushing ...")
break 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 ...") log.Debug(".. notifying insertbuffer ...")
fmsg.segmentIDs = localSegs ddNode.ddMsg.flushMessage = fmsg
ddNode.ddMsg.flushMessages = append(ddNode.ddMsg.flushMessages, fmsg)
default: default:
} }
...@@ -181,7 +176,7 @@ func (ddNode *ddNode) Operate(in []flowgraph.Msg) []flowgraph.Msg { ...@@ -181,7 +176,7 @@ func (ddNode *ddNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
return []Msg{res} 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 binlogMeta := <-binlogMetaCh
if binlogMeta == nil { if binlogMeta == nil {
return return
...@@ -192,6 +187,10 @@ func (ddNode *ddNode) flushComplete(binlogMetaCh <-chan *datapb.DDLBinlogMeta, c ...@@ -192,6 +187,10 @@ func (ddNode *ddNode) flushComplete(binlogMetaCh <-chan *datapb.DDLBinlogMeta, c
if err != nil { if err != nil {
log.Error("Save binlog meta to etcd Wrong", zap.Error(err)) log.Error("Save binlog meta to etcd Wrong", zap.Error(err))
} }
ddlFlushedCh <- true
// TODO remove above
// ddlFlushCh <- binlogMetaCh
} }
/* /*
......
...@@ -151,7 +151,7 @@ func TestFlowGraphDDNode_Operate(t *testing.T) { ...@@ -151,7 +151,7 @@ func TestFlowGraphDDNode_Operate(t *testing.T) {
inFlushCh <- &flushMsg{ inFlushCh <- &flushMsg{
msgID: 5, msgID: 5,
timestamp: 5, timestamp: 5,
segmentIDs: []UniqueID{1}, segmentID: UniqueID(1),
collectionID: collID, collectionID: collID,
} }
......
...@@ -66,7 +66,7 @@ func (fdmNode *filterDmNode) Operate(in []flowgraph.Msg) []flowgraph.Msg { ...@@ -66,7 +66,7 @@ func (fdmNode *filterDmNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
var iMsg = insertMsg{ var iMsg = insertMsg{
insertMessages: make([]*msgstream.InsertMsg, 0), insertMessages: make([]*msgstream.InsertMsg, 0),
flushMessages: make([]*flushMsg, 0), // flushMessages: make([]*flushMsg, 0),
timeRange: TimeRange{ timeRange: TimeRange{
timestampMin: msgStreamMsg.TimestampMin(), timestampMin: msgStreamMsg.TimestampMin(),
timestampMax: msgStreamMsg.TimestampMax(), timestampMax: msgStreamMsg.TimestampMax(),
...@@ -75,7 +75,8 @@ func (fdmNode *filterDmNode) Operate(in []flowgraph.Msg) []flowgraph.Msg { ...@@ -75,7 +75,8 @@ func (fdmNode *filterDmNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
endPositions: make([]*internalpb.MsgPosition, 0), 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() { for _, msg := range msgStreamMsg.TsMessages() {
switch msg.Type() { switch msg.Type() {
......
...@@ -93,6 +93,7 @@ func (ib *insertBuffer) size(segmentID UniqueID) int32 { ...@@ -93,6 +93,7 @@ func (ib *insertBuffer) size(segmentID UniqueID) int32 {
} }
func (ib *insertBuffer) full(segmentID UniqueID) bool { 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 return ib.size(segmentID) >= ib.maxSize
} }
...@@ -493,22 +494,49 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg { ...@@ -493,22 +494,49 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
} }
} }
// iMsg is Flush() msg from dataservice for _, segToFlush := range segToUpdate {
// 1. insertBuffer(not empty) -> binLogs -> minIO/S3 // If full, auto flush
for _, msg := range iMsg.flushMessages { if ibNode.insertBuffer.full(segToFlush) {
for _, currentSegID := range msg.segmentIDs { log.Debug(". Insert Buffer full, auto flushing ",
log.Debug(". Receiving flush message", zap.Int64("segmentID", currentSegID)) zap.Int32("num of rows", ibNode.insertBuffer.size(segToFlush)))
// finishCh := make(chan bool) collMeta, err := ibNode.getCollMetabySegID(segToFlush)
finishCh := make(chan map[UniqueID]string) if err != nil {
go ibNode.completeFlush(currentSegID, finishCh) log.Error("Auto flush failed .. cannot get collection meta ..", zap.Error(err))
continue
}
if ibNode.insertBuffer.size(currentSegID) <= 0 { ibNode.flushMap.Store(segToFlush, ibNode.insertBuffer.insertData[segToFlush])
log.Debug(".. Buffer empty ...") delete(ibNode.insertBuffer.insertData, segToFlush)
finishCh <- make(map[UniqueID]string)
collID, partitionID, err := ibNode.getCollectionandPartitionIDbySegID(segToFlush)
if err != nil {
log.Error("Auto flush failed .. cannot get collection ID or partition ID..", zap.Error(err))
continue continue
} }
finishCh := make(chan map[UniqueID]string)
go flushSegment(collMeta, segToFlush, partitionID, collID,
&ibNode.flushMap, ibNode.minIOKV, finishCh, ibNode.idAllocator)
go ibNode.bufferAutoFlushPaths(finishCh, segToFlush)
}
}
// 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 ..") log.Debug(".. Buffer not empty, flushing ..")
ibNode.flushMap.Store(currentSegID, ibNode.insertBuffer.insertData[currentSegID]) ibNode.flushMap.Store(currentSegID, ibNode.insertBuffer.insertData[currentSegID])
delete(ibNode.insertBuffer.insertData, currentSegID) delete(ibNode.insertBuffer.insertData, currentSegID)
...@@ -518,21 +546,23 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg { ...@@ -518,21 +546,23 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
ibNode.flushMap.Delete(currentSegID) ibNode.flushMap.Delete(currentSegID)
} }
var collMeta *etcdpb.CollectionMeta
var collSch *schemapb.CollectionSchema
seg, err := ibNode.replica.getSegmentByID(currentSegID) seg, err := ibNode.replica.getSegmentByID(currentSegID)
if err != nil { if err != nil {
log.Error("Flush failed .. cannot get segment ..", zap.Error(err)) log.Error("Flush failed .. cannot get segment ..", zap.Error(err))
clearFn() clearFn()
continue // TODO add error handling
} }
collSch, err := ibNode.getCollectionSchemaByID(seg.collectionID) collSch, err = ibNode.getCollectionSchemaByID(seg.collectionID)
if err != nil { if err != nil {
log.Error("Flush failed .. cannot get collection schema ..", zap.Error(err)) log.Error("Flush failed .. cannot get collection schema ..", zap.Error(err))
clearFn() clearFn()
continue // TODO add error handling
} }
collMeta := &etcdpb.CollectionMeta{ collMeta = &etcdpb.CollectionMeta{
Schema: collSch, Schema: collSch,
ID: seg.collectionID, ID: seg.collectionID,
} }
...@@ -540,34 +570,7 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg { ...@@ -540,34 +570,7 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
go flushSegment(collMeta, currentSegID, seg.partitionID, seg.collectionID, go flushSegment(collMeta, currentSegID, seg.partitionID, seg.collectionID,
&ibNode.flushMap, ibNode.minIOKV, finishCh, ibNode.idAllocator) &ibNode.flushMap, ibNode.minIOKV, finishCh, ibNode.idAllocator)
} }
}
for _, segToFlush := range segToUpdate {
// If full, auto flush
if ibNode.insertBuffer.full(segToFlush) {
log.Debug(". Insert Buffer full, auto flushing ",
zap.Int32("num of rows", ibNode.insertBuffer.size(segToFlush)))
collMeta, err := ibNode.getCollMetabySegID(segToFlush)
if err != nil {
log.Error("Auto flush failed .. cannot get collection meta ..", zap.Error(err))
continue
}
ibNode.flushMap.Store(segToFlush, ibNode.insertBuffer.insertData[segToFlush])
delete(ibNode.insertBuffer.insertData, segToFlush)
collID, partitionID, err := ibNode.getCollectionandPartitionIDbySegID(segToFlush)
if err != nil {
log.Error("Auto flush failed .. cannot get collection ID or partition ID..", zap.Error(err))
continue
}
finishCh := make(chan map[UniqueID]string)
go flushSegment(collMeta, segToFlush, partitionID, collID,
&ibNode.flushMap, ibNode.minIOKV, finishCh, ibNode.idAllocator)
go ibNode.bufferAutoFlushPaths(finishCh, segToFlush)
}
} }
if err := ibNode.writeHardTimeTick(iMsg.timeRange.timestampMax); err != nil { if err := ibNode.writeHardTimeTick(iMsg.timeRange.timestampMax); err != nil {
...@@ -685,14 +688,17 @@ func (ibNode *insertBufferNode) bufferAutoFlushPaths(wait <-chan map[UniqueID]st ...@@ -685,14 +688,17 @@ func (ibNode *insertBufferNode) bufferAutoFlushPaths(wait <-chan map[UniqueID]st
return ibNode.replica.bufferAutoFlushBinlogPaths(segID, field2Path) 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 field2Path := <-wait
if field2Path == nil { if field2Path == nil {
return return
} }
dmlFlushedCh <- true
// TODO Call DataService RPC SaveBinlogPaths // TODO Call DataService RPC SaveBinlogPaths
// TODO GetBufferedAutoFlushBinlogPaths
ibNode.replica.bufferAutoFlushBinlogPaths(segID, field2Path) ibNode.replica.bufferAutoFlushBinlogPaths(segID, field2Path)
bufferField2Paths, err := ibNode.replica.getBufferPaths(segID) bufferField2Paths, err := ibNode.replica.getBufferPaths(segID)
if err != nil { if err != nil {
......
...@@ -70,12 +70,19 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) { ...@@ -70,12 +70,19 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
iBNode := newInsertBufferNode(ctx, newBinlogMeta(), replica, msFactory, NewAllocatorFactory()) 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 var iMsg flowgraph.Msg = &inMsg
iBNode.Operate([]flowgraph.Msg{iMsg}) iBNode.Operate([]flowgraph.Msg{iMsg})
isflushed := <-dmlFlushedCh
assert.True(t, isflushed)
} }
func genInsertMsg() insertMsg { func genInsertMsg(ddlFlushedCh, dmlFlushedCh chan<- bool) insertMsg {
timeRange := TimeRange{ timeRange := TimeRange{
timestampMin: 0, timestampMin: 0,
...@@ -92,7 +99,7 @@ func genInsertMsg() insertMsg { ...@@ -92,7 +99,7 @@ func genInsertMsg() insertMsg {
var iMsg = &insertMsg{ var iMsg = &insertMsg{
insertMessages: make([]*msgstream.InsertMsg, 0), insertMessages: make([]*msgstream.InsertMsg, 0),
flushMessages: make([]*flushMsg, 0), // flushMessages: make([]*flushMsg, 0),
timeRange: TimeRange{ timeRange: TimeRange{
timestampMin: timeRange.timestampMin, timestampMin: timeRange.timestampMin,
timestampMax: timeRange.timestampMax, timestampMax: timeRange.timestampMax,
...@@ -104,14 +111,15 @@ func genInsertMsg() insertMsg { ...@@ -104,14 +111,15 @@ func genInsertMsg() insertMsg {
dataFactory := NewDataFactory() dataFactory := NewDataFactory()
iMsg.insertMessages = append(iMsg.insertMessages, dataFactory.GetMsgStreamInsertMsgs(2)...) iMsg.insertMessages = append(iMsg.insertMessages, dataFactory.GetMsgStreamInsertMsgs(2)...)
fmsg := &flushMsg{ iMsg.flushMessage = &flushMsg{
msgID: 1, msgID: 1,
timestamp: 2000, timestamp: 2000,
segmentIDs: []UniqueID{1}, segmentID: UniqueID(1),
collectionID: UniqueID(1), collectionID: UniqueID(1),
ddlFlushedCh: ddlFlushedCh,
dmlFlushedCh: dmlFlushedCh,
} }
iMsg.flushMessages = append(iMsg.flushMessages, fmsg)
return *iMsg return *iMsg
} }
......
...@@ -30,7 +30,7 @@ type key2SegMsg struct { ...@@ -30,7 +30,7 @@ type key2SegMsg struct {
type ddMsg struct { type ddMsg struct {
collectionRecords map[UniqueID][]*metaOperateRecord collectionRecords map[UniqueID][]*metaOperateRecord
partitionRecords map[UniqueID][]*metaOperateRecord partitionRecords map[UniqueID][]*metaOperateRecord
flushMessages []*flushMsg flushMessage *flushMsg
gcRecord *gcRecord gcRecord *gcRecord
timeRange TimeRange timeRange TimeRange
} }
...@@ -42,7 +42,7 @@ type metaOperateRecord struct { ...@@ -42,7 +42,7 @@ type metaOperateRecord struct {
type insertMsg struct { type insertMsg struct {
insertMessages []*msgstream.InsertMsg insertMessages []*msgstream.InsertMsg
flushMessages []*flushMsg flushMessage *flushMsg
gcRecord *gcRecord gcRecord *gcRecord
timeRange TimeRange timeRange TimeRange
startPositions []*internalpb.MsgPosition startPositions []*internalpb.MsgPosition
...@@ -66,8 +66,10 @@ type gcRecord struct { ...@@ -66,8 +66,10 @@ type gcRecord struct {
type flushMsg struct { type flushMsg struct {
msgID UniqueID msgID UniqueID
timestamp Timestamp timestamp Timestamp
segmentIDs []UniqueID segmentID UniqueID
collectionID UniqueID collectionID UniqueID
ddlFlushedCh chan<- bool
dmlFlushedCh chan<- bool
} }
func (ksMsg *key2SegMsg) TimeTick() Timestamp { func (ksMsg *key2SegMsg) TimeTick() Timestamp {
......
...@@ -13,38 +13,40 @@ package datanode ...@@ -13,38 +13,40 @@ package datanode
import ( import (
"context" "context"
"strings"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/msgstream" "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/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 maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism maxParallelism := Params.FlowGraphMaxParallelism
consumeChannels := Params.InsertChannelNames // consumeChannels := Params.InsertChannelNames
consumeSubName := Params.MsgChannelSubName consumeSubName := Params.MsgChannelSubName
insertStream, _ := factory.NewTtMsgStream(ctx) insertStream, _ := factory.NewTtMsgStream(ctx)
insertStream.AsConsumer(consumeChannels, consumeSubName) insertStream.AsConsumer([]string{vchannelName}, consumeSubName)
log.Debug("datanode AsConsumer: " + strings.Join(consumeChannels, ", ") + " : " + consumeSubName) log.Debug("datanode AsConsumer: " + vchannelName + " : " + consumeSubName)
var stream msgstream.MsgStream = insertStream var stream msgstream.MsgStream = insertStream
node := flowgraph.NewInputNode(&stream, "dmInputNode", maxQueueLength, maxParallelism) node := flowgraph.NewInputNode(&stream, "dmInputNode", maxQueueLength, maxParallelism)
return node 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 maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism maxParallelism := Params.FlowGraphMaxParallelism
consumeSubName := Params.MsgChannelSubName consumeSubName := Params.MsgChannelSubName
tmpStream, _ := factory.NewTtMsgStream(ctx) tmpStream, _ := factory.NewTtMsgStream(ctx)
tmpStream.AsConsumer(Params.DDChannelNames, consumeSubName) tmpStream.AsConsumer([]string{vchannelName}, consumeSubName)
log.Debug("datanode AsConsumer: " + strings.Join(Params.DDChannelNames, ", ") + " : " + consumeSubName) log.Debug("datanode AsConsumer: " + vchannelName + " : " + consumeSubName)
var stream msgstream.MsgStream = tmpStream var stream msgstream.MsgStream = tmpStream
node := flowgraph.NewInputNode(&stream, "ddInputNode", maxQueueLength, maxParallelism) node := flowgraph.NewInputNode(&stream, "ddInputNode", maxQueueLength, maxParallelism)
......
...@@ -32,6 +32,7 @@ import ( ...@@ -32,6 +32,7 @@ import (
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/proto/commonpb" "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/etcdpb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/masterpb" "github.com/milvus-io/milvus/internal/proto/masterpb"
...@@ -42,7 +43,7 @@ import ( ...@@ -42,7 +43,7 @@ import (
const ctxTimeInMillisecond = 5000 const ctxTimeInMillisecond = 5000
const debug = false const debug = false
func newDataNodeMock() *DataNode { func newIDLEDataNodeMock() *DataNode {
var ctx context.Context var ctx context.Context
if debug { if debug {
...@@ -59,27 +60,59 @@ func newDataNodeMock() *DataNode { ...@@ -59,27 +60,59 @@ func newDataNodeMock() *DataNode {
msFactory := msgstream.NewPmsFactory() msFactory := msgstream.NewPmsFactory()
node := NewDataNode(ctx, msFactory) node := NewDataNode(ctx, msFactory)
replica := newReplica()
ms := &MasterServiceFactory{ ms := &MasterServiceFactory{
ID: 0, ID: 0,
collectionID: 1, collectionID: 1,
collectionName: "collection-1", collectionName: "collection-1",
} }
node.SetMasterServiceInterface(ms)
ds := &DataServiceFactory{}
node.SetDataServiceInterface(ds)
return node
}
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) node.SetMasterServiceInterface(ms)
ds := &DataServiceFactory{} ds := &DataServiceFactory{}
node.SetDataServiceInterface(ds) node.SetDataServiceInterface(ds)
var alloc allocatorInterface = NewAllocatorFactory(100) vpair := &datapb.VchannelPair{
CollectionID: 1,
DmlVchannelName: dmChannelName,
DdlVchannelName: ddChannelName,
DdlPosition: &datapb.PositionPair{},
DmlPosition: &datapb.PositionPair{},
}
chanSize := 100 _ = node.NewDataSyncService(vpair)
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 node return node
} }
...@@ -160,6 +193,28 @@ type DataServiceFactory struct { ...@@ -160,6 +193,28 @@ type DataServiceFactory struct {
types.DataService 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 { func (mf *MetaFactory) CollectionMetaFactory(collectionID UniqueID, collectionName string) *etcdpb.CollectionMeta {
sch := schemapb.CollectionSchema{ sch := schemapb.CollectionSchema{
Name: collectionName, Name: collectionName,
...@@ -377,7 +432,7 @@ func GenRowData() (rawData []byte) { ...@@ -377,7 +432,7 @@ func GenRowData() (rawData []byte) {
return return
} }
func (df *DataFactory) GenMsgStreamInsertMsg(idx int) *msgstream.InsertMsg { func (df *DataFactory) GenMsgStreamInsertMsg(idx int, chanName string) *msgstream.InsertMsg {
var msg = &msgstream.InsertMsg{ var msg = &msgstream.InsertMsg{
BaseMsg: msgstream.BaseMsg{ BaseMsg: msgstream.BaseMsg{
HashValues: []uint32{uint32(idx)}, HashValues: []uint32{uint32(idx)},
...@@ -385,14 +440,14 @@ func (df *DataFactory) GenMsgStreamInsertMsg(idx int) *msgstream.InsertMsg { ...@@ -385,14 +440,14 @@ func (df *DataFactory) GenMsgStreamInsertMsg(idx int) *msgstream.InsertMsg {
InsertRequest: internalpb.InsertRequest{ InsertRequest: internalpb.InsertRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Insert, MsgType: commonpb.MsgType_Insert,
MsgID: 0, // GOOSE TODO MsgID: 0,
Timestamp: Timestamp(idx + 1000), Timestamp: Timestamp(idx + 1000),
SourceID: 0, SourceID: 0,
}, },
CollectionName: "col1", // GOOSE TODO CollectionName: "col1",
PartitionName: "default", PartitionName: "default",
SegmentID: 1, // GOOSE TODO SegmentID: 1,
ChannelID: "0", // GOOSE TODO ChannelID: chanName,
Timestamps: []Timestamp{Timestamp(idx + 1000)}, Timestamps: []Timestamp{Timestamp(idx + 1000)},
RowIDs: []UniqueID{UniqueID(idx)}, RowIDs: []UniqueID{UniqueID(idx)},
RowData: []*commonpb.Blob{{Value: df.rawData}}, RowData: []*commonpb.Blob{{Value: df.rawData}},
...@@ -401,9 +456,9 @@ func (df *DataFactory) GenMsgStreamInsertMsg(idx int) *msgstream.InsertMsg { ...@@ -401,9 +456,9 @@ func (df *DataFactory) GenMsgStreamInsertMsg(idx int) *msgstream.InsertMsg {
return msg 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++ { for i := 0; i < n; i++ {
var msg = df.GenMsgStreamInsertMsg(i) var msg = df.GenMsgStreamInsertMsg(i, chanName)
var tsMsg msgstream.TsMsg = msg var tsMsg msgstream.TsMsg = msg
inMsgs = append(inMsgs, tsMsg) inMsgs = append(inMsgs, tsMsg)
} }
...@@ -412,7 +467,7 @@ func (df *DataFactory) GetMsgStreamTsInsertMsgs(n int) (inMsgs []msgstream.TsMsg ...@@ -412,7 +467,7 @@ func (df *DataFactory) GetMsgStreamTsInsertMsgs(n int) (inMsgs []msgstream.TsMsg
func (df *DataFactory) GetMsgStreamInsertMsgs(n int) (inMsgs []*msgstream.InsertMsg) { func (df *DataFactory) GetMsgStreamInsertMsgs(n int) (inMsgs []*msgstream.InsertMsg) {
for i := 0; i < n; i++ { for i := 0; i < n; i++ {
var msg = df.GenMsgStreamInsertMsg(i) var msg = df.GenMsgStreamInsertMsg(i, "")
inMsgs = append(inMsgs, msg) inMsgs = append(inMsgs, msg)
} }
return return
...@@ -436,7 +491,7 @@ func NewAllocatorFactory(id ...UniqueID) *AllocatorFactory { ...@@ -436,7 +491,7 @@ func NewAllocatorFactory(id ...UniqueID) *AllocatorFactory {
func (alloc *AllocatorFactory) allocID() (UniqueID, error) { func (alloc *AllocatorFactory) allocID() (UniqueID, error) {
alloc.Lock() alloc.Lock()
defer alloc.Unlock() 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) { func (alloc *AllocatorFactory) genKey(isalloc bool, ids ...UniqueID) (key string, err error) {
......
...@@ -107,7 +107,7 @@ func (c *dataNodeCluster) WatchInsertChannels(channels []string) { ...@@ -107,7 +107,7 @@ func (c *dataNodeCluster) WatchInsertChannels(channels []string) {
Timestamp: 0, // todo Timestamp: 0, // todo
SourceID: Params.NodeID, SourceID: Params.NodeID,
}, },
ChannelNames: group, // ChannelNames: group, // TODO
}) })
if err = VerifyResponse(resp, err); err != nil { if err = VerifyResponse(resp, err); err != nil {
log.Error("watch dm channels error", zap.Stringer("dataNode", c.nodes[i]), zap.Error(err)) log.Error("watch dm channels error", zap.Stringer("dataNode", c.nodes[i]), zap.Error(err))
......
...@@ -116,7 +116,7 @@ func (c *cluster) watch(nodes []*datapb.DataNodeInfo) []*datapb.DataNodeInfo { ...@@ -116,7 +116,7 @@ func (c *cluster) watch(nodes []*datapb.DataNodeInfo) []*datapb.DataNodeInfo {
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
SourceID: Params.NodeID, SourceID: Params.NodeID,
}, },
ChannelNames: uncompletes, // ChannelNames: uncompletes, // TODO
} }
resp, err := cli.WatchDmChannels(context.Background(), req) resp, err := cli.WatchDmChannels(context.Background(), req)
if err != nil { if err != nil {
......
...@@ -77,7 +77,7 @@ func (s *Server) RegisterNode(ctx context.Context, req *datapb.RegisterNodeReque ...@@ -77,7 +77,7 @@ func (s *Server) RegisterNode(ctx context.Context, req *datapb.RegisterNodeReque
Timestamp: 0, Timestamp: 0,
SourceID: Params.NodeID, SourceID: Params.NodeID,
}, },
ChannelNames: s.insertChannels, // ChannelNames: s.insertChannels, // TODO
}) })
if err = VerifyResponse(resp, err); err != nil { if err = VerifyResponse(resp, err); err != nil {
......
...@@ -169,9 +169,17 @@ message GetPartitionStatisticsResponse { ...@@ -169,9 +169,17 @@ message GetPartitionStatisticsResponse {
message GetSegmentInfoChannelRequest { 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 { message WatchDmChannelsRequest {
common.MsgBase base = 1; common.MsgBase base = 1;
repeated string channel_names = 2; repeated VchannelPair vchannels = 2;
} }
message FlushSegmentsRequest { message FlushSegmentsRequest {
...@@ -232,10 +240,10 @@ message SegmentInfo { ...@@ -232,10 +240,10 @@ message SegmentInfo {
uint64 last_expire_time = 10; uint64 last_expire_time = 10;
} }
message ID2PathList { message ID2PathList {
int64 ID = 1; int64 ID = 1;
repeated string Paths = 2; repeated string Paths = 2;
} }
message PositionPair { message PositionPair {
internal.MsgPosition start_position = 1; internal.MsgPosition start_position = 1;
...@@ -252,6 +260,12 @@ message SaveBinlogPathsRequest { ...@@ -252,6 +260,12 @@ message SaveBinlogPathsRequest {
PositionPair ddl_position =7; PositionPair ddl_position =7;
} }
message DataNodeTtMsg {
common.MsgBase base =1;
string channel_name = 2;
uint64 timestamp = 3;
}
message CheckPoint { message CheckPoint {
int64 segmentID = 1; int64 segmentID = 1;
internal.MsgPosition position = 2; internal.MsgPosition position = 2;
...@@ -304,9 +318,5 @@ message GetRecoveryInfoRequest { ...@@ -304,9 +318,5 @@ message GetRecoveryInfoRequest {
int64 partitionID = 3; int64 partitionID = 3;
} }
message DataNodeTtMsg {
common.MsgBase base =1;
string channel_name = 2;
uint64 timestamp = 3;
}
...@@ -1223,9 +1223,80 @@ func (m *GetSegmentInfoChannelRequest) XXX_DiscardUnknown() { ...@@ -1223,9 +1223,80 @@ func (m *GetSegmentInfoChannelRequest) XXX_DiscardUnknown() {
var xxx_messageInfo_GetSegmentInfoChannelRequest proto.InternalMessageInfo 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 { type WatchDmChannelsRequest struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` 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_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"` XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"` XXX_sizecache int32 `json:"-"`
...@@ -1235,7 +1306,7 @@ func (m *WatchDmChannelsRequest) Reset() { *m = WatchDmChannelsRequest{} ...@@ -1235,7 +1306,7 @@ func (m *WatchDmChannelsRequest) Reset() { *m = WatchDmChannelsRequest{}
func (m *WatchDmChannelsRequest) String() string { return proto.CompactTextString(m) } func (m *WatchDmChannelsRequest) String() string { return proto.CompactTextString(m) }
func (*WatchDmChannelsRequest) ProtoMessage() {} func (*WatchDmChannelsRequest) ProtoMessage() {}
func (*WatchDmChannelsRequest) Descriptor() ([]byte, []int) { func (*WatchDmChannelsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{22} return fileDescriptor_3385cd32ad6cfe64, []int{23}
} }
func (m *WatchDmChannelsRequest) XXX_Unmarshal(b []byte) error { func (m *WatchDmChannelsRequest) XXX_Unmarshal(b []byte) error {
...@@ -1263,9 +1334,9 @@ func (m *WatchDmChannelsRequest) GetBase() *commonpb.MsgBase { ...@@ -1263,9 +1334,9 @@ func (m *WatchDmChannelsRequest) GetBase() *commonpb.MsgBase {
return nil return nil
} }
func (m *WatchDmChannelsRequest) GetChannelNames() []string { func (m *WatchDmChannelsRequest) GetVchannels() []*VchannelPair {
if m != nil { if m != nil {
return m.ChannelNames return m.Vchannels
} }
return nil return nil
} }
...@@ -1284,7 +1355,7 @@ func (m *FlushSegmentsRequest) Reset() { *m = FlushSegmentsRequest{} } ...@@ -1284,7 +1355,7 @@ func (m *FlushSegmentsRequest) Reset() { *m = FlushSegmentsRequest{} }
func (m *FlushSegmentsRequest) String() string { return proto.CompactTextString(m) } func (m *FlushSegmentsRequest) String() string { return proto.CompactTextString(m) }
func (*FlushSegmentsRequest) ProtoMessage() {} func (*FlushSegmentsRequest) ProtoMessage() {}
func (*FlushSegmentsRequest) Descriptor() ([]byte, []int) { func (*FlushSegmentsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{23} return fileDescriptor_3385cd32ad6cfe64, []int{24}
} }
func (m *FlushSegmentsRequest) XXX_Unmarshal(b []byte) error { func (m *FlushSegmentsRequest) XXX_Unmarshal(b []byte) error {
...@@ -1345,7 +1416,7 @@ func (m *SegmentMsg) Reset() { *m = SegmentMsg{} } ...@@ -1345,7 +1416,7 @@ func (m *SegmentMsg) Reset() { *m = SegmentMsg{} }
func (m *SegmentMsg) String() string { return proto.CompactTextString(m) } func (m *SegmentMsg) String() string { return proto.CompactTextString(m) }
func (*SegmentMsg) ProtoMessage() {} func (*SegmentMsg) ProtoMessage() {}
func (*SegmentMsg) Descriptor() ([]byte, []int) { func (*SegmentMsg) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{24} return fileDescriptor_3385cd32ad6cfe64, []int{25}
} }
func (m *SegmentMsg) XXX_Unmarshal(b []byte) error { func (m *SegmentMsg) XXX_Unmarshal(b []byte) error {
...@@ -1393,7 +1464,7 @@ func (m *SegmentFieldBinlogMeta) Reset() { *m = SegmentFieldBinlogMeta{} ...@@ -1393,7 +1464,7 @@ func (m *SegmentFieldBinlogMeta) Reset() { *m = SegmentFieldBinlogMeta{}
func (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) } func (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) }
func (*SegmentFieldBinlogMeta) ProtoMessage() {} func (*SegmentFieldBinlogMeta) ProtoMessage() {}
func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) { func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{25} return fileDescriptor_3385cd32ad6cfe64, []int{26}
} }
func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error { func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error {
...@@ -1441,7 +1512,7 @@ func (m *DDLBinlogMeta) Reset() { *m = DDLBinlogMeta{} } ...@@ -1441,7 +1512,7 @@ func (m *DDLBinlogMeta) Reset() { *m = DDLBinlogMeta{} }
func (m *DDLBinlogMeta) String() string { return proto.CompactTextString(m) } func (m *DDLBinlogMeta) String() string { return proto.CompactTextString(m) }
func (*DDLBinlogMeta) ProtoMessage() {} func (*DDLBinlogMeta) ProtoMessage() {}
func (*DDLBinlogMeta) Descriptor() ([]byte, []int) { func (*DDLBinlogMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{26} return fileDescriptor_3385cd32ad6cfe64, []int{27}
} }
func (m *DDLBinlogMeta) XXX_Unmarshal(b []byte) error { func (m *DDLBinlogMeta) XXX_Unmarshal(b []byte) error {
...@@ -1488,7 +1559,7 @@ func (m *FieldFlushMeta) Reset() { *m = FieldFlushMeta{} } ...@@ -1488,7 +1559,7 @@ func (m *FieldFlushMeta) Reset() { *m = FieldFlushMeta{} }
func (m *FieldFlushMeta) String() string { return proto.CompactTextString(m) } func (m *FieldFlushMeta) String() string { return proto.CompactTextString(m) }
func (*FieldFlushMeta) ProtoMessage() {} func (*FieldFlushMeta) ProtoMessage() {}
func (*FieldFlushMeta) Descriptor() ([]byte, []int) { func (*FieldFlushMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{27} return fileDescriptor_3385cd32ad6cfe64, []int{28}
} }
func (m *FieldFlushMeta) XXX_Unmarshal(b []byte) error { func (m *FieldFlushMeta) XXX_Unmarshal(b []byte) error {
...@@ -1536,7 +1607,7 @@ func (m *SegmentFlushMeta) Reset() { *m = SegmentFlushMeta{} } ...@@ -1536,7 +1607,7 @@ func (m *SegmentFlushMeta) Reset() { *m = SegmentFlushMeta{} }
func (m *SegmentFlushMeta) String() string { return proto.CompactTextString(m) } func (m *SegmentFlushMeta) String() string { return proto.CompactTextString(m) }
func (*SegmentFlushMeta) ProtoMessage() {} func (*SegmentFlushMeta) ProtoMessage() {}
func (*SegmentFlushMeta) Descriptor() ([]byte, []int) { func (*SegmentFlushMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{28} return fileDescriptor_3385cd32ad6cfe64, []int{29}
} }
func (m *SegmentFlushMeta) XXX_Unmarshal(b []byte) error { func (m *SegmentFlushMeta) XXX_Unmarshal(b []byte) error {
...@@ -1590,7 +1661,7 @@ func (m *DDLFlushMeta) Reset() { *m = DDLFlushMeta{} } ...@@ -1590,7 +1661,7 @@ func (m *DDLFlushMeta) Reset() { *m = DDLFlushMeta{} }
func (m *DDLFlushMeta) String() string { return proto.CompactTextString(m) } func (m *DDLFlushMeta) String() string { return proto.CompactTextString(m) }
func (*DDLFlushMeta) ProtoMessage() {} func (*DDLFlushMeta) ProtoMessage() {}
func (*DDLFlushMeta) Descriptor() ([]byte, []int) { func (*DDLFlushMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{29} return fileDescriptor_3385cd32ad6cfe64, []int{30}
} }
func (m *DDLFlushMeta) XXX_Unmarshal(b []byte) error { func (m *DDLFlushMeta) XXX_Unmarshal(b []byte) error {
...@@ -1638,7 +1709,7 @@ func (m *CollectionInfo) Reset() { *m = CollectionInfo{} } ...@@ -1638,7 +1709,7 @@ func (m *CollectionInfo) Reset() { *m = CollectionInfo{} }
func (m *CollectionInfo) String() string { return proto.CompactTextString(m) } func (m *CollectionInfo) String() string { return proto.CompactTextString(m) }
func (*CollectionInfo) ProtoMessage() {} func (*CollectionInfo) ProtoMessage() {}
func (*CollectionInfo) Descriptor() ([]byte, []int) { func (*CollectionInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{30} return fileDescriptor_3385cd32ad6cfe64, []int{31}
} }
func (m *CollectionInfo) XXX_Unmarshal(b []byte) error { func (m *CollectionInfo) XXX_Unmarshal(b []byte) error {
...@@ -1700,7 +1771,7 @@ func (m *SegmentInfo) Reset() { *m = SegmentInfo{} } ...@@ -1700,7 +1771,7 @@ func (m *SegmentInfo) Reset() { *m = SegmentInfo{} }
func (m *SegmentInfo) String() string { return proto.CompactTextString(m) } func (m *SegmentInfo) String() string { return proto.CompactTextString(m) }
func (*SegmentInfo) ProtoMessage() {} func (*SegmentInfo) ProtoMessage() {}
func (*SegmentInfo) Descriptor() ([]byte, []int) { func (*SegmentInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{31} return fileDescriptor_3385cd32ad6cfe64, []int{32}
} }
func (m *SegmentInfo) XXX_Unmarshal(b []byte) error { func (m *SegmentInfo) XXX_Unmarshal(b []byte) error {
...@@ -1803,7 +1874,7 @@ func (m *ID2PathList) Reset() { *m = ID2PathList{} } ...@@ -1803,7 +1874,7 @@ func (m *ID2PathList) Reset() { *m = ID2PathList{} }
func (m *ID2PathList) String() string { return proto.CompactTextString(m) } func (m *ID2PathList) String() string { return proto.CompactTextString(m) }
func (*ID2PathList) ProtoMessage() {} func (*ID2PathList) ProtoMessage() {}
func (*ID2PathList) Descriptor() ([]byte, []int) { func (*ID2PathList) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{32} return fileDescriptor_3385cd32ad6cfe64, []int{33}
} }
func (m *ID2PathList) XXX_Unmarshal(b []byte) error { func (m *ID2PathList) XXX_Unmarshal(b []byte) error {
...@@ -1850,7 +1921,7 @@ func (m *PositionPair) Reset() { *m = PositionPair{} } ...@@ -1850,7 +1921,7 @@ func (m *PositionPair) Reset() { *m = PositionPair{} }
func (m *PositionPair) String() string { return proto.CompactTextString(m) } func (m *PositionPair) String() string { return proto.CompactTextString(m) }
func (*PositionPair) ProtoMessage() {} func (*PositionPair) ProtoMessage() {}
func (*PositionPair) Descriptor() ([]byte, []int) { func (*PositionPair) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{33} return fileDescriptor_3385cd32ad6cfe64, []int{34}
} }
func (m *PositionPair) XXX_Unmarshal(b []byte) error { func (m *PositionPair) XXX_Unmarshal(b []byte) error {
...@@ -1902,7 +1973,7 @@ func (m *SaveBinlogPathsRequest) Reset() { *m = SaveBinlogPathsRequest{} ...@@ -1902,7 +1973,7 @@ func (m *SaveBinlogPathsRequest) Reset() { *m = SaveBinlogPathsRequest{}
func (m *SaveBinlogPathsRequest) String() string { return proto.CompactTextString(m) } func (m *SaveBinlogPathsRequest) String() string { return proto.CompactTextString(m) }
func (*SaveBinlogPathsRequest) ProtoMessage() {} func (*SaveBinlogPathsRequest) ProtoMessage() {}
func (*SaveBinlogPathsRequest) Descriptor() ([]byte, []int) { func (*SaveBinlogPathsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{34} return fileDescriptor_3385cd32ad6cfe64, []int{35}
} }
func (m *SaveBinlogPathsRequest) XXX_Unmarshal(b []byte) error { func (m *SaveBinlogPathsRequest) XXX_Unmarshal(b []byte) error {
...@@ -1972,6 +2043,61 @@ func (m *SaveBinlogPathsRequest) GetDdlPosition() *PositionPair { ...@@ -1972,6 +2043,61 @@ func (m *SaveBinlogPathsRequest) GetDdlPosition() *PositionPair {
return nil 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 { type CheckPoint struct {
SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` 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"` Position *internalpb.MsgPosition `protobuf:"bytes,2,opt,name=position,proto3" json:"position,omitempty"`
...@@ -1985,7 +2111,7 @@ func (m *CheckPoint) Reset() { *m = CheckPoint{} } ...@@ -1985,7 +2111,7 @@ func (m *CheckPoint) Reset() { *m = CheckPoint{} }
func (m *CheckPoint) String() string { return proto.CompactTextString(m) } func (m *CheckPoint) String() string { return proto.CompactTextString(m) }
func (*CheckPoint) ProtoMessage() {} func (*CheckPoint) ProtoMessage() {}
func (*CheckPoint) Descriptor() ([]byte, []int) { func (*CheckPoint) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{35} return fileDescriptor_3385cd32ad6cfe64, []int{37}
} }
func (m *CheckPoint) XXX_Unmarshal(b []byte) error { func (m *CheckPoint) XXX_Unmarshal(b []byte) error {
...@@ -2042,7 +2168,7 @@ func (m *VchannelInfo) Reset() { *m = VchannelInfo{} } ...@@ -2042,7 +2168,7 @@ func (m *VchannelInfo) Reset() { *m = VchannelInfo{} }
func (m *VchannelInfo) String() string { return proto.CompactTextString(m) } func (m *VchannelInfo) String() string { return proto.CompactTextString(m) }
func (*VchannelInfo) ProtoMessage() {} func (*VchannelInfo) ProtoMessage() {}
func (*VchannelInfo) Descriptor() ([]byte, []int) { func (*VchannelInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{36} return fileDescriptor_3385cd32ad6cfe64, []int{38}
} }
func (m *VchannelInfo) XXX_Unmarshal(b []byte) error { func (m *VchannelInfo) XXX_Unmarshal(b []byte) error {
...@@ -2110,7 +2236,7 @@ func (m *SegmentBinlogs) Reset() { *m = SegmentBinlogs{} } ...@@ -2110,7 +2236,7 @@ func (m *SegmentBinlogs) Reset() { *m = SegmentBinlogs{} }
func (m *SegmentBinlogs) String() string { return proto.CompactTextString(m) } func (m *SegmentBinlogs) String() string { return proto.CompactTextString(m) }
func (*SegmentBinlogs) ProtoMessage() {} func (*SegmentBinlogs) ProtoMessage() {}
func (*SegmentBinlogs) Descriptor() ([]byte, []int) { func (*SegmentBinlogs) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{37} return fileDescriptor_3385cd32ad6cfe64, []int{39}
} }
func (m *SegmentBinlogs) XXX_Unmarshal(b []byte) error { func (m *SegmentBinlogs) XXX_Unmarshal(b []byte) error {
...@@ -2157,7 +2283,7 @@ func (m *FieldBinlog) Reset() { *m = FieldBinlog{} } ...@@ -2157,7 +2283,7 @@ func (m *FieldBinlog) Reset() { *m = FieldBinlog{} }
func (m *FieldBinlog) String() string { return proto.CompactTextString(m) } func (m *FieldBinlog) String() string { return proto.CompactTextString(m) }
func (*FieldBinlog) ProtoMessage() {} func (*FieldBinlog) ProtoMessage() {}
func (*FieldBinlog) Descriptor() ([]byte, []int) { func (*FieldBinlog) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{38} return fileDescriptor_3385cd32ad6cfe64, []int{40}
} }
func (m *FieldBinlog) XXX_Unmarshal(b []byte) error { func (m *FieldBinlog) XXX_Unmarshal(b []byte) error {
...@@ -2204,7 +2330,7 @@ func (m *ChannelStatus) Reset() { *m = ChannelStatus{} } ...@@ -2204,7 +2330,7 @@ func (m *ChannelStatus) Reset() { *m = ChannelStatus{} }
func (m *ChannelStatus) String() string { return proto.CompactTextString(m) } func (m *ChannelStatus) String() string { return proto.CompactTextString(m) }
func (*ChannelStatus) ProtoMessage() {} func (*ChannelStatus) ProtoMessage() {}
func (*ChannelStatus) Descriptor() ([]byte, []int) { func (*ChannelStatus) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{39} return fileDescriptor_3385cd32ad6cfe64, []int{41}
} }
func (m *ChannelStatus) XXX_Unmarshal(b []byte) error { func (m *ChannelStatus) XXX_Unmarshal(b []byte) error {
...@@ -2252,7 +2378,7 @@ func (m *DataNodeInfo) Reset() { *m = DataNodeInfo{} } ...@@ -2252,7 +2378,7 @@ func (m *DataNodeInfo) Reset() { *m = DataNodeInfo{} }
func (m *DataNodeInfo) String() string { return proto.CompactTextString(m) } func (m *DataNodeInfo) String() string { return proto.CompactTextString(m) }
func (*DataNodeInfo) ProtoMessage() {} func (*DataNodeInfo) ProtoMessage() {}
func (*DataNodeInfo) Descriptor() ([]byte, []int) { func (*DataNodeInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{40} return fileDescriptor_3385cd32ad6cfe64, []int{42}
} }
func (m *DataNodeInfo) XXX_Unmarshal(b []byte) error { func (m *DataNodeInfo) XXX_Unmarshal(b []byte) error {
...@@ -2307,7 +2433,7 @@ func (m *GetRecoveryInfoResponse) Reset() { *m = GetRecoveryInfoResponse ...@@ -2307,7 +2433,7 @@ func (m *GetRecoveryInfoResponse) Reset() { *m = GetRecoveryInfoResponse
func (m *GetRecoveryInfoResponse) String() string { return proto.CompactTextString(m) } func (m *GetRecoveryInfoResponse) String() string { return proto.CompactTextString(m) }
func (*GetRecoveryInfoResponse) ProtoMessage() {} func (*GetRecoveryInfoResponse) ProtoMessage() {}
func (*GetRecoveryInfoResponse) Descriptor() ([]byte, []int) { func (*GetRecoveryInfoResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{41} return fileDescriptor_3385cd32ad6cfe64, []int{43}
} }
func (m *GetRecoveryInfoResponse) XXX_Unmarshal(b []byte) error { func (m *GetRecoveryInfoResponse) XXX_Unmarshal(b []byte) error {
...@@ -2362,7 +2488,7 @@ func (m *GetRecoveryInfoRequest) Reset() { *m = GetRecoveryInfoRequest{} ...@@ -2362,7 +2488,7 @@ func (m *GetRecoveryInfoRequest) Reset() { *m = GetRecoveryInfoRequest{}
func (m *GetRecoveryInfoRequest) String() string { return proto.CompactTextString(m) } func (m *GetRecoveryInfoRequest) String() string { return proto.CompactTextString(m) }
func (*GetRecoveryInfoRequest) ProtoMessage() {} func (*GetRecoveryInfoRequest) ProtoMessage() {}
func (*GetRecoveryInfoRequest) Descriptor() ([]byte, []int) { func (*GetRecoveryInfoRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{42} return fileDescriptor_3385cd32ad6cfe64, []int{44}
} }
func (m *GetRecoveryInfoRequest) XXX_Unmarshal(b []byte) error { func (m *GetRecoveryInfoRequest) XXX_Unmarshal(b []byte) error {
...@@ -2404,61 +2530,6 @@ func (m *GetRecoveryInfoRequest) GetPartitionID() int64 { ...@@ -2404,61 +2530,6 @@ func (m *GetRecoveryInfoRequest) GetPartitionID() int64 {
return 0 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() { func init() {
proto.RegisterEnum("milvus.proto.data.ChannelWatchState", ChannelWatchState_name, ChannelWatchState_value) proto.RegisterEnum("milvus.proto.data.ChannelWatchState", ChannelWatchState_name, ChannelWatchState_value)
proto.RegisterType((*RegisterNodeRequest)(nil), "milvus.proto.data.RegisterNodeRequest") proto.RegisterType((*RegisterNodeRequest)(nil), "milvus.proto.data.RegisterNodeRequest")
...@@ -2483,6 +2554,7 @@ func init() { ...@@ -2483,6 +2554,7 @@ func init() {
proto.RegisterType((*GetPartitionStatisticsRequest)(nil), "milvus.proto.data.GetPartitionStatisticsRequest") proto.RegisterType((*GetPartitionStatisticsRequest)(nil), "milvus.proto.data.GetPartitionStatisticsRequest")
proto.RegisterType((*GetPartitionStatisticsResponse)(nil), "milvus.proto.data.GetPartitionStatisticsResponse") proto.RegisterType((*GetPartitionStatisticsResponse)(nil), "milvus.proto.data.GetPartitionStatisticsResponse")
proto.RegisterType((*GetSegmentInfoChannelRequest)(nil), "milvus.proto.data.GetSegmentInfoChannelRequest") 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((*WatchDmChannelsRequest)(nil), "milvus.proto.data.WatchDmChannelsRequest")
proto.RegisterType((*FlushSegmentsRequest)(nil), "milvus.proto.data.FlushSegmentsRequest") proto.RegisterType((*FlushSegmentsRequest)(nil), "milvus.proto.data.FlushSegmentsRequest")
proto.RegisterType((*SegmentMsg)(nil), "milvus.proto.data.SegmentMsg") proto.RegisterType((*SegmentMsg)(nil), "milvus.proto.data.SegmentMsg")
...@@ -2496,6 +2568,7 @@ func init() { ...@@ -2496,6 +2568,7 @@ func init() {
proto.RegisterType((*ID2PathList)(nil), "milvus.proto.data.ID2PathList") proto.RegisterType((*ID2PathList)(nil), "milvus.proto.data.ID2PathList")
proto.RegisterType((*PositionPair)(nil), "milvus.proto.data.PositionPair") proto.RegisterType((*PositionPair)(nil), "milvus.proto.data.PositionPair")
proto.RegisterType((*SaveBinlogPathsRequest)(nil), "milvus.proto.data.SaveBinlogPathsRequest") 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((*CheckPoint)(nil), "milvus.proto.data.CheckPoint")
proto.RegisterType((*VchannelInfo)(nil), "milvus.proto.data.VchannelInfo") proto.RegisterType((*VchannelInfo)(nil), "milvus.proto.data.VchannelInfo")
proto.RegisterType((*SegmentBinlogs)(nil), "milvus.proto.data.SegmentBinlogs") proto.RegisterType((*SegmentBinlogs)(nil), "milvus.proto.data.SegmentBinlogs")
...@@ -2504,145 +2577,146 @@ func init() { ...@@ -2504,145 +2577,146 @@ func init() {
proto.RegisterType((*DataNodeInfo)(nil), "milvus.proto.data.DataNodeInfo") proto.RegisterType((*DataNodeInfo)(nil), "milvus.proto.data.DataNodeInfo")
proto.RegisterType((*GetRecoveryInfoResponse)(nil), "milvus.proto.data.GetRecoveryInfoResponse") proto.RegisterType((*GetRecoveryInfoResponse)(nil), "milvus.proto.data.GetRecoveryInfoResponse")
proto.RegisterType((*GetRecoveryInfoRequest)(nil), "milvus.proto.data.GetRecoveryInfoRequest") 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) } func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad6cfe64) }
var fileDescriptor_3385cd32ad6cfe64 = []byte{ var fileDescriptor_3385cd32ad6cfe64 = []byte{
// 2101 bytes of a gzipped FileDescriptorProto // 2144 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0x5b, 0x6f, 0x1b, 0xc7, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x19, 0x5b, 0x6f, 0x1b, 0x59,
0x15, 0xd6, 0x92, 0xba, 0xf1, 0x70, 0x49, 0x49, 0x63, 0x55, 0x61, 0x69, 0x5b, 0x96, 0x37, 0x89, 0x39, 0x63, 0xe7, 0x62, 0x7f, 0xbe, 0x24, 0x39, 0x1b, 0xb2, 0xc6, 0x6d, 0xd3, 0x74, 0xd8, 0xed,
0xa3, 0x38, 0xa8, 0x14, 0xd3, 0x45, 0x6f, 0xee, 0x05, 0x96, 0x19, 0xab, 0x44, 0x2d, 0x57, 0x1d, 0x66, 0xb3, 0x22, 0xd9, 0xba, 0x88, 0x5b, 0x59, 0x50, 0x53, 0x6f, 0x83, 0x45, 0x53, 0xc2, 0x49,
0x39, 0x09, 0xd0, 0xa0, 0x20, 0x56, 0xdc, 0x11, 0xb5, 0x15, 0x77, 0x97, 0xd9, 0x59, 0xca, 0xf2, 0xb7, 0x2b, 0xb1, 0x42, 0xd6, 0xc4, 0x73, 0xe2, 0x0c, 0xf1, 0xcc, 0x78, 0xe7, 0x8c, 0xd3, 0xf4,
0x93, 0x83, 0xb4, 0x0f, 0x45, 0x51, 0xd4, 0xed, 0x3f, 0x68, 0x0b, 0x14, 0x28, 0xd0, 0x3e, 0xf4, 0xa9, 0xab, 0x85, 0x07, 0x40, 0x88, 0x05, 0x7e, 0x01, 0x20, 0x21, 0x21, 0xc1, 0x03, 0xcf, 0xbc,
0xb9, 0xef, 0xfd, 0x0b, 0xfd, 0x3d, 0xc5, 0x5c, 0xf6, 0x3e, 0x24, 0xd7, 0x94, 0x0d, 0xbd, 0x71, 0xf3, 0x17, 0xf8, 0x3d, 0xe8, 0x5c, 0xe6, 0x7e, 0x6c, 0x4f, 0x9c, 0x56, 0x79, 0x9b, 0x73, 0xe6,
0x86, 0xe7, 0x36, 0xe7, 0xf2, 0xcd, 0x99, 0xb3, 0x80, 0x2c, 0x33, 0x30, 0xbb, 0x94, 0xf8, 0xe7, 0xbb, 0x9d, 0xef, 0x7e, 0xbe, 0x03, 0xc8, 0x34, 0x7c, 0xa3, 0x4b, 0x89, 0x77, 0x6e, 0xf5, 0xc8,
0x76, 0x8f, 0xec, 0x0c, 0x7d, 0x2f, 0xf0, 0xd0, 0x9a, 0x63, 0x0f, 0xce, 0x47, 0x54, 0xac, 0x76, 0xce, 0xd0, 0x73, 0x7d, 0x17, 0xad, 0xda, 0xd6, 0xe0, 0x7c, 0x44, 0xc5, 0x6a, 0x87, 0x01, 0x34,
0x18, 0x41, 0x53, 0xef, 0x79, 0x8e, 0xe3, 0xb9, 0x62, 0xab, 0x59, 0xb7, 0xdd, 0x80, 0xf8, 0xae, 0xab, 0x3d, 0xd7, 0xb6, 0x5d, 0x47, 0x6c, 0x35, 0xeb, 0x96, 0xe3, 0x13, 0xcf, 0x31, 0x06, 0x72,
0x39, 0x90, 0x6b, 0x3d, 0xc9, 0xd0, 0xd4, 0x69, 0xef, 0x94, 0x38, 0xa6, 0x58, 0x19, 0x2f, 0xe1, 0x5d, 0x8d, 0x23, 0x34, 0xab, 0xb4, 0x77, 0x4a, 0x6c, 0x43, 0xac, 0xf4, 0x57, 0xf0, 0x16, 0x26,
0x1a, 0x26, 0x7d, 0x9b, 0x06, 0xc4, 0x7f, 0xea, 0x59, 0x04, 0x93, 0x2f, 0x47, 0x84, 0x06, 0xe8, 0x7d, 0x8b, 0xfa, 0xc4, 0x7b, 0xea, 0x9a, 0x04, 0x93, 0xcf, 0x47, 0x84, 0xfa, 0xe8, 0x43, 0x98,
0x63, 0x98, 0x3f, 0x36, 0x29, 0x69, 0x68, 0x5b, 0xda, 0x76, 0xb5, 0x75, 0x63, 0x27, 0xa5, 0x52, 0x3f, 0x36, 0x28, 0x69, 0x68, 0x9b, 0xda, 0x56, 0xa5, 0x75, 0x73, 0x27, 0xc1, 0x52, 0x32, 0x3b,
0x2a, 0x3b, 0xa0, 0xfd, 0x3d, 0x93, 0x12, 0xcc, 0x29, 0xd1, 0x77, 0x60, 0xc9, 0xb4, 0x2c, 0x9f, 0xa0, 0xfd, 0x3d, 0x83, 0x12, 0xcc, 0x21, 0xd1, 0xb7, 0x61, 0xc9, 0x30, 0x4d, 0x8f, 0x50, 0xda,
0x50, 0xda, 0x28, 0x4d, 0x60, 0x7a, 0x28, 0x68, 0x70, 0x48, 0x6c, 0xbc, 0xd2, 0x60, 0x3d, 0x6d, 0x28, 0x4c, 0x40, 0x7a, 0x28, 0x60, 0x70, 0x00, 0xac, 0x7f, 0xa5, 0xc1, 0x5a, 0x52, 0x02, 0x3a,
0x01, 0x1d, 0x7a, 0x2e, 0x25, 0x68, 0x0f, 0xaa, 0xb6, 0x6b, 0x07, 0xdd, 0xa1, 0xe9, 0x9b, 0x0e, 0x74, 0x1d, 0x4a, 0xd0, 0x1e, 0x54, 0x2c, 0xc7, 0xf2, 0xbb, 0x43, 0xc3, 0x33, 0x6c, 0x2a, 0x25,
0x95, 0x96, 0xdc, 0x4e, 0x0b, 0x8d, 0x0e, 0xda, 0x71, 0xed, 0xe0, 0x90, 0x13, 0x62, 0xb0, 0xa3, 0xb9, 0x93, 0x24, 0x1a, 0x1e, 0xb4, 0xe3, 0x58, 0xfe, 0x21, 0x07, 0xc4, 0x60, 0x85, 0xdf, 0xe8,
0xdf, 0xe8, 0x3e, 0x2c, 0xd2, 0xc0, 0x0c, 0x46, 0xa1, 0x4d, 0xd7, 0x95, 0x36, 0x1d, 0x71, 0x12, 0x3e, 0x2c, 0x52, 0xdf, 0xf0, 0x47, 0x81, 0x4c, 0x37, 0x94, 0x32, 0x1d, 0x71, 0x10, 0x2c, 0x41,
0x2c, 0x49, 0x8d, 0x0b, 0xd0, 0x1f, 0x0f, 0x46, 0xf4, 0x74, 0x76, 0x5f, 0x20, 0x98, 0xb7, 0x8e, 0xf5, 0x0b, 0xa8, 0x3e, 0x1e, 0x8c, 0xe8, 0xe9, 0xec, 0xba, 0x40, 0x30, 0x6f, 0x1e, 0x77, 0xda,
0x3b, 0x6d, 0xae, 0xb4, 0x8c, 0xf9, 0x6f, 0x64, 0x80, 0xde, 0xf3, 0x06, 0x03, 0xd2, 0x0b, 0x6c, 0x9c, 0x69, 0x11, 0xf3, 0x6f, 0xa4, 0x43, 0xb5, 0xe7, 0x0e, 0x06, 0xa4, 0xe7, 0x5b, 0xae, 0xd3,
0xcf, 0xed, 0xb4, 0x1b, 0xf3, 0xfc, 0xbf, 0xd4, 0x9e, 0xf1, 0x67, 0x0d, 0x56, 0x8f, 0x48, 0xdf, 0x69, 0x37, 0xe6, 0xf9, 0xbf, 0xc4, 0x9e, 0xfe, 0x27, 0x0d, 0x56, 0x8e, 0x48, 0xdf, 0x26, 0x8e,
0x21, 0x6e, 0xd0, 0x69, 0x87, 0xea, 0xd7, 0x61, 0xa1, 0xe7, 0x8d, 0xdc, 0x80, 0xeb, 0xaf, 0x61, 0xdf, 0x69, 0x07, 0xec, 0xd7, 0x60, 0xa1, 0xe7, 0x8e, 0x1c, 0x9f, 0xf3, 0xaf, 0x61, 0xb1, 0x40,
0xb1, 0x40, 0xb7, 0x41, 0xef, 0x9d, 0x9a, 0xae, 0x4b, 0x06, 0x5d, 0xd7, 0x74, 0x08, 0x57, 0x55, 0x77, 0xa0, 0xda, 0x3b, 0x35, 0x1c, 0x87, 0x0c, 0xba, 0x8e, 0x61, 0x13, 0xce, 0xaa, 0x8c, 0x2b,
0xc1, 0x55, 0xb9, 0xf7, 0xd4, 0x74, 0x48, 0x4e, 0x63, 0x39, 0xaf, 0x11, 0x6d, 0x41, 0x75, 0x68, 0x72, 0xef, 0xa9, 0x61, 0x93, 0x0c, 0xc7, 0x62, 0x96, 0x23, 0xda, 0x84, 0xca, 0xd0, 0xf0, 0x7c,
0xfa, 0x81, 0x9d, 0x32, 0x2a, 0xb9, 0x65, 0xfc, 0x45, 0x83, 0x8d, 0x87, 0x94, 0xda, 0x7d, 0x37, 0x2b, 0x21, 0x54, 0x7c, 0x4b, 0xff, 0x8b, 0x06, 0xeb, 0x0f, 0x29, 0xb5, 0xfa, 0x4e, 0x46, 0xb2,
0x67, 0xd9, 0x06, 0x2c, 0xba, 0x9e, 0x45, 0x3a, 0x6d, 0x6e, 0x5a, 0x19, 0xcb, 0x15, 0xba, 0x0e, 0x75, 0x58, 0x74, 0x5c, 0x93, 0x74, 0xda, 0x5c, 0xb4, 0x22, 0x96, 0x2b, 0x74, 0x03, 0xca, 0x43,
0x95, 0x21, 0x21, 0x7e, 0xd7, 0xf7, 0x06, 0xa1, 0x61, 0xcb, 0x6c, 0x03, 0x7b, 0x03, 0x82, 0x7e, 0x42, 0xbc, 0xae, 0xe7, 0x0e, 0x02, 0xc1, 0x4a, 0x6c, 0x03, 0xbb, 0x03, 0x82, 0x7e, 0x06, 0xab,
0x01, 0x6b, 0x34, 0x23, 0x88, 0x36, 0xca, 0x5b, 0xe5, 0xed, 0x6a, 0xeb, 0xdd, 0x9d, 0x5c, 0x66, 0x34, 0x45, 0x88, 0x36, 0x8a, 0x9b, 0xc5, 0xad, 0x4a, 0xeb, 0x1b, 0x3b, 0x19, 0xcf, 0xde, 0x49,
0xef, 0x64, 0x95, 0xe2, 0x3c, 0xb7, 0xf1, 0x55, 0x09, 0xae, 0x45, 0x74, 0xc2, 0x56, 0xf6, 0x9b, 0x33, 0xc5, 0x59, 0x6c, 0xfd, 0x8b, 0x02, 0xbc, 0x15, 0xc2, 0x09, 0x59, 0xd9, 0x37, 0xd3, 0x1c,
0x79, 0x8e, 0x92, 0x7e, 0x64, 0x9e, 0x58, 0x14, 0xf1, 0x5c, 0xe4, 0xf2, 0x72, 0xd2, 0xe5, 0x05, 0x25, 0xfd, 0x50, 0x3c, 0xb1, 0xc8, 0xa3, 0xb9, 0x50, 0xe5, 0xc5, 0xb8, 0xca, 0x73, 0x58, 0x30,
0x22, 0x98, 0xf5, 0xe7, 0x42, 0xce, 0x9f, 0xe8, 0x16, 0x54, 0xc9, 0xc5, 0xd0, 0xf6, 0x49, 0x37, 0xad, 0xcf, 0x85, 0x8c, 0x3e, 0xd1, 0x6d, 0xa8, 0x90, 0x8b, 0xa1, 0xe5, 0x91, 0xae, 0x6f, 0xd9,
0xb0, 0x1d, 0xd2, 0x58, 0xdc, 0xd2, 0xb6, 0xe7, 0x31, 0x88, 0xad, 0x67, 0xb6, 0x43, 0x12, 0x39, 0xa4, 0xb1, 0xb8, 0xa9, 0x6d, 0xcd, 0x63, 0x10, 0x5b, 0xcf, 0x2c, 0x9b, 0xc4, 0x7c, 0x76, 0x29,
0xbb, 0x54, 0x3c, 0x67, 0xff, 0xa6, 0xc1, 0x3b, 0xb9, 0x28, 0xc9, 0x42, 0xc2, 0xb0, 0xca, 0x4f, 0xbf, 0xcf, 0xfe, 0x4d, 0x83, 0xb7, 0x33, 0x56, 0x92, 0x81, 0x84, 0x61, 0x85, 0x9f, 0x3c, 0xd2,
0x1e, 0x7b, 0x86, 0x55, 0x13, 0x73, 0xf8, 0x9d, 0x49, 0x0e, 0x8f, 0xc9, 0x71, 0x8e, 0x7f, 0xb6, 0x0c, 0x8b, 0x26, 0xa6, 0xf0, 0xbb, 0x93, 0x14, 0x1e, 0x81, 0xe3, 0x0c, 0xfe, 0x6c, 0x81, 0xf5,
0xc2, 0xfa, 0xab, 0x06, 0xd7, 0x8e, 0x4e, 0xbd, 0xe7, 0x52, 0x05, 0x9d, 0xbd, 0xc0, 0xb2, 0xa1, 0x57, 0x0d, 0xde, 0x3a, 0x3a, 0x75, 0x5f, 0x48, 0x16, 0x74, 0xf6, 0x00, 0x4b, 0x9b, 0xa2, 0x30,
0x28, 0x4d, 0x0f, 0x45, 0x39, 0x1f, 0x8a, 0xb0, 0x4c, 0xe7, 0xe3, 0x32, 0x35, 0xce, 0x60, 0x3d, 0xdd, 0x14, 0xc5, 0xac, 0x29, 0x82, 0x30, 0x9d, 0x8f, 0xc2, 0x54, 0x3f, 0x83, 0xb5, 0xa4, 0x88,
0x6d, 0xa2, 0x74, 0xe2, 0x26, 0x40, 0x94, 0x78, 0xc2, 0x7d, 0x65, 0x9c, 0xd8, 0x99, 0xcd, 0x21, 0x52, 0x89, 0x1b, 0x00, 0xa1, 0xe3, 0x09, 0xf5, 0x15, 0x71, 0x6c, 0x67, 0x36, 0x85, 0x9c, 0xc1,
0x67, 0xf0, 0xce, 0x3e, 0x09, 0xa4, 0x2e, 0xf6, 0x1f, 0xb9, 0x84, 0x4f, 0xd2, 0x16, 0x96, 0xb2, 0xdb, 0xfb, 0xc4, 0x97, 0xbc, 0xd8, 0x3f, 0x72, 0x05, 0x9d, 0x24, 0x25, 0x2c, 0xa4, 0x25, 0xd4,
0x16, 0x1a, 0xff, 0x2e, 0x45, 0xe0, 0xc2, 0x55, 0x75, 0xdc, 0x13, 0x0f, 0xdd, 0x80, 0x4a, 0x44, 0xff, 0x5d, 0x08, 0x93, 0x0b, 0x67, 0xd5, 0x71, 0x4e, 0x5c, 0x74, 0x13, 0xca, 0x21, 0x88, 0x0c,
0x22, 0xcb, 0x24, 0xde, 0x40, 0xdf, 0x85, 0x05, 0x66, 0xa9, 0xa8, 0x91, 0x7a, 0x16, 0x7c, 0xc3, 0x93, 0x68, 0x03, 0x7d, 0x07, 0x16, 0x98, 0xa4, 0x22, 0x46, 0xea, 0xe9, 0xe4, 0x1b, 0x9c, 0x29,
0x33, 0x25, 0x64, 0x62, 0x41, 0x8f, 0x3a, 0x50, 0xa7, 0x81, 0xe9, 0x07, 0xdd, 0xa1, 0x47, 0xb9, 0x46, 0x13, 0x0b, 0x78, 0xd4, 0x81, 0x3a, 0xf5, 0x0d, 0xcf, 0xef, 0x0e, 0x5d, 0xca, 0xb5, 0xcd,
0xb7, 0xb9, 0xfb, 0xab, 0x2d, 0x63, 0x0c, 0x7c, 0x1f, 0xd0, 0xfe, 0xa1, 0xa4, 0xc4, 0x35, 0xce, 0xd5, 0x5f, 0x69, 0xe9, 0x63, 0xd2, 0xf7, 0x01, 0xed, 0x1f, 0x4a, 0x48, 0x5c, 0xe3, 0x98, 0xc1,
0x19, 0x2e, 0xd1, 0x27, 0xa0, 0x13, 0xd7, 0x8a, 0x05, 0xcd, 0x17, 0x16, 0x54, 0x25, 0xae, 0x15, 0x12, 0x7d, 0x0c, 0x55, 0xe2, 0x98, 0x11, 0xa1, 0xf9, 0xdc, 0x84, 0x2a, 0xc4, 0x31, 0x43, 0x32,
0x89, 0x89, 0xe3, 0xb3, 0x50, 0x3c, 0x3e, 0x7f, 0xd0, 0xa0, 0x91, 0x0f, 0x90, 0xcc, 0x88, 0x58, 0x91, 0x7d, 0x16, 0xf2, 0xdb, 0xe7, 0xf7, 0x1a, 0x34, 0xb2, 0x06, 0x92, 0x1e, 0x11, 0x51, 0xd4,
0xa2, 0x56, 0x58, 0x22, 0x7a, 0x20, 0x98, 0x88, 0x08, 0xd0, 0x44, 0xc8, 0x8b, 0x82, 0x84, 0x25, 0x72, 0x53, 0x44, 0x0f, 0x04, 0x12, 0x11, 0x06, 0x9a, 0x98, 0xf2, 0x42, 0x23, 0x61, 0x89, 0xa2,
0x8b, 0x61, 0xc3, 0x37, 0x62, 0x6b, 0xf8, 0x3f, 0x6f, 0x2d, 0x59, 0x7e, 0xa3, 0xc1, 0x46, 0x56, 0x5b, 0xf0, 0xb5, 0x48, 0x1a, 0xfe, 0xe7, 0x8d, 0x39, 0xcb, 0xaf, 0x34, 0x58, 0x4f, 0xf3, 0xba,
0xd7, 0x65, 0xce, 0xfd, 0x6d, 0x58, 0xb0, 0xdd, 0x13, 0x2f, 0x3c, 0xf6, 0xe6, 0x04, 0xe0, 0x61, 0xca, 0xb9, 0xbf, 0x05, 0x0b, 0x96, 0x73, 0xe2, 0x06, 0xc7, 0xde, 0x98, 0x90, 0x78, 0x18, 0x2f,
0xba, 0x04, 0xb1, 0xe1, 0xc0, 0xf5, 0x7d, 0x12, 0x74, 0x5c, 0x4a, 0xfc, 0x60, 0xcf, 0x76, 0x07, 0x01, 0xac, 0xdb, 0x70, 0x63, 0x9f, 0xf8, 0x1d, 0x87, 0x12, 0xcf, 0xdf, 0xb3, 0x9c, 0x81, 0xdb,
0x5e, 0xff, 0xd0, 0x0c, 0x4e, 0x2f, 0x51, 0x23, 0xa9, 0x74, 0x2f, 0x65, 0xd2, 0xdd, 0xf8, 0x87, 0x3f, 0x34, 0xfc, 0xd3, 0x2b, 0xc4, 0x48, 0xc2, 0xdd, 0x0b, 0x29, 0x77, 0xd7, 0xff, 0xa1, 0xc1,
0x06, 0x37, 0xd4, 0xfa, 0xe4, 0xd1, 0x9b, 0xb0, 0x7c, 0x62, 0x93, 0x81, 0x15, 0x43, 0x40, 0xb4, 0x4d, 0x35, 0x3f, 0x79, 0xf4, 0x26, 0x94, 0x4e, 0x2c, 0x32, 0x30, 0xa3, 0x14, 0x10, 0xae, 0x59,
0x66, 0xb5, 0x32, 0x64, 0xc4, 0xf2, 0x84, 0xe3, 0x1a, 0x95, 0xa3, 0xc0, 0xb7, 0xdd, 0xfe, 0x13, 0xac, 0x0c, 0x19, 0xb0, 0x3c, 0xe1, 0xb8, 0x46, 0xe5, 0xc8, 0xf7, 0x2c, 0xa7, 0xff, 0xc4, 0xa2,
0x9b, 0x06, 0x58, 0xd0, 0x27, 0xfc, 0x59, 0x2e, 0x9e, 0x99, 0xbf, 0x15, 0x99, 0x29, 0x4c, 0x7d, 0x3e, 0x16, 0xf0, 0x31, 0x7d, 0x16, 0xf3, 0x7b, 0xe6, 0xaf, 0x85, 0x67, 0x0a, 0x51, 0x1f, 0x89,
0x24, 0xae, 0x2e, 0xfa, 0x76, 0x1b, 0x16, 0x45, 0xfb, 0x60, 0xfc, 0x5e, 0x83, 0xcd, 0x7d, 0x12, 0xd2, 0x45, 0xdf, 0x6c, 0xc3, 0xa2, 0x68, 0x1f, 0xf4, 0xdf, 0x69, 0xb0, 0xb1, 0x4f, 0xfc, 0x47,
0x3c, 0x8a, 0xf6, 0x98, 0x99, 0x36, 0x0d, 0xec, 0xde, 0x15, 0x18, 0xf3, 0x4a, 0x83, 0x5b, 0x63, 0xe1, 0x1e, 0x13, 0xd3, 0xa2, 0xbe, 0xd5, 0xbb, 0x06, 0x61, 0xbe, 0xd2, 0xe0, 0xf6, 0x58, 0x61,
0x8d, 0x91, 0x11, 0x94, 0x88, 0x16, 0x5e, 0x80, 0x6a, 0x44, 0xfb, 0x19, 0x79, 0xf1, 0x99, 0x39, 0xa4, 0x05, 0x65, 0x46, 0x0b, 0x0a, 0xa0, 0x3a, 0xa3, 0xfd, 0x84, 0xbc, 0x7c, 0x6e, 0x0c, 0x46,
0x18, 0x91, 0x43, 0xd3, 0xf6, 0x05, 0xa2, 0xcd, 0x88, 0xef, 0xff, 0xd4, 0xe0, 0xe6, 0x3e, 0x61, 0xe4, 0xd0, 0xb0, 0x3c, 0x91, 0xd1, 0x66, 0xcc, 0xef, 0xff, 0xd4, 0xe0, 0xd6, 0x3e, 0x61, 0xcd,
0xcd, 0xa8, 0xb8, 0x73, 0xae, 0xd0, 0x3b, 0x05, 0x3a, 0xbd, 0x3f, 0x8a, 0x60, 0x2a, 0xad, 0xbd, 0xa8, 0xa8, 0x39, 0xd7, 0xa8, 0x9d, 0x1c, 0x9d, 0xde, 0x1f, 0x84, 0x31, 0x95, 0xd2, 0x5e, 0x8b,
0x12, 0xf7, 0x6d, 0xf2, 0x72, 0x4c, 0xe0, 0x82, 0x4c, 0x74, 0xe9, 0x3c, 0xc3, 0x83, 0x8d, 0xcf, 0xfa, 0x36, 0x78, 0x38, 0xc6, 0xf2, 0x82, 0x74, 0x74, 0xa9, 0x3c, 0xfd, 0xcf, 0x05, 0xa8, 0x3e,
0xcd, 0xa0, 0x77, 0xda, 0x76, 0x2e, 0x5f, 0x01, 0xef, 0x42, 0x2d, 0xd9, 0x15, 0x8a, 0x32, 0xae, 0x97, 0x7d, 0x1b, 0x63, 0x96, 0xd1, 0x83, 0xa6, 0xd0, 0xc3, 0x36, 0xac, 0x9a, 0xf6, 0xa0, 0x7b,
0x60, 0x3d, 0xd1, 0x16, 0x52, 0xd6, 0x0b, 0xaf, 0xf3, 0xa7, 0xc1, 0xe5, 0x3b, 0x98, 0x59, 0xc3, 0xae, 0xe8, 0x01, 0x97, 0x4d, 0x7b, 0xf0, 0x3c, 0xde, 0x07, 0x32, 0x58, 0x33, 0x0d, 0x5b, 0x94,
0x98, 0x06, 0xee, 0xf9, 0x1c, 0x70, 0x5f, 0x00, 0x48, 0xe3, 0x0e, 0x68, 0x7f, 0x06, 0xbb, 0xbe, 0xb0, 0x66, 0x12, 0x76, 0x0f, 0xaa, 0x0c, 0x36, 0x55, 0xa7, 0x6e, 0x2b, 0x12, 0x5d, 0x50, 0x93,
0x07, 0x4b, 0x52, 0x9a, 0x8c, 0xd4, 0x34, 0xa8, 0x0e, 0xc9, 0x8d, 0x23, 0xd8, 0x90, 0xfb, 0x8f, 0xb8, 0x7e, 0x2a, 0xa6, 0x39, 0x08, 0x8b, 0x14, 0xa3, 0x61, 0xc7, 0x68, 0x2c, 0xe4, 0xa5, 0x61,
0x19, 0x26, 0x0a, 0xfc, 0x3c, 0x20, 0x81, 0x89, 0x1a, 0xb0, 0x24, 0x61, 0x52, 0xb6, 0x18, 0xe1, 0x87, 0x34, 0xf4, 0xdf, 0x6a, 0xb0, 0xfe, 0xa9, 0xe1, 0xf7, 0x4e, 0xdb, 0xf6, 0xd5, 0xf3, 0xc2,
0x92, 0x35, 0xc3, 0xc7, 0x9c, 0xae, 0xcb, 0xb0, 0x50, 0xb6, 0xe2, 0x70, 0x1c, 0x41, 0xaf, 0xf1, 0x47, 0x50, 0x0e, 0x0e, 0x1f, 0x24, 0x36, 0x95, 0x34, 0x71, 0x23, 0xe0, 0x08, 0x83, 0xdd, 0x1d,
0x2b, 0xa8, 0xb5, 0xdb, 0x4f, 0x12, 0xb2, 0xee, 0xc0, 0x8a, 0x65, 0x0d, 0xba, 0x49, 0x2e, 0x8d, 0xd6, 0xf8, 0x55, 0xea, 0xea, 0x1d, 0xdf, 0xac, 0x6e, 0x9f, 0x2c, 0x74, 0xf3, 0x99, 0x42, 0x77,
0x73, 0xd5, 0x2c, 0x6b, 0x10, 0x63, 0x36, 0x7a, 0x0f, 0xea, 0x01, 0xed, 0xe6, 0x85, 0xeb, 0x01, 0x01, 0x20, 0x85, 0x3b, 0xa0, 0xfd, 0x19, 0xe4, 0xfa, 0x2e, 0x2c, 0x49, 0x6a, 0xd2, 0xb3, 0xa7,
0x8d, 0xa9, 0x8c, 0x03, 0xa8, 0x73, 0x63, 0x79, 0x50, 0xa7, 0xd8, 0x7a, 0x1b, 0xf4, 0x84, 0xb8, 0x95, 0xb6, 0x00, 0x5c, 0x3f, 0x82, 0x75, 0xb9, 0xff, 0x98, 0xd5, 0x10, 0x51, 0x6f, 0x0e, 0x88,
0x30, 0x41, 0xaa, 0xb1, 0xb1, 0x94, 0xc1, 0x61, 0xd8, 0x62, 0xc5, 0x12, 0x27, 0xb7, 0x58, 0x37, 0x6f, 0xa0, 0x06, 0x2c, 0xc9, 0xb2, 0x22, 0x3d, 0x38, 0x58, 0xb2, 0xcb, 0xc3, 0x31, 0x87, 0xeb,
0x01, 0x6c, 0xda, 0x3d, 0x61, 0xd4, 0xc4, 0xe2, 0x36, 0x2e, 0xe3, 0x8a, 0x4d, 0x1f, 0x8b, 0x0d, 0xb2, 0xda, 0x21, 0xdd, 0x16, 0x8e, 0xc3, 0x52, 0xa5, 0xff, 0x02, 0x6a, 0xed, 0xf6, 0x93, 0x18,
0xf4, 0x7d, 0x58, 0xe4, 0xfa, 0x59, 0xdb, 0xa2, 0xa8, 0x38, 0x1e, 0x8d, 0xf4, 0x09, 0xb0, 0x64, 0xad, 0xbb, 0xc0, 0x3c, 0xb5, 0x1b, 0xc7, 0xd2, 0x38, 0x56, 0xcd, 0x34, 0x07, 0x51, 0x8d, 0x43,
0x30, 0x3e, 0x05, 0xbd, 0xdd, 0x7e, 0x12, 0xdb, 0x91, 0xcd, 0x2e, 0x4d, 0x91, 0x5d, 0x05, 0xce, 0xef, 0x40, 0xdd, 0xa7, 0xdd, 0x2c, 0xf1, 0xaa, 0x4f, 0x23, 0x28, 0xfd, 0x00, 0xea, 0x5c, 0x58,
0xf8, 0x12, 0xea, 0x31, 0xc2, 0xf2, 0x1e, 0xb2, 0x0e, 0xa5, 0x48, 0x5c, 0xa9, 0xd3, 0x46, 0x3f, 0x6e, 0xd4, 0x29, 0xb2, 0xde, 0x81, 0x6a, 0x8c, 0x9c, 0x70, 0x9f, 0x32, 0xae, 0x44, 0xc2, 0x52,
0x82, 0x45, 0x31, 0x62, 0x90, 0x19, 0xf4, 0x7e, 0xda, 0x66, 0x39, 0x7e, 0x48, 0xc0, 0x34, 0xdf, 0x56, 0x3e, 0x82, 0x96, 0x34, 0xa2, 0x38, 0xb9, 0x25, 0xbd, 0x05, 0x60, 0xd1, 0xee, 0x09, 0x83,
0xc0, 0x92, 0x89, 0x65, 0x78, 0x84, 0x4a, 0xe2, 0x65, 0x58, 0xc6, 0x89, 0x1d, 0xe3, 0x3f, 0x65, 0x26, 0x26, 0x97, 0xb1, 0x84, 0xcb, 0x16, 0x7d, 0x2c, 0x36, 0xd0, 0xf7, 0x60, 0x91, 0xf3, 0x67,
0xa8, 0x26, 0x12, 0x30, 0xa7, 0xfe, 0xcd, 0xbc, 0x0d, 0xde, 0x87, 0xba, 0xcd, 0x2f, 0xd7, 0xae, 0x6d, 0x9e, 0x22, 0x43, 0x71, 0x6b, 0x24, 0x4f, 0x80, 0x25, 0x82, 0xfe, 0x09, 0x54, 0xdb, 0xed,
0x44, 0x00, 0x8e, 0x98, 0x15, 0x5c, 0xb3, 0x93, 0x57, 0x2e, 0xfa, 0x26, 0x2c, 0xbb, 0x23, 0xa7, 0x27, 0x91, 0x1c, 0x79, 0x92, 0x49, 0x8e, 0x33, 0xbe, 0x82, 0x7a, 0x54, 0x91, 0x78, 0xcf, 0x5d,
0xeb, 0x7b, 0xcf, 0xa9, 0x7c, 0xec, 0x2d, 0xb9, 0x23, 0x07, 0x7b, 0xcf, 0x69, 0xdc, 0x3c, 0x2f, 0x87, 0x42, 0x48, 0xae, 0xd0, 0x69, 0xa3, 0x8f, 0x60, 0x51, 0x8c, 0x64, 0xa4, 0x07, 0xbd, 0x9b,
0x5e, 0xba, 0x79, 0x5e, 0x7a, 0x53, 0xcd, 0xf3, 0xf2, 0x6c, 0xcd, 0xf3, 0x26, 0x54, 0x1d, 0xf3, 0x94, 0x59, 0x8e, 0x6b, 0x62, 0x65, 0x8d, 0x6f, 0x60, 0x89, 0xc4, 0x3c, 0x3c, 0xcc, 0xe2, 0xe2,
0x82, 0x9d, 0xb2, 0xeb, 0x8e, 0x9c, 0x46, 0x45, 0x24, 0xb1, 0x63, 0x5e, 0x60, 0xef, 0xf9, 0xd3, 0x26, 0x5d, 0xc4, 0xb1, 0x1d, 0xfd, 0x3f, 0x45, 0xa8, 0xc4, 0x1c, 0x30, 0xc3, 0xfe, 0xf5, 0xdc,
0x91, 0x83, 0xb6, 0x61, 0x75, 0x60, 0xd2, 0xa0, 0x9b, 0x7c, 0xd8, 0x02, 0x7f, 0xd8, 0xd6, 0xd9, 0xa5, 0xde, 0x85, 0xba, 0xc5, 0x9b, 0x91, 0xae, 0x8c, 0x7e, 0x9e, 0x00, 0xcb, 0xb8, 0x66, 0xc5,
0xfe, 0x27, 0xd1, 0xe3, 0xd6, 0xb8, 0x0f, 0xd5, 0x4e, 0xbb, 0xc5, 0x32, 0x89, 0xb5, 0x40, 0xb9, 0x5b, 0x14, 0xf4, 0x75, 0x28, 0x39, 0x23, 0xbb, 0xeb, 0xb9, 0x2f, 0xa8, 0xbc, 0x1c, 0x2f, 0x39,
0xd8, 0xad, 0xc3, 0xc2, 0x61, 0x22, 0xf1, 0xc4, 0x82, 0xc1, 0xae, 0x1e, 0xda, 0xc2, 0x2e, 0x15, 0x23, 0x1b, 0xbb, 0x2f, 0x68, 0x74, 0xd9, 0x58, 0xbc, 0xf2, 0x65, 0x63, 0xe9, 0x75, 0x5d, 0x36,
0x85, 0x87, 0xb4, 0x37, 0xe5, 0xa1, 0xd2, 0x4c, 0x1e, 0x32, 0xfe, 0x55, 0x86, 0x8d, 0x23, 0xf3, 0x4a, 0xb3, 0x5d, 0x36, 0x36, 0xa0, 0x62, 0x1b, 0x17, 0xec, 0x94, 0x5d, 0x67, 0x64, 0x37, 0xca,
0x9c, 0xbc, 0xfd, 0x2e, 0xb5, 0xd0, 0x2d, 0xf1, 0x04, 0xd6, 0x38, 0x0a, 0xb4, 0x12, 0xf6, 0xc8, 0xc2, 0x89, 0x6d, 0xe3, 0x02, 0xbb, 0x2f, 0x9e, 0x8e, 0x6c, 0xb4, 0x05, 0x2b, 0x03, 0x83, 0xfa,
0x97, 0x93, 0x0a, 0xcf, 0x13, 0x21, 0xc1, 0x79, 0x46, 0xf4, 0x53, 0xa8, 0xa7, 0xc0, 0x35, 0x04, 0xdd, 0xf8, 0x20, 0x00, 0xf8, 0x20, 0xa0, 0xce, 0xf6, 0x3f, 0x0e, 0x87, 0x01, 0xfa, 0x7d, 0xa8,
0xa3, 0x2d, 0x85, 0xa8, 0x14, 0x5a, 0xe3, 0x0c, 0x1f, 0xda, 0x03, 0xdd, 0x72, 0x06, 0xb1, 0xb7, 0x74, 0xda, 0x2d, 0xe6, 0x49, 0xac, 0x65, 0xcc, 0xd8, 0x6e, 0x0d, 0x16, 0x0e, 0x63, 0x8e, 0x27,
0x17, 0xb9, 0x49, 0xb7, 0x14, 0x72, 0x92, 0xf1, 0xc6, 0x55, 0xcb, 0x19, 0x44, 0x11, 0x63, 0x32, 0x16, 0x2c, 0xed, 0x56, 0xe3, 0x05, 0x42, 0xa1, 0x21, 0xed, 0x75, 0x69, 0xa8, 0x30, 0x93, 0x86,
0xac, 0x41, 0xb6, 0x38, 0x0a, 0xc8, 0xb0, 0x22, 0x19, 0x0c, 0xa8, 0xe1, 0xd1, 0x29, 0xe9, 0x9d, 0xf4, 0x7f, 0x15, 0x61, 0xfd, 0xc8, 0x38, 0x27, 0x6f, 0xbe, 0xab, 0xcf, 0x55, 0x25, 0x9e, 0xc0,
0x1d, 0x7a, 0xb6, 0x1b, 0x4c, 0x81, 0xe8, 0x1f, 0xc3, 0xf2, 0x0c, 0xe9, 0x11, 0xf1, 0xb0, 0xea, 0x2a, 0xcf, 0x02, 0xad, 0x98, 0x3c, 0xb2, 0x82, 0xab, 0xf2, 0x79, 0xcc, 0x24, 0x38, 0x8b, 0x88,
0x61, 0x18, 0xe1, 0x9d, 0x08, 0x98, 0x10, 0xf1, 0xaa, 0xb8, 0x23, 0xe7, 0xe7, 0x27, 0x0c, 0x28, 0x7e, 0x0c, 0xf5, 0x44, 0x72, 0x0d, 0x92, 0xd1, 0xa6, 0x82, 0x54, 0x22, 0x5b, 0xe3, 0x14, 0x5e,
0x8c, 0xdf, 0x95, 0x40, 0xff, 0x4c, 0xa2, 0x0c, 0x47, 0xb4, 0x22, 0x48, 0xbd, 0x05, 0xc9, 0x89, 0xa6, 0x21, 0x58, 0xbc, 0x7c, 0x43, 0x90, 0x69, 0x4c, 0x96, 0x2e, 0xdf, 0x98, 0xe8, 0x5f, 0x6a,
0x95, 0x6a, 0x88, 0xb5, 0x0f, 0x35, 0x4a, 0xc8, 0xd9, 0x2c, 0x4f, 0x70, 0x9d, 0x31, 0x46, 0x0e, 0x50, 0x6b, 0x1b, 0xbe, 0xf1, 0xd4, 0x35, 0xc9, 0xb3, 0x19, 0x2b, 0x65, 0x8e, 0xb9, 0xdb, 0x4d,
0xff, 0x09, 0x53, 0x15, 0xfa, 0x4a, 0xf4, 0x1c, 0xd5, 0xd6, 0x4d, 0x85, 0xbf, 0x63, 0x8f, 0xe2, 0x28, 0xb3, 0x58, 0xa0, 0xbe, 0x61, 0x0f, 0xb9, 0x9d, 0xe6, 0x71, 0xb4, 0xc1, 0xaa, 0x05, 0x3c,
0x24, 0x07, 0xda, 0x86, 0x15, 0x79, 0xc1, 0x85, 0x7d, 0x13, 0x4f, 0xa0, 0x32, 0xce, 0x6e, 0x1b, 0x3a, 0x25, 0xbd, 0xb3, 0x43, 0xd7, 0x72, 0xfc, 0x29, 0x75, 0xe2, 0x87, 0x50, 0x9a, 0xc1, 0x47,
0x3e, 0xd4, 0xe5, 0x6f, 0x91, 0x35, 0x74, 0x4a, 0x68, 0xf6, 0x40, 0x3f, 0x89, 0x9b, 0x8d, 0x49, 0x43, 0x1c, 0x16, 0xc2, 0x2c, 0x51, 0xb9, 0x27, 0x22, 0x57, 0x09, 0xa7, 0x29, 0x3b, 0x23, 0xfb,
0xaf, 0xcb, 0x44, 0x4f, 0x82, 0x53, 0x3c, 0xc6, 0x43, 0xa8, 0x26, 0xfe, 0x9c, 0xd0, 0x00, 0x34, 0xa7, 0x27, 0x2c, 0x5b, 0xe9, 0xbf, 0x89, 0xf5, 0x9e, 0x3c, 0xad, 0xe6, 0x29, 0x17, 0x9b, 0x10,
0x60, 0xe9, 0x38, 0xa1, 0xa7, 0x82, 0xc3, 0xa5, 0xd1, 0x85, 0x9a, 0xbc, 0x10, 0x44, 0x07, 0xcb, 0x3f, 0xae, 0x4a, 0x03, 0xfb, 0x50, 0xa3, 0x84, 0x9c, 0xcd, 0x32, 0x37, 0xa9, 0x32, 0xc4, 0xd0,
0xba, 0x3b, 0x3e, 0x5b, 0x14, 0xad, 0x09, 0xff, 0x8d, 0x7e, 0x90, 0x1e, 0xa6, 0xbc, 0xa7, 0x74, 0xea, 0x3f, 0x62, 0xac, 0x02, 0x5d, 0x89, 0xc6, 0xa7, 0xd2, 0xba, 0xa5, 0x30, 0x7a, 0xa4, 0x51,
0x20, 0x17, 0xc2, 0x9b, 0xda, 0xe4, 0x95, 0x60, 0x7c, 0xa5, 0x81, 0xde, 0x36, 0x03, 0xf3, 0xa9, 0x1c, 0xc7, 0x40, 0x5b, 0xb0, 0x2c, 0xab, 0x6c, 0xd0, 0xbc, 0x71, 0x2f, 0x2e, 0xe2, 0xf4, 0xb6,
0x67, 0x89, 0xb9, 0x4d, 0x23, 0x9e, 0xb6, 0x0b, 0x1d, 0xe1, 0x92, 0xfd, 0x73, 0x4e, 0x7c, 0x1a, 0xee, 0x41, 0x5d, 0x7e, 0x0b, 0xd7, 0xa5, 0x53, 0x4c, 0xb3, 0x07, 0xd5, 0x93, 0xa8, 0xe3, 0x99,
0x26, 0x6b, 0x19, 0x87, 0x4b, 0xf4, 0x43, 0x58, 0x96, 0xf9, 0x11, 0x0e, 0x5c, 0xb7, 0xc6, 0xdb, 0x34, 0x12, 0x88, 0x35, 0x46, 0x38, 0x81, 0xa3, 0x3f, 0x84, 0x4a, 0xec, 0xe7, 0x84, 0x2e, 0xa4,
0x20, 0x5b, 0xf1, 0x88, 0xc3, 0xf8, 0xaf, 0xc6, 0x87, 0x55, 0x98, 0xf4, 0xbc, 0x73, 0xe2, 0xbf, 0x01, 0x4b, 0xc7, 0x31, 0x3e, 0x65, 0x1c, 0x2c, 0xf5, 0x2e, 0xd4, 0x64, 0x55, 0x12, 0xd7, 0x0e,
0x48, 0x8d, 0x04, 0x5e, 0x1f, 0xe2, 0x1e, 0x24, 0x6c, 0x11, 0x41, 0x53, 0x15, 0x70, 0xb2, 0x2a, 0xd6, 0x62, 0x72, 0xc7, 0x14, 0xfd, 0x11, 0xff, 0x46, 0xdf, 0x4f, 0x4e, 0xc0, 0xde, 0x51, 0x2a,
0x62, 0x53, 0xd0, 0x83, 0x38, 0x10, 0xe5, 0xb1, 0x5d, 0x51, 0x3a, 0x8f, 0xe2, 0x58, 0xfd, 0x49, 0x90, 0x13, 0xe1, 0x3d, 0x77, 0xbc, 0x2e, 0xe9, 0x5f, 0x68, 0x50, 0x0d, 0x82, 0x86, 0xbb, 0x48,
0x4c, 0x36, 0xd2, 0xe7, 0xb8, 0xd2, 0x39, 0xa4, 0xf1, 0xb5, 0x06, 0xb5, 0x30, 0xba, 0xcf, 0x66, 0x23, 0x7a, 0x22, 0x11, 0x3c, 0x82, 0x25, 0xfb, 0x73, 0x4e, 0x3c, 0x1a, 0x38, 0x6b, 0x11, 0x07,
0xec, 0xdb, 0x0b, 0x4c, 0xb5, 0x6f, 0x40, 0x85, 0xdd, 0xcc, 0x34, 0x30, 0x9d, 0x21, 0x37, 0x63, 0x4b, 0xf4, 0x03, 0x28, 0x85, 0x0d, 0x78, 0x71, 0x6c, 0x16, 0x49, 0x1c, 0x04, 0x87, 0x18, 0xfa,
0x1e, 0xc7, 0x1b, 0x77, 0xef, 0xc1, 0x5a, 0x2e, 0xfd, 0x50, 0x1d, 0xe0, 0x53, 0xb7, 0xe7, 0x39, 0x7f, 0x35, 0x3e, 0x61, 0xc4, 0xa4, 0xe7, 0x9e, 0x13, 0xef, 0x65, 0x62, 0x8e, 0x73, 0xf9, 0x08,
0xc3, 0x01, 0x09, 0xc8, 0xea, 0x1c, 0xd2, 0x61, 0xf9, 0x51, 0xb8, 0xd2, 0x5a, 0xff, 0xab, 0x41, 0x7e, 0x10, 0x93, 0x65, 0xfa, 0x65, 0x80, 0x33, 0x0b, 0x11, 0xd0, 0x83, 0xc8, 0x10, 0xc5, 0xb1,
0x95, 0xd9, 0x7d, 0x24, 0x3e, 0x4f, 0xa1, 0x21, 0x20, 0xfe, 0xfe, 0x76, 0x86, 0x9e, 0x1b, 0xcd, 0xad, 0x59, 0xd2, 0x8f, 0x22, 0x5b, 0xfd, 0x51, 0x8c, 0xa3, 0x92, 0xe7, 0xb8, 0xd6, 0xe1, 0xf1,
0xcb, 0xd0, 0xc7, 0x63, 0x00, 0x27, 0x4f, 0x2a, 0xe3, 0xd0, 0xbc, 0x33, 0x86, 0x23, 0x43, 0x6e, 0xf6, 0x3d, 0x58, 0xcd, 0x58, 0x1e, 0xd5, 0x01, 0x3e, 0x71, 0x7a, 0xae, 0x3d, 0x1c, 0x10, 0x9f,
0xcc, 0x21, 0x87, 0x6b, 0x64, 0x9d, 0xc5, 0x33, 0xbb, 0x77, 0x16, 0x36, 0x65, 0x13, 0x34, 0x66, 0xac, 0xcc, 0xa1, 0x2a, 0x94, 0x1e, 0x05, 0x2b, 0xad, 0xf5, 0xbf, 0x1a, 0x54, 0x98, 0x43, 0x1c,
0x48, 0x43, 0x8d, 0x99, 0x31, 0x9c, 0x5c, 0x88, 0x59, 0x4d, 0x98, 0xe5, 0xc6, 0x1c, 0xfa, 0x12, 0x89, 0xe7, 0x3c, 0x34, 0x04, 0xc4, 0xe7, 0x15, 0xf6, 0xd0, 0x75, 0xc2, 0xf9, 0x22, 0xfa, 0x70,
0xd6, 0xd9, 0x83, 0x34, 0x7a, 0x17, 0x87, 0x0a, 0x5b, 0xe3, 0x15, 0xe6, 0x88, 0x5f, 0x53, 0xa5, 0x4c, 0xac, 0x67, 0x41, 0xa5, 0x0a, 0x9a, 0x77, 0xc7, 0x60, 0xa4, 0xc0, 0xf5, 0x39, 0x64, 0x73,
0x09, 0x7a, 0xf2, 0xeb, 0x18, 0x52, 0x8d, 0xec, 0x15, 0x1f, 0xf0, 0x9a, 0x1f, 0x4c, 0xa5, 0x8b, 0x8e, 0xac, 0xb3, 0x78, 0x66, 0xf5, 0xce, 0x82, 0xa6, 0x6c, 0x02, 0xc7, 0x14, 0x68, 0xc0, 0x31,
0x54, 0xec, 0xc3, 0x02, 0x7f, 0x25, 0x20, 0x55, 0x09, 0x26, 0xbf, 0x84, 0x35, 0x27, 0xbd, 0xda, 0x35, 0xb6, 0x94, 0x0b, 0x31, 0xdb, 0x0a, 0x1c, 0x4c, 0x9f, 0x43, 0x9f, 0xc3, 0x1a, 0xbb, 0xc0,
0x8d, 0x39, 0xf4, 0x6b, 0x58, 0xc9, 0x7c, 0x83, 0x40, 0x1f, 0x2a, 0x44, 0xaa, 0xbf, 0x26, 0x35, 0x87, 0x73, 0x84, 0x80, 0x61, 0x6b, 0x3c, 0xc3, 0x0c, 0xf0, 0x25, 0x59, 0x1a, 0x50, 0x8d, 0xbf,
0xef, 0x16, 0x21, 0x4d, 0xfa, 0x25, 0x39, 0xa7, 0x57, 0xfa, 0x45, 0xf1, 0xad, 0x41, 0xe9, 0x17, 0x26, 0x22, 0xd5, 0x13, 0x87, 0xe2, 0xc1, 0xb3, 0xf9, 0xde, 0x54, 0xb8, 0x90, 0xc5, 0x3e, 0x2c,
0xd5, 0xc0, 0xdf, 0x98, 0x43, 0x7d, 0xa8, 0xa7, 0xc7, 0x0f, 0x68, 0x5b, 0xc1, 0xac, 0x9c, 0xc8, 0xf0, 0x5b, 0x02, 0x52, 0x79, 0x7f, 0xfc, 0xe5, 0xb0, 0x39, 0x69, 0xca, 0xa1, 0xcf, 0xa1, 0x5f,
0x36, 0x3f, 0x2c, 0x40, 0x19, 0x29, 0x72, 0x60, 0x35, 0x3b, 0x65, 0x46, 0x77, 0x27, 0x0a, 0x48, 0xc2, 0x72, 0xea, 0xcd, 0x06, 0xbd, 0xaf, 0x20, 0xa9, 0x7e, 0x7d, 0x6b, 0x6e, 0xe7, 0x01, 0x8d,
0xd7, 0xcb, 0x47, 0x85, 0x68, 0x23, 0x75, 0x2f, 0x78, 0x16, 0xe7, 0xa6, 0x9c, 0x68, 0x47, 0x2d, 0xeb, 0x25, 0xfe, 0xae, 0xa1, 0xd4, 0x8b, 0xe2, 0x6d, 0x46, 0xa9, 0x17, 0xd5, 0x03, 0x89, 0x3e,
0x66, 0xdc, 0xf8, 0xb5, 0xb9, 0x5b, 0x98, 0x3e, 0x52, 0x4d, 0x60, 0x2d, 0x37, 0xb5, 0x44, 0x1f, 0x87, 0xfa, 0x50, 0x4f, 0x8e, 0x6b, 0xd0, 0x96, 0x02, 0x59, 0x39, 0xc1, 0x6e, 0xbe, 0x9f, 0x03,
0x4d, 0x92, 0x93, 0x99, 0xec, 0x34, 0xa7, 0xcf, 0x55, 0x8d, 0x39, 0xf4, 0xb5, 0xb8, 0xab, 0x54, 0x32, 0x64, 0x64, 0xc3, 0x4a, 0x7a, 0x2a, 0x8f, 0xb6, 0x27, 0x12, 0x48, 0xc6, 0xcb, 0x07, 0xb9,
0x93, 0x40, 0x74, 0x4f, 0xad, 0x6d, 0xc2, 0x08, 0xb3, 0xd9, 0x7a, 0x1d, 0x96, 0xe8, 0xac, 0x2f, 0x60, 0x43, 0x76, 0x2f, 0xb9, 0x17, 0x67, 0xa6, 0xc2, 0x68, 0x47, 0x4d, 0x66, 0xdc, 0xb8, 0xba,
0xf9, 0x3d, 0xa3, 0x98, 0xa6, 0x65, 0xf1, 0x29, 0x94, 0x37, 0x7e, 0x4c, 0xd8, 0xbc, 0xf7, 0x1a, 0xb9, 0x9b, 0x1b, 0x3e, 0x64, 0x4d, 0x60, 0x35, 0x33, 0xe5, 0x45, 0x1f, 0x4c, 0xa2, 0x93, 0x9a,
0x1c, 0x91, 0x01, 0x5e, 0xf6, 0x73, 0x41, 0x08, 0x57, 0xbb, 0x53, 0x93, 0x73, 0x36, 0xac, 0xfa, 0xf9, 0x34, 0xa7, 0xcf, 0xa1, 0xf5, 0x39, 0xf4, 0xa5, 0x28, 0x13, 0xaa, 0xc9, 0x29, 0xba, 0xa7,
0x02, 0x56, 0x32, 0x6f, 0x20, 0x65, 0xfd, 0xab, 0xdf, 0x49, 0x05, 0xc0, 0x25, 0x73, 0x6d, 0xa3, 0xe6, 0x36, 0x61, 0xe4, 0xdb, 0x6c, 0x5d, 0x06, 0x25, 0x3c, 0xeb, 0x2b, 0x9e, 0xe2, 0x15, 0xd3,
0x31, 0x45, 0xa6, 0xb8, 0xda, 0x9b, 0x77, 0x8b, 0x90, 0x86, 0x07, 0x69, 0xfd, 0xbd, 0x0c, 0xcb, 0xc7, 0x74, 0x7e, 0x0a, 0xe8, 0x8d, 0x1f, 0xab, 0x36, 0xef, 0x5d, 0x02, 0x23, 0x14, 0xc0, 0x4d,
0xe1, 0x85, 0x7c, 0x05, 0xb7, 0xda, 0x15, 0x5c, 0x33, 0x5f, 0xc0, 0x4a, 0x66, 0x94, 0xaa, 0xf4, 0x3f, 0xaf, 0x04, 0xe9, 0x6a, 0x77, 0xaa, 0x73, 0xce, 0x96, 0xab, 0x3e, 0x83, 0xe5, 0xd4, 0x1d,
0xae, 0x7a, 0xdc, 0x3a, 0x2d, 0x74, 0x9f, 0x43, 0x2d, 0x35, 0x35, 0x45, 0x1f, 0x8c, 0xbb, 0x68, 0x48, 0x19, 0xff, 0xea, 0x7b, 0x52, 0x8e, 0xe4, 0x92, 0xaa, 0x98, 0x68, 0x4c, 0x90, 0x29, 0xaa,
0xb2, 0x68, 0x3d, 0x59, 0xf0, 0xde, 0xfd, 0x5f, 0xde, 0xeb, 0xdb, 0xc1, 0xe9, 0xe8, 0x98, 0xfd, 0x6a, 0x73, 0x3b, 0x0f, 0x68, 0x70, 0x90, 0xd6, 0xdf, 0x8b, 0x50, 0x0a, 0x3a, 0x9d, 0x6b, 0xa8,
0xb3, 0x2b, 0x48, 0xbf, 0x65, 0x7b, 0xf2, 0xd7, 0x6e, 0xe8, 0xa0, 0x5d, 0xce, 0xbd, 0xcb, 0xd4, 0x6a, 0xd7, 0x50, 0x66, 0x3e, 0x83, 0xe5, 0xd4, 0x90, 0x55, 0xa9, 0x5d, 0xf5, 0x20, 0x76, 0x9a,
0x0c, 0x8f, 0x8f, 0x17, 0xf9, 0xea, 0xfe, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0xa8, 0x93, 0x0b, 0xe9, 0x3e, 0x85, 0x5a, 0x62, 0x6a, 0x8a, 0xde, 0x1b, 0x57, 0x68, 0xd2, 0xd9, 0x7a, 0x32, 0xe1,
0x43, 0x5b, 0x23, 0x00, 0x00, 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. // Reference imports to suppress errors if they are not otherwise used.
......
0% or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment