mirror of https://github.com/milvus-io/milvus.git
Fix bugs (#5676)
* Remove redundant session startup Signed-off-by: sunby <bingyi.sun@zilliz.com> * Register datanode after start success Signed-off-by: sunby <bingyi.sun@zilliz.com> * fix meta snap shot Signed-off-by: yefu.chen <yefu.chen@zilliz.com> * fix datanode message stream channel Signed-off-by: yangxuan <xuan.yang@zilliz.com> * Fix bugs when drop empty collection Signed-off-by: sunby <bingyi.sun@zilliz.com> * Fix bug of getting pchan statistics from task scheduler Signed-off-by: dragondriver <jiquan.long@zilliz.com> * Fix i/dist/dataservice test code Signed-off-by: Congqi Xia <congqi.xia@zilliz.com> * Fix epoch lifetime not applied Signed-off-by: Congqi Xia <congqi.xia@zilliz.com> * fix datanode flowgraph dd node Signed-off-by: yangxuan <xuan.yang@zilliz.com> * Fix handle datanode timetick bug Signed-off-by: sunby <bingyi.sun@zilliz.com> * Remove repack function of dml stream Signed-off-by: dragondriver <jiquan.long@zilliz.com> * fix proxynode Signed-off-by: yefu.chen <yefu.chen@zilliz.com> * Apply extended seal policy Signed-off-by: Congqi Xia <congqi.xia@zilliz.com> * add check for time tick Signed-off-by: yefu.chen <yefu.chen@zilliz.com> * fix check Signed-off-by: yefu.chen <yefu.chen@zilliz.com> * Fix the repack function of dml stream Signed-off-by: dragondriver <jiquan.long@zilliz.com> * Fix the bug when send statistics of pchan Signed-off-by: dragondriver <jiquan.long@zilliz.com> * Fix the repack function when craete dml stream Signed-off-by: dragondriver <jiquan.long@zilliz.com> * fix bugs Signed-off-by: yefu.chen <yefu.chen@zilliz.com> * fix describe collection Signed-off-by: yefu.chen <yefu.chen@zilliz.com> * Fix bug when send timestamp statistics Signed-off-by: dragondriver <jiquan.long@zilliz.com> * fix data node Signed-off-by: yefu.chen <yefu.chen@zilliz.com> * Add length check before flush request Signed-off-by: Congqi Xia <congqi.xia@zilliz.com> * add log for data node Signed-off-by: yefu.chen <yefu.chen@zilliz.com> * Fix SaveBinlog bugs Signed-off-by: sunby <bingyi.sun@zilliz.com> * Add more log in datanode Signed-off-by: yangxuan <xuan.yang@zilliz.com> * Put SegmentState.Flushing as the last one in enum to fit the client Signed-off-by: sunby <bingyi.sun@zilliz.com> * Fix params in GetInsertBinlogPaths Signed-off-by: sunby <bingyi.sun@zilliz.com> * Rename policy Signed-off-by: sunby <bingyi.sun@zilliz.com> * Remove unused ddl functions and fields Signed-off-by: Congqi Xia <congqi.xia@zilliz.com> * Remove pchan when drop collection Signed-off-by: dragondriver <jiquan.long@zilliz.com> * Add balanced assignment policy Signed-off-by: sunby <bingyi.sun@zilliz.com> * fix master ut Signed-off-by: yefu.chen <yefu.chen@zilliz.com> * Add lock in session manager Signed-off-by: Congqi Xia <congqi.xia@zilliz.com> * add log for debug Signed-off-by: yefu.chen <yefu.chen@zilliz.com> * Fix some logic bug and typo Signed-off-by: Congqi Xia <congqi.xia@zilliz.com> * Fix recover bugs Signed-off-by: sunby <bingyi.sun@zilliz.com> * Get collection scheme of a specific timestamp Signed-off-by: yangxuan <xuan.yang@zilliz.com> * Change CheckPoint to SegmentInfo in VchannelInfo Signed-off-by: sunby <bingyi.sun@zilliz.com> * Recover Unflushed segment numOfRows Signed-off-by: yangxuan <xuan.yang@zilliz.com> * Fix dataservice unit tests Signed-off-by: sunby <bingyi.sun@zilliz.com> Co-authored-by: yefu.chen <yefu.chen@zilliz.com> Co-authored-by: yangxuan <xuan.yang@zilliz.com> Co-authored-by: dragondriver <jiquan.long@zilliz.com> Co-authored-by: Congqi Xia <congqi.xia@zilliz.com>pull/5779/head
parent
e7fd3aa926
commit
189ac881f3
|
@ -12,6 +12,7 @@
|
|||
package datanode
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
|
@ -20,20 +21,17 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
)
|
||||
|
||||
type Replica interface {
|
||||
init(initTs Timestamp) error
|
||||
|
||||
// collection
|
||||
getCollectionNum() int
|
||||
addCollection(collectionID UniqueID, schema *schemapb.CollectionSchema) error
|
||||
removeCollection(collectionID UniqueID) error
|
||||
getCollectionByID(collectionID UniqueID) (*Collection, error)
|
||||
getCollectionByID(collectionID UniqueID, ts Timestamp) (*Collection, error)
|
||||
hasCollection(collectionID UniqueID) bool
|
||||
|
||||
// segment
|
||||
addSegment(segmentID UniqueID, collID UniqueID, partitionID UniqueID, channelName string) error
|
||||
addSegment(segmentID, collID, partitionID UniqueID, channelName string) error
|
||||
removeSegment(segmentID UniqueID) error
|
||||
hasSegment(segmentID UniqueID) bool
|
||||
updateStatistics(segmentID UniqueID, numRows int64) error
|
||||
|
@ -64,8 +62,9 @@ type Segment struct {
|
|||
// It implements `Replica` interface.
|
||||
type CollectionSegmentReplica struct {
|
||||
mu sync.RWMutex
|
||||
collection *Collection
|
||||
segments map[UniqueID]*Segment
|
||||
collections map[UniqueID]*Collection
|
||||
metaService *metaService
|
||||
|
||||
posMu sync.Mutex
|
||||
startPositions map[UniqueID][]*internalpb.MsgPosition
|
||||
|
@ -74,19 +73,33 @@ type CollectionSegmentReplica struct {
|
|||
|
||||
var _ Replica = &CollectionSegmentReplica{}
|
||||
|
||||
func newReplica() Replica {
|
||||
func newReplica(ms types.MasterService, collectionID UniqueID) Replica {
|
||||
metaService := newMetaService(ms, collectionID)
|
||||
segments := make(map[UniqueID]*Segment)
|
||||
collections := make(map[UniqueID]*Collection)
|
||||
|
||||
var replica Replica = &CollectionSegmentReplica{
|
||||
segments: segments,
|
||||
collections: collections,
|
||||
collection: &Collection{id: collectionID},
|
||||
metaService: metaService,
|
||||
startPositions: make(map[UniqueID][]*internalpb.MsgPosition),
|
||||
endPositions: make(map[UniqueID][]*internalpb.MsgPosition),
|
||||
}
|
||||
return replica
|
||||
}
|
||||
|
||||
func (replica *CollectionSegmentReplica) init(initTs Timestamp) error {
|
||||
log.Debug("Initing replica ...")
|
||||
ctx := context.Background()
|
||||
schema, err := replica.metaService.getCollectionSchema(ctx, replica.collection.GetID(), initTs)
|
||||
if err != nil {
|
||||
log.Error("Replica init fail", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
replica.collection.schema = schema
|
||||
return nil
|
||||
}
|
||||
|
||||
func (replica *CollectionSegmentReplica) getChannelName(segID UniqueID) (string, error) {
|
||||
replica.mu.RLock()
|
||||
defer replica.mu.RUnlock()
|
||||
|
@ -226,59 +239,38 @@ func (replica *CollectionSegmentReplica) getSegmentStatisticsUpdates(segmentID U
|
|||
}
|
||||
|
||||
// --- collection ---
|
||||
func (replica *CollectionSegmentReplica) getCollectionNum() int {
|
||||
replica.mu.RLock()
|
||||
defer replica.mu.RUnlock()
|
||||
|
||||
return len(replica.collections)
|
||||
}
|
||||
|
||||
func (replica *CollectionSegmentReplica) addCollection(collectionID UniqueID, schema *schemapb.CollectionSchema) error {
|
||||
// getCollectionByID will get collection schema from masterservice if not exist.
|
||||
// If you want the latest collection schema, ts should be 0
|
||||
func (replica *CollectionSegmentReplica) getCollectionByID(collectionID UniqueID, ts Timestamp) (*Collection, error) {
|
||||
replica.mu.Lock()
|
||||
defer replica.mu.Unlock()
|
||||
|
||||
if _, ok := replica.collections[collectionID]; ok {
|
||||
return fmt.Errorf("Create an existing collection=%s", schema.GetName())
|
||||
if collectionID != replica.collection.GetID() {
|
||||
return nil, fmt.Errorf("Not supported collection %v", collectionID)
|
||||
}
|
||||
|
||||
newCollection, err := newCollection(collectionID, schema)
|
||||
if err != nil {
|
||||
return err
|
||||
if replica.collection.GetSchema() == nil {
|
||||
sch, err := replica.metaService.getCollectionSchema(context.Background(), collectionID, ts)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
replica.collection.schema = sch
|
||||
}
|
||||
|
||||
replica.collections[collectionID] = newCollection
|
||||
log.Debug("Create collection", zap.String("collection name", newCollection.GetName()))
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (replica *CollectionSegmentReplica) removeCollection(collectionID UniqueID) error {
|
||||
replica.mu.Lock()
|
||||
defer replica.mu.Unlock()
|
||||
|
||||
delete(replica.collections, collectionID)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (replica *CollectionSegmentReplica) getCollectionByID(collectionID UniqueID) (*Collection, error) {
|
||||
replica.mu.RLock()
|
||||
defer replica.mu.RUnlock()
|
||||
|
||||
coll, ok := replica.collections[collectionID]
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("Cannot get collection %d by ID: not exist", collectionID)
|
||||
}
|
||||
|
||||
return coll, nil
|
||||
return replica.collection, nil
|
||||
}
|
||||
|
||||
func (replica *CollectionSegmentReplica) hasCollection(collectionID UniqueID) bool {
|
||||
replica.mu.RLock()
|
||||
defer replica.mu.RUnlock()
|
||||
|
||||
_, ok := replica.collections[collectionID]
|
||||
return ok
|
||||
if replica.collection != nil &&
|
||||
collectionID == replica.collection.GetID() &&
|
||||
replica.collection.schema != nil {
|
||||
return true
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
// getSegmentsCheckpoints get current open segments checkpoints
|
||||
|
|
|
@ -19,99 +19,32 @@ import (
|
|||
)
|
||||
|
||||
func TestReplica_Collection(t *testing.T) {
|
||||
Factory := &MetaFactory{}
|
||||
collID := UniqueID(100)
|
||||
collMetaMock := Factory.CollectionMetaFactory(collID, "test-coll-name-0")
|
||||
|
||||
t.Run("get_collection_num", func(t *testing.T) {
|
||||
replica := newReplica()
|
||||
assert.Zero(t, replica.getCollectionNum())
|
||||
|
||||
replica = new(CollectionSegmentReplica)
|
||||
assert.Zero(t, replica.getCollectionNum())
|
||||
|
||||
replica = &CollectionSegmentReplica{
|
||||
collections: map[UniqueID]*Collection{
|
||||
0: {id: 0},
|
||||
1: {id: 1},
|
||||
2: {id: 2},
|
||||
},
|
||||
}
|
||||
assert.Equal(t, 3, replica.getCollectionNum())
|
||||
})
|
||||
|
||||
t.Run("add_collection", func(t *testing.T) {
|
||||
replica := newReplica()
|
||||
require.Zero(t, replica.getCollectionNum())
|
||||
|
||||
err := replica.addCollection(collID, nil)
|
||||
assert.Error(t, err)
|
||||
assert.Zero(t, replica.getCollectionNum())
|
||||
|
||||
err = replica.addCollection(collID, collMetaMock.Schema)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, replica.getCollectionNum())
|
||||
assert.True(t, replica.hasCollection(collID))
|
||||
coll, err := replica.getCollectionByID(collID)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, coll)
|
||||
assert.Equal(t, collID, coll.GetID())
|
||||
assert.Equal(t, collMetaMock.Schema.GetName(), coll.GetName())
|
||||
assert.Equal(t, collMetaMock.Schema, coll.GetSchema())
|
||||
|
||||
sameID := collID
|
||||
otherSchema := Factory.CollectionMetaFactory(sameID, "test-coll-name-1").GetSchema()
|
||||
err = replica.addCollection(sameID, otherSchema)
|
||||
assert.Error(t, err)
|
||||
|
||||
})
|
||||
|
||||
t.Run("remove_collection", func(t *testing.T) {
|
||||
replica := newReplica()
|
||||
require.False(t, replica.hasCollection(collID))
|
||||
require.Zero(t, replica.getCollectionNum())
|
||||
|
||||
err := replica.removeCollection(collID)
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = replica.addCollection(collID, collMetaMock.Schema)
|
||||
require.NoError(t, err)
|
||||
require.True(t, replica.hasCollection(collID))
|
||||
require.Equal(t, 1, replica.getCollectionNum())
|
||||
|
||||
err = replica.removeCollection(collID)
|
||||
assert.NoError(t, err)
|
||||
assert.False(t, replica.hasCollection(collID))
|
||||
assert.Zero(t, replica.getCollectionNum())
|
||||
err = replica.removeCollection(collID)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("get_collection_by_id", func(t *testing.T) {
|
||||
replica := newReplica()
|
||||
mockMaster := &MasterServiceFactory{}
|
||||
replica := newReplica(mockMaster, collID)
|
||||
require.False(t, replica.hasCollection(collID))
|
||||
|
||||
coll, err := replica.getCollectionByID(collID)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, coll)
|
||||
coll, err := replica.getCollectionByID(collID, 0)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, coll)
|
||||
assert.NotNil(t, coll.GetSchema())
|
||||
assert.True(t, replica.hasCollection(collID))
|
||||
|
||||
err = replica.addCollection(collID, collMetaMock.Schema)
|
||||
require.NoError(t, err)
|
||||
require.True(t, replica.hasCollection(collID))
|
||||
require.Equal(t, 1, replica.getCollectionNum())
|
||||
|
||||
coll, err = replica.getCollectionByID(collID)
|
||||
coll, err = replica.getCollectionByID(collID, 0)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, coll)
|
||||
assert.Equal(t, collID, coll.GetID())
|
||||
assert.Equal(t, collMetaMock.Schema.GetName(), coll.GetName())
|
||||
assert.Equal(t, collMetaMock.Schema, coll.GetSchema())
|
||||
})
|
||||
}
|
||||
|
||||
func TestReplica_Segment(t *testing.T) {
|
||||
mockMaster := &MasterServiceFactory{}
|
||||
collID := UniqueID(1)
|
||||
|
||||
t.Run("Test segment", func(t *testing.T) {
|
||||
replica := newReplica()
|
||||
replica := newReplica(mockMaster, collID)
|
||||
assert.False(t, replica.hasSegment(0))
|
||||
|
||||
err := replica.addSegment(0, 1, 2, "insert-01")
|
||||
|
@ -162,7 +95,7 @@ func TestReplica_Segment(t *testing.T) {
|
|||
})
|
||||
|
||||
t.Run("Test errors", func(t *testing.T) {
|
||||
replica := newReplica()
|
||||
replica := newReplica(mockMaster, collID)
|
||||
require.False(t, replica.hasSegment(0))
|
||||
|
||||
seg, err := replica.getSegmentByID(0)
|
||||
|
|
|
@ -132,60 +132,11 @@ func (node *DataNode) Register() error {
|
|||
}
|
||||
|
||||
// Init function supposes data service is in INITIALIZING state.
|
||||
//
|
||||
// In Init process, data node will register itself to data service with its node id
|
||||
// and address. Therefore, `SetDataServiceInterface()` must be called before this func.
|
||||
// Registering return several channel names data node need.
|
||||
//
|
||||
// At last, data node initializes its `dataSyncService` and `metaService`.
|
||||
func (node *DataNode) Init() error {
|
||||
ctx := context.Background()
|
||||
|
||||
node.session = sessionutil.NewSession(ctx, Params.MetaRootPath, []string{Params.EtcdAddress})
|
||||
node.session.Init(typeutil.DataNodeRole, Params.IP+":"+strconv.Itoa(Params.Port), false)
|
||||
|
||||
req := &datapb.RegisterNodeRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
SourceID: node.NodeID,
|
||||
},
|
||||
Address: &commonpb.Address{
|
||||
Ip: Params.IP,
|
||||
Port: int64(Params.Port),
|
||||
},
|
||||
}
|
||||
|
||||
resp, err := node.dataService.RegisterNode(ctx, req)
|
||||
if err != nil {
|
||||
err = fmt.Errorf("Register node failed: %v", err)
|
||||
log.Debug("DataNode RegisterNode failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
err = fmt.Errorf("Receive error when registering data node, msg: %s", resp.Status.Reason)
|
||||
log.Debug("DataNode RegisterNode failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
if resp.InitParams != nil {
|
||||
for _, kv := range resp.InitParams.StartParams {
|
||||
switch kv.Key {
|
||||
case "DDChannelName":
|
||||
Params.DDChannelNames = []string{kv.Value}
|
||||
case "SegmentStatisticsChannelName":
|
||||
Params.SegmentStatisticsChannelName = kv.Value
|
||||
case "TimeTickChannelName":
|
||||
Params.TimeTickChannelName = kv.Value
|
||||
case "CompleteFlushChannelName":
|
||||
Params.CompleteFlushChannelName = kv.Value
|
||||
default:
|
||||
return fmt.Errorf("Invalid key: %v", kv.Key)
|
||||
}
|
||||
}
|
||||
}
|
||||
log.Debug("DataNode Init", zap.Any("DDChannelName", Params.DDChannelNames),
|
||||
log.Debug("DataNode Init",
|
||||
zap.Any("SegmentStatisticsChannelName", Params.SegmentStatisticsChannelName),
|
||||
zap.Any("TimeTickChannelName", Params.TimeTickChannelName),
|
||||
zap.Any("CompleteFlushChannelName", Params.TimeTickChannelName))
|
||||
)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
@ -198,20 +149,27 @@ func (node *DataNode) NewDataSyncService(vchan *datapb.VchannelInfo) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
replica := newReplica()
|
||||
var initTs Timestamp = 0
|
||||
if vchan.SeekPosition != nil {
|
||||
initTs = vchan.SeekPosition.Timestamp
|
||||
}
|
||||
|
||||
replica := newReplica(node.masterService, vchan.CollectionID)
|
||||
if err := replica.init(initTs); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
var alloc allocatorInterface = newAllocator(node.masterService)
|
||||
metaService := newMetaService(node.ctx, replica, node.masterService)
|
||||
|
||||
flushChan := make(chan *flushMsg, 100)
|
||||
dataSyncService := newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory, vchan, node.clearSignal)
|
||||
// TODO metaService using timestamp in DescribeCollection
|
||||
dataSyncService := newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory, vchan, node.clearSignal, node.dataService)
|
||||
node.vchan2SyncService[vchan.GetChannelName()] = dataSyncService
|
||||
node.vchan2FlushCh[vchan.GetChannelName()] = flushChan
|
||||
|
||||
metaService.init()
|
||||
go dataSyncService.start()
|
||||
|
||||
log.Info("New dataSyncService started!")
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -298,10 +256,15 @@ func (node *DataNode) WatchDmChannels(ctx context.Context, in *datapb.WatchDmCha
|
|||
|
||||
default:
|
||||
for _, chanInfo := range in.GetVchannels() {
|
||||
log.Info("DataNode new dataSyncService",
|
||||
zap.String("channel name", chanInfo.ChannelName),
|
||||
zap.Any("channal Info", chanInfo),
|
||||
)
|
||||
node.NewDataSyncService(chanInfo)
|
||||
}
|
||||
|
||||
status.ErrorCode = commonpb.ErrorCode_Success
|
||||
log.Debug("DataNode WatchDmChannels Done")
|
||||
return status, nil
|
||||
}
|
||||
}
|
||||
|
@ -396,7 +359,7 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
|
|||
return status, nil
|
||||
}
|
||||
|
||||
log.Debug("FlushSegments ...", zap.Int("num", len(req.SegmentIDs)))
|
||||
log.Debug("FlushSegments ...", zap.Int("num", len(req.SegmentIDs)), zap.Int64s("segments", req.SegmentIDs))
|
||||
dmlFlushedCh := make(chan []*datapb.ID2PathList, len(req.SegmentIDs))
|
||||
for _, id := range req.SegmentIDs {
|
||||
chanName := node.getChannelNamebySegmentID(id)
|
||||
|
@ -439,6 +402,7 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
|
|||
status.Reason = fmt.Sprintf("flush failed segment list = %s", failedSegments)
|
||||
return status, nil
|
||||
}
|
||||
log.Debug("FlushSegments Done")
|
||||
|
||||
status.ErrorCode = commonpb.ErrorCode_Success
|
||||
return status, nil
|
||||
|
|
|
@ -44,12 +44,13 @@ func TestDataNode(t *testing.T) {
|
|||
node1 := newIDLEDataNodeMock()
|
||||
node1.Start()
|
||||
vchannels := []*datapb.VchannelInfo{}
|
||||
for _, ch := range Params.InsertChannelNames {
|
||||
for _, ch := range []string{"datanode-01-test-WatchDmChannel",
|
||||
"datanode-02-test-WatchDmChannels"} {
|
||||
log.Debug("InsertChannels", zap.String("name", ch))
|
||||
vchan := &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: ch,
|
||||
CheckPoints: []*datapb.CheckPoint{},
|
||||
CollectionID: 1,
|
||||
ChannelName: ch,
|
||||
UnflushedSegments: []*datapb.SegmentInfo{},
|
||||
}
|
||||
vchannels = append(vchannels, vchan)
|
||||
}
|
||||
|
@ -67,7 +68,7 @@ func TestDataNode(t *testing.T) {
|
|||
assert.NoError(t, err)
|
||||
assert.NotNil(t, node1.vchan2FlushCh)
|
||||
assert.NotNil(t, node1.vchan2SyncService)
|
||||
sync, ok := node1.vchan2SyncService[Params.InsertChannelNames[0]]
|
||||
sync, ok := node1.vchan2SyncService["datanode-01-test-WatchDmChannel"]
|
||||
assert.True(t, ok)
|
||||
assert.NotNil(t, sync)
|
||||
assert.Equal(t, UniqueID(1), sync.collectionID)
|
||||
|
@ -94,9 +95,9 @@ func TestDataNode(t *testing.T) {
|
|||
dmChannelName := "fake-dm-channel-test-NewDataSyncService"
|
||||
|
||||
vchan := &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: dmChannelName,
|
||||
CheckPoints: []*datapb.CheckPoint{},
|
||||
CollectionID: 1,
|
||||
ChannelName: dmChannelName,
|
||||
UnflushedSegments: []*datapb.SegmentInfo{},
|
||||
}
|
||||
|
||||
require.Equal(t, 0, len(node2.vchan2FlushCh))
|
||||
|
@ -198,9 +199,9 @@ func TestDataNode(t *testing.T) {
|
|||
dmChannelName := "fake-dm-channel-test-NewDataSyncService"
|
||||
|
||||
vchan := &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: dmChannelName,
|
||||
CheckPoints: []*datapb.CheckPoint{},
|
||||
CollectionID: 1,
|
||||
ChannelName: dmChannelName,
|
||||
UnflushedSegments: []*datapb.SegmentInfo{},
|
||||
}
|
||||
|
||||
err := node.NewDataSyncService(vchan)
|
||||
|
@ -226,9 +227,9 @@ func TestDataNode(t *testing.T) {
|
|||
dmChannelName := "fake-dm-channel-test-BackGroundGC"
|
||||
|
||||
vchan := &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: dmChannelName,
|
||||
CheckPoints: []*datapb.CheckPoint{},
|
||||
CollectionID: 1,
|
||||
ChannelName: dmChannelName,
|
||||
UnflushedSegments: []*datapb.SegmentInfo{},
|
||||
}
|
||||
require.Equal(t, 0, len(node.vchan2FlushCh))
|
||||
require.Equal(t, 0, len(node.vchan2SyncService))
|
||||
|
|
|
@ -45,6 +45,7 @@ func newDataSyncService(ctx context.Context,
|
|||
factory msgstream.Factory,
|
||||
vchan *datapb.VchannelInfo,
|
||||
clearSignal chan<- UniqueID,
|
||||
dataService types.DataService,
|
||||
|
||||
) *dataSyncService {
|
||||
|
||||
|
@ -59,6 +60,7 @@ func newDataSyncService(ctx context.Context,
|
|||
idAllocator: alloc,
|
||||
msFactory: factory,
|
||||
collectionID: vchan.GetCollectionID(),
|
||||
dataService: dataService,
|
||||
}
|
||||
|
||||
service.initNodes(vchan)
|
||||
|
@ -69,6 +71,7 @@ func (dsService *dataSyncService) start() {
|
|||
if dsService.fg != nil {
|
||||
log.Debug("Data Sync Service starting flowgraph")
|
||||
dsService.fg.Start()
|
||||
log.Debug("Data Sync Service starting flowgraph Done")
|
||||
} else {
|
||||
log.Debug("Data Sync Service flowgraph nil")
|
||||
}
|
||||
|
@ -112,23 +115,28 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) {
|
|||
Position: &v.pos,
|
||||
})
|
||||
}
|
||||
log.Debug("SaveBinlogPath",
|
||||
zap.Int64("SegmentID", fu.segID),
|
||||
zap.Int64("CollectionID", fu.collID),
|
||||
zap.Int("Length of Field2BinlogPaths", len(id2path)),
|
||||
)
|
||||
|
||||
req := &datapb.SaveBinlogPathsRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: 0, //TOD msg type
|
||||
MsgID: 0, //TODO,msg id
|
||||
Timestamp: 0, //TODO, time stamp
|
||||
MsgType: 0, //TODO msg type
|
||||
MsgID: 0, //TODO msg id
|
||||
Timestamp: 0, //TODO time stamp
|
||||
SourceID: Params.NodeID,
|
||||
},
|
||||
SegmentID: fu.segID,
|
||||
CollectionID: 0, //TODO
|
||||
CollectionID: fu.collID,
|
||||
Field2BinlogPaths: id2path,
|
||||
CheckPoints: checkPoints,
|
||||
Flushed: fu.flushed,
|
||||
}
|
||||
rsp, err := dsService.dataService.SaveBinlogPaths(dsService.ctx, req)
|
||||
if err != nil {
|
||||
return fmt.Errorf("data service save bin log path failed, err = %w", err)
|
||||
return fmt.Errorf(err.Error())
|
||||
}
|
||||
if rsp.ErrorCode != commonpb.ErrorCode_Success {
|
||||
return fmt.Errorf("data service save bin log path failed, reason = %s", rsp.Reason)
|
||||
|
@ -149,8 +157,20 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) {
|
|||
dsService.idAllocator,
|
||||
dsService.flushChan,
|
||||
saveBinlog,
|
||||
vchanInfo.GetChannelName(),
|
||||
)
|
||||
|
||||
// recover segment checkpoints
|
||||
for _, us := range vchanInfo.GetUnflushedSegments() {
|
||||
if us.CollectionID != dsService.collectionID ||
|
||||
us.GetInsertChannel() != vchanInfo.ChannelName {
|
||||
continue
|
||||
}
|
||||
|
||||
dsService.replica.addSegment(us.GetID(), us.CollectionID, us.PartitionID, us.GetInsertChannel())
|
||||
dsService.replica.updateStatistics(us.GetID(), us.GetNumOfRows())
|
||||
}
|
||||
|
||||
dsService.fg.AddNode(dmStreamNode)
|
||||
dsService.fg.AddNode(ddNode)
|
||||
dsService.fg.AddNode(insertBufferNode)
|
||||
|
|
|
@ -18,6 +18,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
|
@ -39,32 +40,37 @@ func TestDataSyncService_Start(t *testing.T) {
|
|||
|
||||
Factory := &MetaFactory{}
|
||||
collMeta := Factory.CollectionMetaFactory(UniqueID(0), "coll1")
|
||||
mockMaster := &MasterServiceFactory{}
|
||||
collectionID := UniqueID(1)
|
||||
|
||||
flushChan := make(chan *flushMsg, 100)
|
||||
replica := newReplica()
|
||||
replica := newReplica(mockMaster, collectionID)
|
||||
err := replica.init(0)
|
||||
require.NoError(t, err)
|
||||
|
||||
allocFactory := NewAllocatorFactory(1)
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"pulsarAddress": pulsarURL,
|
||||
"receiveBufSize": 1024,
|
||||
"pulsarBufSize": 1024}
|
||||
err := msFactory.SetParams(m)
|
||||
err = msFactory.SetParams(m)
|
||||
|
||||
insertChannelName := "data_sync_service_test_dml"
|
||||
ddlChannelName := "data_sync_service_test_ddl"
|
||||
Params.FlushInsertBufferSize = 1
|
||||
|
||||
vchan := &datapb.VchannelInfo{
|
||||
CollectionID: collMeta.GetID(),
|
||||
ChannelName: insertChannelName,
|
||||
CheckPoints: []*datapb.CheckPoint{},
|
||||
FlushedSegments: []int64{},
|
||||
CollectionID: collMeta.GetID(),
|
||||
ChannelName: insertChannelName,
|
||||
UnflushedSegments: []*datapb.SegmentInfo{},
|
||||
FlushedSegments: []int64{},
|
||||
}
|
||||
|
||||
signalCh := make(chan UniqueID, 100)
|
||||
sync := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchan, signalCh)
|
||||
sync := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchan, signalCh, &DataServiceFactory{})
|
||||
|
||||
sync.replica.addCollection(collMeta.ID, collMeta.Schema)
|
||||
// sync.replica.addCollection(collMeta.ID, collMeta.Schema)
|
||||
go sync.start()
|
||||
|
||||
timeRange := TimeRange{
|
||||
|
|
|
@ -30,9 +30,9 @@ type ddNode struct {
|
|||
clearSignal chan<- UniqueID
|
||||
collectionID UniqueID
|
||||
|
||||
mu sync.RWMutex
|
||||
seg2cp map[UniqueID]*datapb.CheckPoint // Segment ID
|
||||
vchanInfo *datapb.VchannelInfo
|
||||
mu sync.RWMutex
|
||||
seg2SegInfo map[UniqueID]*datapb.SegmentInfo // Segment ID to UnFlushed Segment
|
||||
vchanInfo *datapb.VchannelInfo
|
||||
}
|
||||
|
||||
func (ddn *ddNode) Name() string {
|
||||
|
@ -40,6 +40,9 @@ func (ddn *ddNode) Name() string {
|
|||
}
|
||||
|
||||
func (ddn *ddNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
||||
|
||||
// log.Debug("DDNode Operating")
|
||||
|
||||
if len(in) != 1 {
|
||||
log.Error("Invalid operate message input in ddNode", zap.Int("input length", len(in)))
|
||||
// TODO: add error handling
|
||||
|
@ -77,7 +80,12 @@ func (ddn *ddNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||
log.Info("Destroying current flowgraph")
|
||||
}
|
||||
case commonpb.MsgType_Insert:
|
||||
log.Debug("DDNode with insert messages")
|
||||
if msg.EndTs() < FilterThreshold {
|
||||
log.Info("Filtering Insert Messages",
|
||||
zap.Uint64("Message endts", msg.EndTs()),
|
||||
zap.Uint64("FilterThreshold", FilterThreshold),
|
||||
)
|
||||
resMsg := ddn.filterFlushedSegmentInsertMessages(msg.(*msgstream.InsertMsg))
|
||||
if resMsg != nil {
|
||||
iMsg.insertMessages = append(iMsg.insertMessages, resMsg)
|
||||
|
@ -97,20 +105,19 @@ func (ddn *ddNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||
}
|
||||
|
||||
func (ddn *ddNode) filterFlushedSegmentInsertMessages(msg *msgstream.InsertMsg) *msgstream.InsertMsg {
|
||||
ddn.mu.Lock()
|
||||
defer ddn.mu.Unlock()
|
||||
|
||||
if ddn.isFlushed(msg.GetSegmentID()) {
|
||||
return nil
|
||||
}
|
||||
|
||||
if cp, ok := ddn.seg2cp[msg.GetSegmentID()]; ok {
|
||||
if msg.EndTs() > cp.GetPosition().GetTimestamp() {
|
||||
ddn.mu.Lock()
|
||||
if si, ok := ddn.seg2SegInfo[msg.GetSegmentID()]; ok {
|
||||
if msg.EndTs() > si.GetDmlPosition().GetTimestamp() {
|
||||
delete(ddn.seg2SegInfo, msg.GetSegmentID())
|
||||
return nil
|
||||
}
|
||||
delete(ddn.seg2cp, msg.GetSegmentID())
|
||||
}
|
||||
|
||||
ddn.mu.Unlock()
|
||||
return msg
|
||||
}
|
||||
|
||||
|
@ -130,16 +137,16 @@ func newDDNode(clearSignal chan<- UniqueID, collID UniqueID, vchanInfo *datapb.V
|
|||
baseNode := BaseNode{}
|
||||
baseNode.SetMaxParallelism(Params.FlowGraphMaxQueueLength)
|
||||
|
||||
cp := make(map[UniqueID]*datapb.CheckPoint)
|
||||
for _, c := range vchanInfo.GetCheckPoints() {
|
||||
cp[c.GetSegmentID()] = c
|
||||
si := make(map[UniqueID]*datapb.SegmentInfo)
|
||||
for _, us := range vchanInfo.GetUnflushedSegments() {
|
||||
si[us.GetID()] = us
|
||||
}
|
||||
|
||||
return &ddNode{
|
||||
BaseNode: baseNode,
|
||||
clearSignal: clearSignal,
|
||||
collectionID: collID,
|
||||
seg2cp: cp,
|
||||
seg2SegInfo: si,
|
||||
vchanInfo: vchanInfo,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,6 +16,12 @@ import (
|
|||
)
|
||||
|
||||
func TestFlowGraphDDNode_Operate(t *testing.T) {
|
||||
// clearSignal := make(chan UniqueID)
|
||||
// collectionID := UniqueID(1)
|
||||
// vchanInfo := &datapb.VchannelInfo{
|
||||
// CollectionID: collectionID,
|
||||
// }
|
||||
// ddNode := newDDNode(clearSignal)
|
||||
// ddNode := newDDNode()
|
||||
|
||||
// var inMsg Msg = msgStream
|
||||
|
|
|
@ -21,17 +21,18 @@ import (
|
|||
)
|
||||
|
||||
func newDmInputNode(ctx context.Context, factory msgstream.Factory, vchannelName string, seekPos *internalpb.MsgPosition) *flowgraph.InputNode {
|
||||
// TODO seek
|
||||
maxQueueLength := Params.FlowGraphMaxQueueLength
|
||||
maxParallelism := Params.FlowGraphMaxParallelism
|
||||
// consumeSubName := Params.MsgChannelSubName
|
||||
|
||||
consumeSubName := Params.MsgChannelSubName
|
||||
insertStream, _ := factory.NewTtMsgStream(ctx)
|
||||
insertStream.Seek([]*internalpb.MsgPosition{seekPos})
|
||||
|
||||
// insertStream.AsConsumer([]string{vchannelName}, consumeSubName)
|
||||
// log.Debug("datanode AsConsumer: " + vchannelName + " : " + consumeSubName)
|
||||
log.Debug("datanode Seek: " + vchannelName)
|
||||
insertStream.AsConsumer([]string{vchannelName}, consumeSubName)
|
||||
log.Debug("datanode AsConsumer: " + vchannelName + " : " + consumeSubName)
|
||||
|
||||
if seekPos != nil {
|
||||
insertStream.Seek([]*internalpb.MsgPosition{seekPos})
|
||||
log.Debug("datanode Seek: " + vchannelName)
|
||||
}
|
||||
|
||||
var stream msgstream.MsgStream = insertStream
|
||||
node := flowgraph.NewInputNode(&stream, "dmInputNode", maxQueueLength, maxParallelism)
|
||||
|
|
|
@ -49,6 +49,7 @@ type (
|
|||
)
|
||||
type insertBufferNode struct {
|
||||
BaseNode
|
||||
channelName string
|
||||
insertBuffer *insertBuffer
|
||||
replica Replica
|
||||
idAllocator allocatorInterface
|
||||
|
@ -119,6 +120,8 @@ func (ibNode *insertBufferNode) Name() string {
|
|||
|
||||
func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
||||
|
||||
// log.Debug("InsertBufferNode Operating")
|
||||
|
||||
if len(in) != 1 {
|
||||
log.Error("Invalid operate message input in insertBufferNode", zap.Int("input length", len(in)))
|
||||
// TODO: add error handling
|
||||
|
@ -149,6 +152,8 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||
collID := msg.GetCollectionID()
|
||||
partitionID := msg.GetPartitionID()
|
||||
|
||||
log.Debug("InsertBufferNode Operating Segment", zap.Int64("ID", currentSegID))
|
||||
|
||||
if !ibNode.replica.hasSegment(currentSegID) {
|
||||
err := ibNode.replica.addSegment(currentSegID, collID, partitionID, msg.GetChannelID())
|
||||
if err != nil {
|
||||
|
@ -201,17 +206,15 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||
}
|
||||
|
||||
// 1.1 Get CollectionMeta
|
||||
collection, err := ibNode.replica.getCollectionByID(collectionID)
|
||||
collSchema, err := ibNode.getCollectionSchemaByID(collectionID, msg.EndTs())
|
||||
if err != nil {
|
||||
// GOOSE TODO add error handler
|
||||
log.Error("Get meta wrong:", zap.Error(err))
|
||||
log.Error("Get schema wrong:", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
collSchema := collection.schema
|
||||
// 1.2 Get Fields
|
||||
var pos int = 0 // Record position of blob
|
||||
log.Debug("DataNode flow_graph_insert_buffer_node", zap.Any("Fields", collSchema.Fields))
|
||||
var fieldIDs []int64
|
||||
var fieldTypes []schemapb.DataType
|
||||
for _, field := range collSchema.Fields {
|
||||
|
@ -219,9 +222,6 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||
fieldTypes = append(fieldTypes, field.DataType)
|
||||
}
|
||||
|
||||
log.Debug("DataNode flow_graph_insert_buffer_node", zap.Any("FieldIDs", fieldIDs))
|
||||
log.Debug("DataNode flow_graph_insert_buffer_node", zap.Any("fieldTypes", fieldTypes))
|
||||
|
||||
for _, field := range collSchema.Fields {
|
||||
switch field.DataType {
|
||||
case schemapb.DataType_FloatVector:
|
||||
|
@ -484,7 +484,7 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||
log.Debug(". Insert Buffer full, auto flushing ",
|
||||
zap.Int32("num of rows", ibNode.insertBuffer.size(segToFlush)))
|
||||
|
||||
collMeta, err := ibNode.getCollMetabySegID(segToFlush)
|
||||
collMeta, err := ibNode.getCollMetabySegID(segToFlush, iMsg.timeRange.timestampMax)
|
||||
if err != nil {
|
||||
log.Error("Auto flush failed .. cannot get collection meta ..", zap.Error(err))
|
||||
continue
|
||||
|
@ -523,7 +523,10 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||
select {
|
||||
case fmsg := <-ibNode.flushChan:
|
||||
currentSegID := fmsg.segmentID
|
||||
log.Debug(". Receiving flush message", zap.Int64("segmentID", currentSegID))
|
||||
log.Debug(". Receiving flush message",
|
||||
zap.Int64("segmentID", currentSegID),
|
||||
zap.Int64("collectionID", fmsg.collectionID),
|
||||
)
|
||||
|
||||
if ibNode.insertBuffer.size(currentSegID) <= 0 {
|
||||
log.Debug(".. Buffer empty ...")
|
||||
|
@ -560,7 +563,7 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||
// TODO add error handling
|
||||
}
|
||||
|
||||
collSch, err = ibNode.getCollectionSchemaByID(seg.collectionID)
|
||||
collSch, err = ibNode.getCollectionSchemaByID(seg.collectionID, iMsg.timeRange.timestampMax)
|
||||
if err != nil {
|
||||
log.Error("Flush failed .. cannot get collection schema ..", zap.Error(err))
|
||||
clearFn()
|
||||
|
@ -580,8 +583,8 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||
if fu.field2Path != nil {
|
||||
fu.checkPoint = ibNode.listSegmentCheckPoints()
|
||||
fu.flushed = true
|
||||
if ibNode.dsSaveBinlog(&fu) != nil {
|
||||
log.Debug("data service save bin log path failed", zap.Error(err))
|
||||
if err := ibNode.dsSaveBinlog(&fu); err != nil {
|
||||
log.Debug("Data service save binlog path failed", zap.Error(err))
|
||||
} else {
|
||||
// this segment has flushed, so it's not `open segment`, so remove from the check point
|
||||
ibNode.removeSegmentCheckPoint(fu.segID)
|
||||
|
@ -727,19 +730,21 @@ func (ibNode *insertBufferNode) listSegmentCheckPoints() map[UniqueID]segmentChe
|
|||
|
||||
func (ibNode *insertBufferNode) writeHardTimeTick(ts Timestamp) error {
|
||||
msgPack := msgstream.MsgPack{}
|
||||
timeTickMsg := msgstream.TimeTickMsg{
|
||||
timeTickMsg := msgstream.DataNodeTtMsg{
|
||||
// timeTickMsg := msgstream.TimeTickMsg{
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
BeginTimestamp: ts,
|
||||
EndTimestamp: ts,
|
||||
HashValues: []uint32{0},
|
||||
},
|
||||
TimeTickMsg: internalpb.TimeTickMsg{
|
||||
DataNodeTtMsg: datapb.DataNodeTtMsg{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_TimeTick,
|
||||
MsgID: 0, // GOOSE TODO
|
||||
Timestamp: ts, // GOOSE TODO
|
||||
SourceID: Params.NodeID,
|
||||
MsgType: commonpb.MsgType_DataNodeTt,
|
||||
MsgID: 0,
|
||||
Timestamp: ts,
|
||||
},
|
||||
ChannelName: ibNode.channelName,
|
||||
Timestamp: ts,
|
||||
},
|
||||
}
|
||||
msgPack.Msgs = append(msgPack.Msgs, &timeTickMsg)
|
||||
|
@ -781,27 +786,29 @@ func (ibNode *insertBufferNode) updateSegStatistics(segIDs []UniqueID) error {
|
|||
return ibNode.segmentStatisticsStream.Produce(&msgPack)
|
||||
}
|
||||
|
||||
func (ibNode *insertBufferNode) getCollectionSchemaByID(collectionID UniqueID) (*schemapb.CollectionSchema, error) {
|
||||
ret, err := ibNode.replica.getCollectionByID(collectionID)
|
||||
func (ibNode *insertBufferNode) getCollectionSchemaByID(collectionID UniqueID, ts Timestamp) (*schemapb.CollectionSchema, error) {
|
||||
ret, err := ibNode.replica.getCollectionByID(collectionID, ts)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return ret.schema, nil
|
||||
}
|
||||
|
||||
func (ibNode *insertBufferNode) getCollMetabySegID(segmentID UniqueID) (meta *etcdpb.CollectionMeta, err error) {
|
||||
func (ibNode *insertBufferNode) getCollMetabySegID(segmentID UniqueID, ts Timestamp) (meta *etcdpb.CollectionMeta, err error) {
|
||||
ret, err := ibNode.replica.getSegmentByID(segmentID)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
meta = &etcdpb.CollectionMeta{}
|
||||
meta.ID = ret.collectionID
|
||||
|
||||
coll, err := ibNode.replica.getCollectionByID(ret.collectionID)
|
||||
coll, err := ibNode.replica.getCollectionByID(ret.collectionID, ts)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
meta.Schema = coll.GetSchema()
|
||||
|
||||
meta = &etcdpb.CollectionMeta{
|
||||
ID: ret.collectionID,
|
||||
Schema: coll.GetSchema(),
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -822,6 +829,7 @@ func newInsertBufferNode(
|
|||
idAllocator allocatorInterface,
|
||||
flushCh <-chan *flushMsg,
|
||||
saveBinlog func(*segmentFlushUnit) error,
|
||||
channelName string,
|
||||
) *insertBufferNode {
|
||||
|
||||
maxQueueLength := Params.FlowGraphMaxQueueLength
|
||||
|
@ -870,6 +878,7 @@ func newInsertBufferNode(
|
|||
BaseNode: baseNode,
|
||||
insertBuffer: iBuffer,
|
||||
minIOKV: minIOKV,
|
||||
channelName: channelName,
|
||||
|
||||
timeTickStream: wTtMsgStream,
|
||||
segmentStatisticsStream: segStatisticsMsgStream,
|
||||
|
|
|
@ -40,6 +40,8 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
|||
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
|
||||
defer cancel()
|
||||
|
||||
insertChannelName := "datanode-01-test-flowgraphinsertbuffernode-operate"
|
||||
|
||||
testPath := "/test/datanode/root/meta"
|
||||
err := clearEtcd(testPath)
|
||||
require.NoError(t, err)
|
||||
|
@ -47,11 +49,13 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
|||
|
||||
Factory := &MetaFactory{}
|
||||
collMeta := Factory.CollectionMetaFactory(UniqueID(0), "coll1")
|
||||
mockMaster := &MasterServiceFactory{}
|
||||
|
||||
replica := newReplica()
|
||||
err = replica.addCollection(collMeta.ID, collMeta.Schema)
|
||||
replica := newReplica(mockMaster, collMeta.ID)
|
||||
err = replica.init(0)
|
||||
require.NoError(t, err)
|
||||
err = replica.addSegment(1, collMeta.ID, 0, Params.InsertChannelNames[0])
|
||||
|
||||
err = replica.addSegment(1, collMeta.ID, 0, insertChannelName)
|
||||
require.NoError(t, err)
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
|
@ -68,7 +72,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
|||
}
|
||||
|
||||
flushChan := make(chan *flushMsg, 100)
|
||||
iBNode := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog)
|
||||
iBNode := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string")
|
||||
|
||||
dmlFlushedCh := make(chan []*datapb.ID2PathList, 1)
|
||||
|
||||
|
@ -80,7 +84,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
|||
dmlFlushedCh: dmlFlushedCh,
|
||||
}
|
||||
|
||||
inMsg := genInsertMsg()
|
||||
inMsg := genInsertMsg(insertChannelName)
|
||||
var iMsg flowgraph.Msg = &inMsg
|
||||
iBNode.Operate([]flowgraph.Msg{iMsg})
|
||||
isflushed := <-dmlFlushedCh
|
||||
|
@ -88,7 +92,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
|||
log.Debug("DML binlog paths", zap.Any("paths", isflushed))
|
||||
}
|
||||
|
||||
func genInsertMsg() insertMsg {
|
||||
func genInsertMsg(insertChannelName string) insertMsg {
|
||||
|
||||
timeRange := TimeRange{
|
||||
timestampMin: 0,
|
||||
|
@ -97,7 +101,7 @@ func genInsertMsg() insertMsg {
|
|||
|
||||
startPos := []*internalpb.MsgPosition{
|
||||
{
|
||||
ChannelName: Params.InsertChannelNames[0],
|
||||
ChannelName: insertChannelName,
|
||||
MsgID: make([]byte, 0),
|
||||
Timestamp: 0,
|
||||
},
|
||||
|
@ -125,6 +129,7 @@ func TestFlushSegment(t *testing.T) {
|
|||
defer cancel()
|
||||
idAllocMock := NewAllocatorFactory(1)
|
||||
mockMinIO := memkv.NewMemoryKV()
|
||||
insertChannelName := "datanode-02-test-flushsegment"
|
||||
|
||||
segmentID, _ := idAllocMock.allocID()
|
||||
partitionID, _ := idAllocMock.allocID()
|
||||
|
@ -134,10 +139,13 @@ func TestFlushSegment(t *testing.T) {
|
|||
|
||||
collMeta := genCollectionMeta(collectionID, "test_flush_segment_txn")
|
||||
flushMap := sync.Map{}
|
||||
replica := newReplica()
|
||||
err := replica.addCollection(collMeta.ID, collMeta.Schema)
|
||||
mockMaster := &MasterServiceFactory{}
|
||||
|
||||
replica := newReplica(mockMaster, collMeta.ID)
|
||||
err := replica.init(0)
|
||||
require.NoError(t, err)
|
||||
err = replica.addSegment(segmentID, collMeta.ID, 0, Params.InsertChannelNames[0])
|
||||
|
||||
err = replica.addSegment(segmentID, collMeta.ID, 0, insertChannelName)
|
||||
require.NoError(t, err)
|
||||
replica.setEndPositions(segmentID, []*internalpb.MsgPosition{{ChannelName: "TestChannel"}})
|
||||
|
||||
|
@ -171,7 +179,7 @@ func TestFlushSegment(t *testing.T) {
|
|||
saveBinlog := func(*segmentFlushUnit) error {
|
||||
return nil
|
||||
}
|
||||
ibNode := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog)
|
||||
ibNode := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string")
|
||||
|
||||
flushSegment(collMeta,
|
||||
segmentID,
|
||||
|
@ -259,13 +267,17 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
|
|||
collMeta := Factory.CollectionMetaFactory(UniqueID(0), "coll1")
|
||||
dataFactory := NewDataFactory()
|
||||
|
||||
mockMaster := &MasterServiceFactory{}
|
||||
|
||||
colRep := &CollectionSegmentReplica{
|
||||
segments: make(map[UniqueID]*Segment),
|
||||
collections: make(map[UniqueID]*Collection),
|
||||
collection: &Collection{id: collMeta.ID},
|
||||
startPositions: make(map[UniqueID][]*internalpb.MsgPosition),
|
||||
endPositions: make(map[UniqueID][]*internalpb.MsgPosition),
|
||||
}
|
||||
err = colRep.addCollection(collMeta.ID, collMeta.Schema)
|
||||
|
||||
colRep.metaService = newMetaService(mockMaster, collMeta.ID)
|
||||
err = colRep.init(0)
|
||||
require.NoError(t, err)
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
|
@ -283,9 +295,9 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
|
|||
}
|
||||
|
||||
flushChan := make(chan *flushMsg, 100)
|
||||
iBNode := newInsertBufferNode(ctx, colRep, msFactory, NewAllocatorFactory(), flushChan, saveBinlog)
|
||||
iBNode := newInsertBufferNode(ctx, colRep, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string")
|
||||
|
||||
inMsg := genInsertMsg()
|
||||
inMsg := genInsertMsg("datanode-03-test-autoflush")
|
||||
inMsg.insertMessages = dataFactory.GetMsgStreamInsertMsgs(100)
|
||||
inMsg.insertMessages = append(inMsg.insertMessages, dataFactory.GetMsgStreamInsertMsgs(32000)...)
|
||||
for i := range inMsg.insertMessages {
|
||||
|
|
|
@ -24,69 +24,27 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
)
|
||||
|
||||
// metaService initialize replica collections in data node from master service.
|
||||
// Initializing replica collections happens on data node starting. It depends on
|
||||
// a healthy master service and a valid master service grpc client.
|
||||
type metaService struct {
|
||||
ctx context.Context
|
||||
replica Replica
|
||||
collectionID UniqueID
|
||||
masterClient types.MasterService
|
||||
}
|
||||
|
||||
func newMetaService(ctx context.Context, replica Replica, m types.MasterService) *metaService {
|
||||
func newMetaService(m types.MasterService, collectionID UniqueID) *metaService {
|
||||
return &metaService{
|
||||
ctx: ctx,
|
||||
replica: replica,
|
||||
masterClient: m,
|
||||
collectionID: collectionID,
|
||||
}
|
||||
}
|
||||
|
||||
func (mService *metaService) init() {
|
||||
log.Debug("Initing meta ...")
|
||||
ctx := context.Background()
|
||||
err := mService.loadCollections(ctx)
|
||||
if err != nil {
|
||||
log.Error("metaService init failed", zap.Error(err))
|
||||
}
|
||||
}
|
||||
|
||||
func (mService *metaService) loadCollections(ctx context.Context) error {
|
||||
names, err := mService.getCollectionNames(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, name := range names {
|
||||
err := mService.createCollection(ctx, name)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (mService *metaService) getCollectionNames(ctx context.Context) ([]string, error) {
|
||||
req := &milvuspb.ShowCollectionsRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_ShowCollections,
|
||||
MsgID: 0, //GOOSE TODO
|
||||
Timestamp: 0, // GOOSE TODO
|
||||
SourceID: Params.NodeID,
|
||||
},
|
||||
DbName: "default", // GOOSE TODO
|
||||
}
|
||||
|
||||
response, err := mService.masterClient.ShowCollections(ctx, req)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("Get collection names from master service wrong: %v", err)
|
||||
}
|
||||
return response.GetCollectionNames(), nil
|
||||
}
|
||||
|
||||
func (mService *metaService) createCollection(ctx context.Context, name string) error {
|
||||
log.Debug("Describing collections")
|
||||
func (mService *metaService) getCollectionSchema(ctx context.Context, collID UniqueID, timestamp Timestamp) (*schemapb.CollectionSchema, error) {
|
||||
log.Debug("Describing collection", zap.Int64("ID", collID))
|
||||
req := &milvuspb.DescribeCollectionRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_DescribeCollection,
|
||||
|
@ -94,21 +52,17 @@ func (mService *metaService) createCollection(ctx context.Context, name string)
|
|||
Timestamp: 0, // GOOSE TODO
|
||||
SourceID: Params.NodeID,
|
||||
},
|
||||
DbName: "default", // GOOSE TODO
|
||||
CollectionName: name,
|
||||
DbName: "default", // GOOSE TODO
|
||||
CollectionID: collID,
|
||||
TimeStamp: timestamp,
|
||||
}
|
||||
|
||||
response, err := mService.masterClient.DescribeCollection(ctx, req)
|
||||
if err != nil {
|
||||
return fmt.Errorf("Describe collection %v from master service wrong: %v", name, err)
|
||||
return nil, fmt.Errorf("Describe collection %v from master service wrong: %v", collID, err)
|
||||
}
|
||||
|
||||
err = mService.replica.addCollection(response.GetCollectionID(), response.GetSchema())
|
||||
if err != nil {
|
||||
return fmt.Errorf("Add collection %v into collReplica wrong: %v", name, err)
|
||||
}
|
||||
|
||||
return nil
|
||||
return response.GetSchema(), nil
|
||||
}
|
||||
|
||||
func printCollectionStruct(obj *etcdpb.CollectionMeta) {
|
||||
|
@ -116,7 +70,7 @@ func printCollectionStruct(obj *etcdpb.CollectionMeta) {
|
|||
v = reflect.Indirect(v)
|
||||
typeOfS := v.Type()
|
||||
|
||||
for i := 0; i < v.NumField(); i++ {
|
||||
for i := 0; i < v.NumField()-3; i++ {
|
||||
if typeOfS.Field(i).Name == "GrpcMarshalString" {
|
||||
continue
|
||||
}
|
||||
|
|
|
@ -29,47 +29,17 @@ func TestMetaService_All(t *testing.T) {
|
|||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
replica := newReplica()
|
||||
mFactory := &MasterServiceFactory{}
|
||||
mFactory.setCollectionID(collectionID0)
|
||||
mFactory.setCollectionName(collectionName0)
|
||||
ms := newMetaService(ctx, replica, mFactory)
|
||||
ms := newMetaService(mFactory, collectionID0)
|
||||
|
||||
t.Run("Test getCollectionNames", func(t *testing.T) {
|
||||
names, err := ms.getCollectionNames(ctx)
|
||||
t.Run("Test getCollectionSchema", func(t *testing.T) {
|
||||
|
||||
sch, err := ms.getCollectionSchema(ctx, collectionID0, 0)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(names))
|
||||
assert.Equal(t, collectionName0, names[0])
|
||||
})
|
||||
|
||||
t.Run("Test createCollection", func(t *testing.T) {
|
||||
hasColletion := ms.replica.hasCollection(collectionID0)
|
||||
assert.False(t, hasColletion)
|
||||
|
||||
err := ms.createCollection(ctx, collectionName0)
|
||||
assert.NoError(t, err)
|
||||
hasColletion = ms.replica.hasCollection(collectionID0)
|
||||
assert.True(t, hasColletion)
|
||||
})
|
||||
|
||||
t.Run("Test loadCollections", func(t *testing.T) {
|
||||
hasColletion := ms.replica.hasCollection(collectionID1)
|
||||
assert.False(t, hasColletion)
|
||||
|
||||
mFactory.setCollectionID(1)
|
||||
mFactory.setCollectionName(collectionName1)
|
||||
err := ms.loadCollections(ctx)
|
||||
assert.NoError(t, err)
|
||||
|
||||
hasColletion = ms.replica.hasCollection(collectionID0)
|
||||
assert.True(t, hasColletion)
|
||||
hasColletion = ms.replica.hasCollection(collectionID1)
|
||||
assert.True(t, hasColletion)
|
||||
})
|
||||
|
||||
t.Run("Test Init", func(t *testing.T) {
|
||||
ms1 := newMetaService(ctx, replica, mFactory)
|
||||
ms1.init()
|
||||
assert.NotNil(t, sch)
|
||||
assert.Equal(t, sch.Name, collectionName0)
|
||||
})
|
||||
|
||||
t.Run("Test printCollectionStruct", func(t *testing.T) {
|
||||
|
|
|
@ -104,10 +104,10 @@ func newHEALTHDataNodeMock(dmChannelName string) *DataNode {
|
|||
node.SetDataServiceInterface(ds)
|
||||
|
||||
vchan := &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: dmChannelName,
|
||||
CheckPoints: []*datapb.CheckPoint{},
|
||||
FlushedSegments: []int64{},
|
||||
CollectionID: 1,
|
||||
ChannelName: dmChannelName,
|
||||
UnflushedSegments: []*datapb.SegmentInfo{},
|
||||
FlushedSegments: []int64{},
|
||||
}
|
||||
node.Start()
|
||||
|
||||
|
@ -125,14 +125,8 @@ func makeNewChannelNames(names []string, suffix string) []string {
|
|||
}
|
||||
|
||||
func refreshChannelNames() {
|
||||
Params.DDChannelNames = []string{"datanode-test"}
|
||||
Params.SegmentStatisticsChannelName = "segment-statistics"
|
||||
Params.CompleteFlushChannelName = "flush-completed"
|
||||
Params.InsertChannelNames = []string{"intsert-a-1", "insert-b-1"}
|
||||
Params.TimeTickChannelName = "hard-timetick"
|
||||
suffix := "-test-data-node" + strconv.FormatInt(rand.Int63n(100), 10)
|
||||
Params.DDChannelNames = makeNewChannelNames(Params.DDChannelNames, suffix)
|
||||
Params.InsertChannelNames = makeNewChannelNames(Params.InsertChannelNames, suffix)
|
||||
Params.SegmentStatisticsChannelName = "datanode-refresh-segment-statistics"
|
||||
Params.TimeTickChannelName = "datanode-refresh-hard-timetick"
|
||||
}
|
||||
|
||||
func clearEtcd(rootPath string) error {
|
||||
|
|
|
@ -32,7 +32,6 @@ type ParamTable struct {
|
|||
FlowGraphMaxParallelism int32
|
||||
FlushInsertBufferSize int32
|
||||
InsertBinlogRootPath string
|
||||
DdlBinlogRootPath string
|
||||
StatsBinlogRootPath string
|
||||
Log log.Config
|
||||
|
||||
|
@ -40,21 +39,12 @@ type ParamTable struct {
|
|||
// --- Pulsar ---
|
||||
PulsarAddress string
|
||||
|
||||
// - insert channel -
|
||||
InsertChannelNames []string
|
||||
|
||||
// - dd channel -
|
||||
DDChannelNames []string
|
||||
|
||||
// - seg statistics channel -
|
||||
SegmentStatisticsChannelName string
|
||||
|
||||
// - timetick channel -
|
||||
TimeTickChannelName string
|
||||
|
||||
// - complete flush channel -
|
||||
CompleteFlushChannelName string
|
||||
|
||||
// - channel subname -
|
||||
MsgChannelSubName string
|
||||
|
||||
|
@ -87,7 +77,6 @@ func (p *ParamTable) Init() {
|
|||
p.initFlowGraphMaxParallelism()
|
||||
p.initFlushInsertBufferSize()
|
||||
p.initInsertBinlogRootPath()
|
||||
p.initDdlBinlogRootPath()
|
||||
p.initStatsBinlogRootPath()
|
||||
p.initLogCfg()
|
||||
|
||||
|
@ -95,11 +84,11 @@ func (p *ParamTable) Init() {
|
|||
// --- Pulsar ---
|
||||
p.initPulsarAddress()
|
||||
|
||||
// - insert channel -
|
||||
p.initInsertChannelNames()
|
||||
// - seg statistics channel -
|
||||
p.initSegmentStatisticsChannelName()
|
||||
|
||||
// - dd channel -
|
||||
p.initDDChannelNames()
|
||||
// - timetick channel -
|
||||
p.initTimeTickChannelName()
|
||||
|
||||
// - channel subname -
|
||||
p.initMsgChannelSubName()
|
||||
|
@ -155,15 +144,6 @@ func (p *ParamTable) initInsertBinlogRootPath() {
|
|||
p.InsertBinlogRootPath = path.Join(rootPath, "insert_log")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initDdlBinlogRootPath() {
|
||||
// GOOSE TODO: rootPath change to TenentID
|
||||
rootPath, err := p.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.DdlBinlogRootPath = path.Join(rootPath, "data_definition_log")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initStatsBinlogRootPath() {
|
||||
rootPath, err := p.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
|
@ -181,13 +161,21 @@ func (p *ParamTable) initPulsarAddress() {
|
|||
p.PulsarAddress = url
|
||||
}
|
||||
|
||||
// - insert channel -
|
||||
func (p *ParamTable) initInsertChannelNames() {
|
||||
p.InsertChannelNames = make([]string, 0)
|
||||
func (p *ParamTable) initSegmentStatisticsChannelName() {
|
||||
|
||||
path, err := p.Load("msgChannel.chanNamePrefix.dataServiceStatistic")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.SegmentStatisticsChannelName = path
|
||||
}
|
||||
|
||||
func (p *ParamTable) initDDChannelNames() {
|
||||
p.DDChannelNames = make([]string, 0)
|
||||
func (p *ParamTable) initTimeTickChannelName() {
|
||||
path, err := p.Load("msgChannel.chanNamePrefix.dataServiceTimeTick")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.TimeTickChannelName = path
|
||||
}
|
||||
|
||||
// - msg channel subname -
|
||||
|
@ -220,6 +208,7 @@ func (p *ParamTable) initMetaRootPath() {
|
|||
p.MetaRootPath = path.Join(rootPath, subPath)
|
||||
}
|
||||
|
||||
// --- MinIO ---
|
||||
func (p *ParamTable) initMinioAddress() {
|
||||
endpoint, err := p.Load("_MinioAddress")
|
||||
if err != nil {
|
||||
|
|
|
@ -45,32 +45,17 @@ func TestParamTable_DataNode(t *testing.T) {
|
|||
log.Println("InsertBinlogRootPath:", path)
|
||||
})
|
||||
|
||||
t.Run("Test DdlBinlogRootPath", func(t *testing.T) {
|
||||
path := Params.DdlBinlogRootPath
|
||||
log.Println("DdBinlogRootPath:", path)
|
||||
})
|
||||
|
||||
t.Run("Test PulsarAddress", func(t *testing.T) {
|
||||
address := Params.PulsarAddress
|
||||
log.Println("PulsarAddress:", address)
|
||||
})
|
||||
|
||||
t.Run("Test insertChannelNames", func(t *testing.T) {
|
||||
names := Params.InsertChannelNames
|
||||
log.Println("InsertChannelNames:", names)
|
||||
})
|
||||
|
||||
t.Run("Test ddChannelNames", func(t *testing.T) {
|
||||
names := Params.DDChannelNames
|
||||
log.Println("DDChannelNames:", names)
|
||||
})
|
||||
|
||||
t.Run("Test SegmentStatisticsChannelName", func(t *testing.T) {
|
||||
name := Params.SegmentStatisticsChannelName
|
||||
log.Println("SegmentStatisticsChannelName:", name)
|
||||
path := Params.SegmentStatisticsChannelName
|
||||
log.Println("SegmentStatisticsChannelName:", path)
|
||||
})
|
||||
|
||||
t.Run("Test timeTickChannelName", func(t *testing.T) {
|
||||
t.Run("Test TimeTickChannelName", func(t *testing.T) {
|
||||
name := Params.TimeTickChannelName
|
||||
log.Println("TimeTickChannelName:", name)
|
||||
})
|
||||
|
|
|
@ -45,7 +45,7 @@ var (
|
|||
errNilSegmentInfo = errors.New("nil segment info")
|
||||
)
|
||||
|
||||
//SaveBinLogMetaTxn saves segment-field2Path, collection-tsPath/ddlPath into kv store in transcation
|
||||
//SaveBinLogMetaTxn saves segment-field2Path, collection-tsPath into kv store in transcation
|
||||
func (s *Server) SaveBinLogMetaTxn(meta map[string]string) error {
|
||||
if s.kvClient == nil {
|
||||
return errNilKvClient
|
||||
|
@ -77,30 +77,6 @@ func (s *Server) prepareField2PathMeta(segID UniqueID, field2Paths *datapb.ID2Pa
|
|||
return result, err
|
||||
}
|
||||
|
||||
// prepareDDLBinlogMeta parses Coll2DdlBinlogPaths & Coll2TsBinlogPaths
|
||||
// into key-value for kv store
|
||||
func (s *Server) prepareDDLBinlogMeta(collID UniqueID, ddlMetas []*datapb.DDLBinlogMeta) (result map[string]string, err error) {
|
||||
if ddlMetas == nil {
|
||||
return nil, errNilID2Paths
|
||||
}
|
||||
|
||||
result = make(map[string]string, len(ddlMetas))
|
||||
|
||||
for _, ddlMeta := range ddlMetas {
|
||||
if ddlMeta == nil {
|
||||
continue
|
||||
}
|
||||
uniqueKey, err := s.genKey(true, collID)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
binlogPathPair := proto.MarshalTextString(ddlMeta)
|
||||
|
||||
result[path.Join(Params.CollectionBinlogSubPath, uniqueKey)] = binlogPathPair
|
||||
}
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// getFieldBinlogMeta querys field binlog meta from kv store
|
||||
func (s *Server) getFieldBinlogMeta(segmentID UniqueID,
|
||||
fieldID UniqueID) (metas []*datapb.SegmentFieldBinlogMeta, err error) {
|
||||
|
@ -151,28 +127,6 @@ func (s *Server) getSegmentBinlogMeta(segmentID UniqueID) (metas []*datapb.Segme
|
|||
return
|
||||
}
|
||||
|
||||
func (s *Server) getDDLBinlogMeta(collID UniqueID) (metas []*datapb.DDLBinlogMeta, err error) {
|
||||
prefix, err := s.genKey(false, collID)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
_, vs, err := s.kvClient.LoadWithPrefix(path.Join(Params.CollectionBinlogSubPath, prefix))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for _, blob := range vs {
|
||||
m := &datapb.DDLBinlogMeta{}
|
||||
if err = proto.UnmarshalText(blob, m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
metas = append(metas, m)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// GetVChanPositions get vchannel latest postitions with provided dml channel names
|
||||
func (s *Server) GetVChanPositions(vchans []vchannel) ([]*datapb.VchannelInfo, error) {
|
||||
if s.kvClient == nil {
|
||||
|
@ -183,7 +137,7 @@ func (s *Server) GetVChanPositions(vchans []vchannel) ([]*datapb.VchannelInfo, e
|
|||
for _, vchan := range vchans {
|
||||
segments := s.meta.GetSegmentsByChannel(vchan.DmlChannel)
|
||||
flushedSegmentIDs := make([]UniqueID, 0)
|
||||
unflushedCheckpoints := make([]*datapb.CheckPoint, 0)
|
||||
unflushed := make([]*datapb.SegmentInfo, 0)
|
||||
var seekPosition *internalpb.MsgPosition
|
||||
var useUnflushedPosition bool
|
||||
for _, s := range segments {
|
||||
|
@ -199,12 +153,7 @@ func (s *Server) GetVChanPositions(vchans []vchannel) ([]*datapb.VchannelInfo, e
|
|||
continue
|
||||
}
|
||||
|
||||
cp := &datapb.CheckPoint{
|
||||
SegmentID: s.ID,
|
||||
Position: s.DmlPosition,
|
||||
NumOfRows: s.NumOfRows,
|
||||
}
|
||||
unflushedCheckpoints = append(unflushedCheckpoints, cp)
|
||||
unflushed = append(unflushed, s)
|
||||
|
||||
if seekPosition == nil || !useUnflushedPosition || s.DmlPosition.Timestamp < seekPosition.Timestamp {
|
||||
useUnflushedPosition = true
|
||||
|
@ -213,11 +162,11 @@ func (s *Server) GetVChanPositions(vchans []vchannel) ([]*datapb.VchannelInfo, e
|
|||
}
|
||||
|
||||
pairs = append(pairs, &datapb.VchannelInfo{
|
||||
CollectionID: vchan.CollectionID,
|
||||
ChannelName: vchan.DmlChannel,
|
||||
SeekPosition: seekPosition,
|
||||
CheckPoints: unflushedCheckpoints,
|
||||
FlushedSegments: flushedSegmentIDs,
|
||||
CollectionID: vchan.CollectionID,
|
||||
ChannelName: vchan.DmlChannel,
|
||||
SeekPosition: seekPosition,
|
||||
UnflushedSegments: unflushed,
|
||||
FlushedSegments: flushedSegmentIDs,
|
||||
})
|
||||
}
|
||||
return pairs, nil
|
||||
|
|
|
@ -11,6 +11,7 @@
|
|||
package dataservice
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
|
@ -30,7 +31,7 @@ type cluster struct {
|
|||
startupPolicy clusterStartupPolicy
|
||||
registerPolicy dataNodeRegisterPolicy
|
||||
unregisterPolicy dataNodeUnregisterPolicy
|
||||
assginPolicy channelAssignPolicy
|
||||
assignPolicy channelAssignPolicy
|
||||
}
|
||||
|
||||
type clusterOption struct {
|
||||
|
@ -57,7 +58,7 @@ func withUnregistorPolicy(p dataNodeUnregisterPolicy) clusterOption {
|
|||
|
||||
func withAssignPolicy(p channelAssignPolicy) clusterOption {
|
||||
return clusterOption{
|
||||
apply: func(c *cluster) { c.assginPolicy = p },
|
||||
apply: func(c *cluster) { c.assignPolicy = p },
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -66,15 +67,15 @@ func defaultStartupPolicy() clusterStartupPolicy {
|
|||
}
|
||||
|
||||
func defaultRegisterPolicy() dataNodeRegisterPolicy {
|
||||
return newDoNothingRegisterPolicy()
|
||||
return newEmptyRegisterPolicy()
|
||||
}
|
||||
|
||||
func defaultUnregisterPolicy() dataNodeUnregisterPolicy {
|
||||
return newDoNothingUnregisterPolicy()
|
||||
return newEmptyUnregisterPolicy()
|
||||
}
|
||||
|
||||
func defaultAssignPolicy() channelAssignPolicy {
|
||||
return newAssignAllPolicy()
|
||||
return newBalancedAssignPolicy()
|
||||
}
|
||||
|
||||
func newCluster(ctx context.Context, dataManager *clusterNodeManager, sessionManager sessionManager, posProvider positionProvider, opts ...clusterOption) *cluster {
|
||||
|
@ -86,7 +87,7 @@ func newCluster(ctx context.Context, dataManager *clusterNodeManager, sessionMan
|
|||
startupPolicy: defaultStartupPolicy(),
|
||||
registerPolicy: defaultRegisterPolicy(),
|
||||
unregisterPolicy: defaultUnregisterPolicy(),
|
||||
assginPolicy: defaultAssignPolicy(),
|
||||
assignPolicy: defaultAssignPolicy(),
|
||||
}
|
||||
for _, opt := range opts {
|
||||
opt.apply(c)
|
||||
|
@ -107,15 +108,27 @@ func (c *cluster) startup(dataNodes []*datapb.DataNodeInfo) error {
|
|||
|
||||
func (c *cluster) watch(nodes []*datapb.DataNodeInfo) []*datapb.DataNodeInfo {
|
||||
for _, n := range nodes {
|
||||
uncompletes := make([]vchannel, 0, len(nodes))
|
||||
logMsg := fmt.Sprintf("Begin to watch channels for node %s:", n.Address)
|
||||
uncompletes := make([]vchannel, 0, len(n.Channels))
|
||||
for _, ch := range n.Channels {
|
||||
if ch.State == datapb.ChannelWatchState_Uncomplete {
|
||||
if len(uncompletes) == 0 {
|
||||
logMsg += ch.Name
|
||||
} else {
|
||||
logMsg += "," + ch.Name
|
||||
}
|
||||
uncompletes = append(uncompletes, vchannel{
|
||||
CollectionID: ch.CollectionID,
|
||||
DmlChannel: ch.Name,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
if len(uncompletes) == 0 {
|
||||
continue
|
||||
}
|
||||
log.Debug(logMsg)
|
||||
|
||||
vchanInfos, err := c.posProvider.GetVChanPositions(uncompletes)
|
||||
if err != nil {
|
||||
log.Warn("get vchannel position failed", zap.Error(err))
|
||||
|
@ -177,14 +190,13 @@ func (c *cluster) watchIfNeeded(channel string, collectionID UniqueID) {
|
|||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
cNodes := c.dataManager.getDataNodes(true)
|
||||
rets := c.assginPolicy.apply(cNodes, channel, collectionID)
|
||||
rets := c.assignPolicy.apply(cNodes, channel, collectionID)
|
||||
c.dataManager.updateDataNodes(rets)
|
||||
rets = c.watch(rets)
|
||||
c.dataManager.updateDataNodes(rets)
|
||||
}
|
||||
|
||||
func (c *cluster) flush(segments []*datapb.SegmentInfo) {
|
||||
log.Debug("prepare to flush", zap.Any("segments", segments))
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@ type sessionManager interface {
|
|||
}
|
||||
|
||||
type clusterSessionManager struct {
|
||||
mu sync.RWMutex
|
||||
sync.RWMutex
|
||||
sessions map[string]types.DataNode
|
||||
dataClientCreator func(addr string) (types.DataNode, error)
|
||||
}
|
||||
|
@ -37,36 +37,49 @@ func newClusterSessionManager(dataClientCreator func(addr string) (types.DataNod
|
|||
}
|
||||
}
|
||||
|
||||
func (m *clusterSessionManager) createSession(addr string) error {
|
||||
// lock acquired
|
||||
func (m *clusterSessionManager) createSession(addr string) (types.DataNode, error) {
|
||||
cli, err := m.dataClientCreator(addr)
|
||||
if err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
if err := cli.Init(); err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
if err := cli.Start(); err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
m.sessions[addr] = cli
|
||||
return nil
|
||||
return cli, nil
|
||||
}
|
||||
|
||||
// entry function
|
||||
func (m *clusterSessionManager) getOrCreateSession(addr string) (types.DataNode, error) {
|
||||
if !m.hasSession(addr) {
|
||||
if err := m.createSession(addr); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
m.RLock()
|
||||
dn, has := m.sessions[addr]
|
||||
m.RUnlock()
|
||||
if has {
|
||||
return dn, nil
|
||||
}
|
||||
return m.sessions[addr], nil
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
dn, has = m.sessions[addr]
|
||||
if has {
|
||||
return dn, nil
|
||||
}
|
||||
dn, err := m.createSession(addr)
|
||||
return dn, err
|
||||
}
|
||||
|
||||
func (m *clusterSessionManager) hasSession(addr string) bool {
|
||||
_, ok := m.sessions[addr]
|
||||
return ok
|
||||
}
|
||||
// // lock acquired
|
||||
// func (m *clusterSessionManager) hasSession(addr string) bool {
|
||||
// _, ok := m.sessions[addr]
|
||||
// return ok
|
||||
// }
|
||||
|
||||
func (m *clusterSessionManager) releaseSession(addr string) {
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
cli, ok := m.sessions[addr]
|
||||
if !ok {
|
||||
return
|
||||
|
@ -76,7 +89,10 @@ func (m *clusterSessionManager) releaseSession(addr string) {
|
|||
}
|
||||
|
||||
func (m *clusterSessionManager) release() {
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
for _, cli := range m.sessions {
|
||||
_ = cli.Stop()
|
||||
}
|
||||
m.sessions = map[string]types.DataNode{}
|
||||
}
|
||||
|
|
|
@ -39,7 +39,7 @@ func TestClusterCreate(t *testing.T) {
|
|||
|
||||
func TestRegister(t *testing.T) {
|
||||
cPolicy := newMockStartupPolicy()
|
||||
registerPolicy := newDoNothingRegisterPolicy()
|
||||
registerPolicy := newEmptyRegisterPolicy()
|
||||
cluster := createCluster(t, nil, withStartupPolicy(cPolicy), withRegisterPolicy(registerPolicy))
|
||||
addr := "localhost:8080"
|
||||
|
||||
|
@ -57,7 +57,7 @@ func TestRegister(t *testing.T) {
|
|||
|
||||
func TestUnregister(t *testing.T) {
|
||||
cPolicy := newMockStartupPolicy()
|
||||
unregisterPolicy := newDoNothingUnregisterPolicy()
|
||||
unregisterPolicy := newEmptyUnregisterPolicy()
|
||||
cluster := createCluster(t, nil, withStartupPolicy(cPolicy), withUnregistorPolicy(unregisterPolicy))
|
||||
addr := "localhost:8080"
|
||||
nodes := []*datapb.DataNodeInfo{
|
||||
|
|
|
@ -33,10 +33,10 @@ func (dp dummyPosProvider) GetVChanPositions(vchans []vchannel) ([]*datapb.Vchan
|
|||
pairs := make([]*datapb.VchannelInfo, len(vchans))
|
||||
for _, vchan := range vchans {
|
||||
pairs = append(pairs, &datapb.VchannelInfo{
|
||||
CollectionID: vchan.CollectionID,
|
||||
ChannelName: vchan.DmlChannel,
|
||||
FlushedSegments: []int64{},
|
||||
CheckPoints: []*datapb.CheckPoint{},
|
||||
CollectionID: vchan.CollectionID,
|
||||
ChannelName: vchan.DmlChannel,
|
||||
FlushedSegments: []int64{},
|
||||
UnflushedSegments: []*datapb.SegmentInfo{},
|
||||
})
|
||||
}
|
||||
return pairs, nil
|
||||
|
|
|
@ -20,6 +20,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
)
|
||||
|
||||
// flushMonitor check segments / channels meet the provided flush policy
|
||||
type flushMonitor struct {
|
||||
meta *meta
|
||||
segmentPolicy SegmentFlushPolicy
|
||||
|
@ -48,7 +49,7 @@ func defaultFlushMonitor(meta *meta) flushMonitor {
|
|||
}
|
||||
}
|
||||
|
||||
// CheckSegments check segemnt sizes
|
||||
// CheckSegments check segments meet flush policy, returns segment id needs to flush
|
||||
func (f flushMonitor) CheckSegments(segments []*datapb.SegmentInfo) []UniqueID {
|
||||
if f.segmentPolicy == nil {
|
||||
return []UniqueID{}
|
||||
|
@ -62,7 +63,7 @@ func (f flushMonitor) CheckSegments(segments []*datapb.SegmentInfo) []UniqueID {
|
|||
return result
|
||||
}
|
||||
|
||||
// CheckChannels check channels changed
|
||||
// CheckChannels check channels changed, apply `ChannelPolicy`
|
||||
func (f flushMonitor) CheckChannels(channels []string, latest *internalpb.MsgPosition) []UniqueID {
|
||||
segHits := make(map[UniqueID]struct{})
|
||||
for _, channel := range channels {
|
||||
|
@ -94,6 +95,7 @@ func (f flushMonitor) CheckChannels(channels []string, latest *internalpb.MsgPos
|
|||
return result
|
||||
}
|
||||
|
||||
// deprecated
|
||||
func estSegmentSizePolicy(rowSize, limit int64) SegmentFlushPolicy {
|
||||
return func(seg *datapb.SegmentInfo) bool {
|
||||
if seg == nil {
|
||||
|
@ -106,6 +108,9 @@ func estSegmentSizePolicy(rowSize, limit int64) SegmentFlushPolicy {
|
|||
}
|
||||
}
|
||||
|
||||
// channelSizeEpochPolicy policy check channel sizes and segment life time
|
||||
// segmentMax is the max number of segment allowed in the channel
|
||||
// epochDuration is the max live time segment has
|
||||
func channelSizeEpochPolicy(segmentMax int, epochDuration uint64) ChannelFlushPolicy {
|
||||
return func(channel string, segments []*datapb.SegmentInfo, latest *internalpb.MsgPosition) []UniqueID {
|
||||
if len(segments) < segmentMax && latest == nil {
|
||||
|
@ -120,7 +125,7 @@ func channelSizeEpochPolicy(segmentMax int, epochDuration uint64) ChannelFlushPo
|
|||
continue
|
||||
}
|
||||
if latest != nil {
|
||||
if segment.DmlPosition == nil || latest.Timestamp-segment.DmlPosition.Timestamp > uint64(time.Hour) {
|
||||
if segment.DmlPosition == nil || latest.Timestamp-segment.DmlPosition.Timestamp > epochDuration {
|
||||
result = append(result, segment.ID)
|
||||
continue
|
||||
}
|
||||
|
@ -131,6 +136,7 @@ func channelSizeEpochPolicy(segmentMax int, epochDuration uint64) ChannelFlushPo
|
|||
}
|
||||
}
|
||||
|
||||
// sortSegmentsByDmlPos sorts input segments in ascending order by `DmlPosition.Timestamp`, nil value is less than 0
|
||||
func sortSegmentsByDmlPos(segments []*datapb.SegmentInfo) {
|
||||
sort.Slice(segments, func(i, j int) bool {
|
||||
if segments[i].DmlPosition == nil {
|
||||
|
|
|
@ -39,6 +39,7 @@ func (s *Server) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResp
|
|||
}
|
||||
|
||||
func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*commonpb.Status, error) {
|
||||
log.Debug("Receive flush request", zap.Int64("dbID", req.GetDbID()), zap.Int64("collectionID", req.GetCollectionID()))
|
||||
resp := &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
}
|
||||
|
@ -76,6 +77,12 @@ func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentI
|
|||
}
|
||||
|
||||
for _, r := range req.SegmentIDRequests {
|
||||
log.Debug("Handle assign segment request",
|
||||
zap.Int64("collectionID", r.GetCollectionID()),
|
||||
zap.Int64("partitionID", r.GetPartitionID()),
|
||||
zap.String("channelName", r.GetChannelName()),
|
||||
zap.Uint32("count", r.GetCount()))
|
||||
|
||||
if !s.meta.HasCollection(r.CollectionID) {
|
||||
if err := s.loadCollectionFromMaster(ctx, r.CollectionID); err != nil {
|
||||
errMsg := fmt.Sprintf("can not load collection %d", r.CollectionID)
|
||||
|
@ -103,6 +110,9 @@ func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentI
|
|||
continue
|
||||
}
|
||||
|
||||
log.Debug("Assign segment success", zap.Int64("segmentID", segmentID),
|
||||
zap.Uint64("expireTs", expireTs))
|
||||
|
||||
result := &datapb.SegmentIDAssignment{
|
||||
SegID: segmentID,
|
||||
ChannelName: r.ChannelName,
|
||||
|
@ -184,7 +194,7 @@ func (s *Server) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsert
|
|||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
},
|
||||
}
|
||||
p := path.Join(Params.SegmentFlushMetaPath, strconv.FormatInt(req.SegmentID, 10))
|
||||
p := path.Join(Params.SegmentBinlogSubPath, strconv.FormatInt(req.SegmentID, 10)) + "/" // prefix/id/ instead of prefix/id
|
||||
_, values, err := s.kvClient.LoadWithPrefix(p)
|
||||
if err != nil {
|
||||
resp.Status.Reason = err.Error()
|
||||
|
@ -295,6 +305,9 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
|
|||
resp.Reason = "server is closed"
|
||||
return resp, nil
|
||||
}
|
||||
log.Debug("Receive SaveBinlogPaths request",
|
||||
zap.Int64("collectionID", req.GetCollectionID()),
|
||||
zap.Int64("segmentID", req.GetSegmentID()))
|
||||
|
||||
// check segment id & collection id matched
|
||||
_, err := s.meta.GetCollection(req.GetCollectionID())
|
||||
|
@ -306,7 +319,7 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
|
|||
|
||||
binlogs, err := s.prepareBinlog(req)
|
||||
if err != nil {
|
||||
log.Error("prepare binlog meta failed", zap.Error(err))
|
||||
log.Error("Prepare binlog meta failed", zap.Error(err))
|
||||
resp.Reason = err.Error()
|
||||
return resp, nil
|
||||
}
|
||||
|
@ -314,10 +327,13 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
|
|||
// set segment to SegmentState_Flushing and save binlogs and checkpoints
|
||||
err = s.meta.SaveBinlogAndCheckPoints(req.SegmentID, req.Flushed, binlogs, req.CheckPoints)
|
||||
if err != nil {
|
||||
log.Error("Save binlog and checkpoints failed",
|
||||
zap.Int64("segmentID", req.GetSegmentID()),
|
||||
zap.Error(err))
|
||||
resp.Reason = err.Error()
|
||||
return resp, nil
|
||||
}
|
||||
log.Debug("flush segment with meta", zap.Int64("id", req.SegmentID),
|
||||
log.Debug("Flush segment with meta", zap.Int64("id", req.SegmentID),
|
||||
zap.Any("meta", binlogs))
|
||||
|
||||
if req.Flushed {
|
||||
|
@ -356,7 +372,8 @@ func (s *Server) GetComponentStates(ctx context.Context) (*internalpb.ComponentS
|
|||
func (s *Server) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error) {
|
||||
collectionID := req.GetCollectionID()
|
||||
partitionID := req.GetPartitionID()
|
||||
log.Info("Receive get recovery info request", zap.Int64("collectionID", collectionID),
|
||||
log.Info("Receive get recovery info request",
|
||||
zap.Int64("collectionID", collectionID),
|
||||
zap.Int64("partitionID", partitionID))
|
||||
resp := &datapb.GetRecoveryInfoResponse{
|
||||
Status: &commonpb.Status{
|
||||
|
@ -422,7 +439,8 @@ func (s *Server) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInf
|
|||
|
||||
channelInfos, err := s.GetVChanPositions(vchans)
|
||||
if err != nil {
|
||||
log.Error("Get channel positions failed", zap.Strings("channels", channels),
|
||||
log.Error("Get channel positions failed",
|
||||
zap.Strings("channels", channels),
|
||||
zap.Error(err))
|
||||
resp.Status.Reason = err.Error()
|
||||
return resp, nil
|
||||
|
|
|
@ -58,8 +58,7 @@ type ParamTable struct {
|
|||
K2SChannelNames []string
|
||||
ProxyTimeTickChannelName string
|
||||
|
||||
SegmentFlushMetaPath string
|
||||
Log log.Config
|
||||
Log log.Config
|
||||
}
|
||||
|
||||
var Params ParamTable
|
||||
|
@ -96,7 +95,6 @@ func (p *ParamTable) Init() {
|
|||
p.initSegmentInfoChannelName()
|
||||
p.initDataServiceSubscriptionName()
|
||||
p.initK2SChannelNames()
|
||||
p.initSegmentFlushMetaPath()
|
||||
p.initLogCfg()
|
||||
p.initProxyServiceTimeTickChannelName()
|
||||
|
||||
|
@ -245,14 +243,6 @@ func (p *ParamTable) initK2SChannelNames() {
|
|||
p.K2SChannelNames = ret
|
||||
}
|
||||
|
||||
func (p *ParamTable) initSegmentFlushMetaPath() {
|
||||
subPath, err := p.Load("etcd.segFlushMetaSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.SegmentFlushMetaPath = subPath
|
||||
}
|
||||
|
||||
func (p *ParamTable) initLogCfg() {
|
||||
p.Log = log.Config{}
|
||||
format, err := p.Load("log.format")
|
||||
|
|
|
@ -12,6 +12,7 @@ package dataservice
|
|||
|
||||
import (
|
||||
"crypto/rand"
|
||||
"math"
|
||||
"math/big"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
|
@ -53,14 +54,14 @@ type dataNodeRegisterPolicy interface {
|
|||
apply(cluster map[string]*datapb.DataNodeInfo, session *datapb.DataNodeInfo) []*datapb.DataNodeInfo
|
||||
}
|
||||
|
||||
type doNothingRegisterPolicy struct {
|
||||
type emptyRegisterPolicy struct {
|
||||
}
|
||||
|
||||
func newDoNothingRegisterPolicy() dataNodeRegisterPolicy {
|
||||
return &doNothingRegisterPolicy{}
|
||||
func newEmptyRegisterPolicy() dataNodeRegisterPolicy {
|
||||
return &emptyRegisterPolicy{}
|
||||
}
|
||||
|
||||
func (p *doNothingRegisterPolicy) apply(cluster map[string]*datapb.DataNodeInfo, session *datapb.DataNodeInfo) []*datapb.DataNodeInfo {
|
||||
func (p *emptyRegisterPolicy) apply(cluster map[string]*datapb.DataNodeInfo, session *datapb.DataNodeInfo) []*datapb.DataNodeInfo {
|
||||
return []*datapb.DataNodeInfo{session}
|
||||
}
|
||||
|
||||
|
@ -69,14 +70,14 @@ type dataNodeUnregisterPolicy interface {
|
|||
apply(cluster map[string]*datapb.DataNodeInfo, session *datapb.DataNodeInfo) []*datapb.DataNodeInfo
|
||||
}
|
||||
|
||||
type doNothingUnregisterPolicy struct {
|
||||
type emptyUnregisterPolicy struct {
|
||||
}
|
||||
|
||||
func newDoNothingUnregisterPolicy() dataNodeUnregisterPolicy {
|
||||
return &doNothingUnregisterPolicy{}
|
||||
func newEmptyUnregisterPolicy() dataNodeUnregisterPolicy {
|
||||
return &emptyUnregisterPolicy{}
|
||||
}
|
||||
|
||||
func (p *doNothingUnregisterPolicy) apply(cluster map[string]*datapb.DataNodeInfo, session *datapb.DataNodeInfo) []*datapb.DataNodeInfo {
|
||||
func (p *emptyUnregisterPolicy) apply(cluster map[string]*datapb.DataNodeInfo, session *datapb.DataNodeInfo) []*datapb.DataNodeInfo {
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -158,3 +159,39 @@ func (p *assignAllPolicy) apply(cluster map[string]*datapb.DataNodeInfo, channel
|
|||
|
||||
return ret
|
||||
}
|
||||
|
||||
type balancedAssignPolicy struct{}
|
||||
|
||||
func newBalancedAssignPolicy() channelAssignPolicy {
|
||||
return &balancedAssignPolicy{}
|
||||
}
|
||||
|
||||
func (p *balancedAssignPolicy) apply(cluster map[string]*datapb.DataNodeInfo, channel string, collectionID UniqueID) []*datapb.DataNodeInfo {
|
||||
if len(cluster) == 0 {
|
||||
return []*datapb.DataNodeInfo{}
|
||||
}
|
||||
// filter existed channel
|
||||
for _, node := range cluster {
|
||||
for _, c := range node.GetChannels() {
|
||||
if c.GetName() == channel && c.GetCollectionID() == collectionID {
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
target, min := "", math.MaxInt32
|
||||
for k, v := range cluster {
|
||||
if len(v.GetChannels()) < min {
|
||||
target = k
|
||||
min = len(v.GetChannels())
|
||||
}
|
||||
}
|
||||
|
||||
ret := make([]*datapb.DataNodeInfo, 0)
|
||||
cluster[target].Channels = append(cluster[target].Channels, &datapb.ChannelStatus{
|
||||
Name: channel,
|
||||
State: datapb.ChannelWatchState_Uncomplete,
|
||||
CollectionID: collectionID,
|
||||
})
|
||||
ret = append(ret, cluster[target])
|
||||
return ret
|
||||
}
|
||||
|
|
|
@ -1,6 +1,19 @@
|
|||
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
// or implied. See the License for the specific language governing permissions and limitations under the License.
|
||||
|
||||
package dataservice
|
||||
|
||||
import (
|
||||
"sort"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
)
|
||||
|
@ -46,6 +59,51 @@ type sealPolicy interface {
|
|||
apply(maxCount, writtenCount, allocatedCount int64) bool
|
||||
}
|
||||
|
||||
// segmentSealPolicy seal policy applies to segment
|
||||
type segmentSealPolicy func(*segmentStatus, Timestamp) bool
|
||||
|
||||
// channelSealPolicy seal policy applies to channel
|
||||
type channelSealPolicy func(string, []*segmentStatus, Timestamp) []*segmentStatus
|
||||
|
||||
// getSegmentCapacityPolicy get segmentSealPolicy with segment size factor policy
|
||||
func getSegmentCapacityPolicy(sizeFactor float64) segmentSealPolicy {
|
||||
return func(status *segmentStatus, ts Timestamp) bool {
|
||||
var allocSize int64
|
||||
for _, allocation := range status.allocations {
|
||||
allocSize += allocation.rowNums
|
||||
}
|
||||
// max, written, allocated := status.total, status.currentRows, allocSize
|
||||
// float64(writtenCount) >= Params.SegmentSizeFactor*float64(maxCount)
|
||||
return float64(status.currentRows) >= sizeFactor*float64(status.total)
|
||||
}
|
||||
}
|
||||
|
||||
// getLastExpiresLifetimePolicy get segmentSealPolicy with lifetime limit compares ts - segment.lastExpireTime
|
||||
func getLastExpiresLifetimePolicy(lifetime uint64) segmentSealPolicy {
|
||||
return func(status *segmentStatus, ts Timestamp) bool {
|
||||
return (ts - status.lastExpireTime) > lifetime
|
||||
}
|
||||
}
|
||||
|
||||
// getChannelCapacityPolicy get channelSealPolicy with channel segment capacity policy
|
||||
func getChannelOpenSegCapacityPolicy(limit int) channelSealPolicy {
|
||||
return func(channel string, segs []*segmentStatus, ts Timestamp) []*segmentStatus {
|
||||
if len(segs) <= limit {
|
||||
return []*segmentStatus{}
|
||||
}
|
||||
sortSegStatusByLastExpires(segs)
|
||||
offLen := len(segs) - limit
|
||||
return segs[0:offLen]
|
||||
}
|
||||
}
|
||||
|
||||
// sortSegStatusByLastExpires sort segmentStatus with lastExpireTime ascending order
|
||||
func sortSegStatusByLastExpires(segs []*segmentStatus) {
|
||||
sort.Slice(segs, func(i, j int) bool {
|
||||
return segs[i].lastExpireTime < segs[j].lastExpireTime
|
||||
})
|
||||
}
|
||||
|
||||
type sealPolicyV1 struct {
|
||||
}
|
||||
|
||||
|
|
|
@ -7,6 +7,7 @@
|
|||
// Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
// or implied. See the License for the specific language governing permissions and limitations under the License.
|
||||
|
||||
package dataservice
|
||||
|
||||
import (
|
||||
|
@ -80,8 +81,10 @@ type SegmentManager struct {
|
|||
|
||||
estimatePolicy calUpperLimitPolicy
|
||||
allocPolicy allocatePolicy
|
||||
sealPolicy sealPolicy
|
||||
flushPolicy flushPolicy
|
||||
// sealPolicy sealPolicy
|
||||
segmentSealPolicies []segmentSealPolicy
|
||||
channelSealPolicies []channelSealPolicy
|
||||
flushPolicy flushPolicy
|
||||
}
|
||||
|
||||
type allocHelper struct {
|
||||
|
@ -116,9 +119,27 @@ func withAllocPolicy(policy allocatePolicy) allocOption {
|
|||
}
|
||||
}
|
||||
|
||||
func withSealPolicy(policy sealPolicy) allocOption {
|
||||
// func withSealPolicy(policy sealPolicy) allocOption {
|
||||
// return allocOption{
|
||||
// apply: func(manager *SegmentManager) { manager.sealPolicy = policy },
|
||||
// }
|
||||
// }
|
||||
|
||||
func withSegmentSealPolices(policies ...segmentSealPolicy) allocOption {
|
||||
return allocOption{
|
||||
apply: func(manager *SegmentManager) { manager.sealPolicy = policy },
|
||||
apply: func(manager *SegmentManager) {
|
||||
// do override instead of append, to override default options
|
||||
manager.segmentSealPolicies = policies
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func withChannelSealPolices(policies ...channelSealPolicy) allocOption {
|
||||
return allocOption{
|
||||
apply: func(manager *SegmentManager) {
|
||||
// do override instead of append, to override default options
|
||||
manager.channelSealPolicies = policies
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -140,6 +161,10 @@ func defaultSealPolicy() sealPolicy {
|
|||
return newSealPolicyV1()
|
||||
}
|
||||
|
||||
func defaultSegmentSealPolicy() segmentSealPolicy {
|
||||
return getSegmentCapacityPolicy(Params.SegmentSizeFactor)
|
||||
}
|
||||
|
||||
func defaultFlushPolicy() flushPolicy {
|
||||
return newFlushPolicyV1()
|
||||
}
|
||||
|
@ -151,10 +176,11 @@ func newSegmentManager(meta *meta, allocator allocator, opts ...allocOption) *Se
|
|||
helper: defaultAllocHelper(),
|
||||
stats: make(map[UniqueID]*segmentStatus),
|
||||
|
||||
estimatePolicy: defaultCalUpperLimitPolicy(),
|
||||
allocPolicy: defaultAlocatePolicy(),
|
||||
sealPolicy: defaultSealPolicy(),
|
||||
flushPolicy: defaultFlushPolicy(),
|
||||
estimatePolicy: defaultCalUpperLimitPolicy(),
|
||||
allocPolicy: defaultAlocatePolicy(),
|
||||
segmentSealPolicies: []segmentSealPolicy{defaultSegmentSealPolicy()}, // default only segment size policy
|
||||
channelSealPolicies: []channelSealPolicy{}, // no default channel seal policy
|
||||
flushPolicy: defaultFlushPolicy(),
|
||||
}
|
||||
for _, opt := range opts {
|
||||
opt.apply(manager)
|
||||
|
@ -164,9 +190,10 @@ func newSegmentManager(meta *meta, allocator allocator, opts ...allocOption) *Se
|
|||
}
|
||||
|
||||
func (s *SegmentManager) loadSegmentsFromMeta() {
|
||||
// load unflushed segments from meta
|
||||
segments := s.meta.GetUnFlushedSegments()
|
||||
ids := make([]UniqueID, 0, len(segments))
|
||||
for _, seg := range segments {
|
||||
ids = append(ids, seg.ID)
|
||||
stat := &segmentStatus{
|
||||
id: seg.ID,
|
||||
collectionID: seg.CollectionID,
|
||||
|
@ -179,6 +206,7 @@ func (s *SegmentManager) loadSegmentsFromMeta() {
|
|||
}
|
||||
s.stats[seg.ID] = stat
|
||||
}
|
||||
log.Debug("Restore segment allocation", zap.Int64s("segments", ids))
|
||||
}
|
||||
func (s *SegmentManager) AllocSegment(ctx context.Context, collectionID UniqueID,
|
||||
partitionID UniqueID, channelName string, requestRows int64) (segID UniqueID, retCount int64, expireTime Timestamp, err error) {
|
||||
|
@ -305,7 +333,8 @@ func (s *SegmentManager) openNewSegment(ctx context.Context, collectionID Unique
|
|||
log.Debug("dataservice: estimateTotalRows: ",
|
||||
zap.Int64("CollectionID", segmentInfo.CollectionID),
|
||||
zap.Int64("SegmentID", segmentInfo.ID),
|
||||
zap.Int("Rows", totalRows))
|
||||
zap.Int("Rows", totalRows),
|
||||
zap.String("channel", segmentInfo.InsertChannel))
|
||||
|
||||
s.helper.afterCreateSegment(segmentInfo)
|
||||
|
||||
|
@ -334,15 +363,13 @@ func (s *SegmentManager) SealAllSegments(ctx context.Context, collectionID Uniqu
|
|||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
for _, status := range s.stats {
|
||||
if status.collectionID == collectionID {
|
||||
if status.sealed {
|
||||
continue
|
||||
}
|
||||
if err := s.meta.SealSegment(status.id); err != nil {
|
||||
return err
|
||||
}
|
||||
status.sealed = true
|
||||
if status.sealed || status.collectionID != collectionID {
|
||||
continue
|
||||
}
|
||||
if err := s.meta.SealSegment(status.id); err != nil {
|
||||
return err
|
||||
}
|
||||
status.sealed = true
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -353,7 +380,7 @@ func (s *SegmentManager) GetFlushableSegments(ctx context.Context, channel strin
|
|||
defer s.mu.Unlock()
|
||||
sp, _ := trace.StartSpanFromContext(ctx)
|
||||
defer sp.Finish()
|
||||
if err := s.tryToSealSegment(); err != nil {
|
||||
if err := s.tryToSealSegment(t); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -380,35 +407,51 @@ func (s *SegmentManager) UpdateSegmentStats(stat *internalpb.SegmentStatisticsUp
|
|||
segment.currentRows = stat.NumRows
|
||||
}
|
||||
|
||||
func (s *SegmentManager) tryToSealSegment() error {
|
||||
// tryToSealSegment applies segment & channel seal policies
|
||||
func (s *SegmentManager) tryToSealSegment(ts Timestamp) error {
|
||||
channelInfo := make(map[string][]*segmentStatus)
|
||||
for _, segStatus := range s.stats {
|
||||
channelInfo[segStatus.insertChannel] = append(channelInfo[segStatus.insertChannel], segStatus)
|
||||
if segStatus.sealed {
|
||||
continue
|
||||
}
|
||||
sealed, err := s.shouldSeal(segStatus)
|
||||
if err != nil {
|
||||
return err
|
||||
// change shouldSeal to segment seal policy logic
|
||||
for _, policy := range s.segmentSealPolicies {
|
||||
if policy(segStatus, ts) {
|
||||
if err := s.meta.SealSegment(segStatus.id); err != nil {
|
||||
return err
|
||||
}
|
||||
segStatus.sealed = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !sealed {
|
||||
continue
|
||||
}
|
||||
if err := s.meta.SealSegment(segStatus.id); err != nil {
|
||||
return err
|
||||
}
|
||||
segStatus.sealed = true
|
||||
}
|
||||
|
||||
}
|
||||
for channel, segs := range channelInfo {
|
||||
for _, policy := range s.channelSealPolicies {
|
||||
vs := policy(channel, segs, ts)
|
||||
for _, seg := range vs {
|
||||
if seg.sealed {
|
||||
continue
|
||||
}
|
||||
if err := s.meta.SealSegment(seg.id); err != nil {
|
||||
return err
|
||||
}
|
||||
seg.sealed = true
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *SegmentManager) shouldSeal(segStatus *segmentStatus) (bool, error) {
|
||||
var allocSize int64
|
||||
for _, allocation := range segStatus.allocations {
|
||||
allocSize += allocation.rowNums
|
||||
}
|
||||
ret := s.sealPolicy.apply(segStatus.total, segStatus.currentRows, allocSize)
|
||||
return ret, nil
|
||||
}
|
||||
// func (s *SegmentManager) shouldSeal(segStatus *segmentStatus) (bool, error) {
|
||||
// var allocSize int64
|
||||
// for _, allocation := range segStatus.allocations {
|
||||
// allocSize += allocation.rowNums
|
||||
// }
|
||||
// ret := s.sealPolicy.apply(segStatus.total, segStatus.currentRows, allocSize)
|
||||
// return ret, nil
|
||||
// }
|
||||
|
||||
// only for test
|
||||
func (s *SegmentManager) SealSegment(ctx context.Context, segmentID UniqueID) error {
|
||||
|
|
|
@ -304,7 +304,7 @@ func (s *Server) startDataNodeTtLoop(ctx context.Context) {
|
|||
}
|
||||
for _, msg := range msgPack.Msgs {
|
||||
if msg.Type() != commonpb.MsgType_DataNodeTt {
|
||||
log.Warn("receive unexpected msg type from tt channel",
|
||||
log.Warn("Receive unexpected msg type from tt channel",
|
||||
zap.Stringer("msgType", msg.Type()))
|
||||
continue
|
||||
}
|
||||
|
@ -312,13 +312,18 @@ func (s *Server) startDataNodeTtLoop(ctx context.Context) {
|
|||
|
||||
ch := ttMsg.ChannelName
|
||||
ts := ttMsg.Timestamp
|
||||
log.Debug("Receive datanode timetick msg", zap.String("channel", ch),
|
||||
zap.Any("ts", ts))
|
||||
segments, err := s.segmentManager.GetFlushableSegments(ctx, ch, ts)
|
||||
if err != nil {
|
||||
log.Warn("get flushable segments failed", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
log.Debug("flushable segments", zap.Any("segments", segments))
|
||||
if len(segments) == 0 {
|
||||
continue
|
||||
}
|
||||
log.Debug("Flush segments", zap.Int64s("segmentIDs", segments))
|
||||
segmentInfos := make([]*datapb.SegmentInfo, 0, len(segments))
|
||||
for _, id := range segments {
|
||||
sInfo, err := s.meta.GetSegment(id)
|
||||
|
@ -329,8 +334,9 @@ func (s *Server) startDataNodeTtLoop(ctx context.Context) {
|
|||
}
|
||||
segmentInfos = append(segmentInfos, sInfo)
|
||||
}
|
||||
|
||||
s.cluster.flush(segmentInfos)
|
||||
if len(segmentInfos) > 0 {
|
||||
s.cluster.flush(segmentInfos)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -351,8 +357,14 @@ func (s *Server) startWatchService(ctx context.Context) {
|
|||
}
|
||||
switch event.EventType {
|
||||
case sessionutil.SessionAddEvent:
|
||||
log.Info("Received datanode register",
|
||||
zap.String("address", datanode.Address),
|
||||
zap.Int64("serverID", datanode.Version))
|
||||
s.cluster.register(datanode)
|
||||
case sessionutil.SessionDelEvent:
|
||||
log.Info("Received datanode unregister",
|
||||
zap.String("address", datanode.Address),
|
||||
zap.Int64("serverID", datanode.Version))
|
||||
s.cluster.unregister(datanode)
|
||||
default:
|
||||
log.Warn("receive unknown service event type",
|
||||
|
@ -528,13 +540,6 @@ func (s *Server) prepareBinlog(req *datapb.SaveBinlogPathsRequest) (map[string]s
|
|||
}
|
||||
}
|
||||
|
||||
ddlMeta, err := s.prepareDDLBinlogMeta(req.CollectionID, req.GetDdlBinlogPaths())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for k, v := range ddlMeta {
|
||||
meta[k] = v
|
||||
}
|
||||
return meta, nil
|
||||
}
|
||||
|
||||
|
|
|
@ -1,5 +1,4 @@
|
|||
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
|
||||
//
|
||||
// Copyright (C) 2019-2020 Zilliz. All rights reserved.//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
|
@ -486,16 +485,6 @@ func TestSaveBinlogPaths(t *testing.T) {
|
|||
},
|
||||
},
|
||||
},
|
||||
DdlBinlogPaths: []*datapb.DDLBinlogMeta{
|
||||
{
|
||||
DdlBinlogPath: "/by-dev/test/0/ddl/Allo7",
|
||||
TsBinlogPath: "/by-dev/test/0/ts/Allo5",
|
||||
},
|
||||
{
|
||||
DdlBinlogPath: "/by-dev/test/0/ddl/Allo9",
|
||||
TsBinlogPath: "/by-dev/test/0/ts/Allo8",
|
||||
},
|
||||
},
|
||||
CheckPoints: []*datapb.CheckPoint{
|
||||
{
|
||||
SegmentID: 0,
|
||||
|
@ -531,15 +520,6 @@ func TestSaveBinlogPaths(t *testing.T) {
|
|||
assert.EqualValues(t, "/by-dev/test/0/1/2/1/Allo2", metas[1].BinlogPath)
|
||||
}
|
||||
|
||||
collMetas, err := svr.getDDLBinlogMeta(0)
|
||||
assert.Nil(t, err)
|
||||
if assert.EqualValues(t, 2, len(collMetas)) {
|
||||
assert.EqualValues(t, "/by-dev/test/0/ts/Allo5", collMetas[0].TsBinlogPath)
|
||||
assert.EqualValues(t, "/by-dev/test/0/ddl/Allo7", collMetas[0].DdlBinlogPath)
|
||||
assert.EqualValues(t, "/by-dev/test/0/ts/Allo8", collMetas[1].TsBinlogPath)
|
||||
assert.EqualValues(t, "/by-dev/test/0/ddl/Allo9", collMetas[1].DdlBinlogPath)
|
||||
}
|
||||
|
||||
segmentInfo, err := svr.meta.GetSegment(0)
|
||||
assert.Nil(t, err)
|
||||
assert.EqualValues(t, segmentInfo.DmlPosition.ChannelName, "ch1")
|
||||
|
@ -557,16 +537,6 @@ func TestSaveBinlogPaths(t *testing.T) {
|
|||
Paths: []string{"/by-dev/test/0/1/2/1/Allo1", "/by-dev/test/0/1/2/1/Allo2"},
|
||||
},
|
||||
},
|
||||
DdlBinlogPaths: []*datapb.DDLBinlogMeta{
|
||||
{
|
||||
DdlBinlogPath: "/by-dev/test/0/ddl/Allo7",
|
||||
TsBinlogPath: "/by-dev/test/0/ts/Allo5",
|
||||
},
|
||||
{
|
||||
DdlBinlogPath: "/by-dev/test/0/ddl/Allo9",
|
||||
TsBinlogPath: "/by-dev/test/0/ts/Allo8",
|
||||
},
|
||||
},
|
||||
})
|
||||
assert.Nil(t, err)
|
||||
assert.EqualValues(t, resp.ErrorCode, commonpb.ErrorCode_UnexpectedError)
|
||||
|
@ -713,7 +683,7 @@ func TestGetVChannelPos(t *testing.T) {
|
|||
})
|
||||
assert.Nil(t, err)
|
||||
assert.EqualValues(t, 1, len(pair))
|
||||
assert.Empty(t, pair[0].CheckPoints)
|
||||
assert.Empty(t, pair[0].UnflushedSegments)
|
||||
assert.Empty(t, pair[0].FlushedSegments)
|
||||
})
|
||||
|
||||
|
@ -729,9 +699,9 @@ func TestGetVChannelPos(t *testing.T) {
|
|||
assert.EqualValues(t, 0, pair[0].CollectionID)
|
||||
assert.EqualValues(t, 1, len(pair[0].FlushedSegments))
|
||||
assert.EqualValues(t, 1, pair[0].FlushedSegments[0])
|
||||
assert.EqualValues(t, 1, len(pair[0].CheckPoints))
|
||||
assert.EqualValues(t, 2, pair[0].CheckPoints[0].SegmentID)
|
||||
assert.EqualValues(t, []byte{1, 2, 3}, pair[0].CheckPoints[0].Position.MsgID)
|
||||
assert.EqualValues(t, 1, len(pair[0].UnflushedSegments))
|
||||
assert.EqualValues(t, 2, pair[0].UnflushedSegments[0].ID)
|
||||
assert.EqualValues(t, []byte{1, 2, 3}, pair[0].UnflushedSegments[0].DmlPosition.MsgID)
|
||||
})
|
||||
}
|
||||
|
||||
|
@ -789,7 +759,7 @@ func TestGetRecoveryInfo(t *testing.T) {
|
|||
assert.Nil(t, err)
|
||||
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
|
||||
assert.EqualValues(t, 1, len(resp.GetChannels()))
|
||||
assert.EqualValues(t, 0, len(resp.GetChannels()[0].GetCheckPoints()))
|
||||
assert.EqualValues(t, 0, len(resp.GetChannels()[0].GetUnflushedSegments()))
|
||||
assert.ElementsMatch(t, []UniqueID{0, 1}, resp.GetChannels()[0].GetFlushedSegments())
|
||||
assert.EqualValues(t, 20, resp.GetChannels()[0].GetSeekPosition().GetTimestamp())
|
||||
})
|
||||
|
@ -813,14 +783,14 @@ func TestGetRecoveryInfo(t *testing.T) {
|
|||
assert.Nil(t, err)
|
||||
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
|
||||
assert.EqualValues(t, 1, len(resp.GetChannels()))
|
||||
assert.EqualValues(t, 2, len(resp.GetChannels()[0].GetCheckPoints()))
|
||||
assert.EqualValues(t, 2, len(resp.GetChannels()[0].GetUnflushedSegments()))
|
||||
assert.ElementsMatch(t, []UniqueID{0, 1}, resp.GetChannels()[0].GetFlushedSegments())
|
||||
assert.EqualValues(t, 30, resp.GetChannels()[0].GetSeekPosition().GetTimestamp())
|
||||
cps := resp.GetChannels()[0].GetCheckPoints()
|
||||
cps := resp.GetChannels()[0].GetUnflushedSegments()
|
||||
for _, cp := range cps {
|
||||
seg, ok := expectedCps[cp.GetSegmentID()]
|
||||
seg, ok := expectedCps[cp.GetID()]
|
||||
assert.True(t, ok)
|
||||
assert.EqualValues(t, seg.GetDmlPosition().GetTimestamp(), cp.GetPosition().GetTimestamp())
|
||||
assert.EqualValues(t, seg.GetDmlPosition().GetTimestamp(), cp.GetDmlPosition().GetTimestamp())
|
||||
assert.EqualValues(t, seg.GetNumOfRows(), cp.GetNumOfRows())
|
||||
}
|
||||
})
|
||||
|
@ -838,7 +808,6 @@ func TestGetRecoveryInfo(t *testing.T) {
|
|||
},
|
||||
},
|
||||
},
|
||||
DdlBinlogPaths: []*datapb.DDLBinlogMeta{},
|
||||
}
|
||||
meta, err := svr.prepareBinlog(binlogReq)
|
||||
assert.Nil(t, err)
|
||||
|
|
|
@ -169,12 +169,7 @@ func (s *Server) init() error {
|
|||
addr := Params.IP + ":" + strconv.Itoa(Params.Port)
|
||||
log.Debug("DataNode address", zap.String("address", addr))
|
||||
|
||||
err := s.datanode.Register()
|
||||
if err != nil {
|
||||
log.Debug("DataNode Register etcd failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
err = s.startGrpc()
|
||||
err := s.startGrpc()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -243,7 +238,15 @@ func (s *Server) init() error {
|
|||
}
|
||||
|
||||
func (s *Server) start() error {
|
||||
return s.datanode.Start()
|
||||
if err := s.datanode.Start(); err != nil {
|
||||
return err
|
||||
}
|
||||
err := s.datanode.Register()
|
||||
if err != nil {
|
||||
log.Debug("DataNode Register etcd failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) GetComponentStates(ctx context.Context, req *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) {
|
||||
|
|
|
@ -90,10 +90,6 @@ func TestRun(t *testing.T) {
|
|||
assert.NotNil(t, err)
|
||||
assert.EqualError(t, err, "listen tcp: address 1000000: invalid port")
|
||||
|
||||
dsServer.newMasterServiceClient = func() (types.MasterService, error) {
|
||||
return &mockMaster{}, nil
|
||||
}
|
||||
|
||||
Params.Port = rand.Int()%100 + 10000
|
||||
err = dsServer.Run()
|
||||
assert.Nil(t, err)
|
||||
|
|
|
@ -18,7 +18,6 @@ import (
|
|||
"net"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/logutil"
|
||||
|
||||
|
@ -28,7 +27,6 @@ import (
|
|||
|
||||
grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
|
||||
"github.com/milvus-io/milvus/internal/dataservice"
|
||||
msc "github.com/milvus-io/milvus/internal/distributed/masterservice/client"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
|
@ -54,8 +52,6 @@ type Server struct {
|
|||
grpcServer *grpc.Server
|
||||
masterService types.MasterService
|
||||
|
||||
newMasterServiceClient func() (types.MasterService, error)
|
||||
|
||||
closer io.Closer
|
||||
}
|
||||
|
||||
|
@ -68,9 +64,6 @@ func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error)
|
|||
ctx: ctx1,
|
||||
cancel: cancel,
|
||||
grpcErrChan: make(chan error),
|
||||
newMasterServiceClient: func() (types.MasterService, error) {
|
||||
return msc.NewClient(dataservice.Params.MetaRootPath, []string{dataservice.Params.EtcdAddress}, 3*time.Second)
|
||||
},
|
||||
}
|
||||
s.dataService, err = dataservice.CreateServer(s.ctx, factory)
|
||||
if err != nil {
|
||||
|
@ -83,8 +76,6 @@ func (s *Server) init() error {
|
|||
Params.Init()
|
||||
Params.LoadFromEnv()
|
||||
|
||||
ctx := context.Background()
|
||||
|
||||
closer := trace.InitTracing("data_service")
|
||||
s.closer = closer
|
||||
|
||||
|
@ -105,30 +96,6 @@ func (s *Server) init() error {
|
|||
return err
|
||||
}
|
||||
|
||||
if s.newMasterServiceClient != nil {
|
||||
log.Debug("DataService try to new master service client", zap.String("address", Params.MasterAddress))
|
||||
masterServiceClient, err := s.newMasterServiceClient()
|
||||
if err != nil {
|
||||
log.Debug("DataService new master service client failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if err = masterServiceClient.Init(); err != nil {
|
||||
log.Debug("DataService masterServiceClient Init failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
if err = masterServiceClient.Start(); err != nil {
|
||||
log.Debug("DataService masterServiceClient Start failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
log.Debug("DataService start to wait for MasterService ready")
|
||||
if err = funcutil.WaitForComponentInitOrHealthy(ctx, masterServiceClient, "MasterService", 1000000, 200*time.Millisecond); err != nil {
|
||||
log.Debug("DataService wait for MasterService Ready failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
log.Debug("DataService report MasterService is ready")
|
||||
}
|
||||
|
||||
if err := s.dataService.Init(); err != nil {
|
||||
log.Error("dataService init error", zap.Error(err))
|
||||
return err
|
||||
|
|
|
@ -84,6 +84,9 @@ func (d *dmlChannels) Broadcast(name string, pack *msgstream.MsgPack) error {
|
|||
d.lock.Lock()
|
||||
defer d.lock.Unlock()
|
||||
|
||||
if len(name) == 0 {
|
||||
return fmt.Errorf("channel name is empty")
|
||||
}
|
||||
var err error
|
||||
ms, ok := d.dml[name]
|
||||
if !ok {
|
||||
|
@ -97,12 +100,13 @@ func (d *dmlChannels) Broadcast(name string, pack *msgstream.MsgPack) error {
|
|||
return ms.Broadcast(pack)
|
||||
}
|
||||
|
||||
func (d *dmlChannels) AddProducerChannles(names ...string) {
|
||||
func (d *dmlChannels) AddProducerChannels(names ...string) {
|
||||
d.lock.Lock()
|
||||
defer d.lock.Unlock()
|
||||
|
||||
var err error
|
||||
for _, name := range names {
|
||||
log.Debug("add dml channel", zap.String("channel name", name))
|
||||
ms, ok := d.dml[name]
|
||||
if !ok {
|
||||
ms, err = d.core.msFactory.NewMsgStream(d.core.ctx)
|
||||
|
@ -121,9 +125,23 @@ func (d *dmlChannels) RemoveProducerChannels(names ...string) {
|
|||
defer d.lock.Unlock()
|
||||
|
||||
for _, name := range names {
|
||||
log.Debug("delete dml channel", zap.String("channel name", name))
|
||||
if ms, ok := d.dml[name]; ok {
|
||||
ms.Close()
|
||||
delete(d.dml, name)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (d *dmlChannels) HasChannel(names ...string) bool {
|
||||
d.lock.Lock()
|
||||
defer d.lock.Unlock()
|
||||
|
||||
for _, name := range names {
|
||||
if _, ok := d.dml[name]; !ok {
|
||||
log.Debug("unknown channel", zap.String("channel name", name))
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
|
|
@ -21,9 +21,6 @@ import (
|
|||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
|
@ -46,6 +43,8 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// internalpb -> internalpb
|
||||
|
@ -316,8 +315,15 @@ func (c *Core) startDataServiceSegmentLoop() {
|
|||
if msg.Type() != commonpb.MsgType_SegmentInfo {
|
||||
continue
|
||||
}
|
||||
segInfoMsg := msg.(*ms.SegmentInfoMsg)
|
||||
segInfos = append(segInfos, segInfoMsg.Segment)
|
||||
segInfoMsg, ok := msg.(*ms.SegmentInfoMsg)
|
||||
if !ok {
|
||||
log.Debug("input msg is not SegmentInfoMsg")
|
||||
continue
|
||||
}
|
||||
if segInfoMsg.Segment != nil {
|
||||
segInfos = append(segInfos, segInfoMsg.Segment)
|
||||
log.Debug("open segment", zap.Int64("segmentID", segInfoMsg.Segment.ID))
|
||||
}
|
||||
}
|
||||
if len(segInfos) > 0 {
|
||||
startPosStr, err := EncodeMsgPositions(segMsg.StartPositions)
|
||||
|
@ -371,7 +377,11 @@ func (c *Core) startDataNodeFlushedSegmentLoop() {
|
|||
if msg.Type() != commonpb.MsgType_SegmentFlushDone {
|
||||
continue
|
||||
}
|
||||
flushMsg := msg.(*ms.FlushCompletedMsg)
|
||||
flushMsg, ok := msg.(*ms.FlushCompletedMsg)
|
||||
if !ok {
|
||||
log.Debug("input msg is not FlushCompletedMsg")
|
||||
continue
|
||||
}
|
||||
segID := flushMsg.SegmentID
|
||||
log.Debug("flush segment", zap.Int64("id", segID))
|
||||
|
||||
|
@ -386,6 +396,10 @@ func (c *Core) startDataNodeFlushedSegmentLoop() {
|
|||
continue
|
||||
}
|
||||
|
||||
if len(coll.FieldIndexes) == 0 {
|
||||
log.Debug("no index params on collection", zap.String("collection_name", coll.Schema.Name))
|
||||
}
|
||||
|
||||
for _, f := range coll.FieldIndexes {
|
||||
fieldSch, err := GetFieldSchemaByID(coll, f.FiledID)
|
||||
if err != nil {
|
||||
|
@ -972,7 +986,7 @@ func (c *Core) Init() error {
|
|||
|
||||
c.dmlChannels = newDMLChannels(c)
|
||||
pc := c.MetaTable.ListCollectionPhysicalChannels()
|
||||
c.dmlChannels.AddProducerChannles(pc...)
|
||||
c.dmlChannels.AddProducerChannels(pc...)
|
||||
|
||||
c.chanTimeTick = newTimeTickSync(c)
|
||||
c.chanTimeTick.AddProxyNode(c.session)
|
||||
|
@ -1335,6 +1349,7 @@ func (c *Core) DescribeCollection(ctx context.Context, in *milvuspb.DescribeColl
|
|||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
Reason: "",
|
||||
}
|
||||
// log.Debug("describe collection", zap.Any("schema", t.Rsp.Schema))
|
||||
return t.Rsp, nil
|
||||
}
|
||||
|
||||
|
@ -1858,6 +1873,12 @@ func (c *Core) UpdateChannelTimeTick(ctx context.Context, in *internalpb.Channel
|
|||
status.Reason = fmt.Sprintf("UpdateChannelTimeTick receive invalid message %d", in.Base.GetMsgType())
|
||||
return status, nil
|
||||
}
|
||||
if !c.dmlChannels.HasChannel(in.ChannelNames...) {
|
||||
log.Debug("update time tick with unkonw channel", zap.Int("input channel size", len(in.ChannelNames)), zap.Strings("input channels", in.ChannelNames))
|
||||
status.ErrorCode = commonpb.ErrorCode_UnexpectedError
|
||||
status.Reason = fmt.Sprintf("update time tick with unknown channel name, input channels = %v", in.ChannelNames)
|
||||
return status, nil
|
||||
}
|
||||
err := c.chanTimeTick.UpdateTimeTick(in)
|
||||
if err != nil {
|
||||
status.ErrorCode = commonpb.ErrorCode_UnexpectedError
|
||||
|
|
|
@ -1485,6 +1485,8 @@ func TestMasterService(t *testing.T) {
|
|||
assert.Nil(t, err)
|
||||
time.Sleep(time.Second)
|
||||
|
||||
core.dmlChannels.AddProducerChannels("c0", "c1", "c2")
|
||||
|
||||
msg0 := &internalpb.ChannelTimeTickMsg{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_TimeTick,
|
||||
|
|
|
@ -393,7 +393,7 @@ func (ms *metaSnapshot) MultiSaveAndRemoveWithPrefix(saves map[string]string, re
|
|||
}
|
||||
}
|
||||
for _, key := range removals {
|
||||
ops = append(ops, clientv3.OpDelete(path.Join(ms.root, key)))
|
||||
ops = append(ops, clientv3.OpDelete(path.Join(ms.root, key), clientv3.WithPrefix()))
|
||||
}
|
||||
ops = append(ops, clientv3.OpPut(path.Join(ms.root, ms.tsKey), strTs))
|
||||
resp, err := ms.cli.Txn(ctx).If().Then(ops...).Commit()
|
||||
|
|
|
@ -345,13 +345,13 @@ func TestMultiSaveAndRemoveWithPrefix(t *testing.T) {
|
|||
|
||||
for i := 0; i < 20; i++ {
|
||||
vtso = typeutil.Timestamp(100 + i*5)
|
||||
ts, err := ms.Save(fmt.Sprintf("kd-%d", i), fmt.Sprintf("value-%d", i))
|
||||
ts, err := ms.Save(fmt.Sprintf("kd-%04d", i), fmt.Sprintf("value-%d", i))
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, vtso, ts)
|
||||
}
|
||||
for i := 20; i < 40; i++ {
|
||||
sm := map[string]string{"ks": fmt.Sprintf("value-%d", i)}
|
||||
dm := []string{fmt.Sprintf("kd-%d", i-20)}
|
||||
dm := []string{fmt.Sprintf("kd-%04d", i-20)}
|
||||
vtso = typeutil.Timestamp(100 + i*5)
|
||||
ts, err := ms.MultiSaveAndRemoveWithPrefix(sm, dm, nil)
|
||||
assert.Nil(t, err)
|
||||
|
@ -359,7 +359,7 @@ func TestMultiSaveAndRemoveWithPrefix(t *testing.T) {
|
|||
}
|
||||
|
||||
for i := 0; i < 20; i++ {
|
||||
val, err := ms.Load(fmt.Sprintf("kd-%d", i), typeutil.Timestamp(100+i*5+2))
|
||||
val, err := ms.Load(fmt.Sprintf("kd-%04d", i), typeutil.Timestamp(100+i*5+2))
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, fmt.Sprintf("value-%d", i), val)
|
||||
_, vals, err := ms.LoadWithPrefix("kd-", typeutil.Timestamp(100+i*5+2))
|
||||
|
@ -380,7 +380,7 @@ func TestMultiSaveAndRemoveWithPrefix(t *testing.T) {
|
|||
assert.NotNil(t, ms)
|
||||
|
||||
for i := 0; i < 20; i++ {
|
||||
val, err := ms.Load(fmt.Sprintf("kd-%d", i), typeutil.Timestamp(100+i*5+2))
|
||||
val, err := ms.Load(fmt.Sprintf("kd-%04d", i), typeutil.Timestamp(100+i*5+2))
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, fmt.Sprintf("value-%d", i), val)
|
||||
_, vals, err := ms.LoadWithPrefix("kd-", typeutil.Timestamp(100+i*5+2))
|
||||
|
|
|
@ -237,7 +237,7 @@ func (t *CreateCollectionReqTask) Execute(ctx context.Context) error {
|
|||
}
|
||||
|
||||
// add dml channel before send dd msg
|
||||
t.core.dmlChannels.AddProducerChannles(chanNames...)
|
||||
t.core.dmlChannels.AddProducerChannels(chanNames...)
|
||||
|
||||
err = t.core.SendDdCreateCollectionReq(ctx, &ddCollReq)
|
||||
if err != nil {
|
||||
|
@ -393,13 +393,13 @@ func (t *DescribeCollectionReqTask) Execute(ctx context.Context) error {
|
|||
|
||||
t.Rsp.Schema = proto.Clone(collInfo.Schema).(*schemapb.CollectionSchema)
|
||||
t.Rsp.CollectionID = collInfo.ID
|
||||
var newField []*schemapb.FieldSchema
|
||||
for _, field := range t.Rsp.Schema.Fields {
|
||||
if field.FieldID >= StartOfUserFieldID {
|
||||
newField = append(newField, field)
|
||||
}
|
||||
}
|
||||
t.Rsp.Schema.Fields = newField
|
||||
//var newField []*schemapb.FieldSchema
|
||||
//for _, field := range t.Rsp.Schema.Fields {
|
||||
// if field.FieldID >= StartOfUserFieldID {
|
||||
// newField = append(newField, field)
|
||||
// }
|
||||
//}
|
||||
//t.Rsp.Schema.Fields = newField
|
||||
|
||||
t.Rsp.VirtualChannelNames = collInfo.VirtualChannelNames
|
||||
t.Rsp.PhysicalChannelNames = collInfo.PhysicalChannelNames
|
||||
|
|
|
@ -12,8 +12,10 @@
|
|||
package msgstream
|
||||
|
||||
import (
|
||||
"log"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// Reference: https://blog.cyeam.com/golang/2018/08/27/retry
|
||||
|
@ -25,7 +27,7 @@ func Retry(attempts int, sleep time.Duration, fn func() error) error {
|
|||
}
|
||||
|
||||
if attempts--; attempts > 0 {
|
||||
log.Printf("retry func error: %s. attempts #%d after %s.", err.Error(), attempts, sleep)
|
||||
log.Debug("retry func error", zap.Int("attempts", attempts), zap.Duration("sleep", sleep), zap.Error(err))
|
||||
time.Sleep(sleep)
|
||||
return Retry(attempts, 2*sleep, fn)
|
||||
}
|
||||
|
|
|
@ -48,8 +48,8 @@ enum SegmentState {
|
|||
NotExist = 1;
|
||||
Growing = 2;
|
||||
Sealed = 3;
|
||||
Flushing = 4;
|
||||
Flushed = 5;
|
||||
Flushed = 4;
|
||||
Flushing = 5;
|
||||
}
|
||||
|
||||
message Status {
|
||||
|
|
|
@ -159,8 +159,8 @@ const (
|
|||
SegmentState_NotExist SegmentState = 1
|
||||
SegmentState_Growing SegmentState = 2
|
||||
SegmentState_Sealed SegmentState = 3
|
||||
SegmentState_Flushing SegmentState = 4
|
||||
SegmentState_Flushed SegmentState = 5
|
||||
SegmentState_Flushed SegmentState = 4
|
||||
SegmentState_Flushing SegmentState = 5
|
||||
)
|
||||
|
||||
var SegmentState_name = map[int32]string{
|
||||
|
@ -168,8 +168,8 @@ var SegmentState_name = map[int32]string{
|
|||
1: "NotExist",
|
||||
2: "Growing",
|
||||
3: "Sealed",
|
||||
4: "Flushing",
|
||||
5: "Flushed",
|
||||
4: "Flushed",
|
||||
5: "Flushing",
|
||||
}
|
||||
|
||||
var SegmentState_value = map[string]int32{
|
||||
|
@ -177,8 +177,8 @@ var SegmentState_value = map[string]int32{
|
|||
"NotExist": 1,
|
||||
"Growing": 2,
|
||||
"Sealed": 3,
|
||||
"Flushing": 4,
|
||||
"Flushed": 5,
|
||||
"Flushed": 4,
|
||||
"Flushing": 5,
|
||||
}
|
||||
|
||||
func (x SegmentState) String() string {
|
||||
|
@ -691,85 +691,86 @@ func init() {
|
|||
func init() { proto.RegisterFile("common.proto", fileDescriptor_555bd8c177793206) }
|
||||
|
||||
var fileDescriptor_555bd8c177793206 = []byte{
|
||||
// 1280 bytes of a gzipped FileDescriptorProto
|
||||
// 1282 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x55, 0xcb, 0x72, 0xdc, 0xb6,
|
||||
0x12, 0x15, 0x87, 0x23, 0x8d, 0x08, 0x8d, 0x24, 0x08, 0x7a, 0x58, 0xf6, 0x55, 0xdd, 0x72, 0x69,
|
||||
0xe5, 0x52, 0x95, 0xa5, 0x7b, 0xaf, 0xeb, 0x26, 0x2b, 0x2f, 0xac, 0xa1, 0x25, 0x4d, 0xd9, 0x7a,
|
||||
0x84, 0x23, 0x3b, 0xa9, 0x6c, 0x5c, 0x10, 0xd9, 0x33, 0x83, 0x98, 0x04, 0x26, 0x00, 0x28, 0x4b,
|
||||
0xfb, 0x7c, 0x40, 0xe2, 0x7f, 0xc8, 0x2e, 0x49, 0xe5, 0x9d, 0x7c, 0x42, 0xde, 0xeb, 0x7c, 0x42,
|
||||
0x3e, 0x20, 0x4f, 0x3f, 0x53, 0x0d, 0x72, 0x66, 0xe8, 0x2a, 0x67, 0xc7, 0x3e, 0xdd, 0x38, 0x38,
|
||||
0x38, 0x8d, 0x06, 0x49, 0x33, 0x56, 0x59, 0xa6, 0xe4, 0xe6, 0x40, 0x2b, 0xab, 0xd8, 0x62, 0x26,
|
||||
0xd2, 0xd3, 0xdc, 0x14, 0xd1, 0x66, 0x91, 0x5a, 0xbf, 0x47, 0xa6, 0x3a, 0x96, 0xdb, 0xdc, 0xb0,
|
||||
0xeb, 0x84, 0x80, 0xd6, 0x4a, 0xdf, 0x8b, 0x55, 0x02, 0xab, 0xde, 0x65, 0xef, 0xca, 0xdc, 0xff,
|
||||
0xfe, 0xbd, 0xf9, 0x92, 0x35, 0x9b, 0x37, 0xb1, 0xac, 0xa5, 0x12, 0x88, 0x02, 0x18, 0x7e, 0xb2,
|
||||
0x15, 0x32, 0xa5, 0x81, 0x1b, 0x25, 0x57, 0x6b, 0x97, 0xbd, 0x2b, 0x41, 0x54, 0x46, 0xeb, 0xaf,
|
||||
0x90, 0xe6, 0x2d, 0x38, 0xbf, 0xcb, 0xd3, 0x1c, 0x8e, 0xb8, 0xd0, 0x8c, 0x12, 0xff, 0x3e, 0x9c,
|
||||
0x3b, 0xfe, 0x20, 0xc2, 0x4f, 0xb6, 0x44, 0x26, 0x4f, 0x31, 0x5d, 0x2e, 0x2c, 0x82, 0xf5, 0x35,
|
||||
0x52, 0xdf, 0x4e, 0xd5, 0xc9, 0x38, 0x8b, 0x2b, 0x9a, 0xc3, 0xec, 0x55, 0xd2, 0xb8, 0x91, 0x24,
|
||||
0x1a, 0x8c, 0x61, 0x73, 0xa4, 0x26, 0x06, 0x25, 0x5f, 0x4d, 0x0c, 0x18, 0x23, 0xf5, 0x81, 0xd2,
|
||||
0xd6, 0xb1, 0xf9, 0x91, 0xfb, 0x5e, 0x7f, 0xe8, 0x91, 0xc6, 0xbe, 0xe9, 0x6d, 0x73, 0x03, 0xec,
|
||||
0x55, 0x32, 0x9d, 0x99, 0xde, 0x3d, 0x7b, 0x3e, 0x18, 0x9e, 0x72, 0xed, 0xa5, 0xa7, 0xdc, 0x37,
|
||||
0xbd, 0xe3, 0xf3, 0x01, 0x44, 0x8d, 0xac, 0xf8, 0x40, 0x25, 0x99, 0xe9, 0xb5, 0xc3, 0x92, 0xb9,
|
||||
0x08, 0xd8, 0x1a, 0x09, 0xac, 0xc8, 0xc0, 0x58, 0x9e, 0x0d, 0x56, 0xfd, 0xcb, 0xde, 0x95, 0x7a,
|
||||
0x34, 0x06, 0xd8, 0x25, 0x32, 0x6d, 0x54, 0xae, 0x63, 0x68, 0x87, 0xab, 0x75, 0xb7, 0x6c, 0x14,
|
||||
0xaf, 0x5f, 0x27, 0xc1, 0xbe, 0xe9, 0xed, 0x01, 0x4f, 0x40, 0xb3, 0xff, 0x90, 0xfa, 0x09, 0x37,
|
||||
0x85, 0xa2, 0x99, 0x7f, 0x56, 0x84, 0x27, 0x88, 0x5c, 0xe5, 0xc6, 0xfb, 0x75, 0x12, 0x8c, 0x3a,
|
||||
0xc1, 0x66, 0x48, 0xa3, 0x93, 0xc7, 0x31, 0x18, 0x43, 0x27, 0xd8, 0x22, 0x99, 0xbf, 0x23, 0xe1,
|
||||
0x6c, 0x00, 0xb1, 0x85, 0xc4, 0xd5, 0x50, 0x8f, 0x2d, 0x90, 0xd9, 0x96, 0x92, 0x12, 0x62, 0xbb,
|
||||
0xc3, 0x45, 0x0a, 0x09, 0xad, 0xb1, 0x25, 0x42, 0x8f, 0x40, 0x67, 0xc2, 0x18, 0xa1, 0x64, 0x08,
|
||||
0x52, 0x40, 0x42, 0x7d, 0x76, 0x81, 0x2c, 0xb6, 0x54, 0x9a, 0x42, 0x6c, 0x85, 0x92, 0x07, 0xca,
|
||||
0xde, 0x3c, 0x13, 0xc6, 0x1a, 0x5a, 0x47, 0xda, 0x76, 0x9a, 0x42, 0x8f, 0xa7, 0x37, 0x74, 0x2f,
|
||||
0xcf, 0x40, 0x5a, 0x3a, 0x89, 0x1c, 0x25, 0x18, 0x8a, 0x0c, 0x24, 0x32, 0xd1, 0x46, 0x05, 0x6d,
|
||||
0xcb, 0x04, 0xce, 0xd0, 0x3f, 0x3a, 0xcd, 0x2e, 0x92, 0xe5, 0x12, 0xad, 0x6c, 0xc0, 0x33, 0xa0,
|
||||
0x01, 0x9b, 0x27, 0x33, 0x65, 0xea, 0xf8, 0xf0, 0xe8, 0x16, 0x25, 0x15, 0x86, 0x48, 0x3d, 0x88,
|
||||
0x20, 0x56, 0x3a, 0xa1, 0x33, 0x15, 0x09, 0x77, 0x21, 0xb6, 0x4a, 0xb7, 0x43, 0xda, 0x44, 0xc1,
|
||||
0x25, 0xd8, 0x01, 0xae, 0xe3, 0x7e, 0x04, 0x26, 0x4f, 0x2d, 0x9d, 0x65, 0x94, 0x34, 0x77, 0x44,
|
||||
0x0a, 0x07, 0xca, 0xee, 0xa8, 0x5c, 0x26, 0x74, 0x8e, 0xcd, 0x11, 0xb2, 0x0f, 0x96, 0x97, 0x0e,
|
||||
0xcc, 0xe3, 0xb6, 0x2d, 0x1e, 0xf7, 0xa1, 0x04, 0x28, 0x5b, 0x21, 0xac, 0xc5, 0xa5, 0x54, 0xb6,
|
||||
0xa5, 0x81, 0x5b, 0xd8, 0x51, 0x69, 0x02, 0x9a, 0x2e, 0xa0, 0x9c, 0x17, 0x70, 0x91, 0x02, 0x65,
|
||||
0xe3, 0xea, 0x10, 0x52, 0x18, 0x55, 0x2f, 0x8e, 0xab, 0x4b, 0x1c, 0xab, 0x97, 0x50, 0xfc, 0x76,
|
||||
0x2e, 0xd2, 0xc4, 0x59, 0x52, 0xb4, 0x65, 0x19, 0x35, 0x96, 0xe2, 0x0f, 0x6e, 0xb7, 0x3b, 0xc7,
|
||||
0x74, 0x85, 0x2d, 0x93, 0x85, 0x12, 0xd9, 0x07, 0xab, 0x45, 0xec, 0xcc, 0xbb, 0x80, 0x52, 0x0f,
|
||||
0x73, 0x7b, 0xd8, 0xdd, 0x87, 0x4c, 0xe9, 0x73, 0xba, 0x8a, 0x0d, 0x75, 0x4c, 0xc3, 0x16, 0xd1,
|
||||
0x8b, 0x8c, 0x91, 0xd9, 0x30, 0x8c, 0xe0, 0xed, 0x1c, 0x8c, 0x8d, 0x78, 0x0c, 0xf4, 0x97, 0xc6,
|
||||
0xc6, 0x1b, 0x84, 0xb8, 0x32, 0x1c, 0x73, 0x60, 0x8c, 0xcc, 0x8d, 0xa3, 0x03, 0x25, 0x81, 0x4e,
|
||||
0xb0, 0x26, 0x99, 0xbe, 0x23, 0x85, 0x31, 0x39, 0x24, 0xd4, 0x43, 0x8b, 0xda, 0xf2, 0x48, 0xab,
|
||||
0x1e, 0x4e, 0x17, 0xad, 0x61, 0x76, 0x47, 0x48, 0x61, 0xfa, 0xee, 0x72, 0x10, 0x32, 0x55, 0x7a,
|
||||
0x55, 0xdf, 0xe8, 0x92, 0x66, 0x07, 0x7a, 0x78, 0x0f, 0x0a, 0xee, 0x25, 0x42, 0xab, 0xf1, 0x98,
|
||||
0x7d, 0xa4, 0xd0, 0xc3, 0x7b, 0xba, 0xab, 0xd5, 0x03, 0x21, 0x7b, 0xb4, 0x86, 0x64, 0x1d, 0xe0,
|
||||
0xa9, 0x23, 0xc6, 0x6d, 0xd2, 0xdc, 0xf4, 0x31, 0x53, 0xc7, 0x32, 0x17, 0x41, 0x42, 0x27, 0x37,
|
||||
0xde, 0x99, 0x76, 0xd3, 0xeb, 0x86, 0x70, 0x96, 0x04, 0x77, 0x64, 0x02, 0x5d, 0x21, 0x21, 0xa1,
|
||||
0x13, 0xce, 0x68, 0xd7, 0x90, 0xf1, 0x85, 0xa2, 0x09, 0x1e, 0x32, 0xd4, 0x6a, 0x50, 0xc1, 0x00,
|
||||
0xdd, 0xda, 0xe3, 0xa6, 0x02, 0x75, 0xb1, 0x7b, 0x21, 0x98, 0x58, 0x8b, 0x93, 0xea, 0xf2, 0x1e,
|
||||
0xf6, 0xa9, 0xd3, 0x57, 0x0f, 0xc6, 0x98, 0xa1, 0x7d, 0xdc, 0x69, 0x17, 0x6c, 0xe7, 0xdc, 0x58,
|
||||
0xc8, 0x5a, 0x4a, 0x76, 0x45, 0xcf, 0x50, 0x81, 0x3b, 0xdd, 0x56, 0x3c, 0xa9, 0x2c, 0x7f, 0x0b,
|
||||
0xfb, 0x17, 0x41, 0x0a, 0xdc, 0x54, 0x59, 0xef, 0xb3, 0x25, 0x32, 0x5f, 0x48, 0x3d, 0xe2, 0xda,
|
||||
0x0a, 0x07, 0x7e, 0xe3, 0xb9, 0x8e, 0x69, 0x35, 0x18, 0x63, 0xdf, 0xe2, 0xa4, 0x36, 0xf7, 0xb8,
|
||||
0x19, 0x43, 0xdf, 0x79, 0x6c, 0x85, 0x2c, 0x0c, 0xa5, 0x8e, 0xf1, 0xef, 0x3d, 0xb6, 0x48, 0xe6,
|
||||
0x50, 0xea, 0x08, 0x33, 0xf4, 0x07, 0x07, 0xa2, 0xa8, 0x0a, 0xf8, 0xa3, 0x63, 0x28, 0x55, 0x55,
|
||||
0xf0, 0x9f, 0xdc, 0x66, 0xc8, 0x50, 0x36, 0xce, 0xd0, 0x47, 0x1e, 0x2a, 0x1d, 0x6e, 0x56, 0xc2,
|
||||
0xf4, 0xb1, 0x2b, 0x44, 0xd6, 0x51, 0xe1, 0x13, 0x57, 0x58, 0x72, 0x8e, 0xd0, 0xa7, 0x0e, 0xdd,
|
||||
0xe3, 0x32, 0x51, 0xdd, 0xee, 0x08, 0x7d, 0xe6, 0xb1, 0x55, 0xb2, 0x88, 0xcb, 0xb7, 0x79, 0xca,
|
||||
0x65, 0x3c, 0xae, 0x7f, 0xee, 0x31, 0x4a, 0x66, 0x0a, 0x63, 0xdc, 0xc5, 0xa4, 0x1f, 0xd4, 0x9c,
|
||||
0x29, 0xa5, 0x80, 0x02, 0xfb, 0xb0, 0xc6, 0xe6, 0x48, 0x80, 0x46, 0x15, 0xf1, 0x47, 0x35, 0x36,
|
||||
0x43, 0xa6, 0xda, 0xd2, 0x80, 0xb6, 0xf4, 0x5d, 0x1f, 0x83, 0x62, 0xd2, 0xe8, 0x7b, 0x78, 0x45,
|
||||
0x27, 0xdd, 0xdd, 0xa1, 0x0f, 0x5d, 0xa2, 0x78, 0x13, 0xe8, 0xaf, 0xbe, 0x3b, 0x6a, 0xf5, 0x81,
|
||||
0xf8, 0xcd, 0xc7, 0x9d, 0x76, 0xc1, 0x8e, 0x27, 0x82, 0xfe, 0xee, 0xb3, 0x4b, 0x64, 0x79, 0x88,
|
||||
0xb9, 0x71, 0x1d, 0xcd, 0xc2, 0x1f, 0x3e, 0x5b, 0x23, 0x17, 0x76, 0xc1, 0x8e, 0xfb, 0x8a, 0x8b,
|
||||
0x84, 0xb1, 0x22, 0x36, 0xf4, 0x4f, 0x9f, 0xfd, 0x8b, 0xac, 0xec, 0x82, 0x1d, 0xf9, 0x5b, 0x49,
|
||||
0xfe, 0xe5, 0xb3, 0x59, 0x32, 0x1d, 0xe1, 0x3c, 0xc3, 0x29, 0xd0, 0x47, 0x3e, 0x36, 0x69, 0x18,
|
||||
0x96, 0x72, 0x1e, 0xfb, 0x68, 0xdd, 0xeb, 0xdc, 0xc6, 0xfd, 0x30, 0x6b, 0xf5, 0xb9, 0x94, 0x90,
|
||||
0x1a, 0xfa, 0xc4, 0x67, 0xcb, 0x84, 0x46, 0x90, 0xa9, 0x53, 0xa8, 0xc0, 0x4f, 0xf1, 0x9d, 0x66,
|
||||
0xae, 0xf8, 0xb5, 0x1c, 0xf4, 0xf9, 0x28, 0xf1, 0xcc, 0x47, 0xab, 0x8b, 0xfa, 0x17, 0x33, 0xcf,
|
||||
0x7d, 0xb4, 0xba, 0x74, 0xbe, 0x2d, 0xbb, 0x8a, 0xfe, 0x5c, 0x47, 0x55, 0xc7, 0x22, 0x83, 0x63,
|
||||
0x11, 0xdf, 0xa7, 0x1f, 0x07, 0xa8, 0xca, 0x2d, 0x3a, 0x50, 0x09, 0xa0, 0x7c, 0x43, 0x3f, 0x09,
|
||||
0xd0, 0x7a, 0x6c, 0x5d, 0x61, 0xfd, 0xa7, 0x2e, 0x2e, 0xdf, 0x98, 0x76, 0x48, 0x3f, 0xc3, 0xb7,
|
||||
0x9b, 0x94, 0xf1, 0x71, 0xe7, 0x90, 0x7e, 0x1e, 0xe0, 0x31, 0x6e, 0xa4, 0xa9, 0x8a, 0xb9, 0x1d,
|
||||
0x5d, 0xa0, 0x2f, 0x02, 0xbc, 0x81, 0x95, 0xe7, 0xa1, 0x34, 0xe6, 0xcb, 0x00, 0x8f, 0x57, 0xe2,
|
||||
0xae, 0x6d, 0x21, 0x3e, 0x1b, 0x5f, 0x39, 0xd6, 0x90, 0x5b, 0x8e, 0x4a, 0x8e, 0x2d, 0xfd, 0x3a,
|
||||
0xd8, 0x58, 0x27, 0x8d, 0xd0, 0xa4, 0xee, 0x15, 0x68, 0x10, 0x3f, 0x34, 0x29, 0x9d, 0xc0, 0xc7,
|
||||
0x6a, 0x5b, 0xa9, 0xf4, 0xe6, 0xd9, 0x40, 0xdf, 0xfd, 0x2f, 0xf5, 0xb6, 0xff, 0xff, 0xe6, 0xb5,
|
||||
0x9e, 0xb0, 0xfd, 0xfc, 0x04, 0xff, 0x99, 0x5b, 0xc5, 0x4f, 0xf4, 0xaa, 0x50, 0xe5, 0xd7, 0x96,
|
||||
0x90, 0x16, 0xb4, 0xe4, 0xe9, 0x96, 0xfb, 0xaf, 0x6e, 0x15, 0xff, 0xd5, 0xc1, 0xc9, 0xc9, 0x94,
|
||||
0x8b, 0xaf, 0xfd, 0x1d, 0x00, 0x00, 0xff, 0xff, 0xc7, 0x97, 0xd8, 0x68, 0x31, 0x09, 0x00, 0x00,
|
||||
0x3e, 0x20, 0x4f, 0x3f, 0x53, 0x0d, 0x72, 0x66, 0xe8, 0x2a, 0x67, 0xc7, 0x3e, 0x68, 0x1c, 0x1c,
|
||||
0x9c, 0xee, 0x06, 0x49, 0x33, 0x56, 0x59, 0xa6, 0xe4, 0xe6, 0x40, 0x2b, 0xab, 0xd8, 0x62, 0x26,
|
||||
0xd2, 0xd3, 0xdc, 0x14, 0xd1, 0x66, 0xb1, 0xb4, 0x7e, 0x8f, 0x4c, 0x75, 0x2c, 0xb7, 0xb9, 0x61,
|
||||
0xd7, 0x09, 0x01, 0xad, 0x95, 0xbe, 0x17, 0xab, 0x04, 0x56, 0xbd, 0xcb, 0xde, 0x95, 0xb9, 0xff,
|
||||
0xfd, 0x7b, 0xf3, 0x25, 0x7b, 0x36, 0x6f, 0x62, 0x5a, 0x4b, 0x25, 0x10, 0x05, 0x30, 0xfc, 0x64,
|
||||
0x2b, 0x64, 0x4a, 0x03, 0x37, 0x4a, 0xae, 0xd6, 0x2e, 0x7b, 0x57, 0x82, 0xa8, 0x8c, 0xd6, 0x5f,
|
||||
0x21, 0xcd, 0x5b, 0x70, 0x7e, 0x97, 0xa7, 0x39, 0x1c, 0x71, 0xa1, 0x19, 0x25, 0xfe, 0x7d, 0x38,
|
||||
0x77, 0xfc, 0x41, 0x84, 0x9f, 0x6c, 0x89, 0x4c, 0x9e, 0xe2, 0x72, 0xb9, 0xb1, 0x08, 0xd6, 0xd7,
|
||||
0x48, 0x7d, 0x3b, 0x55, 0x27, 0xe3, 0x55, 0xdc, 0xd1, 0x1c, 0xae, 0x5e, 0x25, 0x8d, 0x1b, 0x49,
|
||||
0xa2, 0xc1, 0x18, 0x36, 0x47, 0x6a, 0x62, 0x50, 0xf2, 0xd5, 0xc4, 0x80, 0x31, 0x52, 0x1f, 0x28,
|
||||
0x6d, 0x1d, 0x9b, 0x1f, 0xb9, 0xef, 0xf5, 0x87, 0x1e, 0x69, 0xec, 0x9b, 0xde, 0x36, 0x37, 0xc0,
|
||||
0x5e, 0x25, 0xd3, 0x99, 0xe9, 0xdd, 0xb3, 0xe7, 0x83, 0xe1, 0x2d, 0xd7, 0x5e, 0x7a, 0xcb, 0x7d,
|
||||
0xd3, 0x3b, 0x3e, 0x1f, 0x40, 0xd4, 0xc8, 0x8a, 0x0f, 0x54, 0x92, 0x99, 0x5e, 0x3b, 0x2c, 0x99,
|
||||
0x8b, 0x80, 0xad, 0x91, 0xc0, 0x8a, 0x0c, 0x8c, 0xe5, 0xd9, 0x60, 0xd5, 0xbf, 0xec, 0x5d, 0xa9,
|
||||
0x47, 0x63, 0x80, 0x5d, 0x22, 0xd3, 0x46, 0xe5, 0x3a, 0x86, 0x76, 0xb8, 0x5a, 0x77, 0xdb, 0x46,
|
||||
0xf1, 0xfa, 0x75, 0x12, 0xec, 0x9b, 0xde, 0x1e, 0xf0, 0x04, 0x34, 0xfb, 0x0f, 0xa9, 0x9f, 0x70,
|
||||
0x53, 0x28, 0x9a, 0xf9, 0x67, 0x45, 0x78, 0x83, 0xc8, 0x65, 0x6e, 0xbc, 0x5f, 0x27, 0xc1, 0xa8,
|
||||
0x12, 0x6c, 0x86, 0x34, 0x3a, 0x79, 0x1c, 0x83, 0x31, 0x74, 0x82, 0x2d, 0x92, 0xf9, 0x3b, 0x12,
|
||||
0xce, 0x06, 0x10, 0x5b, 0x48, 0x5c, 0x0e, 0xf5, 0xd8, 0x02, 0x99, 0x6d, 0x29, 0x29, 0x21, 0xb6,
|
||||
0x3b, 0x5c, 0xa4, 0x90, 0xd0, 0x1a, 0x5b, 0x22, 0xf4, 0x08, 0x74, 0x26, 0x8c, 0x11, 0x4a, 0x86,
|
||||
0x20, 0x05, 0x24, 0xd4, 0x67, 0x17, 0xc8, 0x62, 0x4b, 0xa5, 0x29, 0xc4, 0x56, 0x28, 0x79, 0xa0,
|
||||
0xec, 0xcd, 0x33, 0x61, 0xac, 0xa1, 0x75, 0xa4, 0x6d, 0xa7, 0x29, 0xf4, 0x78, 0x7a, 0x43, 0xf7,
|
||||
0xf2, 0x0c, 0xa4, 0xa5, 0x93, 0xc8, 0x51, 0x82, 0xa1, 0xc8, 0x40, 0x22, 0x13, 0x6d, 0x54, 0xd0,
|
||||
0xb6, 0x4c, 0xe0, 0x0c, 0xfd, 0xa3, 0xd3, 0xec, 0x22, 0x59, 0x2e, 0xd1, 0xca, 0x01, 0x3c, 0x03,
|
||||
0x1a, 0xb0, 0x79, 0x32, 0x53, 0x2e, 0x1d, 0x1f, 0x1e, 0xdd, 0xa2, 0xa4, 0xc2, 0x10, 0xa9, 0x07,
|
||||
0x11, 0xc4, 0x4a, 0x27, 0x74, 0xa6, 0x22, 0xe1, 0x2e, 0xc4, 0x56, 0xe9, 0x76, 0x48, 0x9b, 0x28,
|
||||
0xb8, 0x04, 0x3b, 0xc0, 0x75, 0xdc, 0x8f, 0xc0, 0xe4, 0xa9, 0xa5, 0xb3, 0x8c, 0x92, 0xe6, 0x8e,
|
||||
0x48, 0xe1, 0x40, 0xd9, 0x1d, 0x95, 0xcb, 0x84, 0xce, 0xb1, 0x39, 0x42, 0xf6, 0xc1, 0xf2, 0xd2,
|
||||
0x81, 0x79, 0x3c, 0xb6, 0xc5, 0xe3, 0x3e, 0x94, 0x00, 0x65, 0x2b, 0x84, 0xb5, 0xb8, 0x94, 0xca,
|
||||
0xb6, 0x34, 0x70, 0x0b, 0x3b, 0x2a, 0x4d, 0x40, 0xd3, 0x05, 0x94, 0xf3, 0x02, 0x2e, 0x52, 0xa0,
|
||||
0x6c, 0x9c, 0x1d, 0x42, 0x0a, 0xa3, 0xec, 0xc5, 0x71, 0x76, 0x89, 0x63, 0xf6, 0x12, 0x8a, 0xdf,
|
||||
0xce, 0x45, 0x9a, 0x38, 0x4b, 0x8a, 0xb2, 0x2c, 0xa3, 0xc6, 0x52, 0xfc, 0xc1, 0xed, 0x76, 0xe7,
|
||||
0x98, 0xae, 0xb0, 0x65, 0xb2, 0x50, 0x22, 0xfb, 0x60, 0xb5, 0x88, 0x9d, 0x79, 0x17, 0x50, 0xea,
|
||||
0x61, 0x6e, 0x0f, 0xbb, 0xfb, 0x90, 0x29, 0x7d, 0x4e, 0x57, 0xb1, 0xa0, 0x8e, 0x69, 0x58, 0x22,
|
||||
0x7a, 0x91, 0x31, 0x32, 0x1b, 0x86, 0x11, 0xbc, 0x9d, 0x83, 0xb1, 0x11, 0x8f, 0x81, 0xfe, 0xd2,
|
||||
0xd8, 0x78, 0x83, 0x10, 0x97, 0x86, 0x63, 0x0e, 0x8c, 0x91, 0xb9, 0x71, 0x74, 0xa0, 0x24, 0xd0,
|
||||
0x09, 0xd6, 0x24, 0xd3, 0x77, 0xa4, 0x30, 0x26, 0x87, 0x84, 0x7a, 0x68, 0x51, 0x5b, 0x1e, 0x69,
|
||||
0xd5, 0xc3, 0xe9, 0xa2, 0x35, 0x5c, 0xdd, 0x11, 0x52, 0x98, 0xbe, 0x6b, 0x0e, 0x42, 0xa6, 0x4a,
|
||||
0xaf, 0xea, 0x1b, 0x5d, 0xd2, 0xec, 0x40, 0x0f, 0xfb, 0xa0, 0xe0, 0x5e, 0x22, 0xb4, 0x1a, 0x8f,
|
||||
0xd9, 0x47, 0x0a, 0x3d, 0xec, 0xd3, 0x5d, 0xad, 0x1e, 0x08, 0xd9, 0xa3, 0x35, 0x24, 0xeb, 0x00,
|
||||
0x4f, 0x1d, 0xf1, 0x0c, 0x69, 0xec, 0xa4, 0xb9, 0x3b, 0xa5, 0xee, 0xce, 0xc4, 0x00, 0xd3, 0x26,
|
||||
0x37, 0xde, 0x99, 0x76, 0xd3, 0xeb, 0x86, 0x70, 0x96, 0x04, 0x77, 0x64, 0x02, 0x5d, 0x21, 0x21,
|
||||
0xa1, 0x13, 0xce, 0x68, 0x57, 0x90, 0x71, 0x43, 0xd1, 0x04, 0x2f, 0x19, 0x6a, 0x35, 0xa8, 0x60,
|
||||
0x80, 0x6e, 0xed, 0x71, 0x53, 0x81, 0xba, 0x58, 0xbd, 0x10, 0x4c, 0xac, 0xc5, 0x49, 0x75, 0x7b,
|
||||
0x0f, 0xeb, 0xd4, 0xe9, 0xab, 0x07, 0x63, 0xcc, 0xd0, 0x3e, 0x9e, 0xb4, 0x0b, 0xb6, 0x73, 0x6e,
|
||||
0x2c, 0x64, 0x2d, 0x25, 0xbb, 0xa2, 0x67, 0xa8, 0xc0, 0x93, 0x6e, 0x2b, 0x9e, 0x54, 0xb6, 0xbf,
|
||||
0x85, 0xf5, 0x8b, 0x20, 0x05, 0x6e, 0xaa, 0xac, 0xf7, 0xd9, 0x12, 0x99, 0x2f, 0xa4, 0x1e, 0x71,
|
||||
0x6d, 0x85, 0x03, 0xbf, 0xf1, 0x5c, 0xc5, 0xb4, 0x1a, 0x8c, 0xb1, 0x6f, 0x71, 0x52, 0x9b, 0x7b,
|
||||
0xdc, 0x8c, 0xa1, 0xef, 0x3c, 0xb6, 0x42, 0x16, 0x86, 0x52, 0xc7, 0xf8, 0xf7, 0x1e, 0x5b, 0x24,
|
||||
0x73, 0x28, 0x75, 0x84, 0x19, 0xfa, 0x83, 0x03, 0x51, 0x54, 0x05, 0xfc, 0xd1, 0x31, 0x94, 0xaa,
|
||||
0x2a, 0xf8, 0x4f, 0xee, 0x30, 0x64, 0x28, 0x0b, 0x67, 0xe8, 0x23, 0x0f, 0x95, 0x0e, 0x0f, 0x2b,
|
||||
0x61, 0xfa, 0xd8, 0x25, 0x22, 0xeb, 0x28, 0xf1, 0x89, 0x4b, 0x2c, 0x39, 0x47, 0xe8, 0x53, 0x87,
|
||||
0xee, 0x71, 0x99, 0xa8, 0x6e, 0x77, 0x84, 0x3e, 0xf3, 0xd8, 0x2a, 0x59, 0xc4, 0xed, 0xdb, 0x3c,
|
||||
0xe5, 0x32, 0x1e, 0xe7, 0x3f, 0xf7, 0x18, 0x25, 0x33, 0x85, 0x31, 0xae, 0x31, 0xe9, 0x07, 0x35,
|
||||
0x67, 0x4a, 0x29, 0xa0, 0xc0, 0x3e, 0xac, 0xb1, 0x39, 0x12, 0xa0, 0x51, 0x45, 0xfc, 0x51, 0x8d,
|
||||
0xcd, 0x90, 0xa9, 0xb6, 0x34, 0xa0, 0x2d, 0x7d, 0x17, 0x9b, 0x67, 0xaa, 0x98, 0x34, 0xfa, 0x1e,
|
||||
0xb6, 0xe8, 0xa4, 0x6b, 0x1e, 0xfa, 0xd0, 0x2d, 0x14, 0x6f, 0x02, 0xfd, 0xd5, 0x77, 0x57, 0xad,
|
||||
0x3e, 0x10, 0xbf, 0xf9, 0x78, 0xd2, 0x2e, 0xd8, 0xf1, 0x44, 0xd0, 0xdf, 0x7d, 0x76, 0x89, 0x2c,
|
||||
0x0f, 0x31, 0x37, 0xae, 0xa3, 0x59, 0xf8, 0xc3, 0x67, 0x6b, 0xe4, 0xc2, 0x2e, 0xd8, 0x71, 0x5d,
|
||||
0x71, 0x93, 0x30, 0x56, 0xc4, 0x86, 0xfe, 0xe9, 0xb3, 0x7f, 0x91, 0x95, 0x5d, 0xb0, 0x23, 0x7f,
|
||||
0x2b, 0x8b, 0x7f, 0xf9, 0x6c, 0x96, 0x4c, 0x47, 0x38, 0xcf, 0x70, 0x0a, 0xf4, 0x91, 0x8f, 0x45,
|
||||
0x1a, 0x86, 0xa5, 0x9c, 0xc7, 0x3e, 0x5a, 0xf7, 0x3a, 0xb7, 0x71, 0x3f, 0xcc, 0x5a, 0x7d, 0x2e,
|
||||
0x25, 0xa4, 0x86, 0x3e, 0xf1, 0xd9, 0x32, 0xa1, 0x11, 0x64, 0xea, 0x14, 0x2a, 0xf0, 0x53, 0x7c,
|
||||
0xa7, 0x99, 0x4b, 0x7e, 0x2d, 0x07, 0x7d, 0x3e, 0x5a, 0x78, 0xe6, 0xa3, 0xd5, 0x45, 0xfe, 0x8b,
|
||||
0x2b, 0xcf, 0x7d, 0xb4, 0xba, 0x74, 0xbe, 0x2d, 0xbb, 0x8a, 0xfe, 0x5c, 0x47, 0x55, 0xc7, 0x22,
|
||||
0x83, 0x63, 0x11, 0xdf, 0xa7, 0x1f, 0x07, 0xa8, 0xca, 0x6d, 0x3a, 0x50, 0x09, 0xa0, 0x7c, 0x43,
|
||||
0x3f, 0x09, 0xd0, 0x7a, 0x2c, 0x5d, 0x61, 0xfd, 0xa7, 0x2e, 0x2e, 0xdf, 0x98, 0x76, 0x48, 0x3f,
|
||||
0xc3, 0xb7, 0x9b, 0x94, 0xf1, 0x71, 0xe7, 0x90, 0x7e, 0x1e, 0xe0, 0x35, 0x6e, 0xa4, 0xa9, 0x8a,
|
||||
0xb9, 0x1d, 0x35, 0xd0, 0x17, 0x01, 0x76, 0x60, 0xe5, 0x79, 0x28, 0x8d, 0xf9, 0x32, 0xc0, 0xeb,
|
||||
0x95, 0xb8, 0x2b, 0x5b, 0x88, 0xcf, 0xc6, 0x57, 0x8e, 0x35, 0xe4, 0x96, 0xa3, 0x92, 0x63, 0x4b,
|
||||
0xbf, 0x0e, 0x36, 0xd6, 0x49, 0x23, 0x34, 0xa9, 0x7b, 0x05, 0x1a, 0xc4, 0x0f, 0x4d, 0x4a, 0x27,
|
||||
0xf0, 0xb1, 0xda, 0x56, 0x2a, 0xbd, 0x79, 0x36, 0xd0, 0x77, 0xff, 0x4b, 0xbd, 0xed, 0xff, 0xbf,
|
||||
0x79, 0xad, 0x27, 0x6c, 0x3f, 0x3f, 0xc1, 0x7f, 0xe6, 0x56, 0xf1, 0x13, 0xbd, 0x2a, 0x54, 0xf9,
|
||||
0xb5, 0x25, 0xa4, 0x05, 0x2d, 0x79, 0xba, 0xe5, 0xfe, 0xab, 0x5b, 0xc5, 0x7f, 0x75, 0x70, 0x72,
|
||||
0x32, 0xe5, 0xe2, 0x6b, 0x7f, 0x07, 0x00, 0x00, 0xff, 0xff, 0xc9, 0xcf, 0x59, 0x73, 0x31, 0x09,
|
||||
0x00, 0x00,
|
||||
}
|
||||
|
|
|
@ -174,7 +174,7 @@ message VchannelInfo {
|
|||
int64 collectionID = 1;
|
||||
string channelName = 2;
|
||||
internal.MsgPosition seek_position = 3;
|
||||
repeated CheckPoint checkPoints = 4;
|
||||
repeated SegmentInfo unflushedSegments = 4;
|
||||
repeated int64 flushedSegments = 5;
|
||||
}
|
||||
|
||||
|
@ -255,15 +255,15 @@ message SaveBinlogPathsRequest {
|
|||
int64 segmentID = 2;
|
||||
int64 collectionID = 3;
|
||||
repeated ID2PathList field2BinlogPaths = 4;
|
||||
repeated DDLBinlogMeta ddlBinlogPaths = 5;
|
||||
repeated CheckPoint checkPoints = 6;
|
||||
bool flushed = 7;
|
||||
repeated CheckPoint checkPoints = 5;
|
||||
bool flushed = 6;
|
||||
}
|
||||
|
||||
message CheckPoint {
|
||||
int64 segmentID = 1;
|
||||
internal.MsgPosition position = 2;
|
||||
int64 num_of_rows = 3;
|
||||
|
||||
}
|
||||
|
||||
message DataNodeTtMsg {
|
||||
|
|
|
@ -1227,7 +1227,7 @@ type VchannelInfo struct {
|
|||
CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
|
||||
ChannelName string `protobuf:"bytes,2,opt,name=channelName,proto3" json:"channelName,omitempty"`
|
||||
SeekPosition *internalpb.MsgPosition `protobuf:"bytes,3,opt,name=seek_position,json=seekPosition,proto3" json:"seek_position,omitempty"`
|
||||
CheckPoints []*CheckPoint `protobuf:"bytes,4,rep,name=checkPoints,proto3" json:"checkPoints,omitempty"`
|
||||
UnflushedSegments []*SegmentInfo `protobuf:"bytes,4,rep,name=unflushedSegments,proto3" json:"unflushedSegments,omitempty"`
|
||||
FlushedSegments []int64 `protobuf:"varint,5,rep,packed,name=flushedSegments,proto3" json:"flushedSegments,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
|
@ -1280,9 +1280,9 @@ func (m *VchannelInfo) GetSeekPosition() *internalpb.MsgPosition {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (m *VchannelInfo) GetCheckPoints() []*CheckPoint {
|
||||
func (m *VchannelInfo) GetUnflushedSegments() []*SegmentInfo {
|
||||
if m != nil {
|
||||
return m.CheckPoints
|
||||
return m.UnflushedSegments
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -1953,9 +1953,8 @@ type SaveBinlogPathsRequest struct {
|
|||
SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
|
||||
CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
|
||||
Field2BinlogPaths []*ID2PathList `protobuf:"bytes,4,rep,name=field2BinlogPaths,proto3" json:"field2BinlogPaths,omitempty"`
|
||||
DdlBinlogPaths []*DDLBinlogMeta `protobuf:"bytes,5,rep,name=ddlBinlogPaths,proto3" json:"ddlBinlogPaths,omitempty"`
|
||||
CheckPoints []*CheckPoint `protobuf:"bytes,6,rep,name=checkPoints,proto3" json:"checkPoints,omitempty"`
|
||||
Flushed bool `protobuf:"varint,7,opt,name=flushed,proto3" json:"flushed,omitempty"`
|
||||
CheckPoints []*CheckPoint `protobuf:"bytes,5,rep,name=checkPoints,proto3" json:"checkPoints,omitempty"`
|
||||
Flushed bool `protobuf:"varint,6,opt,name=flushed,proto3" json:"flushed,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
|
@ -2014,13 +2013,6 @@ func (m *SaveBinlogPathsRequest) GetField2BinlogPaths() []*ID2PathList {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (m *SaveBinlogPathsRequest) GetDdlBinlogPaths() []*DDLBinlogMeta {
|
||||
if m != nil {
|
||||
return m.DdlBinlogPaths
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *SaveBinlogPathsRequest) GetCheckPoints() []*CheckPoint {
|
||||
if m != nil {
|
||||
return m.CheckPoints
|
||||
|
@ -2510,138 +2502,137 @@ func init() {
|
|||
func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad6cfe64) }
|
||||
|
||||
var fileDescriptor_3385cd32ad6cfe64 = []byte{
|
||||
// 2083 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0xdb, 0x6f, 0x1b, 0x59,
|
||||
0x19, 0xcf, 0xd8, 0xce, 0xc5, 0x9f, 0xc7, 0x4e, 0x72, 0x1a, 0xb2, 0x96, 0xdb, 0xa6, 0xe9, 0xb0,
|
||||
0xdb, 0xcd, 0x76, 0x45, 0xb2, 0x75, 0x11, 0xb7, 0xb2, 0xa0, 0xa6, 0x6e, 0x83, 0x45, 0x53, 0xc2,
|
||||
0x49, 0x77, 0x57, 0x62, 0x85, 0xac, 0x89, 0xe7, 0xc4, 0x19, 0x32, 0x17, 0xef, 0x9c, 0x71, 0x92,
|
||||
0x3e, 0x75, 0xb5, 0xf0, 0x00, 0x08, 0xb1, 0xf0, 0x1f, 0x00, 0x12, 0x12, 0x12, 0x2f, 0xfc, 0x19,
|
||||
0xfc, 0x05, 0xfc, 0x25, 0xbc, 0x83, 0xce, 0x65, 0xee, 0x63, 0x7b, 0xe2, 0xb4, 0xca, 0x9b, 0xcf,
|
||||
0xf1, 0x77, 0x3b, 0xdf, 0xf9, 0xce, 0xef, 0xbb, 0x0c, 0x20, 0x43, 0xf7, 0xf5, 0x1e, 0x25, 0xde,
|
||||
0x99, 0xd9, 0x27, 0xdb, 0x43, 0xcf, 0xf5, 0x5d, 0xb4, 0x6a, 0x9b, 0xd6, 0xd9, 0x88, 0x8a, 0xd5,
|
||||
0x36, 0x23, 0x68, 0xa9, 0x7d, 0xd7, 0xb6, 0x5d, 0x47, 0x6c, 0xb5, 0x1a, 0xa6, 0xe3, 0x13, 0xcf,
|
||||
0xd1, 0x2d, 0xb9, 0x56, 0xe3, 0x0c, 0x2d, 0x95, 0xf6, 0x4f, 0x88, 0xad, 0x8b, 0x95, 0xf6, 0x1a,
|
||||
0x6e, 0x60, 0x32, 0x30, 0xa9, 0x4f, 0xbc, 0x17, 0xae, 0x41, 0x30, 0xf9, 0x62, 0x44, 0xa8, 0x8f,
|
||||
0x3e, 0x82, 0xca, 0x91, 0x4e, 0x49, 0x53, 0xd9, 0x54, 0xb6, 0x6a, 0xed, 0x5b, 0xdb, 0x09, 0x95,
|
||||
0x52, 0xd9, 0x3e, 0x1d, 0xec, 0xea, 0x94, 0x60, 0x4e, 0x89, 0xbe, 0x03, 0x8b, 0xba, 0x61, 0x78,
|
||||
0x84, 0xd2, 0x66, 0x69, 0x02, 0xd3, 0x63, 0x41, 0x83, 0x03, 0x62, 0xed, 0x6b, 0x05, 0xd6, 0x92,
|
||||
0x16, 0xd0, 0xa1, 0xeb, 0x50, 0x82, 0x76, 0xa1, 0x66, 0x3a, 0xa6, 0xdf, 0x1b, 0xea, 0x9e, 0x6e,
|
||||
0x53, 0x69, 0xc9, 0xdd, 0xa4, 0xd0, 0xf0, 0xa0, 0x5d, 0xc7, 0xf4, 0x0f, 0x38, 0x21, 0x06, 0x33,
|
||||
0xfc, 0x8d, 0x1e, 0xc2, 0x02, 0xf5, 0x75, 0x7f, 0x14, 0xd8, 0x74, 0x33, 0xd7, 0xa6, 0x43, 0x4e,
|
||||
0x82, 0x25, 0xa9, 0x76, 0x01, 0xea, 0x33, 0x6b, 0x44, 0x4f, 0x66, 0xf7, 0x05, 0x82, 0x8a, 0x71,
|
||||
0xd4, 0xed, 0x70, 0xa5, 0x65, 0xcc, 0x7f, 0x23, 0x0d, 0xd4, 0xbe, 0x6b, 0x59, 0xa4, 0xef, 0x9b,
|
||||
0xae, 0xd3, 0xed, 0x34, 0x2b, 0xfc, 0xbf, 0xc4, 0x9e, 0xf6, 0x67, 0x05, 0x56, 0x0e, 0xc9, 0xc0,
|
||||
0x26, 0x8e, 0xdf, 0xed, 0x04, 0xea, 0xd7, 0x60, 0xbe, 0xef, 0x8e, 0x1c, 0x9f, 0xeb, 0xaf, 0x63,
|
||||
0xb1, 0x40, 0x77, 0x41, 0xed, 0x9f, 0xe8, 0x8e, 0x43, 0xac, 0x9e, 0xa3, 0xdb, 0x84, 0xab, 0xaa,
|
||||
0xe2, 0x9a, 0xdc, 0x7b, 0xa1, 0xdb, 0x24, 0xa3, 0xb1, 0x9c, 0xd5, 0x88, 0x36, 0xa1, 0x36, 0xd4,
|
||||
0x3d, 0xdf, 0x4c, 0x18, 0x15, 0xdf, 0xd2, 0xfe, 0xa2, 0xc0, 0xfa, 0x63, 0x4a, 0xcd, 0x81, 0x93,
|
||||
0xb1, 0x6c, 0x1d, 0x16, 0x1c, 0xd7, 0x20, 0xdd, 0x0e, 0x37, 0xad, 0x8c, 0xe5, 0x0a, 0xdd, 0x84,
|
||||
0xea, 0x90, 0x10, 0xaf, 0xe7, 0xb9, 0x56, 0x60, 0xd8, 0x12, 0xdb, 0xc0, 0xae, 0x45, 0xd0, 0xcf,
|
||||
0x61, 0x95, 0xa6, 0x04, 0xd1, 0x66, 0x79, 0xb3, 0xbc, 0x55, 0x6b, 0x7f, 0x73, 0x3b, 0x13, 0xd9,
|
||||
0xdb, 0x69, 0xa5, 0x38, 0xcb, 0xad, 0x7d, 0x59, 0x82, 0x1b, 0x21, 0x9d, 0xb0, 0x95, 0xfd, 0x66,
|
||||
0x9e, 0xa3, 0x64, 0x10, 0x9a, 0x27, 0x16, 0x45, 0x3c, 0x17, 0xba, 0xbc, 0x1c, 0x77, 0x79, 0x81,
|
||||
0x1b, 0x4c, 0xfb, 0x73, 0x3e, 0xe3, 0x4f, 0x74, 0x07, 0x6a, 0xe4, 0x62, 0x68, 0x7a, 0xa4, 0xe7,
|
||||
0x9b, 0x36, 0x69, 0x2e, 0x6c, 0x2a, 0x5b, 0x15, 0x0c, 0x62, 0xeb, 0xa5, 0x69, 0x93, 0x58, 0xcc,
|
||||
0x2e, 0x16, 0x8f, 0xd9, 0xbf, 0x29, 0xf0, 0x4e, 0xe6, 0x96, 0xe4, 0x43, 0xc2, 0xb0, 0xc2, 0x4f,
|
||||
0x1e, 0x79, 0x86, 0xbd, 0x26, 0xe6, 0xf0, 0x7b, 0x93, 0x1c, 0x1e, 0x91, 0xe3, 0x0c, 0xff, 0x6c,
|
||||
0x0f, 0xeb, 0xaf, 0x0a, 0xdc, 0x38, 0x3c, 0x71, 0xcf, 0xa5, 0x0a, 0x3a, 0xfb, 0x03, 0x4b, 0x5f,
|
||||
0x45, 0x69, 0xfa, 0x55, 0x94, 0xb3, 0x57, 0x11, 0x3c, 0xd3, 0x4a, 0xf4, 0x4c, 0xb5, 0x53, 0x58,
|
||||
0x4b, 0x9a, 0x28, 0x9d, 0xb8, 0x01, 0x10, 0x06, 0x9e, 0x70, 0x5f, 0x19, 0xc7, 0x76, 0x66, 0x73,
|
||||
0xc8, 0x29, 0xbc, 0xb3, 0x47, 0x7c, 0xa9, 0x8b, 0xfd, 0x47, 0xae, 0xe0, 0x93, 0xa4, 0x85, 0xa5,
|
||||
0xb4, 0x85, 0xda, 0xbf, 0x4a, 0x21, 0xb8, 0x70, 0x55, 0x5d, 0xe7, 0xd8, 0x45, 0xb7, 0xa0, 0x1a,
|
||||
0x92, 0xc8, 0x67, 0x12, 0x6d, 0xa0, 0xef, 0xc2, 0x3c, 0xb3, 0x54, 0xbc, 0x91, 0x46, 0x1a, 0x7c,
|
||||
0x83, 0x33, 0xc5, 0x64, 0x62, 0x41, 0x8f, 0xba, 0xd0, 0xa0, 0xbe, 0xee, 0xf9, 0xbd, 0xa1, 0x4b,
|
||||
0xb9, 0xb7, 0xb9, 0xfb, 0x6b, 0x6d, 0x6d, 0x0c, 0x7c, 0xef, 0xd3, 0xc1, 0x81, 0xa4, 0xc4, 0x75,
|
||||
0xce, 0x19, 0x2c, 0xd1, 0x53, 0x50, 0x89, 0x63, 0x44, 0x82, 0x2a, 0x85, 0x05, 0xd5, 0x88, 0x63,
|
||||
0x84, 0x62, 0xa2, 0xfb, 0x99, 0x2f, 0x7e, 0x3f, 0x7f, 0x50, 0xa0, 0x99, 0xbd, 0x20, 0x19, 0x11,
|
||||
0x91, 0x44, 0xa5, 0xb0, 0x44, 0xf4, 0x48, 0x30, 0x11, 0x71, 0x41, 0x13, 0x21, 0x2f, 0xbc, 0x24,
|
||||
0x2c, 0x59, 0x34, 0x13, 0xbe, 0x11, 0x59, 0xc3, 0xff, 0x79, 0x6b, 0xc1, 0xf2, 0x6b, 0x05, 0xd6,
|
||||
0xd3, 0xba, 0xae, 0x72, 0xee, 0x6f, 0xc3, 0xbc, 0xe9, 0x1c, 0xbb, 0xc1, 0xb1, 0x37, 0x26, 0x00,
|
||||
0x0f, 0xd3, 0x25, 0x88, 0x35, 0x1b, 0x6e, 0xee, 0x11, 0xbf, 0xeb, 0x50, 0xe2, 0xf9, 0xbb, 0xa6,
|
||||
0x63, 0xb9, 0x83, 0x03, 0xdd, 0x3f, 0xb9, 0xc2, 0x1b, 0x49, 0x84, 0x7b, 0x29, 0x15, 0xee, 0xda,
|
||||
0x3f, 0x14, 0xb8, 0x95, 0xaf, 0x4f, 0x1e, 0xbd, 0x05, 0x4b, 0xc7, 0x26, 0xb1, 0x8c, 0x08, 0x02,
|
||||
0xc2, 0x35, 0x7b, 0x2b, 0x43, 0x46, 0x2c, 0x4f, 0x38, 0xae, 0x50, 0x39, 0xf4, 0x3d, 0xd3, 0x19,
|
||||
0x3c, 0x37, 0xa9, 0x8f, 0x05, 0x7d, 0xcc, 0x9f, 0xe5, 0xe2, 0x91, 0xf9, 0x1b, 0x11, 0x99, 0xc2,
|
||||
0xd4, 0x27, 0x22, 0x75, 0xd1, 0xb7, 0x5b, 0xb0, 0xe4, 0x94, 0x0f, 0xda, 0xef, 0x15, 0xd8, 0xd8,
|
||||
0x23, 0xfe, 0x93, 0x70, 0x8f, 0x99, 0x69, 0x52, 0xdf, 0xec, 0x5f, 0x83, 0x31, 0x5f, 0x2b, 0x70,
|
||||
0x67, 0xac, 0x31, 0xf2, 0x06, 0x25, 0xa2, 0x05, 0x09, 0x30, 0x1f, 0xd1, 0x7e, 0x4a, 0x5e, 0x7d,
|
||||
0xaa, 0x5b, 0x23, 0x72, 0xa0, 0x9b, 0x9e, 0x40, 0xb4, 0x19, 0xf1, 0xfd, 0x9f, 0x0a, 0xdc, 0xde,
|
||||
0x23, 0xac, 0x18, 0x15, 0x39, 0xe7, 0x1a, 0xbd, 0x53, 0xa0, 0xd2, 0xfb, 0xa3, 0xb8, 0xcc, 0x5c,
|
||||
0x6b, 0xaf, 0xc5, 0x7d, 0x1b, 0xfc, 0x39, 0xc6, 0x70, 0x41, 0x06, 0xba, 0x74, 0x9e, 0xf6, 0xdb,
|
||||
0x12, 0xa8, 0x9f, 0xca, 0xba, 0x8d, 0x67, 0xb3, 0xb4, 0x1f, 0x94, 0x7c, 0x3f, 0xc4, 0x4a, 0xbd,
|
||||
0xbc, 0xea, 0x6f, 0x0f, 0xea, 0x94, 0x90, 0xd3, 0x59, 0x72, 0x97, 0xca, 0x18, 0xc3, 0x9c, 0xf3,
|
||||
0x63, 0xa6, 0x8a, 0xf4, 0x4f, 0x0f, 0x5c, 0x93, 0xd5, 0x5c, 0x15, 0xee, 0xb3, 0xdb, 0x39, 0xd0,
|
||||
0xf7, 0x24, 0xa4, 0xc2, 0x71, 0x0e, 0xb4, 0x05, 0xcb, 0xc7, 0xac, 0x13, 0x21, 0x46, 0x50, 0x8f,
|
||||
0x34, 0xe7, 0x39, 0xec, 0xa4, 0xb7, 0xb5, 0xdf, 0x29, 0xb0, 0xfe, 0x99, 0xee, 0xf7, 0x4f, 0x3a,
|
||||
0xf6, 0xd5, 0xd1, 0xe0, 0x63, 0xa8, 0x9e, 0x49, 0x87, 0x04, 0x70, 0x76, 0x27, 0xc7, 0xea, 0xb8,
|
||||
0xeb, 0x71, 0xc4, 0xc1, 0x3a, 0x86, 0x35, 0xde, 0x40, 0x5d, 0xbd, 0xce, 0x9b, 0x35, 0xd8, 0x93,
|
||||
0xe9, 0xad, 0x92, 0x49, 0x6f, 0x17, 0x00, 0xd2, 0xb8, 0x7d, 0x3a, 0x98, 0xc1, 0xae, 0xef, 0xc1,
|
||||
0xa2, 0x94, 0x26, 0xe3, 0x79, 0x5a, 0x42, 0x0b, 0xc8, 0xb5, 0x43, 0x58, 0x97, 0xfb, 0xcf, 0x58,
|
||||
0xe6, 0x10, 0x59, 0x66, 0x9f, 0xf8, 0x3a, 0x6a, 0xc2, 0xa2, 0x4c, 0x26, 0x32, 0x6e, 0x83, 0x25,
|
||||
0x6b, 0x19, 0x8e, 0x38, 0x5d, 0x8f, 0x65, 0x0c, 0x19, 0xb2, 0x70, 0x14, 0x26, 0x28, 0xed, 0x97,
|
||||
0x50, 0xef, 0x74, 0x9e, 0xc7, 0x64, 0xdd, 0x83, 0x65, 0xc3, 0xb0, 0x7a, 0x71, 0x2e, 0x85, 0x73,
|
||||
0xd5, 0x0d, 0xc3, 0x8a, 0x32, 0x1b, 0x7a, 0x17, 0x1a, 0x3e, 0xed, 0x65, 0x85, 0xab, 0x3e, 0x8d,
|
||||
0xa8, 0xb4, 0x7d, 0x68, 0x70, 0x63, 0xf9, 0xa5, 0x4e, 0xb1, 0xf5, 0x2e, 0xa8, 0x31, 0x71, 0x22,
|
||||
0x7c, 0xaa, 0xb8, 0x16, 0x19, 0x4b, 0x59, 0xd2, 0x08, 0x0a, 0xd1, 0x48, 0xe2, 0xe4, 0x42, 0xf4,
|
||||
0x36, 0x80, 0x49, 0x7b, 0x32, 0xe8, 0xb9, 0x8d, 0x4b, 0xb8, 0x6a, 0xd2, 0x67, 0x62, 0x03, 0x7d,
|
||||
0x1f, 0x16, 0xb8, 0x7e, 0xf1, 0x3c, 0x32, 0xb8, 0xc4, 0x6f, 0x23, 0x79, 0x02, 0x2c, 0x19, 0xb4,
|
||||
0x4f, 0x40, 0xed, 0x74, 0x9e, 0x47, 0x76, 0x14, 0x81, 0x90, 0x02, 0x67, 0x7c, 0x0d, 0x8d, 0x28,
|
||||
0x0f, 0x71, 0x6c, 0x6a, 0x40, 0x29, 0x14, 0x57, 0xea, 0x76, 0xd0, 0xc7, 0xb0, 0x20, 0x06, 0x31,
|
||||
0x32, 0x82, 0xde, 0x4b, 0xda, 0x2c, 0x87, 0x34, 0xb1, 0x64, 0xc6, 0x37, 0xb0, 0x64, 0x62, 0x11,
|
||||
0x1e, 0x62, 0xb7, 0xe8, 0x9f, 0xcb, 0x38, 0xb6, 0xa3, 0xfd, 0xb7, 0x04, 0xb5, 0x58, 0x00, 0x66,
|
||||
0xd4, 0xbf, 0x99, 0x0e, 0xea, 0x3d, 0x68, 0x98, 0xbc, 0x04, 0xe9, 0xc9, 0xd7, 0xcf, 0xf3, 0x4a,
|
||||
0x15, 0xd7, 0xcd, 0x78, 0x61, 0x82, 0x36, 0xa0, 0xe6, 0x8c, 0xec, 0x9e, 0x7b, 0xdc, 0xf3, 0xdc,
|
||||
0x73, 0x2a, 0xbb, 0xe2, 0xaa, 0x33, 0xb2, 0x7f, 0x76, 0x8c, 0xdd, 0x73, 0x1a, 0xf5, 0x19, 0x0b,
|
||||
0x97, 0xec, 0x33, 0x9e, 0x82, 0x6a, 0xd8, 0x56, 0x84, 0xd4, 0x8b, 0xc5, 0x9b, 0x03, 0xc3, 0xb6,
|
||||
0x42, 0xa0, 0xde, 0x80, 0x9a, 0xad, 0x5f, 0x30, 0xe3, 0x7a, 0xce, 0xc8, 0x6e, 0x2e, 0x09, 0xfb,
|
||||
0x6c, 0xfd, 0x02, 0xbb, 0xe7, 0x2f, 0x46, 0x36, 0xda, 0x82, 0x15, 0x4b, 0xa7, 0x7e, 0x2f, 0xde,
|
||||
0xb8, 0x57, 0x79, 0xe3, 0xde, 0x60, 0xfb, 0x4f, 0xc3, 0xe6, 0x5d, 0x7b, 0x08, 0xb5, 0x6e, 0xa7,
|
||||
0xcd, 0x62, 0x80, 0x95, 0x78, 0x19, 0xaf, 0xaf, 0xc1, 0xfc, 0x41, 0x2c, 0x64, 0xc4, 0x82, 0x01,
|
||||
0xa6, 0x1a, 0xd8, 0xc2, 0x92, 0x66, 0x4e, 0xfb, 0xa4, 0xbc, 0xa9, 0xf6, 0xa9, 0x34, 0x53, 0xfb,
|
||||
0xa4, 0xfd, 0xaf, 0x04, 0xeb, 0x87, 0xfa, 0x19, 0x79, 0xfb, 0x55, 0x78, 0x21, 0x7c, 0x7f, 0x0e,
|
||||
0xab, 0xfc, 0xfd, 0xb6, 0x63, 0xf6, 0xc8, 0xfc, 0x9a, 0x87, 0xc4, 0xb1, 0x2b, 0xc1, 0x59, 0x46,
|
||||
0xf4, 0x13, 0x68, 0x24, 0x60, 0x31, 0x80, 0x91, 0xcd, 0x1c, 0x51, 0x09, 0x9c, 0xc5, 0x29, 0xbe,
|
||||
0x74, 0xc6, 0x5f, 0xb8, 0x74, 0xc6, 0x67, 0xc0, 0x2a, 0x51, 0x6e, 0x91, 0xa3, 0x5c, 0xb0, 0x64,
|
||||
0xa8, 0x09, 0x11, 0xd7, 0x14, 0xbc, 0xfc, 0x11, 0x2c, 0xcd, 0x70, 0xe3, 0x21, 0x4f, 0xfa, 0xc1,
|
||||
0x96, 0x53, 0x0f, 0x56, 0xfb, 0x4a, 0x81, 0x7a, 0x47, 0xf7, 0xf5, 0x17, 0xae, 0x41, 0x5e, 0xce,
|
||||
0x98, 0x43, 0x0b, 0xcc, 0xe1, 0x6e, 0x41, 0x95, 0xbd, 0x35, 0xea, 0xeb, 0xf6, 0x90, 0x1b, 0x51,
|
||||
0xc1, 0xd1, 0x06, 0xeb, 0x51, 0xeb, 0x12, 0x61, 0x44, 0xe1, 0xc8, 0xca, 0x05, 0x2e, 0x4a, 0xe4,
|
||||
0x3a, 0xfe, 0x1b, 0xfd, 0x20, 0x39, 0xc3, 0x78, 0x37, 0xf7, 0x32, 0xb8, 0x10, 0x5e, 0x3f, 0x25,
|
||||
0xe0, 0xa5, 0x48, 0xd7, 0xf1, 0xa5, 0x02, 0x6a, 0xe0, 0x0a, 0x8e, 0xb4, 0xcd, 0x68, 0x10, 0x2e,
|
||||
0xec, 0x08, 0x96, 0xec, 0x9f, 0x33, 0xe2, 0xd1, 0xe0, 0x52, 0xca, 0x38, 0x58, 0xa2, 0x1f, 0xc2,
|
||||
0x52, 0x58, 0x70, 0x95, 0xc7, 0xc6, 0x5e, 0xe2, 0xb0, 0x38, 0xe4, 0xd0, 0x3c, 0x68, 0x48, 0x70,
|
||||
0x14, 0xb1, 0x48, 0xa7, 0x44, 0xc7, 0x2e, 0xa8, 0xc7, 0x51, 0xf1, 0x31, 0xa9, 0x27, 0x8f, 0xd5,
|
||||
0x28, 0x38, 0xc1, 0xa3, 0x3d, 0x86, 0x5a, 0xec, 0xcf, 0x09, 0x05, 0x41, 0x13, 0x16, 0x8f, 0x62,
|
||||
0x7a, 0xaa, 0x38, 0x58, 0x6a, 0xff, 0x56, 0xf8, 0xf8, 0x0b, 0x93, 0xbe, 0x7b, 0x46, 0xbc, 0x57,
|
||||
0x57, 0x1f, 0x32, 0x3c, 0x8a, 0x79, 0xb1, 0x60, 0xd9, 0x1a, 0x32, 0xa0, 0x47, 0x91, 0x9d, 0xe5,
|
||||
0xb1, 0x45, 0x44, 0xd2, 0xcd, 0xd1, 0x51, 0xfe, 0x24, 0xc6, 0x25, 0xc9, 0xa3, 0x5c, 0xeb, 0x70,
|
||||
0xf3, 0xfe, 0x03, 0x58, 0xcd, 0xc4, 0x35, 0x6a, 0x00, 0x7c, 0xe2, 0xf4, 0x5d, 0x7b, 0x68, 0x11,
|
||||
0x9f, 0xac, 0xcc, 0x21, 0x15, 0x96, 0x9e, 0x04, 0x2b, 0xa5, 0xfd, 0x9f, 0x3a, 0xd4, 0x58, 0x28,
|
||||
0x1f, 0x8a, 0xcf, 0x4d, 0x68, 0x08, 0x88, 0xf7, 0xd3, 0xf6, 0xd0, 0x75, 0xc2, 0xf9, 0x17, 0xfa,
|
||||
0x68, 0x0c, 0x92, 0x64, 0x49, 0xa5, 0x0b, 0x5a, 0xf7, 0xc6, 0x70, 0xa4, 0xc8, 0xb5, 0x39, 0x64,
|
||||
0x73, 0x8d, 0x2c, 0x93, 0xbe, 0x34, 0xfb, 0xa7, 0x41, 0xf9, 0x30, 0x41, 0x63, 0x8a, 0x34, 0xd0,
|
||||
0x98, 0x1a, 0xab, 0xc9, 0x85, 0x98, 0xbd, 0x04, 0x31, 0xa6, 0xcd, 0xa1, 0x2f, 0x60, 0x8d, 0x35,
|
||||
0x98, 0x61, 0x9f, 0x1b, 0x28, 0x6c, 0x8f, 0x57, 0x98, 0x21, 0xbe, 0xa4, 0x4a, 0x1d, 0xd4, 0xf8,
|
||||
0xd7, 0x2e, 0x94, 0x37, 0x82, 0xcf, 0xf9, 0x20, 0xd7, 0x7a, 0x7f, 0x2a, 0x5d, 0xa8, 0x62, 0x0f,
|
||||
0xe6, 0x79, 0x3d, 0x8b, 0xf2, 0xa2, 0x3f, 0xfe, 0x65, 0xab, 0x35, 0xe9, 0x55, 0x69, 0x73, 0xe8,
|
||||
0x57, 0xb0, 0x9c, 0xfa, 0xa6, 0x80, 0x3e, 0xc8, 0x11, 0x99, 0xff, 0x75, 0xa8, 0x75, 0xbf, 0x08,
|
||||
0x69, 0xdc, 0x2f, 0xf1, 0xb9, 0x7b, 0xae, 0x5f, 0x72, 0xbe, 0x1d, 0xe4, 0xfa, 0x25, 0x6f, 0x80,
|
||||
0xaf, 0xcd, 0xa1, 0x01, 0x34, 0x92, 0xe3, 0x04, 0xb4, 0x95, 0xc3, 0x9c, 0x3b, 0x61, 0x6d, 0x7d,
|
||||
0x50, 0x80, 0x32, 0x54, 0x64, 0xc3, 0x4a, 0x7a, 0x6a, 0x8c, 0xee, 0x4f, 0x14, 0x90, 0x7c, 0x2f,
|
||||
0x1f, 0x16, 0xa2, 0x0d, 0xd5, 0xbd, 0xe2, 0x51, 0x9c, 0x99, 0x5a, 0xa2, 0xed, 0x7c, 0x31, 0xe3,
|
||||
0xc6, 0xa9, 0xad, 0x9d, 0xc2, 0xf4, 0xa1, 0x6a, 0x02, 0xab, 0x99, 0x29, 0x24, 0xfa, 0x70, 0x92,
|
||||
0x9c, 0xd4, 0x74, 0xa2, 0x35, 0x7d, 0x4e, 0xaa, 0xcd, 0xa1, 0xaf, 0x44, 0xa6, 0xc8, 0x9b, 0xec,
|
||||
0xa1, 0x07, 0xf9, 0xda, 0x26, 0x8c, 0x24, 0x5b, 0xed, 0xcb, 0xb0, 0x84, 0x67, 0x7d, 0xcd, 0x21,
|
||||
0x3e, 0x67, 0x3a, 0x96, 0xc6, 0xa7, 0x40, 0xde, 0xf8, 0xb1, 0x5f, 0xeb, 0xc1, 0x25, 0x38, 0x42,
|
||||
0x03, 0xdc, 0xf4, 0xf8, 0x3f, 0x80, 0xab, 0x9d, 0xa9, 0xc1, 0x39, 0x1b, 0x56, 0x7d, 0x0e, 0xcb,
|
||||
0xa9, 0x9a, 0x3f, 0xf7, 0xfd, 0xe7, 0xf7, 0x05, 0x05, 0xc0, 0x25, 0x95, 0x31, 0xd1, 0x98, 0x47,
|
||||
0x96, 0x93, 0x55, 0x5b, 0xf7, 0x8b, 0x90, 0x06, 0x07, 0x69, 0xff, 0xbd, 0x0c, 0x4b, 0x41, 0x8d,
|
||||
0x76, 0x0d, 0x59, 0xed, 0x1a, 0xd2, 0xcc, 0xe7, 0xb0, 0x9c, 0x1a, 0x07, 0xe6, 0x7a, 0x37, 0x7f,
|
||||
0x64, 0x38, 0xed, 0xea, 0x3e, 0x83, 0x7a, 0x62, 0xbe, 0x87, 0xde, 0x1f, 0x97, 0x68, 0xd2, 0x68,
|
||||
0x3d, 0x59, 0xf0, 0xee, 0xc3, 0x5f, 0x3c, 0x18, 0x98, 0xfe, 0xc9, 0xe8, 0x88, 0xfd, 0xb3, 0x23,
|
||||
0x48, 0xbf, 0x65, 0xba, 0xf2, 0xd7, 0x4e, 0xe0, 0xa0, 0x1d, 0xce, 0xbd, 0xc3, 0xd4, 0x0c, 0x8f,
|
||||
0x8e, 0x16, 0xf8, 0xea, 0xe1, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0x11, 0xb6, 0xf8, 0x23, 0x2b,
|
||||
0x23, 0x00, 0x00,
|
||||
// 2069 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x19, 0x5b, 0x6f, 0x1b, 0x59,
|
||||
0x39, 0x63, 0x3b, 0x17, 0x7f, 0x1e, 0x3b, 0xc9, 0x69, 0xc8, 0x5a, 0x6e, 0x9b, 0xa6, 0xc3, 0x6e,
|
||||
0x37, 0xdb, 0x15, 0xc9, 0xd6, 0x45, 0xdc, 0xca, 0x82, 0x9a, 0xba, 0x8d, 0x2c, 0x92, 0x12, 0x4e,
|
||||
0xba, 0xbb, 0x12, 0x2b, 0x64, 0x4d, 0x3c, 0x27, 0xce, 0x90, 0xb9, 0x78, 0xe7, 0x8c, 0xd3, 0xf4,
|
||||
0xa9, 0xab, 0x85, 0x17, 0x10, 0x62, 0xe1, 0x81, 0x77, 0x40, 0x42, 0x42, 0xe2, 0x85, 0x17, 0xfe,
|
||||
0x03, 0xbf, 0x80, 0x5f, 0xc2, 0x0f, 0x58, 0x9d, 0xcb, 0xdc, 0xc7, 0xf6, 0xc4, 0x69, 0x95, 0x37,
|
||||
0x9f, 0xe3, 0xef, 0x76, 0xbe, 0xfb, 0xf7, 0x0d, 0x20, 0x43, 0xf7, 0xf5, 0x1e, 0x25, 0xde, 0xb9,
|
||||
0xd9, 0x27, 0xdb, 0x43, 0xcf, 0xf5, 0x5d, 0xb4, 0x6a, 0x9b, 0xd6, 0xf9, 0x88, 0x8a, 0xd3, 0x36,
|
||||
0x03, 0x68, 0xa9, 0x7d, 0xd7, 0xb6, 0x5d, 0x47, 0x5c, 0xb5, 0x1a, 0xa6, 0xe3, 0x13, 0xcf, 0xd1,
|
||||
0x2d, 0x79, 0x56, 0xe3, 0x08, 0x2d, 0x95, 0xf6, 0x4f, 0x89, 0xad, 0x8b, 0x93, 0xf6, 0x1a, 0x6e,
|
||||
0x60, 0x32, 0x30, 0xa9, 0x4f, 0xbc, 0xe7, 0xae, 0x41, 0x30, 0xf9, 0x62, 0x44, 0xa8, 0x8f, 0x3e,
|
||||
0x82, 0xca, 0xb1, 0x4e, 0x49, 0x53, 0xd9, 0x54, 0xb6, 0x6a, 0xed, 0x5b, 0xdb, 0x09, 0x96, 0x92,
|
||||
0xd9, 0x01, 0x1d, 0xec, 0xea, 0x94, 0x60, 0x0e, 0x89, 0xbe, 0x07, 0x8b, 0xba, 0x61, 0x78, 0x84,
|
||||
0xd2, 0x66, 0x69, 0x02, 0xd2, 0x63, 0x01, 0x83, 0x03, 0x60, 0xed, 0x6b, 0x05, 0xd6, 0x92, 0x12,
|
||||
0xd0, 0xa1, 0xeb, 0x50, 0x82, 0x76, 0xa1, 0x66, 0x3a, 0xa6, 0xdf, 0x1b, 0xea, 0x9e, 0x6e, 0x53,
|
||||
0x29, 0xc9, 0xdd, 0x24, 0xd1, 0xf0, 0xa1, 0x5d, 0xc7, 0xf4, 0x0f, 0x39, 0x20, 0x06, 0x33, 0xfc,
|
||||
0x8d, 0x1e, 0xc2, 0x02, 0xf5, 0x75, 0x7f, 0x14, 0xc8, 0x74, 0x33, 0x57, 0xa6, 0x23, 0x0e, 0x82,
|
||||
0x25, 0xa8, 0x76, 0x01, 0xea, 0x33, 0x6b, 0x44, 0x4f, 0x67, 0xd7, 0x05, 0x82, 0x8a, 0x71, 0xdc,
|
||||
0xed, 0x70, 0xa6, 0x65, 0xcc, 0x7f, 0x23, 0x0d, 0xd4, 0xbe, 0x6b, 0x59, 0xa4, 0xef, 0x9b, 0xae,
|
||||
0xd3, 0xed, 0x34, 0x2b, 0xfc, 0xbf, 0xc4, 0x9d, 0xf6, 0x67, 0x05, 0x56, 0x8e, 0xc8, 0xc0, 0x26,
|
||||
0x8e, 0xdf, 0xed, 0x04, 0xec, 0xd7, 0x60, 0xbe, 0xef, 0x8e, 0x1c, 0x9f, 0xf3, 0xaf, 0x63, 0x71,
|
||||
0x40, 0x77, 0x41, 0xed, 0x9f, 0xea, 0x8e, 0x43, 0xac, 0x9e, 0xa3, 0xdb, 0x84, 0xb3, 0xaa, 0xe2,
|
||||
0x9a, 0xbc, 0x7b, 0xae, 0xdb, 0x24, 0xc3, 0xb1, 0x9c, 0xe5, 0x88, 0x36, 0xa1, 0x36, 0xd4, 0x3d,
|
||||
0xdf, 0x4c, 0x08, 0x15, 0xbf, 0xd2, 0xfe, 0xaa, 0xc0, 0xfa, 0x63, 0x4a, 0xcd, 0x81, 0x93, 0x91,
|
||||
0x6c, 0x1d, 0x16, 0x1c, 0xd7, 0x20, 0xdd, 0x0e, 0x17, 0xad, 0x8c, 0xe5, 0x09, 0xdd, 0x84, 0xea,
|
||||
0x90, 0x10, 0xaf, 0xe7, 0xb9, 0x56, 0x20, 0xd8, 0x12, 0xbb, 0xc0, 0xae, 0x45, 0xd0, 0x2f, 0x60,
|
||||
0x95, 0xa6, 0x08, 0xd1, 0x66, 0x79, 0xb3, 0xbc, 0x55, 0x6b, 0x7f, 0x7b, 0x3b, 0xe3, 0xd9, 0xdb,
|
||||
0x69, 0xa6, 0x38, 0x8b, 0xad, 0x7d, 0x59, 0x82, 0x1b, 0x21, 0x9c, 0x90, 0x95, 0xfd, 0x66, 0x9a,
|
||||
0xa3, 0x64, 0x10, 0x8a, 0x27, 0x0e, 0x45, 0x34, 0x17, 0xaa, 0xbc, 0x1c, 0x57, 0x79, 0x01, 0x0b,
|
||||
0xa6, 0xf5, 0x39, 0x9f, 0xd1, 0x27, 0xba, 0x03, 0x35, 0x72, 0x31, 0x34, 0x3d, 0xd2, 0xf3, 0x4d,
|
||||
0x9b, 0x34, 0x17, 0x36, 0x95, 0xad, 0x0a, 0x06, 0x71, 0xf5, 0xc2, 0xb4, 0x49, 0xcc, 0x67, 0x17,
|
||||
0x8b, 0xfb, 0xec, 0xdf, 0x15, 0x78, 0x27, 0x63, 0x25, 0x19, 0x48, 0x18, 0x56, 0xf8, 0xcb, 0x23,
|
||||
0xcd, 0xb0, 0x68, 0x62, 0x0a, 0xbf, 0x37, 0x49, 0xe1, 0x11, 0x38, 0xce, 0xe0, 0xcf, 0x16, 0x58,
|
||||
0x7f, 0x53, 0xe0, 0xc6, 0xd1, 0xa9, 0xfb, 0x52, 0xb2, 0xa0, 0xb3, 0x07, 0x58, 0xda, 0x14, 0xa5,
|
||||
0xe9, 0xa6, 0x28, 0x67, 0x4d, 0x11, 0x84, 0x69, 0x25, 0x0a, 0x53, 0xed, 0x0c, 0xd6, 0x92, 0x22,
|
||||
0x4a, 0x25, 0x6e, 0x00, 0x84, 0x8e, 0x27, 0xd4, 0x57, 0xc6, 0xb1, 0x9b, 0xd9, 0x14, 0x72, 0x06,
|
||||
0xef, 0xec, 0x11, 0x5f, 0xf2, 0x62, 0xff, 0x91, 0x2b, 0xe8, 0x24, 0x29, 0x61, 0x29, 0x2d, 0xa1,
|
||||
0xf6, 0xef, 0x52, 0x98, 0x5c, 0x38, 0xab, 0xae, 0x73, 0xe2, 0xa2, 0x5b, 0x50, 0x0d, 0x41, 0x64,
|
||||
0x98, 0x44, 0x17, 0xe8, 0xfb, 0x30, 0xcf, 0x24, 0x15, 0x31, 0xd2, 0x48, 0x27, 0xdf, 0xe0, 0x4d,
|
||||
0x31, 0x9a, 0x58, 0xc0, 0xa3, 0x2e, 0x34, 0xa8, 0xaf, 0x7b, 0x7e, 0x6f, 0xe8, 0x52, 0xae, 0x6d,
|
||||
0xae, 0xfe, 0x5a, 0x5b, 0x1b, 0x93, 0xbe, 0x0f, 0xe8, 0xe0, 0x50, 0x42, 0xe2, 0x3a, 0xc7, 0x0c,
|
||||
0x8e, 0xe8, 0x29, 0xa8, 0xc4, 0x31, 0x22, 0x42, 0x95, 0xc2, 0x84, 0x6a, 0xc4, 0x31, 0x42, 0x32,
|
||||
0x91, 0x7d, 0xe6, 0x8b, 0xdb, 0xe7, 0x0f, 0x0a, 0x34, 0xb3, 0x06, 0x92, 0x1e, 0x11, 0x51, 0x54,
|
||||
0x0a, 0x53, 0x44, 0x8f, 0x04, 0x12, 0x11, 0x06, 0x9a, 0x98, 0xf2, 0x42, 0x23, 0x61, 0x89, 0xa2,
|
||||
0x99, 0xf0, 0xad, 0x48, 0x1a, 0xfe, 0xcf, 0x5b, 0x73, 0x96, 0xdf, 0x28, 0xb0, 0x9e, 0xe6, 0x75,
|
||||
0x95, 0x77, 0x7f, 0x17, 0xe6, 0x4d, 0xe7, 0xc4, 0x0d, 0x9e, 0xbd, 0x31, 0x21, 0xf1, 0x30, 0x5e,
|
||||
0x02, 0x58, 0xb3, 0xe1, 0xe6, 0x1e, 0xf1, 0xbb, 0x0e, 0x25, 0x9e, 0xbf, 0x6b, 0x3a, 0x96, 0x3b,
|
||||
0x38, 0xd4, 0xfd, 0xd3, 0x2b, 0xc4, 0x48, 0xc2, 0xdd, 0x4b, 0x29, 0x77, 0xd7, 0xfe, 0xa9, 0xc0,
|
||||
0xad, 0x7c, 0x7e, 0xf2, 0xe9, 0x2d, 0x58, 0x3a, 0x31, 0x89, 0x65, 0x44, 0x29, 0x20, 0x3c, 0xb3,
|
||||
0x58, 0x19, 0x32, 0x60, 0xf9, 0xc2, 0x71, 0x8d, 0xca, 0x91, 0xef, 0x99, 0xce, 0x60, 0xdf, 0xa4,
|
||||
0x3e, 0x16, 0xf0, 0x31, 0x7d, 0x96, 0x8b, 0x7b, 0xe6, 0x6f, 0x85, 0x67, 0x0a, 0x51, 0x9f, 0x88,
|
||||
0xd2, 0x45, 0xdf, 0x6e, 0xc3, 0x92, 0xd3, 0x3e, 0x68, 0xbf, 0x57, 0x60, 0x63, 0x8f, 0xf8, 0x4f,
|
||||
0xc2, 0x3b, 0x26, 0xa6, 0x49, 0x7d, 0xb3, 0x7f, 0x0d, 0xc2, 0x7c, 0xad, 0xc0, 0x9d, 0xb1, 0xc2,
|
||||
0x48, 0x0b, 0xca, 0x8c, 0x16, 0x14, 0xc0, 0xfc, 0x8c, 0xf6, 0x33, 0xf2, 0xea, 0x53, 0xdd, 0x1a,
|
||||
0x91, 0x43, 0xdd, 0xf4, 0x44, 0x46, 0x9b, 0x31, 0xbf, 0xff, 0x4b, 0x81, 0xdb, 0x7b, 0x84, 0x35,
|
||||
0xa3, 0xa2, 0xe6, 0x5c, 0xa3, 0x76, 0x0a, 0x74, 0x7a, 0x7f, 0x14, 0xc6, 0xcc, 0x95, 0xf6, 0x5a,
|
||||
0xd4, 0xb7, 0xc1, 0xc3, 0x31, 0x96, 0x17, 0xa4, 0xa3, 0x4b, 0xe5, 0x69, 0x7f, 0x29, 0x81, 0xfa,
|
||||
0xa9, 0xec, 0xdb, 0x78, 0x35, 0x4b, 0xeb, 0x41, 0xc9, 0xd7, 0x43, 0xac, 0xd5, 0xcb, 0xeb, 0xfe,
|
||||
0xf6, 0xa0, 0x4e, 0x09, 0x39, 0x9b, 0xa5, 0x76, 0xa9, 0x0c, 0x31, 0xac, 0x39, 0xfb, 0xb0, 0x3a,
|
||||
0x72, 0x4e, 0xd8, 0x28, 0x41, 0x8c, 0xa0, 0xa1, 0x68, 0x56, 0x0a, 0x25, 0xc0, 0x2c, 0x22, 0xda,
|
||||
0x82, 0xe5, 0x34, 0xad, 0x79, 0x9e, 0x83, 0xd2, 0xd7, 0xda, 0xef, 0x14, 0x58, 0xff, 0x4c, 0xf7,
|
||||
0xfb, 0xa7, 0x1d, 0xfb, 0xea, 0xa9, 0xe1, 0x63, 0xa8, 0x9e, 0x4b, 0xed, 0x04, 0xb9, 0xed, 0x4e,
|
||||
0x8e, 0xf0, 0x71, 0x3b, 0xe0, 0x08, 0x83, 0x8d, 0x0f, 0x6b, 0x7c, 0x9a, 0xba, 0x7a, 0xd3, 0x37,
|
||||
0xab, 0xe7, 0x27, 0x6b, 0x5d, 0x25, 0x53, 0xeb, 0x2e, 0x00, 0xa4, 0x70, 0x07, 0x74, 0x30, 0x83,
|
||||
0x5c, 0x3f, 0x80, 0x45, 0x49, 0x4d, 0x3a, 0xf7, 0x34, 0xe3, 0x06, 0xe0, 0xda, 0x11, 0xac, 0xcb,
|
||||
0xfb, 0x67, 0xac, 0x8c, 0x88, 0x92, 0x73, 0x40, 0x7c, 0x1d, 0x35, 0x61, 0x51, 0x56, 0x16, 0xe9,
|
||||
0xc4, 0xc1, 0x91, 0xcd, 0x0f, 0xc7, 0x1c, 0xae, 0xc7, 0xca, 0x87, 0xf4, 0x5f, 0x38, 0x0e, 0xab,
|
||||
0x95, 0xf6, 0x2b, 0xa8, 0x77, 0x3a, 0xfb, 0x31, 0x5a, 0xf7, 0x60, 0xd9, 0x30, 0xac, 0x5e, 0x1c,
|
||||
0x4b, 0xe1, 0x58, 0x75, 0xc3, 0xb0, 0xa2, 0x32, 0x87, 0xde, 0x85, 0x86, 0x4f, 0x7b, 0x59, 0xe2,
|
||||
0xaa, 0x4f, 0x23, 0x28, 0xed, 0x00, 0x1a, 0x5c, 0x58, 0x6e, 0xd4, 0x29, 0xb2, 0xde, 0x05, 0x35,
|
||||
0x46, 0x4e, 0xb8, 0x4f, 0x15, 0xd7, 0x22, 0x61, 0x29, 0xab, 0x20, 0x41, 0x57, 0x1a, 0x51, 0x9c,
|
||||
0xdc, 0x95, 0xde, 0x06, 0x30, 0x69, 0x4f, 0x3a, 0x3d, 0x97, 0x71, 0x09, 0x57, 0x4d, 0xfa, 0x4c,
|
||||
0x5c, 0xa0, 0x1f, 0xc2, 0x02, 0xe7, 0x2f, 0xc2, 0x23, 0x93, 0xa4, 0xb8, 0x35, 0x92, 0x2f, 0xc0,
|
||||
0x12, 0x41, 0xfb, 0x04, 0xd4, 0x4e, 0x67, 0x3f, 0x92, 0xa3, 0x48, 0x3e, 0x29, 0xf0, 0xc6, 0xd7,
|
||||
0xd0, 0x88, 0x8a, 0x12, 0x4f, 0x54, 0x0d, 0x28, 0x85, 0xe4, 0x4a, 0xdd, 0x0e, 0xfa, 0x18, 0x16,
|
||||
0xc4, 0x56, 0x46, 0x7a, 0xd0, 0x7b, 0x49, 0x99, 0xe5, 0xc6, 0x26, 0x56, 0xd9, 0xf8, 0x05, 0x96,
|
||||
0x48, 0xcc, 0xc3, 0xc3, 0x44, 0x2e, 0x86, 0xe9, 0x32, 0x8e, 0xdd, 0x68, 0xff, 0x2f, 0x41, 0x2d,
|
||||
0xe6, 0x80, 0x19, 0xf6, 0x6f, 0x66, 0x9c, 0x7a, 0x0f, 0x1a, 0x26, 0xef, 0x47, 0x7a, 0x32, 0xfa,
|
||||
0x79, 0x91, 0xa9, 0xe2, 0xba, 0x19, 0xef, 0x52, 0xd0, 0x06, 0xd4, 0x9c, 0x91, 0xdd, 0x73, 0x4f,
|
||||
0x7a, 0x9e, 0xfb, 0x92, 0xca, 0x11, 0xb9, 0xea, 0x8c, 0xec, 0x9f, 0x9f, 0x60, 0xf7, 0x25, 0x8d,
|
||||
0x86, 0x8e, 0x85, 0x4b, 0x0e, 0x1d, 0x4f, 0x41, 0x35, 0x6c, 0x2b, 0x4a, 0xdb, 0x8b, 0xc5, 0x27,
|
||||
0x05, 0xc3, 0xb6, 0xc2, 0xac, 0xbd, 0x01, 0x35, 0x5b, 0xbf, 0x60, 0xc2, 0xf5, 0x9c, 0x91, 0xdd,
|
||||
0x5c, 0x12, 0xf2, 0xd9, 0xfa, 0x05, 0x76, 0x5f, 0x3e, 0x1f, 0xd9, 0x68, 0x0b, 0x56, 0x2c, 0x9d,
|
||||
0xfa, 0xbd, 0xf8, 0x14, 0x5f, 0xe5, 0x53, 0x7c, 0x83, 0xdd, 0x3f, 0x0d, 0x27, 0x79, 0xed, 0x21,
|
||||
0xd4, 0xba, 0x9d, 0x36, 0xf3, 0x01, 0xd6, 0xef, 0x65, 0xb4, 0xbe, 0x06, 0xf3, 0x87, 0x31, 0x97,
|
||||
0x11, 0x07, 0x96, 0x30, 0xd5, 0x40, 0x16, 0x56, 0x41, 0x73, 0x66, 0x29, 0xe5, 0x4d, 0xcd, 0x52,
|
||||
0xa5, 0x99, 0x66, 0x29, 0xed, 0x3f, 0x25, 0x58, 0x3f, 0xd2, 0xcf, 0xc9, 0xdb, 0x6f, 0xc9, 0x0b,
|
||||
0xe5, 0xf7, 0x7d, 0x58, 0xe5, 0xf1, 0xdb, 0x8e, 0xc9, 0x33, 0xa1, 0xcc, 0xc6, 0x4c, 0x82, 0xb3,
|
||||
0x88, 0xe8, 0xa7, 0xac, 0x3f, 0x20, 0xfd, 0xb3, 0x43, 0xd7, 0x0c, 0x4a, 0x6c, 0xad, 0x7d, 0x3b,
|
||||
0x87, 0xce, 0x93, 0x10, 0x0a, 0xc7, 0x31, 0x78, 0x3a, 0x94, 0xb9, 0x69, 0x81, 0xe7, 0xa6, 0xe0,
|
||||
0xc8, 0x72, 0x1d, 0x44, 0x58, 0x53, 0xb2, 0xdc, 0x4f, 0x60, 0x69, 0x06, 0x3b, 0x85, 0x38, 0xe9,
|
||||
0x30, 0x2b, 0xa7, 0xc2, 0x4c, 0xfb, 0x4a, 0x81, 0x7a, 0x47, 0xf7, 0xf5, 0xe7, 0xae, 0x41, 0x5e,
|
||||
0xcc, 0x58, 0xf9, 0x0a, 0xac, 0xd2, 0x6e, 0x41, 0x95, 0x45, 0x08, 0xf5, 0x75, 0x7b, 0xc8, 0x85,
|
||||
0xa8, 0xe0, 0xe8, 0x82, 0x8d, 0x99, 0x75, 0x99, 0x17, 0x44, 0xef, 0xc7, 0x8a, 0x3c, 0x27, 0x25,
|
||||
0x2a, 0x14, 0xff, 0x8d, 0x7e, 0x94, 0x5c, 0x43, 0xbc, 0x9b, 0x6b, 0x0c, 0x4e, 0x84, 0x77, 0x3d,
|
||||
0x89, 0xa4, 0x50, 0x64, 0x70, 0xf8, 0x52, 0x01, 0x35, 0x50, 0x05, 0xcf, 0x8f, 0xcd, 0x68, 0x97,
|
||||
0x2d, 0xe4, 0x08, 0x8e, 0xec, 0x9f, 0x73, 0xe2, 0xd1, 0xc0, 0x28, 0x65, 0x1c, 0x1c, 0xd1, 0x8f,
|
||||
0x61, 0x29, 0x6c, 0x93, 0xc4, 0x3a, 0x73, 0x73, 0xbc, 0x9c, 0xb2, 0xd1, 0x0d, 0x31, 0x34, 0x0f,
|
||||
0x1a, 0x32, 0xa5, 0x09, 0x5f, 0xa4, 0x53, 0xbc, 0x63, 0x17, 0xd4, 0x93, 0xa8, 0x65, 0x98, 0x34,
|
||||
0x56, 0xc7, 0x3a, 0x0b, 0x9c, 0xc0, 0xd1, 0x1e, 0x43, 0x2d, 0xf6, 0xe7, 0x84, 0x32, 0xde, 0x84,
|
||||
0xc5, 0xe3, 0x18, 0x9f, 0x2a, 0x0e, 0x8e, 0xda, 0x7f, 0x15, 0xbe, 0xc1, 0xc2, 0xa4, 0xef, 0x9e,
|
||||
0x13, 0xef, 0xd5, 0xd5, 0xf7, 0x04, 0x8f, 0x62, 0x5a, 0x2c, 0xd8, 0x6c, 0x86, 0x08, 0xe8, 0x51,
|
||||
0x24, 0x67, 0x79, 0x6c, 0xe9, 0x4f, 0xaa, 0x39, 0x7a, 0xca, 0x9f, 0xc4, 0xc6, 0x23, 0xf9, 0x94,
|
||||
0x6b, 0xdd, 0x4f, 0xde, 0x7f, 0x00, 0xab, 0x19, 0xbf, 0x46, 0x0d, 0x80, 0x4f, 0x9c, 0xbe, 0x6b,
|
||||
0x0f, 0x2d, 0xe2, 0x93, 0x95, 0x39, 0xa4, 0xc2, 0xd2, 0x93, 0xe0, 0xa4, 0xb4, 0xff, 0x57, 0x87,
|
||||
0x1a, 0x73, 0xe5, 0x23, 0xf1, 0xc5, 0x08, 0x0d, 0x01, 0xf1, 0x91, 0xd8, 0x1e, 0xba, 0x4e, 0xb8,
|
||||
0xc2, 0x42, 0x1f, 0x8d, 0xc9, 0x24, 0x59, 0x50, 0xa9, 0x82, 0xd6, 0xbd, 0x31, 0x18, 0x29, 0x70,
|
||||
0x6d, 0x0e, 0xd9, 0x9c, 0x23, 0xab, 0x7f, 0x2f, 0xcc, 0xfe, 0x59, 0x50, 0xf4, 0x27, 0x70, 0x4c,
|
||||
0x81, 0x06, 0x1c, 0x53, 0x9b, 0x31, 0x79, 0x10, 0xeb, 0x93, 0xc0, 0xc7, 0xb4, 0x39, 0xf4, 0x05,
|
||||
0xac, 0xb1, 0x19, 0x31, 0x1c, 0x55, 0x03, 0x86, 0xed, 0xf1, 0x0c, 0x33, 0xc0, 0x97, 0x64, 0xa9,
|
||||
0x83, 0x1a, 0xff, 0x60, 0x85, 0xf2, 0xb6, 0xe8, 0x39, 0xdf, 0xd4, 0x5a, 0xef, 0x4f, 0x85, 0x0b,
|
||||
0x59, 0xec, 0xc1, 0x3c, 0xef, 0x42, 0x51, 0x9e, 0xf7, 0xc7, 0x3f, 0x4e, 0xb5, 0x26, 0x45, 0x95,
|
||||
0x36, 0x87, 0x7e, 0x0d, 0xcb, 0xa9, 0xcf, 0x02, 0xe8, 0x83, 0x1c, 0x92, 0xf9, 0x1f, 0x78, 0x5a,
|
||||
0xf7, 0x8b, 0x80, 0xc6, 0xf5, 0x12, 0x5f, 0x9d, 0xe7, 0xea, 0x25, 0x67, 0xfd, 0x9f, 0xab, 0x97,
|
||||
0xbc, 0x1d, 0xbc, 0x36, 0x87, 0x06, 0xd0, 0x48, 0x6e, 0x04, 0xd0, 0x56, 0x0e, 0x72, 0xee, 0x92,
|
||||
0xb4, 0xf5, 0x41, 0x01, 0xc8, 0x90, 0x91, 0x0d, 0x2b, 0xe9, 0xc5, 0x2f, 0xba, 0x3f, 0x91, 0x40,
|
||||
0x32, 0x5e, 0x3e, 0x2c, 0x04, 0x1b, 0xb2, 0x7b, 0xc5, 0xbd, 0x38, 0xb3, 0x78, 0x44, 0xdb, 0xf9,
|
||||
0x64, 0xc6, 0x6d, 0x44, 0x5b, 0x3b, 0x85, 0xe1, 0x43, 0xd6, 0x04, 0x56, 0x33, 0x8b, 0x44, 0xf4,
|
||||
0xe1, 0x24, 0x3a, 0xa9, 0x9d, 0x42, 0x6b, 0xfa, 0xaa, 0x53, 0x9b, 0x43, 0x5f, 0x89, 0x4a, 0x91,
|
||||
0xb7, 0x9c, 0x43, 0x0f, 0xf2, 0xb9, 0x4d, 0xd8, 0x2a, 0xb6, 0xda, 0x97, 0x41, 0x09, 0xdf, 0xfa,
|
||||
0x9a, 0xa7, 0xf8, 0x9c, 0x05, 0x57, 0x3a, 0x3f, 0x05, 0xf4, 0xc6, 0x6f, 0xee, 0x5a, 0x0f, 0x2e,
|
||||
0x81, 0x11, 0x0a, 0xe0, 0xa6, 0x37, 0xf8, 0x41, 0xba, 0xda, 0x99, 0xea, 0x9c, 0xb3, 0xe5, 0xaa,
|
||||
0xcf, 0x61, 0x39, 0xd5, 0xa9, 0xe7, 0xc6, 0x7f, 0x7e, 0x37, 0x5f, 0x20, 0xb9, 0xa4, 0x2a, 0x26,
|
||||
0x1a, 0x13, 0x64, 0x39, 0x55, 0xb5, 0x75, 0xbf, 0x08, 0x68, 0xf0, 0x90, 0xf6, 0x3f, 0xca, 0xb0,
|
||||
0x14, 0xf4, 0x68, 0xd7, 0x50, 0xd5, 0xae, 0xa1, 0xcc, 0x7c, 0x0e, 0xcb, 0xa9, 0x25, 0x5e, 0xae,
|
||||
0x76, 0xf3, 0x17, 0x7d, 0xd3, 0x4c, 0xf7, 0x19, 0xd4, 0x13, 0x5b, 0x39, 0xf4, 0xfe, 0xb8, 0x42,
|
||||
0x93, 0xce, 0xd6, 0x93, 0x09, 0xef, 0x3e, 0xfc, 0xe5, 0x83, 0x81, 0xe9, 0x9f, 0x8e, 0x8e, 0xd9,
|
||||
0x3f, 0x3b, 0x02, 0xf4, 0x3b, 0xa6, 0x2b, 0x7f, 0xed, 0x04, 0x0a, 0xda, 0xe1, 0xd8, 0x3b, 0x8c,
|
||||
0xcd, 0xf0, 0xf8, 0x78, 0x81, 0x9f, 0x1e, 0x7e, 0x13, 0x00, 0x00, 0xff, 0xff, 0x3a, 0xb4, 0x65,
|
||||
0x6c, 0xee, 0x22, 0x00, 0x00,
|
||||
}
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
|
|
|
@ -8,7 +8,9 @@ import (
|
|||
"sort"
|
||||
"sync"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/msgstream"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
type vChan = string
|
||||
|
@ -154,12 +156,7 @@ func (mgr *singleTypeChannelsMgr) getAllVIDs(collectionID UniqueID) ([]int, erro
|
|||
mgr.collMtx.RLock()
|
||||
defer mgr.collMtx.RUnlock()
|
||||
|
||||
ids, ok := mgr.collectionID2VIDs[collectionID]
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("collection %d not found", collectionID)
|
||||
}
|
||||
|
||||
return ids, nil
|
||||
return mgr.collectionID2VIDs[collectionID], nil
|
||||
}
|
||||
|
||||
func (mgr *singleTypeChannelsMgr) getVChansByVID(vid int) ([]vChan, error) {
|
||||
|
@ -319,6 +316,7 @@ func (mgr *singleTypeChannelsMgr) getVChannels(collectionID UniqueID) ([]vChan,
|
|||
|
||||
func (mgr *singleTypeChannelsMgr) createMsgStream(collectionID UniqueID) error {
|
||||
channels, err := mgr.getChannelsFunc(collectionID)
|
||||
log.Debug("singleTypeChannelsMgr", zap.Any("createMsgStream.getChannels", channels))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -336,8 +334,20 @@ func (mgr *singleTypeChannelsMgr) createMsgStream(collectionID UniqueID) error {
|
|||
pchans := getAllValues(channels)
|
||||
stream.AsProducer(pchans)
|
||||
repack := func(tsMsgs []msgstream.TsMsg, hashKeys [][]int32) (map[int32]*msgstream.MsgPack, error) {
|
||||
// TODO(dragondriver): use new repack function later
|
||||
return nil, nil
|
||||
// after assigning segment id to msg, tsMsgs was already re-bucketed
|
||||
pack := make(map[int32]*msgstream.MsgPack)
|
||||
for idx, msg := range tsMsgs {
|
||||
if len(hashKeys[idx]) <= 0 {
|
||||
continue
|
||||
}
|
||||
key := hashKeys[idx][0]
|
||||
_, ok := pack[key]
|
||||
if !ok {
|
||||
pack[key] = &msgstream.MsgPack{}
|
||||
}
|
||||
pack[key].Msgs = append(pack[key].Msgs, msg)
|
||||
}
|
||||
return pack, nil
|
||||
}
|
||||
stream.SetRepackFunc(repack)
|
||||
runtime.SetFinalizer(stream, func(stream msgstream.MsgStream) {
|
||||
|
|
|
@ -6,8 +6,9 @@ import (
|
|||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
)
|
||||
|
||||
type pChanStatistics struct {
|
||||
|
@ -34,6 +35,7 @@ type channelsTimeTicker interface {
|
|||
start() error
|
||||
close() error
|
||||
addPChan(pchan pChan) error
|
||||
removePChan(pchan pChan) error
|
||||
getLastTick(pchan pChan) (Timestamp, error)
|
||||
getMinTsStatistics() (map[pChan]Timestamp, error)
|
||||
}
|
||||
|
@ -96,6 +98,7 @@ func (ticker *channelsTimeTickerImpl) tick() error {
|
|||
|
||||
stats, err := ticker.getStatisticsFunc(pchan)
|
||||
if err != nil {
|
||||
log.Warn("failed to get statistics from scheduler", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
|
@ -104,9 +107,15 @@ func (ticker *channelsTimeTickerImpl) tick() error {
|
|||
ticker.currents[pchan] = getTs(current+Timestamp(ticker.interval), stats.maxTs, func(ts1, ts2 Timestamp) bool {
|
||||
return ts1 > ts2
|
||||
})
|
||||
//} else if stats.invalid {
|
||||
// ticker.minTsStatistics[pchan] = current
|
||||
// ticker.currents[pchan] = current + Timestamp(ticker.interval)
|
||||
} else if stats.invalid {
|
||||
ticker.minTsStatistics[pchan] = current
|
||||
// ticker.currents[pchan] = current + Timestamp(ticker.interval)
|
||||
t, err := ticker.tso.AllocOne()
|
||||
if err != nil {
|
||||
log.Warn("failed to get ts from tso", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
ticker.currents[pchan] = t
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -166,6 +175,19 @@ func (ticker *channelsTimeTickerImpl) addPChan(pchan pChan) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (ticker *channelsTimeTickerImpl) removePChan(pchan pChan) error {
|
||||
ticker.statisticsMtx.Lock()
|
||||
defer ticker.statisticsMtx.Unlock()
|
||||
|
||||
if _, ok := ticker.minTsStatistics[pchan]; !ok {
|
||||
return fmt.Errorf("pChan %v don't exist", pchan)
|
||||
}
|
||||
|
||||
delete(ticker.minTsStatistics, pchan)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ticker *channelsTimeTickerImpl) getLastTick(pchan pChan) (Timestamp, error) {
|
||||
ticker.statisticsMtx.RLock()
|
||||
defer ticker.statisticsMtx.RUnlock()
|
||||
|
|
|
@ -144,6 +144,7 @@ func (node *ProxyNode) DropCollection(ctx context.Context, request *milvuspb.Dro
|
|||
DropCollectionRequest: request,
|
||||
masterService: node.masterService,
|
||||
chMgr: node.chMgr,
|
||||
chTicker: node.chTicker,
|
||||
}
|
||||
|
||||
err := node.sched.DdQueue.Enqueue(dct)
|
||||
|
|
|
@ -20,8 +20,6 @@ import (
|
|||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
|
@ -29,6 +27,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/proto/proxypb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
|
@ -315,14 +314,21 @@ func (node *ProxyNode) sendChannelsTimeTickLoop() {
|
|||
continue
|
||||
}
|
||||
|
||||
channels := make([]pChan, len(stats))
|
||||
tss := make([]Timestamp, len(stats))
|
||||
log.Debug("send timestamp statistics of pchan", zap.Any("statistics", stats))
|
||||
|
||||
channels := make([]pChan, 0, len(stats))
|
||||
tss := make([]Timestamp, 0, len(stats))
|
||||
|
||||
for channel, ts := range stats {
|
||||
channels = append(channels, channel)
|
||||
tss = append(tss, ts)
|
||||
}
|
||||
|
||||
req := &internalpb.ChannelTimeTickMsg{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Undefined, // todo
|
||||
MsgID: 0, // todo
|
||||
Timestamp: 0, // todo
|
||||
MsgType: commonpb.MsgType_TimeTick, // todo
|
||||
MsgID: 0, // todo
|
||||
Timestamp: 0, // todo
|
||||
SourceID: node.session.ServerID,
|
||||
},
|
||||
ChannelNames: channels,
|
||||
|
|
|
@ -292,8 +292,6 @@ func (sa *SegIDAssigner) syncSegments() (bool, error) {
|
|||
return true, nil
|
||||
}
|
||||
sa.reduceSegReqs()
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||
defer cancel()
|
||||
req := &datapb.AssignSegmentIDRequest{
|
||||
NodeID: sa.PeerID,
|
||||
PeerRole: typeutil.ProxyNodeRole,
|
||||
|
@ -301,7 +299,7 @@ func (sa *SegIDAssigner) syncSegments() (bool, error) {
|
|||
}
|
||||
|
||||
sa.segReqs = []*datapb.SegmentIDRequest{}
|
||||
resp, err := sa.dataService.AssignSegmentID(ctx, req)
|
||||
resp, err := sa.dataService.AssignSegmentID(context.Background(), req)
|
||||
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("syncSegmentID Failed:%w", err)
|
||||
|
|
|
@ -750,6 +750,7 @@ func (it *InsertTask) Execute(ctx context.Context) error {
|
|||
return err
|
||||
}
|
||||
for _, pchan := range pchans {
|
||||
log.Debug("add pchan to time ticker", zap.Any("pchan", pchan))
|
||||
_ = it.chTicker.addPChan(pchan)
|
||||
}
|
||||
|
||||
|
@ -902,6 +903,7 @@ type DropCollectionTask struct {
|
|||
masterService types.MasterService
|
||||
result *commonpb.Status
|
||||
chMgr channelsMgr
|
||||
chTicker channelsTimeTicker
|
||||
}
|
||||
|
||||
func (dct *DropCollectionTask) TraceCtx() context.Context {
|
||||
|
@ -962,11 +964,13 @@ func (dct *DropCollectionTask) Execute(ctx context.Context) error {
|
|||
return err
|
||||
}
|
||||
|
||||
err = dct.chMgr.removeDMLStream(collID)
|
||||
if err != nil {
|
||||
return err
|
||||
pchans, _ := dct.chMgr.getChannels(collID)
|
||||
for _, pchan := range pchans {
|
||||
_ = dct.chTicker.removePChan(pchan)
|
||||
}
|
||||
|
||||
_ = dct.chMgr.removeDMLStream(collID)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -1846,13 +1850,51 @@ func (dct *DescribeCollectionTask) PreExecute(ctx context.Context) error {
|
|||
|
||||
func (dct *DescribeCollectionTask) Execute(ctx context.Context) error {
|
||||
var err error
|
||||
dct.result, err = dct.masterService.DescribeCollection(ctx, dct.DescribeCollectionRequest)
|
||||
if dct.result == nil {
|
||||
dct.result = &milvuspb.DescribeCollectionResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
},
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "",
|
||||
Description: "",
|
||||
AutoID: false,
|
||||
Fields: make([]*schemapb.FieldSchema, 0),
|
||||
},
|
||||
CollectionID: 0,
|
||||
VirtualChannelNames: nil,
|
||||
PhysicalChannelNames: nil,
|
||||
}
|
||||
|
||||
result, err := dct.masterService.DescribeCollection(ctx, dct.DescribeCollectionRequest)
|
||||
|
||||
if result == nil {
|
||||
return errors.New("has collection resp is nil")
|
||||
}
|
||||
if dct.result.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
if result.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
return errors.New(dct.result.Status.Reason)
|
||||
}
|
||||
|
||||
dct.result.Schema.Name = result.Schema.Name
|
||||
dct.result.Schema.Description = result.Schema.Description
|
||||
dct.result.Schema.AutoID = result.Schema.AutoID
|
||||
dct.result.CollectionID = result.CollectionID
|
||||
dct.result.VirtualChannelNames = result.VirtualChannelNames
|
||||
dct.result.PhysicalChannelNames = result.PhysicalChannelNames
|
||||
|
||||
for _, field := range result.Schema.Fields {
|
||||
if field.FieldID >= 100 { // TODO(dragondriver): use StartOfUserFieldID replacing 100
|
||||
dct.result.Schema.Fields = append(dct.result.Schema.Fields, &schemapb.FieldSchema{
|
||||
FieldID: field.FieldID,
|
||||
Name: field.Name,
|
||||
IsPrimaryKey: field.IsPrimaryKey,
|
||||
Description: field.Description,
|
||||
DataType: field.DataType,
|
||||
TypeParams: field.TypeParams,
|
||||
IndexParams: field.IndexParams,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
|
|
|
@ -208,15 +208,33 @@ type DmTaskQueue struct {
|
|||
}
|
||||
|
||||
func (queue *DmTaskQueue) getPChanStatistics(pchan pChan) (pChanStatistics, error) {
|
||||
queue.atLock.RLock()
|
||||
defer queue.atLock.RUnlock()
|
||||
|
||||
stats := pChanStatistics{
|
||||
minTs: 0,
|
||||
maxTs: ^uint64(0),
|
||||
invalid: true,
|
||||
}
|
||||
|
||||
queue.utLock.RLock()
|
||||
defer queue.utLock.RUnlock()
|
||||
|
||||
for e := queue.unissuedTasks.Front(); e != nil; e = e.Next() {
|
||||
dmlT := e.Value.(task).(dmlTask)
|
||||
stat, err := dmlT.getStatistics(pchan)
|
||||
if err != nil {
|
||||
return pChanStatistics{invalid: true}, nil
|
||||
}
|
||||
if stat.minTs < stats.minTs {
|
||||
stats.minTs = stat.minTs
|
||||
}
|
||||
if stat.maxTs > stats.maxTs {
|
||||
stats.maxTs = stat.maxTs
|
||||
}
|
||||
stats.invalid = false
|
||||
}
|
||||
|
||||
queue.atLock.RLock()
|
||||
defer queue.atLock.RUnlock()
|
||||
|
||||
for _, t := range queue.activeTasks {
|
||||
dmlT, _ := t.(dmlTask)
|
||||
stat, err := dmlT.getStatistics(pchan)
|
||||
|
|
|
@ -12,8 +12,10 @@
|
|||
package retry
|
||||
|
||||
import (
|
||||
"log"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// Reference: https://blog.cyeam.com/golang/2018/08/27/retry
|
||||
|
@ -25,7 +27,7 @@ func Impl(attempts int, sleep time.Duration, fn func() error, maxSleepTime time.
|
|||
}
|
||||
|
||||
if attempts--; attempts > 0 {
|
||||
log.Printf("retry func error: %s. attempts #%d after %s.", err.Error(), attempts, sleep)
|
||||
log.Debug("retry func error", zap.Int("attempts", attempts), zap.Duration("sleep", sleep), zap.Error(err))
|
||||
time.Sleep(sleep)
|
||||
if sleep < maxSleepTime {
|
||||
return Impl(attempts, 2*sleep, fn, maxSleepTime)
|
||||
|
|
|
@ -32,7 +32,7 @@ type UniqueID = typeutil.UniqueID
|
|||
|
||||
const (
|
||||
DefaultMessageID = "-1"
|
||||
FixedChannelNameLen = 32
|
||||
FixedChannelNameLen = 320
|
||||
RocksDBLRUCacheCapacity = 3 << 30
|
||||
)
|
||||
|
||||
|
|
Loading…
Reference in New Issue