From 189ac881f3795ca06dc3f82ca3cd82ebd02165b2 Mon Sep 17 00:00:00 2001 From: sunby Date: Tue, 8 Jun 2021 19:25:37 +0800 Subject: [PATCH] Fix bugs (#5676) * Remove redundant session startup Signed-off-by: sunby * Register datanode after start success Signed-off-by: sunby * fix meta snap shot Signed-off-by: yefu.chen * fix datanode message stream channel Signed-off-by: yangxuan * Fix bugs when drop empty collection Signed-off-by: sunby * Fix bug of getting pchan statistics from task scheduler Signed-off-by: dragondriver * Fix i/dist/dataservice test code Signed-off-by: Congqi Xia * Fix epoch lifetime not applied Signed-off-by: Congqi Xia * fix datanode flowgraph dd node Signed-off-by: yangxuan * Fix handle datanode timetick bug Signed-off-by: sunby * Remove repack function of dml stream Signed-off-by: dragondriver * fix proxynode Signed-off-by: yefu.chen * Apply extended seal policy Signed-off-by: Congqi Xia * add check for time tick Signed-off-by: yefu.chen * fix check Signed-off-by: yefu.chen * Fix the repack function of dml stream Signed-off-by: dragondriver * Fix the bug when send statistics of pchan Signed-off-by: dragondriver * Fix the repack function when craete dml stream Signed-off-by: dragondriver * fix bugs Signed-off-by: yefu.chen * fix describe collection Signed-off-by: yefu.chen * Fix bug when send timestamp statistics Signed-off-by: dragondriver * fix data node Signed-off-by: yefu.chen * Add length check before flush request Signed-off-by: Congqi Xia * add log for data node Signed-off-by: yefu.chen * Fix SaveBinlog bugs Signed-off-by: sunby * Add more log in datanode Signed-off-by: yangxuan * Put SegmentState.Flushing as the last one in enum to fit the client Signed-off-by: sunby * Fix params in GetInsertBinlogPaths Signed-off-by: sunby * Rename policy Signed-off-by: sunby * Remove unused ddl functions and fields Signed-off-by: Congqi Xia * Remove pchan when drop collection Signed-off-by: dragondriver * Add balanced assignment policy Signed-off-by: sunby * fix master ut Signed-off-by: yefu.chen * Add lock in session manager Signed-off-by: Congqi Xia * add log for debug Signed-off-by: yefu.chen * Fix some logic bug and typo Signed-off-by: Congqi Xia * Fix recover bugs Signed-off-by: sunby * Get collection scheme of a specific timestamp Signed-off-by: yangxuan * Change CheckPoint to SegmentInfo in VchannelInfo Signed-off-by: sunby * Recover Unflushed segment numOfRows Signed-off-by: yangxuan * Fix dataservice unit tests Signed-off-by: sunby Co-authored-by: yefu.chen Co-authored-by: yangxuan Co-authored-by: dragondriver Co-authored-by: Congqi Xia --- internal/datanode/collection_replica.go | 94 +++--- internal/datanode/collection_replica_test.go | 93 +----- internal/datanode/data_node.go | 78 ++--- internal/datanode/data_node_test.go | 29 +- internal/datanode/data_sync_service.go | 30 +- internal/datanode/data_sync_service_test.go | 22 +- internal/datanode/flow_graph_dd_node.go | 33 +- internal/datanode/flow_graph_dd_node_test.go | 6 + .../flow_graph_dmstream_input_node.go | 15 +- .../datanode/flow_graph_insert_buffer_node.go | 59 ++-- .../flow_graph_insert_buffer_node_test.go | 42 ++- internal/datanode/meta_service.go | 70 +---- internal/datanode/meta_service_test.go | 42 +-- internal/datanode/mock_test.go | 18 +- internal/datanode/param_table.go | 47 ++- internal/datanode/param_table_test.go | 21 +- internal/dataservice/binlog_helper.go | 67 +---- internal/dataservice/cluster.go | 30 +- .../dataservice/cluster_session_manager.go | 46 ++- internal/dataservice/cluster_test.go | 4 +- internal/dataservice/datanode_helper.go | 8 +- internal/dataservice/flush_monitor.go | 12 +- internal/dataservice/grpc_services.go | 28 +- internal/dataservice/param.go | 12 +- internal/dataservice/policy.go | 53 +++- .../dataservice/segment_allocation_policy.go | 58 ++++ internal/dataservice/segment_manager.go | 121 +++++--- internal/dataservice/server.go | 27 +- internal/dataservice/server_test.go | 51 +--- internal/distributed/datanode/service.go | 17 +- .../dataservice/dataservice_test.go | 4 - internal/distributed/dataservice/service.go | 33 -- internal/masterservice/dml_channels.go | 20 +- internal/masterservice/master_service.go | 35 ++- internal/masterservice/master_service_test.go | 2 + internal/masterservice/meta_snapshot.go | 2 +- internal/masterservice/meta_snapshot_test.go | 8 +- internal/masterservice/task.go | 16 +- internal/msgstream/retry.go | 6 +- internal/proto/common.proto | 4 +- internal/proto/commonpb/common.pb.go | 165 +++++----- internal/proto/data_service.proto | 8 +- internal/proto/datapb/data_service.pb.go | 281 +++++++++--------- internal/proxynode/channels_mgr.go | 26 +- internal/proxynode/channels_time_ticker.go | 30 +- internal/proxynode/impl.go | 1 + internal/proxynode/proxy_node.go | 20 +- internal/proxynode/segment.go | 4 +- internal/proxynode/task.go | 54 +++- internal/proxynode/task_scheduler.go | 24 +- internal/util/retry/retry.go | 6 +- .../rocksmq/server/rocksmq/rocksmq_impl.go | 2 +- 52 files changed, 1015 insertions(+), 969 deletions(-) diff --git a/internal/datanode/collection_replica.go b/internal/datanode/collection_replica.go index 7bf3a697aa..ad16163546 100644 --- a/internal/datanode/collection_replica.go +++ b/internal/datanode/collection_replica.go @@ -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 diff --git a/internal/datanode/collection_replica_test.go b/internal/datanode/collection_replica_test.go index e94697d5b7..4462ed3816 100644 --- a/internal/datanode/collection_replica_test.go +++ b/internal/datanode/collection_replica_test.go @@ -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) diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 371c7eb049..788086c94c 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -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 diff --git a/internal/datanode/data_node_test.go b/internal/datanode/data_node_test.go index 629be0b620..5e1937fea8 100644 --- a/internal/datanode/data_node_test.go +++ b/internal/datanode/data_node_test.go @@ -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)) diff --git a/internal/datanode/data_sync_service.go b/internal/datanode/data_sync_service.go index 9fd6db3bb5..c57f97f182 100644 --- a/internal/datanode/data_sync_service.go +++ b/internal/datanode/data_sync_service.go @@ -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) diff --git a/internal/datanode/data_sync_service_test.go b/internal/datanode/data_sync_service_test.go index 5b22120184..e03adf4cf2 100644 --- a/internal/datanode/data_sync_service_test.go +++ b/internal/datanode/data_sync_service_test.go @@ -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{ diff --git a/internal/datanode/flow_graph_dd_node.go b/internal/datanode/flow_graph_dd_node.go index 156717a062..9efeb323fe 100644 --- a/internal/datanode/flow_graph_dd_node.go +++ b/internal/datanode/flow_graph_dd_node.go @@ -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, } } diff --git a/internal/datanode/flow_graph_dd_node_test.go b/internal/datanode/flow_graph_dd_node_test.go index babaf3d39d..76dd897369 100644 --- a/internal/datanode/flow_graph_dd_node_test.go +++ b/internal/datanode/flow_graph_dd_node_test.go @@ -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 diff --git a/internal/datanode/flow_graph_dmstream_input_node.go b/internal/datanode/flow_graph_dmstream_input_node.go index b40ee1012b..e5ff1f548d 100644 --- a/internal/datanode/flow_graph_dmstream_input_node.go +++ b/internal/datanode/flow_graph_dmstream_input_node.go @@ -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) diff --git a/internal/datanode/flow_graph_insert_buffer_node.go b/internal/datanode/flow_graph_insert_buffer_node.go index 3a23a23479..19eef7af9f 100644 --- a/internal/datanode/flow_graph_insert_buffer_node.go +++ b/internal/datanode/flow_graph_insert_buffer_node.go @@ -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, diff --git a/internal/datanode/flow_graph_insert_buffer_node_test.go b/internal/datanode/flow_graph_insert_buffer_node_test.go index bbc61f89ec..7a17baadff 100644 --- a/internal/datanode/flow_graph_insert_buffer_node_test.go +++ b/internal/datanode/flow_graph_insert_buffer_node_test.go @@ -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 { diff --git a/internal/datanode/meta_service.go b/internal/datanode/meta_service.go index 6e5f72f484..9c69084cbe 100644 --- a/internal/datanode/meta_service.go +++ b/internal/datanode/meta_service.go @@ -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 } diff --git a/internal/datanode/meta_service_test.go b/internal/datanode/meta_service_test.go index 6fa8cd7a98..afd6bc1d91 100644 --- a/internal/datanode/meta_service_test.go +++ b/internal/datanode/meta_service_test.go @@ -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) { diff --git a/internal/datanode/mock_test.go b/internal/datanode/mock_test.go index 47e5fd3509..04431efa75 100644 --- a/internal/datanode/mock_test.go +++ b/internal/datanode/mock_test.go @@ -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 { diff --git a/internal/datanode/param_table.go b/internal/datanode/param_table.go index 2f68b33fea..3c2ad57bde 100644 --- a/internal/datanode/param_table.go +++ b/internal/datanode/param_table.go @@ -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 { diff --git a/internal/datanode/param_table_test.go b/internal/datanode/param_table_test.go index 5bdfd97d1a..0e1abeff6c 100644 --- a/internal/datanode/param_table_test.go +++ b/internal/datanode/param_table_test.go @@ -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) }) diff --git a/internal/dataservice/binlog_helper.go b/internal/dataservice/binlog_helper.go index 5df6b93558..4e0762aec0 100644 --- a/internal/dataservice/binlog_helper.go +++ b/internal/dataservice/binlog_helper.go @@ -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 diff --git a/internal/dataservice/cluster.go b/internal/dataservice/cluster.go index ec67b64aa6..fa0ec00795 100644 --- a/internal/dataservice/cluster.go +++ b/internal/dataservice/cluster.go @@ -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() diff --git a/internal/dataservice/cluster_session_manager.go b/internal/dataservice/cluster_session_manager.go index 468488c6e7..046961182a 100644 --- a/internal/dataservice/cluster_session_manager.go +++ b/internal/dataservice/cluster_session_manager.go @@ -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{} } diff --git a/internal/dataservice/cluster_test.go b/internal/dataservice/cluster_test.go index c6244e5abc..2112bf9741 100644 --- a/internal/dataservice/cluster_test.go +++ b/internal/dataservice/cluster_test.go @@ -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{ diff --git a/internal/dataservice/datanode_helper.go b/internal/dataservice/datanode_helper.go index ff74012d89..48e939b4f3 100644 --- a/internal/dataservice/datanode_helper.go +++ b/internal/dataservice/datanode_helper.go @@ -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 diff --git a/internal/dataservice/flush_monitor.go b/internal/dataservice/flush_monitor.go index f63c9b4214..6fbc3d0aa5 100644 --- a/internal/dataservice/flush_monitor.go +++ b/internal/dataservice/flush_monitor.go @@ -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 { diff --git a/internal/dataservice/grpc_services.go b/internal/dataservice/grpc_services.go index 3b89943dc1..839df860bd 100644 --- a/internal/dataservice/grpc_services.go +++ b/internal/dataservice/grpc_services.go @@ -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 diff --git a/internal/dataservice/param.go b/internal/dataservice/param.go index 2a2b1928f0..a5374d659d 100644 --- a/internal/dataservice/param.go +++ b/internal/dataservice/param.go @@ -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") diff --git a/internal/dataservice/policy.go b/internal/dataservice/policy.go index ca17b5b5b9..bf241bc2c0 100644 --- a/internal/dataservice/policy.go +++ b/internal/dataservice/policy.go @@ -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 +} diff --git a/internal/dataservice/segment_allocation_policy.go b/internal/dataservice/segment_allocation_policy.go index e35f4356a3..3f6f4aa616 100644 --- a/internal/dataservice/segment_allocation_policy.go +++ b/internal/dataservice/segment_allocation_policy.go @@ -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 { } diff --git a/internal/dataservice/segment_manager.go b/internal/dataservice/segment_manager.go index 1b07f5684b..dd8c9b0862 100644 --- a/internal/dataservice/segment_manager.go +++ b/internal/dataservice/segment_manager.go @@ -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 { diff --git a/internal/dataservice/server.go b/internal/dataservice/server.go index ea1baee8ab..e1dbc2d573 100644 --- a/internal/dataservice/server.go +++ b/internal/dataservice/server.go @@ -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 } diff --git a/internal/dataservice/server_test.go b/internal/dataservice/server_test.go index 2bf1469d01..4b87ced64a 100644 --- a/internal/dataservice/server_test.go +++ b/internal/dataservice/server_test.go @@ -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) diff --git a/internal/distributed/datanode/service.go b/internal/distributed/datanode/service.go index dd8cff51d3..07869c6455 100644 --- a/internal/distributed/datanode/service.go +++ b/internal/distributed/datanode/service.go @@ -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) { diff --git a/internal/distributed/dataservice/dataservice_test.go b/internal/distributed/dataservice/dataservice_test.go index 35e050a894..5718f9fb59 100644 --- a/internal/distributed/dataservice/dataservice_test.go +++ b/internal/distributed/dataservice/dataservice_test.go @@ -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) diff --git a/internal/distributed/dataservice/service.go b/internal/distributed/dataservice/service.go index d099df8c8e..f91355c98f 100644 --- a/internal/distributed/dataservice/service.go +++ b/internal/distributed/dataservice/service.go @@ -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 diff --git a/internal/masterservice/dml_channels.go b/internal/masterservice/dml_channels.go index 1584b52c0d..ab44c2469d 100644 --- a/internal/masterservice/dml_channels.go +++ b/internal/masterservice/dml_channels.go @@ -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 +} diff --git a/internal/masterservice/master_service.go b/internal/masterservice/master_service.go index 3a1409088b..1544250ef1 100644 --- a/internal/masterservice/master_service.go +++ b/internal/masterservice/master_service.go @@ -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 diff --git a/internal/masterservice/master_service_test.go b/internal/masterservice/master_service_test.go index 00cb911660..25230a405b 100644 --- a/internal/masterservice/master_service_test.go +++ b/internal/masterservice/master_service_test.go @@ -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, diff --git a/internal/masterservice/meta_snapshot.go b/internal/masterservice/meta_snapshot.go index ab0bf00ca6..46fb8e8ea0 100644 --- a/internal/masterservice/meta_snapshot.go +++ b/internal/masterservice/meta_snapshot.go @@ -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() diff --git a/internal/masterservice/meta_snapshot_test.go b/internal/masterservice/meta_snapshot_test.go index 27919b1432..dce55e1f6a 100644 --- a/internal/masterservice/meta_snapshot_test.go +++ b/internal/masterservice/meta_snapshot_test.go @@ -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)) diff --git a/internal/masterservice/task.go b/internal/masterservice/task.go index 02722fc8d5..f934019b38 100644 --- a/internal/masterservice/task.go +++ b/internal/masterservice/task.go @@ -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 diff --git a/internal/msgstream/retry.go b/internal/msgstream/retry.go index 51fd99b91c..14b0c98a4d 100644 --- a/internal/msgstream/retry.go +++ b/internal/msgstream/retry.go @@ -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) } diff --git a/internal/proto/common.proto b/internal/proto/common.proto index 052912b298..88ca3d0f5b 100644 --- a/internal/proto/common.proto +++ b/internal/proto/common.proto @@ -48,8 +48,8 @@ enum SegmentState { NotExist = 1; Growing = 2; Sealed = 3; - Flushing = 4; - Flushed = 5; + Flushed = 4; + Flushing = 5; } message Status { diff --git a/internal/proto/commonpb/common.pb.go b/internal/proto/commonpb/common.pb.go index 9a85b62412..c8190ad003 100644 --- a/internal/proto/commonpb/common.pb.go +++ b/internal/proto/commonpb/common.pb.go @@ -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, } diff --git a/internal/proto/data_service.proto b/internal/proto/data_service.proto index dff2577f51..c5d7fa0f3d 100644 --- a/internal/proto/data_service.proto +++ b/internal/proto/data_service.proto @@ -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 { diff --git a/internal/proto/datapb/data_service.pb.go b/internal/proto/datapb/data_service.pb.go index aabd10dc5c..10dd4fe7e3 100644 --- a/internal/proto/datapb/data_service.pb.go +++ b/internal/proto/datapb/data_service.pb.go @@ -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. diff --git a/internal/proxynode/channels_mgr.go b/internal/proxynode/channels_mgr.go index 7429413888..dc6fff19be 100644 --- a/internal/proxynode/channels_mgr.go +++ b/internal/proxynode/channels_mgr.go @@ -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) { diff --git a/internal/proxynode/channels_time_ticker.go b/internal/proxynode/channels_time_ticker.go index aaa6e750d2..139ee653b7 100644 --- a/internal/proxynode/channels_time_ticker.go +++ b/internal/proxynode/channels_time_ticker.go @@ -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() diff --git a/internal/proxynode/impl.go b/internal/proxynode/impl.go index 70f4cb4c2c..0dccffe06b 100644 --- a/internal/proxynode/impl.go +++ b/internal/proxynode/impl.go @@ -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) diff --git a/internal/proxynode/proxy_node.go b/internal/proxynode/proxy_node.go index 65fd425749..23602924dd 100644 --- a/internal/proxynode/proxy_node.go +++ b/internal/proxynode/proxy_node.go @@ -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, diff --git a/internal/proxynode/segment.go b/internal/proxynode/segment.go index 71fb53e491..1999d96bdb 100644 --- a/internal/proxynode/segment.go +++ b/internal/proxynode/segment.go @@ -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) diff --git a/internal/proxynode/task.go b/internal/proxynode/task.go index ce799d7261..467201778e 100644 --- a/internal/proxynode/task.go +++ b/internal/proxynode/task.go @@ -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 } diff --git a/internal/proxynode/task_scheduler.go b/internal/proxynode/task_scheduler.go index aa13ed62bc..fa6e6682c9 100644 --- a/internal/proxynode/task_scheduler.go +++ b/internal/proxynode/task_scheduler.go @@ -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) diff --git a/internal/util/retry/retry.go b/internal/util/retry/retry.go index 80ba94c65d..c641bb141d 100644 --- a/internal/util/retry/retry.go +++ b/internal/util/retry/retry.go @@ -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) diff --git a/internal/util/rocksmq/server/rocksmq/rocksmq_impl.go b/internal/util/rocksmq/server/rocksmq/rocksmq_impl.go index 7b3e6e5d14..8662dadc5b 100644 --- a/internal/util/rocksmq/server/rocksmq/rocksmq_impl.go +++ b/internal/util/rocksmq/server/rocksmq/rocksmq_impl.go @@ -32,7 +32,7 @@ type UniqueID = typeutil.UniqueID const ( DefaultMessageID = "-1" - FixedChannelNameLen = 32 + FixedChannelNameLen = 320 RocksDBLRUCacheCapacity = 3 << 30 )