From 3662b3f0e57726cc348c12e81927f5515d3cf7df Mon Sep 17 00:00:00 2001 From: sunby Date: Tue, 25 May 2021 15:35:37 +0800 Subject: [PATCH] Refactor dataservice (#5400) Pick up grpc calls to grpc_handler.go and refactor code format Signed-off-by: sunby --- internal/dataservice/grpc_handler.go | 411 ++++++++++++++ internal/dataservice/meta.go | 220 ++++---- internal/dataservice/meta_test.go | 8 +- internal/dataservice/mock_test.go | 4 + .../dataservice/segment_allocation_stats.go | 8 +- internal/dataservice/segment_allocator.go | 9 +- internal/dataservice/server.go | 516 ++---------------- internal/dataservice/server_test.go | 166 +++--- internal/dataservice/watcher.go | 91 --- internal/dataservice/watcher_test.go | 111 ---- internal/msgstream/msg.go | 45 ++ internal/msgstream/unmarshal.go | 2 + internal/proto/common.proto | 5 +- internal/proto/commonpb/common.pb.go | 173 +++--- internal/proto/data_service.proto | 6 + internal/proto/datapb/data_service.pb.go | 294 ++++++---- 16 files changed, 1035 insertions(+), 1034 deletions(-) create mode 100644 internal/dataservice/grpc_handler.go delete mode 100644 internal/dataservice/watcher.go delete mode 100644 internal/dataservice/watcher_test.go diff --git a/internal/dataservice/grpc_handler.go b/internal/dataservice/grpc_handler.go new file mode 100644 index 0000000000..704412a013 --- /dev/null +++ b/internal/dataservice/grpc_handler.go @@ -0,0 +1,411 @@ +package dataservice + +import ( + "context" + "fmt" + "path" + "strconv" + + "github.com/golang/protobuf/proto" + "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/proto/commonpb" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/internal/proto/milvuspb" + "go.uber.org/zap" +) + +func (s *Server) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { + resp := &internalpb.ComponentStates{ + State: &internalpb.ComponentInfo{ + NodeID: Params.NodeID, + Role: role, + StateCode: s.state.Load().(internalpb.StateCode), + }, + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, + } + dataNodeStates, err := s.cluster.GetDataNodeStates(ctx) + if err != nil { + resp.Status.Reason = err.Error() + return resp, nil + } + resp.SubcomponentStates = dataNodeStates + resp.Status.ErrorCode = commonpb.ErrorCode_Success + return resp, nil +} + +func (s *Server) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) { + return &milvuspb.StringResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + Value: Params.TimeTickChannelName, + }, nil +} + +func (s *Server) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) { + return &milvuspb.StringResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + Value: Params.StatisticsChannelName, + }, nil +} + +func (s *Server) RegisterNode(ctx context.Context, req *datapb.RegisterNodeRequest) (*datapb.RegisterNodeResponse, error) { + ret := &datapb.RegisterNodeResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, + } + log.Debug("DataService: RegisterNode:", + zap.String("IP", req.Address.Ip), + zap.Int64("Port", req.Address.Port)) + + node, err := s.newDataNode(req.Address.Ip, req.Address.Port, req.Base.SourceID) + if err != nil { + ret.Status.Reason = err.Error() + return ret, nil + } + + resp, err := node.client.WatchDmChannels(s.ctx, &datapb.WatchDmChannelsRequest{ + Base: &commonpb.MsgBase{ + MsgType: 0, + MsgID: 0, + Timestamp: 0, + SourceID: Params.NodeID, + }, + ChannelNames: s.insertChannels, + }) + + if err = VerifyResponse(resp, err); err != nil { + ret.Status.Reason = err.Error() + return ret, nil + } + + if err := s.getDDChannel(); err != nil { + ret.Status.Reason = err.Error() + return ret, nil + } + + if err = s.cluster.Register(node); err != nil { + ret.Status.Reason = err.Error() + return ret, nil + } + + ret.Status.ErrorCode = commonpb.ErrorCode_Success + ret.InitParams = &internalpb.InitParams{ + NodeID: Params.NodeID, + StartParams: []*commonpb.KeyValuePair{ + {Key: "DDChannelName", Value: s.ddChannelMu.name}, + {Key: "SegmentStatisticsChannelName", Value: Params.StatisticsChannelName}, + {Key: "TimeTickChannelName", Value: Params.TimeTickChannelName}, + {Key: "CompleteFlushChannelName", Value: Params.SegmentInfoChannelName}, + }, + } + return ret, nil +} +func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*commonpb.Status, error) { + if !s.checkStateIsHealthy() { + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "server is initializing", + }, nil + } + if err := s.segAllocator.SealAllSegments(ctx, req.CollectionID); err != nil { + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: fmt.Sprintf("Seal all segments error %s", err), + }, nil + } + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, nil +} + +func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) { + if !s.checkStateIsHealthy() { + return &datapb.AssignSegmentIDResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, + }, nil + } + + assigns := make([]*datapb.SegmentIDAssignment, 0, len(req.SegmentIDRequests)) + + var appendFailedAssignment = func(err string) { + assigns = append(assigns, &datapb.SegmentIDAssignment{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: err, + }, + }) + } + + for _, r := range req.SegmentIDRequests { + 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) + appendFailedAssignment(errMsg) + log.Error("load collection from master error", + zap.Int64("collectionID", r.CollectionID), + zap.Error(err)) + continue + } + } + //if err := s.validateAllocRequest(r.CollectionID, r.PartitionID, r.ChannelName); err != nil { + //result.Status.Reason = err.Error() + //assigns = append(assigns, result) + //continue + //} + + segmentID, retCount, expireTs, err := s.segAllocator.AllocSegment(ctx, + r.CollectionID, r.PartitionID, r.ChannelName, int64(r.Count)) + if err != nil { + errMsg := fmt.Sprintf("allocation of collection %d, partition %d, channel %s, count %d error: %s", + r.CollectionID, r.PartitionID, r.ChannelName, r.Count, err.Error()) + appendFailedAssignment(errMsg) + continue + } + + result := &datapb.SegmentIDAssignment{ + SegID: segmentID, + ChannelName: r.ChannelName, + Count: uint32(retCount), + CollectionID: r.CollectionID, + PartitionID: r.PartitionID, + ExpireTime: expireTs, + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + Reason: "", + }, + } + assigns = append(assigns, result) + } + return &datapb.AssignSegmentIDResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + SegIDAssignments: assigns, + }, nil +} +func (s *Server) ShowSegments(ctx context.Context, req *datapb.ShowSegmentsRequest) (*datapb.ShowSegmentsResponse, error) { + resp := &datapb.ShowSegmentsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, + } + if !s.checkStateIsHealthy() { + resp.Status.Reason = "server is initializing" + return resp, nil + } + ids := s.meta.GetSegmentsOfPartition(req.CollectionID, req.PartitionID) + resp.Status.ErrorCode = commonpb.ErrorCode_Success + resp.SegmentIDs = ids + return resp, nil +} + +func (s *Server) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) { + resp := &datapb.GetSegmentStatesResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, + } + if !s.checkStateIsHealthy() { + resp.Status.Reason = "server is initializing" + return resp, nil + } + + for _, segmentID := range req.SegmentIDs { + state := &datapb.SegmentStateInfo{ + Status: &commonpb.Status{}, + SegmentID: segmentID, + } + segmentInfo, err := s.meta.GetSegment(segmentID) + if err != nil { + state.Status.ErrorCode = commonpb.ErrorCode_UnexpectedError + state.Status.Reason = "get segment states error: " + err.Error() + } else { + state.Status.ErrorCode = commonpb.ErrorCode_Success + state.State = segmentInfo.State + state.StartPosition = segmentInfo.StartPosition + state.EndPosition = segmentInfo.EndPosition + } + resp.States = append(resp.States, state) + } + resp.Status.ErrorCode = commonpb.ErrorCode_Success + + return resp, nil +} + +func (s *Server) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsertBinlogPathsRequest) (*datapb.GetInsertBinlogPathsResponse, error) { + resp := &datapb.GetInsertBinlogPathsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, + } + p := path.Join(Params.SegmentFlushMetaPath, strconv.FormatInt(req.SegmentID, 10)) + _, values, err := s.kvClient.LoadWithPrefix(p) + if err != nil { + resp.Status.Reason = err.Error() + return resp, nil + } + m := make(map[int64][]string) + tMeta := &datapb.SegmentFieldBinlogMeta{} + for _, v := range values { + if err := proto.UnmarshalText(v, tMeta); err != nil { + resp.Status.Reason = fmt.Errorf("DataService GetInsertBinlogPaths UnmarshalText datapb.SegmentFieldBinlogMeta err:%w", err).Error() + return resp, nil + } + m[tMeta.FieldID] = append(m[tMeta.FieldID], tMeta.BinlogPath) + } + + fids := make([]UniqueID, len(m)) + paths := make([]*internalpb.StringList, len(m)) + for k, v := range m { + fids = append(fids, k) + paths = append(paths, &internalpb.StringList{Values: v}) + } + resp.Status.ErrorCode = commonpb.ErrorCode_Success + resp.FieldIDs = fids + resp.Paths = paths + return resp, nil +} + +func (s *Server) GetInsertChannels(ctx context.Context, req *datapb.GetInsertChannelsRequest) (*internalpb.StringList, error) { + return &internalpb.StringList{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + Values: s.insertChannels, + }, nil +} + +func (s *Server) GetCollectionStatistics(ctx context.Context, req *datapb.GetCollectionStatisticsRequest) (*datapb.GetCollectionStatisticsResponse, error) { + resp := &datapb.GetCollectionStatisticsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, + } + nums, err := s.meta.GetNumRowsOfCollection(req.CollectionID) + if err != nil { + resp.Status.Reason = err.Error() + return resp, nil + } + resp.Status.ErrorCode = commonpb.ErrorCode_Success + resp.Stats = append(resp.Stats, &commonpb.KeyValuePair{Key: "row_count", Value: strconv.FormatInt(nums, 10)}) + return resp, nil +} + +func (s *Server) GetPartitionStatistics(ctx context.Context, req *datapb.GetPartitionStatisticsRequest) (*datapb.GetPartitionStatisticsResponse, error) { + resp := &datapb.GetPartitionStatisticsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, + } + nums, err := s.meta.GetNumRowsOfPartition(req.CollectionID, req.PartitionID) + if err != nil { + resp.Status.Reason = err.Error() + return resp, nil + } + resp.Status.ErrorCode = commonpb.ErrorCode_Success + resp.Stats = append(resp.Stats, &commonpb.KeyValuePair{Key: "row_count", Value: strconv.FormatInt(nums, 10)}) + return resp, nil +} + +func (s *Server) GetSegmentInfoChannel(ctx context.Context) (*milvuspb.StringResponse, error) { + return &milvuspb.StringResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + Value: Params.SegmentInfoChannelName, + }, nil +} + +func (s *Server) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) { + resp := &datapb.GetSegmentInfoResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, + } + if !s.checkStateIsHealthy() { + resp.Status.Reason = "data service is not healthy" + return resp, nil + } + infos := make([]*datapb.SegmentInfo, 0, len(req.SegmentIDs)) + for _, id := range req.SegmentIDs { + info, err := s.meta.GetSegment(id) + if err != nil { + resp.Status.Reason = err.Error() + return resp, nil + } + infos = append(infos, info) + } + resp.Status.ErrorCode = commonpb.ErrorCode_Success + resp.Infos = infos + return resp, nil +} + +// SaveBinlogPaths implement DataServiceServer +func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) { + resp := &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + } + if !s.checkStateIsHealthy() { + resp.Reason = "server is initializing" + return resp, nil + } + if s.flushMsgStream == nil { + resp.Reason = "flush msg stream nil" + return resp, nil + } + + // check segment id & collection id matched + _, err := s.meta.GetCollection(req.GetCollectionID()) + if err != nil { + log.Error("Failed to get collection info", zap.Int64("collectionID", req.GetCollectionID()), zap.Error(err)) + resp.Reason = err.Error() + return resp, err + } + + meta, err := s.prepareBinlogAndPos(req) + if err != nil { + log.Error("prepare binlog and pos meta failed", zap.Error(err)) + resp.Reason = err.Error() + return resp, err + } + + // set segment to SegmentState_Flushing + err = s.meta.FlushSegmentWithBinlogAndPos(req.SegmentID, meta) + if err != nil { + resp.Reason = err.Error() + return resp, err + } + log.Debug("flush segment with meta", zap.Int64("id", req.SegmentID), + zap.Any("meta", meta)) + + // write flush msg into segmentInfo/flush stream + msgPack := composeSegmentFlushMsgPack(req.SegmentID) + err = s.flushMsgStream.Produce(&msgPack) + if err != nil { + resp.Reason = err.Error() + return resp, err + } + log.Debug("send segment flush msg", zap.Int64("id", req.SegmentID)) + + // set segment to SegmentState_Flushed + if err = s.meta.FlushSegment(req.SegmentID); err != nil { + log.Error("flush segment complete failed", zap.Error(err)) + resp.Reason = err.Error() + return resp, err + } + log.Debug("flush segment complete", zap.Int64("id", req.SegmentID)) + + s.segAllocator.DropSegment(ctx, req.SegmentID) + + resp.ErrorCode = commonpb.ErrorCode_Success + return resp, nil +} diff --git a/internal/dataservice/meta.go b/internal/dataservice/meta.go index 3b04d59831..1c4a341328 100644 --- a/internal/dataservice/meta.go +++ b/internal/dataservice/meta.go @@ -69,8 +69,8 @@ func newMeta(kv kv.TxnKV) (*meta, error) { return mt, nil } -func (meta *meta) reloadFromKV() error { - _, values, err := meta.client.LoadWithPrefix(segmentPrefix) +func (m *meta) reloadFromKV() error { + _, values, err := m.client.LoadWithPrefix(segmentPrefix) if err != nil { return err } @@ -81,65 +81,65 @@ func (meta *meta) reloadFromKV() error { if err != nil { return fmt.Errorf("DataService reloadFromKV UnMarshalText datapb.SegmentInfo err:%w", err) } - meta.segments[segmentInfo.ID] = segmentInfo + m.segments[segmentInfo.ID] = segmentInfo } return nil } -func (meta *meta) AddCollection(collection *datapb.CollectionInfo) error { - meta.Lock() - defer meta.Unlock() - if _, ok := meta.collections[collection.ID]; ok { +func (m *meta) AddCollection(collection *datapb.CollectionInfo) error { + m.Lock() + defer m.Unlock() + if _, ok := m.collections[collection.ID]; ok { return fmt.Errorf("collection %s with id %d already exist", collection.Schema.Name, collection.ID) } - meta.collections[collection.ID] = collection + m.collections[collection.ID] = collection return nil } -func (meta *meta) DropCollection(collID UniqueID) error { - meta.Lock() - defer meta.Unlock() +func (m *meta) DropCollection(collID UniqueID) error { + m.Lock() + defer m.Unlock() - if _, ok := meta.collections[collID]; !ok { + if _, ok := m.collections[collID]; !ok { return newErrCollectionNotFound(collID) } key := fmt.Sprintf("%s/%d/", segmentPrefix, collID) - if err := meta.client.RemoveWithPrefix(key); err != nil { + if err := m.client.RemoveWithPrefix(key); err != nil { return err } - delete(meta.collections, collID) + delete(m.collections, collID) - for i, info := range meta.segments { + for i, info := range m.segments { if info.CollectionID == collID { - delete(meta.segments, i) + delete(m.segments, i) } } return nil } -func (meta *meta) HasCollection(collID UniqueID) bool { - meta.RLock() - defer meta.RUnlock() - _, ok := meta.collections[collID] +func (m *meta) HasCollection(collID UniqueID) bool { + m.RLock() + defer m.RUnlock() + _, ok := m.collections[collID] return ok } -func (meta *meta) GetCollection(collectionID UniqueID) (*datapb.CollectionInfo, error) { - meta.RLock() - defer meta.RUnlock() +func (m *meta) GetCollection(collectionID UniqueID) (*datapb.CollectionInfo, error) { + m.RLock() + defer m.RUnlock() - collection, ok := meta.collections[collectionID] + collection, ok := m.collections[collectionID] if !ok { return nil, newErrCollectionNotFound(collectionID) } return proto.Clone(collection).(*datapb.CollectionInfo), nil } -func (meta *meta) GetNumRowsOfCollection(collectionID UniqueID) (int64, error) { - meta.RLock() - defer meta.RUnlock() +func (m *meta) GetNumRowsOfCollection(collectionID UniqueID) (int64, error) { + m.RLock() + defer m.RUnlock() var ret int64 = 0 - for _, info := range meta.segments { + for _, info := range m.segments { if info.CollectionID == collectionID { ret += info.NumRows } @@ -147,23 +147,23 @@ func (meta *meta) GetNumRowsOfCollection(collectionID UniqueID) (int64, error) { return ret, nil } -func (meta *meta) AddSegment(segment *datapb.SegmentInfo) error { - meta.Lock() - defer meta.Unlock() - if _, ok := meta.segments[segment.ID]; ok { +func (m *meta) AddSegment(segment *datapb.SegmentInfo) error { + m.Lock() + defer m.Unlock() + if _, ok := m.segments[segment.ID]; ok { return fmt.Errorf("segment %d already exist", segment.ID) } - meta.segments[segment.ID] = segment - if err := meta.saveSegmentInfo(segment); err != nil { + m.segments[segment.ID] = segment + if err := m.saveSegmentInfo(segment); err != nil { return err } return nil } -func (meta *meta) UpdateSegmentStatistic(segment *datapb.SegmentInfo) error { - meta.Lock() - defer meta.Unlock() - seg, ok := meta.segments[segment.ID] +func (m *meta) UpdateSegmentStatistic(segment *datapb.SegmentInfo) error { + m.Lock() + defer m.Unlock() + seg, ok := m.segments[segment.ID] if !ok { return newErrSegmentNotFound(segment.ID) } @@ -171,90 +171,108 @@ func (meta *meta) UpdateSegmentStatistic(segment *datapb.SegmentInfo) error { seg.StartPosition = proto.Clone(segment.StartPosition).(*internalpb.MsgPosition) seg.EndPosition = proto.Clone(segment.EndPosition).(*internalpb.MsgPosition) - if err := meta.saveSegmentInfo(segment); err != nil { + if err := m.saveSegmentInfo(segment); err != nil { return err } return nil } -func (meta *meta) SetLastExpireTime(segmentID UniqueID, expireTs Timestamp) error { - meta.Lock() - defer meta.Unlock() - seg, ok := meta.segments[segmentID] +func (m *meta) SetLastExpireTime(segmentID UniqueID, expireTs Timestamp) error { + m.Lock() + defer m.Unlock() + seg, ok := m.segments[segmentID] if !ok { return newErrSegmentNotFound(segmentID) } seg.LastExpireTime = expireTs - if err := meta.saveSegmentInfo(seg); err != nil { + if err := m.saveSegmentInfo(seg); err != nil { return err } return nil } -func (meta *meta) DropSegment(segmentID UniqueID) error { - meta.Lock() - defer meta.Unlock() +func (m *meta) DropSegment(segmentID UniqueID) error { + m.Lock() + defer m.Unlock() - segment, ok := meta.segments[segmentID] + segment, ok := m.segments[segmentID] if !ok { return newErrSegmentNotFound(segmentID) } - if err := meta.removeSegmentInfo(segment); err != nil { + if err := m.removeSegmentInfo(segment); err != nil { return err } - delete(meta.segments, segmentID) + delete(m.segments, segmentID) return nil } -func (meta *meta) GetSegment(segID UniqueID) (*datapb.SegmentInfo, error) { - meta.RLock() - defer meta.RUnlock() +func (m *meta) GetSegment(segID UniqueID) (*datapb.SegmentInfo, error) { + m.RLock() + defer m.RUnlock() - segment, ok := meta.segments[segID] + segment, ok := m.segments[segID] if !ok { return nil, newErrSegmentNotFound(segID) } return proto.Clone(segment).(*datapb.SegmentInfo), nil } -func (meta *meta) SealSegment(segID UniqueID, timetick Timestamp) error { - meta.Lock() - defer meta.Unlock() +func (m *meta) SealSegment(segID UniqueID) error { + m.Lock() + defer m.Unlock() - segInfo, ok := meta.segments[segID] + segInfo, ok := m.segments[segID] if !ok { return newErrSegmentNotFound(segID) } segInfo.State = commonpb.SegmentState_Sealed - if err := meta.saveSegmentInfo(segInfo); err != nil { + if err := m.saveSegmentInfo(segInfo); err != nil { return err } return nil } -func (meta *meta) FlushSegment(segID UniqueID, timetick Timestamp) error { - meta.Lock() - defer meta.Unlock() +func (m *meta) FlushSegmentWithBinlogAndPos(segID UniqueID, kv map[string]string) error { + m.Lock() + defer m.Unlock() - segInfo, ok := meta.segments[segID] + segInfo, ok := m.segments[segID] + if !ok { + return newErrSegmentNotFound(segID) + } + segInfo.State = commonpb.SegmentState_Flushing + segBytes := proto.MarshalTextString(segInfo) + key := fmt.Sprintf("%s/%d/%d/%d", segmentPrefix, segInfo.CollectionID, segInfo.PartitionID, segInfo.ID) + kv[key] = segBytes + + if err := m.saveKvTxn(kv); err != nil { + return err + } + return nil +} + +func (m *meta) FlushSegment(segID UniqueID) error { + m.Lock() + defer m.Unlock() + segInfo, ok := m.segments[segID] if !ok { return newErrSegmentNotFound(segID) } segInfo.State = commonpb.SegmentState_Flushed - if err := meta.saveSegmentInfo(segInfo); err != nil { + if err := m.saveSegmentInfo(segInfo); err != nil { return err } return nil } -func (meta *meta) GetSegmentsOfCollection(collectionID UniqueID) []UniqueID { - meta.RLock() - defer meta.RUnlock() +func (m *meta) GetSegmentsOfCollection(collectionID UniqueID) []UniqueID { + m.RLock() + defer m.RUnlock() ret := make([]UniqueID, 0) - for _, info := range meta.segments { + for _, info := range m.segments { if info.CollectionID == collectionID { ret = append(ret, info.ID) } @@ -262,12 +280,12 @@ func (meta *meta) GetSegmentsOfCollection(collectionID UniqueID) []UniqueID { return ret } -func (meta *meta) GetSegmentsOfPartition(collectionID, partitionID UniqueID) []UniqueID { - meta.RLock() - defer meta.RUnlock() +func (m *meta) GetSegmentsOfPartition(collectionID, partitionID UniqueID) []UniqueID { + m.RLock() + defer m.RUnlock() ret := make([]UniqueID, 0) - for _, info := range meta.segments { + for _, info := range m.segments { if info.CollectionID == collectionID && info.PartitionID == partitionID { ret = append(ret, info.ID) } @@ -275,10 +293,10 @@ func (meta *meta) GetSegmentsOfPartition(collectionID, partitionID UniqueID) []U return ret } -func (meta *meta) AddPartition(collectionID UniqueID, partitionID UniqueID) error { - meta.Lock() - defer meta.Unlock() - coll, ok := meta.collections[collectionID] +func (m *meta) AddPartition(collectionID UniqueID, partitionID UniqueID) error { + m.Lock() + defer m.Unlock() + coll, ok := m.collections[collectionID] if !ok { return newErrCollectionNotFound(collectionID) } @@ -292,11 +310,11 @@ func (meta *meta) AddPartition(collectionID UniqueID, partitionID UniqueID) erro return nil } -func (meta *meta) DropPartition(collID UniqueID, partitionID UniqueID) error { - meta.Lock() - defer meta.Unlock() +func (m *meta) DropPartition(collID UniqueID, partitionID UniqueID) error { + m.Lock() + defer m.Unlock() - collection, ok := meta.collections[collID] + collection, ok := m.collections[collID] if !ok { return newErrCollectionNotFound(collID) } @@ -312,23 +330,23 @@ func (meta *meta) DropPartition(collID UniqueID, partitionID UniqueID) error { } prefix := fmt.Sprintf("%s/%d/%d/", segmentPrefix, collID, partitionID) - if err := meta.client.RemoveWithPrefix(prefix); err != nil { + if err := m.client.RemoveWithPrefix(prefix); err != nil { return err } collection.Partitions = append(collection.Partitions[:idx], collection.Partitions[idx+1:]...) - for i, info := range meta.segments { + for i, info := range m.segments { if info.PartitionID == partitionID { - delete(meta.segments, i) + delete(m.segments, i) } } return nil } -func (meta *meta) HasPartition(collID UniqueID, partID UniqueID) bool { - meta.RLock() - defer meta.RUnlock() - coll, ok := meta.collections[collID] +func (m *meta) HasPartition(collID UniqueID, partID UniqueID) bool { + m.RLock() + defer m.RUnlock() + coll, ok := m.collections[collID] if !ok { return false } @@ -340,11 +358,11 @@ func (meta *meta) HasPartition(collID UniqueID, partID UniqueID) bool { return false } -func (meta *meta) GetNumRowsOfPartition(collectionID UniqueID, partitionID UniqueID) (int64, error) { - meta.RLock() - defer meta.RUnlock() +func (m *meta) GetNumRowsOfPartition(collectionID UniqueID, partitionID UniqueID) (int64, error) { + m.RLock() + defer m.RUnlock() var ret int64 = 0 - for _, info := range meta.segments { + for _, info := range m.segments { if info.CollectionID == collectionID && info.PartitionID == partitionID { ret += info.NumRows } @@ -352,11 +370,11 @@ func (meta *meta) GetNumRowsOfPartition(collectionID UniqueID, partitionID Uniqu return ret, nil } -func (meta *meta) GetUnFlushedSegments() []*datapb.SegmentInfo { - meta.RLock() - defer meta.RUnlock() +func (m *meta) GetUnFlushedSegments() []*datapb.SegmentInfo { + m.RLock() + defer m.RUnlock() segments := make([]*datapb.SegmentInfo, 0) - for _, info := range meta.segments { + for _, info := range m.segments { if info.State != commonpb.SegmentState_Flushed { cInfo := proto.Clone(info).(*datapb.SegmentInfo) segments = append(segments, cInfo) @@ -365,16 +383,20 @@ func (meta *meta) GetUnFlushedSegments() []*datapb.SegmentInfo { return segments } -func (meta *meta) saveSegmentInfo(segment *datapb.SegmentInfo) error { +func (m *meta) saveSegmentInfo(segment *datapb.SegmentInfo) error { segBytes := proto.MarshalTextString(segment) key := fmt.Sprintf("%s/%d/%d/%d", segmentPrefix, segment.CollectionID, segment.PartitionID, segment.ID) - return meta.client.Save(key, segBytes) + return m.client.Save(key, segBytes) } -func (meta *meta) removeSegmentInfo(segment *datapb.SegmentInfo) error { +func (m *meta) removeSegmentInfo(segment *datapb.SegmentInfo) error { key := fmt.Sprintf("%s/%d/%d/%d", segmentPrefix, segment.CollectionID, segment.PartitionID, segment.ID) - return meta.client.Remove(key) + return m.client.Remove(key) +} + +func (m *meta) saveKvTxn(kv map[string]string) error { + return m.client.MultiSave(kv) } func BuildSegment(collectionID UniqueID, partitionID UniqueID, segmentID UniqueID, channelName string) (*datapb.SegmentInfo, error) { diff --git a/internal/dataservice/meta_test.go b/internal/dataservice/meta_test.go index a0c2e9833b..ba3e235130 100644 --- a/internal/dataservice/meta_test.go +++ b/internal/dataservice/meta_test.go @@ -174,9 +174,9 @@ func TestMeta_Basic(t *testing.T) { assert.EqualValues(t, 1, len(segIDs)) assert.Contains(t, segIDs, segID1_1) - err = meta.SealSegment(segID0_0, 200) + err = meta.SealSegment(segID0_0) assert.Nil(t, err) - err = meta.FlushSegment(segID0_0, 300) + err = meta.FlushSegment(segID0_0) assert.Nil(t, err) info0_0, err = meta.GetSegment(segID0_0) @@ -273,11 +273,11 @@ func TestMeta_Basic(t *testing.T) { assert.NotNil(t, err) // check seal non-exist segment - err = meta.SealSegment(segIDInvalid, 200) + err = meta.SealSegment(segIDInvalid) assert.NotNil(t, err) // check flush non-exist segment - err = meta.FlushSegment(segIDInvalid, 300) + err = meta.FlushSegment(segIDInvalid) assert.NotNil(t, err) err = meta.DropCollection(collID) diff --git a/internal/dataservice/mock_test.go b/internal/dataservice/mock_test.go index 8c2abb5235..4d63fde38d 100644 --- a/internal/dataservice/mock_test.go +++ b/internal/dataservice/mock_test.go @@ -66,6 +66,7 @@ func newTestSchema() *schemapb.CollectionSchema { type mockDataNodeClient struct { id int64 state internalpb.StateCode + ch chan interface{} } func newMockDataNodeClient(id int64) (*mockDataNodeClient, error) { @@ -106,6 +107,9 @@ func (c *mockDataNodeClient) WatchDmChannels(ctx context.Context, in *datapb.Wat } func (c *mockDataNodeClient) FlushSegments(ctx context.Context, in *datapb.FlushSegmentsRequest) (*commonpb.Status, error) { + if c.ch != nil { + c.ch <- in + } return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil } diff --git a/internal/dataservice/segment_allocation_stats.go b/internal/dataservice/segment_allocation_stats.go index d7546178a1..a3909770ea 100644 --- a/internal/dataservice/segment_allocation_stats.go +++ b/internal/dataservice/segment_allocation_stats.go @@ -2,6 +2,7 @@ package dataservice import ( "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" "go.uber.org/zap" ) @@ -60,11 +61,12 @@ func (s *segAllocStats) loadSegmentsFromMeta() { id: seg.ID, collectionID: seg.CollectionID, partitionID: seg.PartitionID, - sealed: false, total: seg.MaxRowNum, allocations: []*allocation{}, insertChannel: seg.InsertChannel, lastExpireTime: seg.LastExpireTime, + sealed: seg.State == commonpb.SegmentState_Sealed || + seg.State == commonpb.SegmentState_Flushing, } s.stats[seg.ID] = stat } @@ -89,7 +91,7 @@ func (s *segAllocStats) appendAllocation(segmentID UniqueID, numRows int64, expi func (s *segAllocStats) sealSegment(id UniqueID) error { s.stats[id].sealed = true - return s.meta.SealSegment(id, 0) + return s.meta.SealSegment(id) } func (s *segAllocStats) sealSegmentsBy(collectionID UniqueID) error { @@ -98,7 +100,7 @@ func (s *segAllocStats) sealSegmentsBy(collectionID UniqueID) error { if status.sealed { continue } - if err := s.meta.SealSegment(status.id, 0); err != nil { + if err := s.meta.SealSegment(status.id); err != nil { return err } status.sealed = true diff --git a/internal/dataservice/segment_allocator.go b/internal/dataservice/segment_allocator.go index a716de6fc6..f7ab21a02e 100644 --- a/internal/dataservice/segment_allocator.go +++ b/internal/dataservice/segment_allocator.go @@ -1,5 +1,4 @@ // 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 // @@ -50,7 +49,7 @@ type segmentAllocatorInterface interface { // SealAllSegments get all opened segment ids of collection. return success and failed segment ids SealAllSegments(ctx context.Context, collectionID UniqueID) error // GetFlushableSegments return flushable segment ids - GetFlushableSegments(ctx context.Context, ts Timestamp) ([]UniqueID, error) + GetFlushableSegments(ctx context.Context, channel string, ts Timestamp) ([]UniqueID, error) } type segmentAllocator struct { @@ -291,7 +290,8 @@ func (s *segmentAllocator) SealAllSegments(ctx context.Context, collectionID Uni return nil } -func (s *segmentAllocator) GetFlushableSegments(ctx context.Context, t Timestamp) ([]UniqueID, error) { +func (s *segmentAllocator) GetFlushableSegments(ctx context.Context, channel string, + t Timestamp) ([]UniqueID, error) { s.mu.Lock() defer s.mu.Unlock() sp, _ := trace.StartSpanFromContext(ctx) @@ -303,6 +303,9 @@ func (s *segmentAllocator) GetFlushableSegments(ctx context.Context, t Timestamp ret := make([]UniqueID, 0) segments := s.allocStats.getAllSegments() for _, segStatus := range segments { + if segStatus.insertChannel != channel { + continue + } if s.flushPolicy.apply(segStatus, t) { ret = append(ret, segStatus.id) } diff --git a/internal/dataservice/server.go b/internal/dataservice/server.go index 0fc26ea2fe..1c5f06f570 100644 --- a/internal/dataservice/server.go +++ b/internal/dataservice/server.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 // @@ -16,21 +15,17 @@ import ( "errors" "fmt" "math/rand" - "path" "strconv" - "strings" "sync" "sync/atomic" "time" "github.com/milvus-io/milvus/internal/logutil" - "github.com/golang/protobuf/proto" grpcdatanodeclient "github.com/milvus-io/milvus/internal/distributed/datanode/client" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/msgstream" - "github.com/milvus-io/milvus/internal/timesync" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/sessionutil" @@ -62,20 +57,15 @@ type Server struct { statsHandler *statsHandler allocator allocatorInterface cluster *dataNodeCluster - msgProducer *timesync.MsgProducer masterClient types.MasterService - ttMsgStream msgstream.MsgStream - k2sMsgStream msgstream.MsgStream ddChannelMu struct { sync.Mutex name string } session *sessionutil.Session - segmentInfoStream msgstream.MsgStream flushMsgStream msgstream.MsgStream insertChannels []string msFactory msgstream.Factory - ttBarrier timesync.TimeTickBarrier createDataNodeClient func(addr string) (types.DataNode, error) } @@ -183,7 +173,9 @@ func (s *Server) checkStateIsHealthy() bool { func (s *Server) initMeta() error { connectEtcdFn := func() error { - etcdClient, err := clientv3.New(clientv3.Config{Endpoints: []string{Params.EtcdAddress}}) + etcdClient, err := clientv3.New(clientv3.Config{ + Endpoints: []string{Params.EtcdAddress}, + }) if err != nil { return err } @@ -199,26 +191,6 @@ func (s *Server) initMeta() error { func (s *Server) initMsgProducer() error { var err error - if s.ttMsgStream, err = s.msFactory.NewMsgStream(s.ctx); err != nil { - return err - } - s.ttMsgStream.AsConsumer([]string{Params.TimeTickChannelName}, Params.DataServiceSubscriptionName) - log.Debug("dataservice AsConsumer: " + Params.TimeTickChannelName + " : " + Params.DataServiceSubscriptionName) - s.ttMsgStream.Start() - s.ttBarrier = timesync.NewHardTimeTickBarrier(s.ctx, s.ttMsgStream, s.cluster.GetNodeIDs()) - s.ttBarrier.Start() - if s.k2sMsgStream, err = s.msFactory.NewMsgStream(s.ctx); err != nil { - return err - } - s.k2sMsgStream.AsProducer(Params.K2SChannelNames) - log.Debug("dataservice AsProducer: " + strings.Join(Params.K2SChannelNames, ", ")) - s.k2sMsgStream.Start() - dataNodeTTWatcher := newDataNodeTimeTickWatcher(s.meta, s.segAllocator, s.cluster) - k2sMsgWatcher := timesync.NewMsgTimeTickWatcher(s.k2sMsgStream) - if s.msgProducer, err = timesync.NewTimeSyncMsgProducer(s.ttBarrier, dataNodeTTWatcher, k2sMsgWatcher); err != nil { - return err - } - s.msgProducer.Start(s.ctx) // segment flush stream s.flushMsgStream, err = s.msFactory.NewMsgStream(s.ctx) if err != nil { @@ -340,7 +312,7 @@ func (s *Server) startServerLoop() { s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.ctx) s.serverLoopWg.Add(2) go s.startStatsChannel(s.serverLoopCtx) - go s.startSegmentFlushChannel(s.serverLoopCtx) + go s.startDataNodeTtLoop(s.serverLoopCtx) } func (s *Server) startStatsChannel(ctx context.Context) { @@ -370,7 +342,7 @@ func (s *Server) startStatsChannel(ctx context.Context) { } msgPack := statsStream.Consume() if msgPack == nil { - continue + return } for _, msg := range msgPack.Msgs { if msg.Type() != commonpb.MsgType_SegmentStatistics { @@ -398,60 +370,69 @@ func (s *Server) startStatsChannel(ctx context.Context) { } } -func (s *Server) startSegmentFlushChannel(ctx context.Context) { +func (s *Server) startDataNodeTtLoop(ctx context.Context) { defer logutil.LogPanic() defer s.serverLoopWg.Done() - flushStream, _ := s.msFactory.NewMsgStream(ctx) - flushStream.AsConsumer([]string{Params.SegmentInfoChannelName}, Params.DataServiceSubscriptionName) - log.Debug("DataService AsConsumer: " + Params.SegmentInfoChannelName + " : " + Params.DataServiceSubscriptionName) - - // try to restore last processed pos - pos, err := s.loadStreamLastPos(streamTypeFlush) - if err == nil { - err = flushStream.Seek([]*internalpb.MsgPosition{pos}) - if err != nil { - log.Error("Failed to seek to last pos for segment flush Stream", - zap.String("SegInfoChannelName", Params.SegmentInfoChannelName), - zap.String("DataServiceSubscriptionName", Params.DataServiceSubscriptionName), - zap.Error(err)) - } + ttMsgStream, err := s.msFactory.NewMsgStream(ctx) + if err != nil { + log.Error("new msg stream failed", zap.Error(err)) + return } - - flushStream.Start() - defer flushStream.Close() + ttMsgStream.AsConsumer([]string{Params.TimeTickChannelName}, + Params.DataServiceSubscriptionName) + log.Debug(fmt.Sprintf("dataservice AsConsumer:%s:%s", + Params.TimeTickChannelName, Params.DataServiceSubscriptionName)) + ttMsgStream.Start() + defer ttMsgStream.Close() for { select { case <-ctx.Done(): - log.Debug("segment flush channel shut down") + log.Debug("data node tt loop done") return default: } - msgPack := flushStream.Consume() + msgPack := ttMsgStream.Consume() if msgPack == nil { - continue + return } for _, msg := range msgPack.Msgs { - if msg.Type() != commonpb.MsgType_SegmentFlushDone { - log.Warn("receive unknown msg from segment flush channel", zap.Stringer("msgType", msg.Type())) + if msg.Type() != commonpb.MsgType_DataNodeTt { + log.Warn("receive unexpected msg type from tt channel", + zap.Stringer("msgType", msg.Type())) continue } - fcMsg := msg.(*msgstream.FlushCompletedMsg) - err := s.meta.FlushSegment(fcMsg.SegmentID, fcMsg.BeginTimestamp) - log.Debug("DataService flushed segment", zap.Any("segmentID", fcMsg.SegmentID), zap.Error(err)) + ttMsg := msg.(*msgstream.DataNodeTtMsg) + + coll2Segs := make(map[UniqueID][]UniqueID) + ch := ttMsg.ChannelName + ts := ttMsg.Timestamp + segments, err := s.segAllocator.GetFlushableSegments(ctx, ch, ts) if err != nil { - log.Error("get segment from meta error", zap.Int64("segmentID", fcMsg.SegmentID), zap.Error(err)) + log.Warn("get flushable segments failed", zap.Error(err)) continue } + for _, id := range segments { + sInfo, err := s.meta.GetSegment(id) + if err != nil { + log.Error("get segment from meta error", zap.Int64("id", id), + zap.Error(err)) + continue + } + collID, segID := sInfo.CollectionID, sInfo.ID + coll2Segs[collID] = append(coll2Segs[collID], segID) + } - if fcMsg.MsgPosition != nil { - err = s.storeStreamPos(streamTypeFlush, fcMsg.MsgPosition) - if err != nil { - log.Error("Fail to store current success pos for segment flush stream", - zap.Stringer("pos", fcMsg.MsgPosition), - zap.Error(err)) - } - } else { - log.Warn("Empty Msg Pos found ", zap.Int64("msgid", msg.ID())) + for collID, segIDs := range coll2Segs { + s.cluster.FlushSegment(&datapb.FlushSegmentsRequest{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_Flush, + MsgID: -1, // todo add msg id + Timestamp: 0, // todo + SourceID: Params.NodeID, + }, + CollectionID: collID, + SegmentIDs: segIDs, + }) } } } @@ -459,10 +440,7 @@ func (s *Server) startSegmentFlushChannel(ctx context.Context) { func (s *Server) Stop() error { s.cluster.ShutDownClients() - s.ttMsgStream.Close() - s.k2sMsgStream.Close() - s.ttBarrier.Close() - s.msgProducer.Close() + s.flushMsgStream.Close() s.stopServerLoop() return nil } @@ -477,105 +455,6 @@ func (s *Server) stopServerLoop() { s.serverLoopWg.Wait() } -func (s *Server) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { - resp := &internalpb.ComponentStates{ - State: &internalpb.ComponentInfo{ - NodeID: Params.NodeID, - Role: role, - StateCode: s.state.Load().(internalpb.StateCode), - }, - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - }, - } - dataNodeStates, err := s.cluster.GetDataNodeStates(ctx) - if err != nil { - resp.Status.Reason = err.Error() - return resp, nil - } - resp.SubcomponentStates = dataNodeStates - resp.Status.ErrorCode = commonpb.ErrorCode_Success - return resp, nil -} - -func (s *Server) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) { - return &milvuspb.StringResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_Success, - }, - Value: Params.TimeTickChannelName, - }, nil -} - -func (s *Server) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) { - return &milvuspb.StringResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_Success, - }, - Value: Params.StatisticsChannelName, - }, nil -} - -func (s *Server) RegisterNode(ctx context.Context, req *datapb.RegisterNodeRequest) (*datapb.RegisterNodeResponse, error) { - ret := &datapb.RegisterNodeResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - }, - } - log.Debug("DataService: RegisterNode:", zap.String("IP", req.Address.Ip), zap.Int64("Port", req.Address.Port), - zap.Any("NodeID", req.Base.SourceID)) - node, err := s.newDataNode(req.Address.Ip, req.Address.Port, req.Base.SourceID) - if err != nil { - ret.Status.Reason = err.Error() - log.Debug("DataService newDataNode failed", zap.Error(err)) - return ret, nil - } - - resp, err := node.client.WatchDmChannels(s.ctx, &datapb.WatchDmChannelsRequest{ - Base: &commonpb.MsgBase{ - MsgType: 0, - MsgID: 0, - Timestamp: 0, - SourceID: Params.NodeID, - }, - ChannelNames: s.insertChannels, - }) - - if err = VerifyResponse(resp, err); err != nil { - ret.Status.Reason = err.Error() - return ret, nil - } - - if err := s.getDDChannel(); err != nil { - ret.Status.Reason = err.Error() - return ret, nil - } - - if s.ttBarrier != nil { - if err = s.ttBarrier.AddPeer(node.id); err != nil { - ret.Status.Reason = err.Error() - return ret, nil - } - } - - if err = s.cluster.Register(node); err != nil { - ret.Status.Reason = err.Error() - return ret, nil - } - - ret.Status.ErrorCode = commonpb.ErrorCode_Success - ret.InitParams = &internalpb.InitParams{ - NodeID: Params.NodeID, - StartParams: []*commonpb.KeyValuePair{ - {Key: "DDChannelName", Value: s.ddChannelMu.name}, - {Key: "SegmentStatisticsChannelName", Value: Params.StatisticsChannelName}, - {Key: "TimeTickChannelName", Value: Params.TimeTickChannelName}, - {Key: "CompleteFlushChannelName", Value: Params.SegmentInfoChannelName}, - }, - } - return ret, nil -} - func (s *Server) newDataNode(ip string, port int64, id UniqueID) (*dataNode, error) { client, err := s.createDataNodeClient(fmt.Sprintf("%s:%d", ip, port)) if err != nil { @@ -599,87 +478,6 @@ func (s *Server) newDataNode(ip string, port int64, id UniqueID) (*dataNode, err }, nil } -func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*commonpb.Status, error) { - if !s.checkStateIsHealthy() { - return &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - Reason: "server is initializing", - }, nil - } - if err := s.segAllocator.SealAllSegments(ctx, req.CollectionID); err != nil { - return &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - Reason: fmt.Sprintf("Seal all segments error %s", err), - }, nil - } - return &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_Success, - }, nil -} - -func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) { - if !s.checkStateIsHealthy() { - return &datapb.AssignSegmentIDResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - }, - }, nil - } - - assigns := make([]*datapb.SegmentIDAssignment, 0, len(req.SegmentIDRequests)) - - var appendFailedAssignment = func(err string) { - assigns = append(assigns, &datapb.SegmentIDAssignment{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - Reason: err, - }, - }) - } - - for _, r := range req.SegmentIDRequests { - if !s.meta.HasCollection(r.CollectionID) { - if err := s.loadCollectionFromMaster(ctx, r.CollectionID); err != nil { - appendFailedAssignment(fmt.Sprintf("can not load collection %d", r.CollectionID)) - log.Error("load collection from master error", zap.Int64("collectionID", r.CollectionID), zap.Error(err)) - continue - } - } - //if err := s.validateAllocRequest(r.CollectionID, r.PartitionID, r.ChannelName); err != nil { - //result.Status.Reason = err.Error() - //assigns = append(assigns, result) - //continue - //} - - segmentID, retCount, expireTs, err := s.segAllocator.AllocSegment(ctx, r.CollectionID, r.PartitionID, r.ChannelName, int64(r.Count)) - if err != nil { - appendFailedAssignment(fmt.Sprintf("allocation of collection %d, partition %d, channel %s, count %d error: %s", - r.CollectionID, r.PartitionID, r.ChannelName, r.Count, err.Error())) - continue - } - - result := &datapb.SegmentIDAssignment{ - SegID: segmentID, - ChannelName: r.ChannelName, - Count: uint32(retCount), - CollectionID: r.CollectionID, - PartitionID: r.PartitionID, - ExpireTime: expireTs, - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_Success, - Reason: "", - }, - } - assigns = append(assigns, result) - } - return &datapb.AssignSegmentIDResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_Success, - }, - SegIDAssignments: assigns, - }, nil -} - //func (s *Server) validateAllocRequest(collID UniqueID, partID UniqueID, channelName string) error { // if !s.meta.HasCollection(collID) { // return fmt.Errorf("can not find collection %d", collID) @@ -730,234 +528,38 @@ func (s *Server) loadCollectionFromMaster(ctx context.Context, collectionID int6 return s.meta.AddCollection(collInfo) } -func (s *Server) ShowSegments(ctx context.Context, req *datapb.ShowSegmentsRequest) (*datapb.ShowSegmentsResponse, error) { - resp := &datapb.ShowSegmentsResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - }, - } - if !s.checkStateIsHealthy() { - resp.Status.Reason = "server is initializing" - return resp, nil - } - ids := s.meta.GetSegmentsOfPartition(req.CollectionID, req.PartitionID) - resp.Status.ErrorCode = commonpb.ErrorCode_Success - resp.SegmentIDs = ids - return resp, nil -} - -func (s *Server) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) { - resp := &datapb.GetSegmentStatesResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - }, - } - if !s.checkStateIsHealthy() { - resp.Status.Reason = "server is initializing" - return resp, nil - } - - for _, segmentID := range req.SegmentIDs { - state := &datapb.SegmentStateInfo{ - Status: &commonpb.Status{}, - SegmentID: segmentID, - } - segmentInfo, err := s.meta.GetSegment(segmentID) - if err != nil { - state.Status.ErrorCode = commonpb.ErrorCode_UnexpectedError - state.Status.Reason = "get segment states error: " + err.Error() - } else { - state.Status.ErrorCode = commonpb.ErrorCode_Success - state.State = segmentInfo.State - state.StartPosition = segmentInfo.StartPosition - state.EndPosition = segmentInfo.EndPosition - } - resp.States = append(resp.States, state) - } - resp.Status.ErrorCode = commonpb.ErrorCode_Success - - return resp, nil -} - -func (s *Server) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsertBinlogPathsRequest) (*datapb.GetInsertBinlogPathsResponse, error) { - resp := &datapb.GetInsertBinlogPathsResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - }, - } - p := path.Join(Params.SegmentFlushMetaPath, strconv.FormatInt(req.SegmentID, 10)) - _, values, err := s.kvClient.LoadWithPrefix(p) - if err != nil { - resp.Status.Reason = err.Error() - return resp, nil - } - m := make(map[int64][]string) - tMeta := &datapb.SegmentFieldBinlogMeta{} - for _, v := range values { - if err := proto.UnmarshalText(v, tMeta); err != nil { - resp.Status.Reason = fmt.Errorf("DataService GetInsertBinlogPaths UnmarshalText datapb.SegmentFieldBinlogMeta err:%w", err).Error() - return resp, nil - } - m[tMeta.FieldID] = append(m[tMeta.FieldID], tMeta.BinlogPath) - } - - fids := make([]UniqueID, len(m)) - paths := make([]*internalpb.StringList, len(m)) - for k, v := range m { - fids = append(fids, k) - paths = append(paths, &internalpb.StringList{Values: v}) - } - resp.Status.ErrorCode = commonpb.ErrorCode_Success - resp.FieldIDs = fids - resp.Paths = paths - return resp, nil -} - -func (s *Server) GetInsertChannels(ctx context.Context, req *datapb.GetInsertChannelsRequest) (*internalpb.StringList, error) { - return &internalpb.StringList{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_Success, - }, - Values: s.insertChannels, - }, nil -} - -func (s *Server) GetCollectionStatistics(ctx context.Context, req *datapb.GetCollectionStatisticsRequest) (*datapb.GetCollectionStatisticsResponse, error) { - resp := &datapb.GetCollectionStatisticsResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - }, - } - nums, err := s.meta.GetNumRowsOfCollection(req.CollectionID) - if err != nil { - resp.Status.Reason = err.Error() - return resp, nil - } - resp.Status.ErrorCode = commonpb.ErrorCode_Success - resp.Stats = append(resp.Stats, &commonpb.KeyValuePair{Key: "row_count", Value: strconv.FormatInt(nums, 10)}) - return resp, nil -} - -func (s *Server) GetPartitionStatistics(ctx context.Context, req *datapb.GetPartitionStatisticsRequest) (*datapb.GetPartitionStatisticsResponse, error) { - resp := &datapb.GetPartitionStatisticsResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - }, - } - nums, err := s.meta.GetNumRowsOfPartition(req.CollectionID, req.PartitionID) - if err != nil { - resp.Status.Reason = err.Error() - return resp, nil - } - resp.Status.ErrorCode = commonpb.ErrorCode_Success - resp.Stats = append(resp.Stats, &commonpb.KeyValuePair{Key: "row_count", Value: strconv.FormatInt(nums, 10)}) - return resp, nil -} - -func (s *Server) GetSegmentInfoChannel(ctx context.Context) (*milvuspb.StringResponse, error) { - return &milvuspb.StringResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_Success, - }, - Value: Params.SegmentInfoChannelName, - }, nil -} - -func (s *Server) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) { - resp := &datapb.GetSegmentInfoResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - }, - } - if !s.checkStateIsHealthy() { - resp.Status.Reason = "data service is not healthy" - return resp, nil - } - infos := make([]*datapb.SegmentInfo, 0, len(req.SegmentIDs)) - for _, id := range req.SegmentIDs { - info, err := s.meta.GetSegment(id) - if err != nil { - resp.Status.Reason = err.Error() - return resp, nil - } - infos = append(infos, info) - } - resp.Status.ErrorCode = commonpb.ErrorCode_Success - resp.Infos = infos - return resp, nil -} - -// SaveBinlogPaths implement DataServiceServer -func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) { - resp := &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - } - if !s.checkStateIsHealthy() { - resp.Reason = "server is initializing" - return resp, nil - } - if s.flushMsgStream == nil { - resp.Reason = "flush msg stream nil" - return resp, nil - } - - // check segment id & collection id matched - _, err := s.meta.GetCollection(req.GetCollectionID()) - if err != nil { - log.Error("Failed to get collection info", zap.Int64("collectionID", req.GetCollectionID()), zap.Error(err)) - resp.Reason = err.Error() - return resp, err - } - +func (s *Server) prepareBinlogAndPos(req *datapb.SaveBinlogPathsRequest) (map[string]string, error) { + meta := make(map[string]string) segInfo, err := s.meta.GetSegment(req.GetSegmentID()) if err != nil { log.Error("Failed to get segment info", zap.Int64("segmentID", req.GetSegmentID()), zap.Error(err)) - resp.Reason = err.Error() - return resp, err + return nil, err } log.Debug("segment", zap.Int64("segment", segInfo.CollectionID)) - meta := make(map[string]string) fieldMeta, err := s.prepareField2PathMeta(req.SegmentID, req.Field2BinlogPaths) if err != nil { - resp.Reason = err.Error() - return resp, err + return nil, err } for k, v := range fieldMeta { meta[k] = v } ddlMeta, err := s.prepareDDLBinlogMeta(req.CollectionID, req.GetDdlBinlogPaths()) if err != nil { - resp.Reason = err.Error() - return resp, err + return nil, err } for k, v := range ddlMeta { meta[k] = v } segmentPos, err := s.prepareSegmentPos(segInfo, req.GetDmlPosition(), req.GetDdlPosition()) if err != nil { - resp.Reason = err.Error() - return resp, err + return nil, err } for k, v := range segmentPos { meta[k] = v } - // Save into k-v store - err = s.SaveBinLogMetaTxn(meta) - if err != nil { - resp.Reason = err.Error() - return resp, err - } - // write flush msg into segmentInfo/flush stream - msgPack := composeSegmentFlushMsgPack(req.SegmentID) - err = s.flushMsgStream.Produce(&msgPack) - if err != nil { - resp.Reason = err.Error() - return resp, err - } - resp.ErrorCode = commonpb.ErrorCode_Success - return resp, nil + return meta, nil } func (s *Server) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error) { diff --git a/internal/dataservice/server_test.go b/internal/dataservice/server_test.go index 984312dca5..cff29be7d7 100644 --- a/internal/dataservice/server_test.go +++ b/internal/dataservice/server_test.go @@ -223,7 +223,7 @@ func TestFlush(t *testing.T) { }) assert.Nil(t, err) assert.EqualValues(t, commonpb.ErrorCode_Success, resp.ErrorCode) - ids, err := svr.segAllocator.GetFlushableSegments(context.TODO(), expireTs) + ids, err := svr.segAllocator.GetFlushableSegments(context.TODO(), "channel-1", expireTs) assert.Nil(t, err) assert.EqualValues(t, 1, len(ids)) assert.EqualValues(t, segID, ids[0]) @@ -606,6 +606,107 @@ func TestSaveBinlogPaths(t *testing.T) { }) } +func TestDataNodeTtChannel(t *testing.T) { + svr := newTestServer(t) + defer closeTestServer(t, svr) + + svr.meta.AddCollection(&datapb.CollectionInfo{ + ID: 0, + Schema: newTestSchema(), + Partitions: []int64{0}, + }) + + ch := make(chan interface{}, 1) + svr.createDataNodeClient = func(addr string) types.DataNode { + cli := newMockDataNodeClient(0) + cli.ch = ch + return cli + } + + ttMsgStream, err := svr.msFactory.NewMsgStream(context.TODO()) + assert.Nil(t, err) + ttMsgStream.AsProducer([]string{Params.TimeTickChannelName}) + ttMsgStream.Start() + defer ttMsgStream.Close() + + genMsg := func(msgType commonpb.MsgType, ch string, t Timestamp) *msgstream.DataNodeTtMsg { + return &msgstream.DataNodeTtMsg{ + BaseMsg: msgstream.BaseMsg{ + HashValues: []uint32{0}, + }, + DataNodeTtMsg: datapb.DataNodeTtMsg{ + Base: &commonpb.MsgBase{ + MsgType: msgType, + MsgID: 0, + Timestamp: t, + SourceID: 0, + }, + ChannelName: ch, + Timestamp: t, + }, + } + } + + resp, err := svr.RegisterNode(context.TODO(), &datapb.RegisterNodeRequest{ + Base: &commonpb.MsgBase{ + MsgType: 0, + MsgID: 0, + Timestamp: 0, + SourceID: 0, + }, + Address: &commonpb.Address{ + Ip: "localhost:7777", + Port: 8080, + }, + }) + assert.Nil(t, err) + assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) + + t.Run("Test segment flush after tt", func(t *testing.T) { + resp, err := svr.AssignSegmentID(context.TODO(), &datapb.AssignSegmentIDRequest{ + NodeID: 0, + PeerRole: "", + SegmentIDRequests: []*datapb.SegmentIDRequest{ + { + CollectionID: 0, + PartitionID: 0, + ChannelName: "ch-1", + Count: 100, + }, + }, + }) + + assert.Nil(t, err) + assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) + assert.EqualValues(t, 1, len(resp.SegIDAssignments)) + assign := resp.SegIDAssignments[0] + + resp2, err := svr.Flush(context.TODO(), &datapb.FlushRequest{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_Flush, + MsgID: 0, + Timestamp: 0, + SourceID: 0, + }, + DbID: 0, + CollectionID: 0, + }) + assert.Nil(t, err) + assert.EqualValues(t, commonpb.ErrorCode_Success, resp2.ErrorCode) + + msgPack := msgstream.MsgPack{} + msg := genMsg(commonpb.MsgType_DataNodeTt, "ch-1", assign.ExpireTime) + msgPack.Msgs = append(msgPack.Msgs, msg) + ttMsgStream.Produce(&msgPack) + + flushMsg := <-ch + flushReq := flushMsg.(*datapb.FlushSegmentsRequest) + assert.EqualValues(t, 1, len(flushReq.SegmentIDs)) + assert.EqualValues(t, assign.SegID, flushReq.SegmentIDs[0]) + }) + +} + func TestResumeChannel(t *testing.T) { Params.Init() @@ -702,69 +803,6 @@ func TestResumeChannel(t *testing.T) { } }) - t.Run("Test ResumeSegmentFlushChannel", func(t *testing.T) { - genMsg := func(msgType commonpb.MsgType, t Timestamp, segID int64) *msgstream.FlushCompletedMsg { - return &msgstream.FlushCompletedMsg{ - BaseMsg: msgstream.BaseMsg{ - HashValues: []uint32{0}, - }, - SegmentFlushCompletedMsg: internalpb.SegmentFlushCompletedMsg{ - Base: &commonpb.MsgBase{ - MsgType: msgType, - MsgID: 0, - Timestamp: t, - SourceID: 0, - }, - SegmentID: segID, - }, - } - } - svr := newTestServer(t) - - ch := make(chan struct{}) - - segInfoStream, _ := svr.msFactory.NewMsgStream(svr.ctx) - segInfoStream.AsProducer([]string{Params.SegmentInfoChannelName}) - segInfoStream.Start() - defer segInfoStream.Close() - go func() { - for _, segID := range segmentIDs { - - msgPack := msgstream.MsgPack{} - msgPack.Msgs = append(msgPack.Msgs, genMsg(commonpb.MsgType_SegmentFlushDone, uint64(time.Now().Unix()), segID)) - - err := segInfoStream.Produce(&msgPack) - assert.Nil(t, err) - time.Sleep(time.Millisecond * 5) - } - ch <- struct{}{} - }() - - time.Sleep(time.Millisecond * 50) - //stop current server, simulating server quit - svr.Stop() - - time.Sleep(time.Second) - // start new test server as restarting - svr = newTestServer(t) - defer svr.Stop() - <-ch - - //wait for Server processing last messages - time.Sleep(time.Second) - - //ASSERT PART - svr.meta.RLock() - defer svr.meta.RUnlock() - for _, segID := range segmentIDs { - seg, has := svr.meta.segments[segID] - assert.True(t, has) - if has { - assert.Equal(t, seg.State, commonpb.SegmentState_Flushed) - } - } - }) - t.Run("Clean up test segments", func(t *testing.T) { svr := newTestServer(t) defer closeTestServer(t, svr) diff --git a/internal/dataservice/watcher.go b/internal/dataservice/watcher.go deleted file mode 100644 index 0b68d8f68a..0000000000 --- a/internal/dataservice/watcher.go +++ /dev/null @@ -1,91 +0,0 @@ -// 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 ( - "github.com/milvus-io/milvus/internal/log" - "github.com/milvus-io/milvus/internal/proto/commonpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/util/trace" - "go.uber.org/zap" - - "golang.org/x/net/context" - - "github.com/milvus-io/milvus/internal/msgstream" -) - -type dataNodeTimeTickWatcher struct { - meta *meta - cluster *dataNodeCluster - allocator segmentAllocatorInterface - msgQueue chan *msgstream.TimeTickMsg -} - -func newDataNodeTimeTickWatcher(meta *meta, allocator segmentAllocatorInterface, cluster *dataNodeCluster) *dataNodeTimeTickWatcher { - return &dataNodeTimeTickWatcher{ - meta: meta, - allocator: allocator, - cluster: cluster, - msgQueue: make(chan *msgstream.TimeTickMsg, 1), - } -} - -func (watcher *dataNodeTimeTickWatcher) Watch(msg *msgstream.TimeTickMsg) { - watcher.msgQueue <- msg -} - -func (watcher *dataNodeTimeTickWatcher) StartBackgroundLoop(ctx context.Context) { - for { - select { - case <-ctx.Done(): - log.Debug("data node time tick watcher closed") - return - case msg := <-watcher.msgQueue: - if err := watcher.handleTimeTickMsg(msg); err != nil { - log.Error("handle time tick error", zap.Error(err)) - } - } - } -} - -func (watcher *dataNodeTimeTickWatcher) handleTimeTickMsg(msg *msgstream.TimeTickMsg) error { - ctx := context.TODO() - sp, _ := trace.StartSpanFromContext(ctx) - defer sp.Finish() - coll2Segs := make(map[UniqueID][]UniqueID) - segments, err := watcher.allocator.GetFlushableSegments(ctx, msg.Base.Timestamp) - if err != nil { - return err - } - for _, id := range segments { - sInfo, err := watcher.meta.GetSegment(id) - if err != nil { - log.Error("get segment from meta error", zap.Int64("segmentID", id), zap.Error(err)) - continue - } - collID, segID := sInfo.CollectionID, sInfo.ID - coll2Segs[collID] = append(coll2Segs[collID], segID) - watcher.allocator.DropSegment(ctx, id) - } - for collID, segIDs := range coll2Segs { - watcher.cluster.FlushSegment(&datapb.FlushSegmentsRequest{ - Base: &commonpb.MsgBase{ - MsgType: commonpb.MsgType_Flush, - MsgID: -1, // todo add msg id - Timestamp: 0, // todo - SourceID: Params.NodeID, - }, - CollectionID: collID, - SegmentIDs: segIDs, - }) - } - return nil -} diff --git a/internal/dataservice/watcher_test.go b/internal/dataservice/watcher_test.go deleted file mode 100644 index f5f23bb48c..0000000000 --- a/internal/dataservice/watcher_test.go +++ /dev/null @@ -1,111 +0,0 @@ -// 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 ( - "context" - "strconv" - "testing" - "time" - - "github.com/milvus-io/milvus/internal/msgstream" - "github.com/milvus-io/milvus/internal/proto/commonpb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - - "github.com/stretchr/testify/assert" -) - -func TestWatcher(t *testing.T) { - const collID = UniqueID(0) - const partID = UniqueID(100) - - Params.Init() - - cluster := newDataNodeCluster() - defer cluster.ShutDownClients() - schema := newTestSchema() - allocator := newMockAllocator() - meta, err := newMemoryMeta(allocator) - assert.Nil(t, err) - segAllocator := newSegmentAllocator(meta, allocator) - assert.Nil(t, err) - - collInfo := &datapb.CollectionInfo{ - Schema: schema, - ID: collID, - } - - t.Run("Test DataNodeTimeTickWatcher", func(t *testing.T) { - datanodeWatcher := newDataNodeTimeTickWatcher(meta, segAllocator, cluster) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - go datanodeWatcher.StartBackgroundLoop(ctx) - - err = meta.AddCollection(collInfo) - assert.Nil(t, err) - - cases := []struct { - sealed bool - expired bool - expected bool - }{ - {false, true, false}, - {false, true, false}, - {false, false, false}, - {true, true, true}, - {true, false, false}, - {true, true, true}, - } - - segIDs := make([]UniqueID, 0) - for i := range cases { - segID, _, _, err := segAllocator.AllocSegment(ctx, collID, partID, "channel"+strconv.Itoa(i), 100) - assert.Nil(t, err) - segIDs = append(segIDs, segID) - } - - time.Sleep(time.Duration(Params.SegIDAssignExpiration+1000) * time.Millisecond) - for i, c := range cases { - if !c.expired { - _, _, _, err := segAllocator.AllocSegment(ctx, collID, partID, "channel"+strconv.Itoa(i), 100) - assert.Nil(t, err) - } - if c.sealed { - err := segAllocator.SealSegment(ctx, segIDs[i]) - assert.Nil(t, err) - } - } - ts, err := allocator.allocTimestamp() - assert.Nil(t, err) - - ttMsg := &msgstream.TimeTickMsg{ - BaseMsg: msgstream.BaseMsg{ - HashValues: []uint32{0}, - }, - TimeTickMsg: internalpb.TimeTickMsg{ - Base: &commonpb.MsgBase{ - MsgType: commonpb.MsgType_TimeTick, - Timestamp: ts, - }, - }, - } - datanodeWatcher.Watch(ttMsg) - - time.Sleep(time.Second) - - // check flushed segments been removed from segAllocator - for i, c := range cases { - ok := segAllocator.HasSegment(ctx, segIDs[i]) - assert.EqualValues(t, !c.expected, ok) - } - }) -} diff --git a/internal/msgstream/msg.go b/internal/msgstream/msg.go index d79a261056..7e15bbd539 100644 --- a/internal/msgstream/msg.go +++ b/internal/msgstream/msg.go @@ -933,3 +933,48 @@ func (l *LoadBalanceSegmentsMsg) Unmarshal(input MarshalType) (TsMsg, error) { return loadMsg, nil } + +type DataNodeTtMsg struct { + BaseMsg + datapb.DataNodeTtMsg +} + +func (m *DataNodeTtMsg) TraceCtx() context.Context { + return m.BaseMsg.Ctx +} + +func (m *DataNodeTtMsg) SetTraceCtx(ctx context.Context) { + m.BaseMsg.Ctx = ctx +} + +func (m *DataNodeTtMsg) ID() UniqueID { + return m.Base.MsgID +} + +func (m *DataNodeTtMsg) Type() MsgType { + return m.Base.MsgType +} + +func (m *DataNodeTtMsg) Marshal(input TsMsg) (MarshalType, error) { + msg := input.(*DataNodeTtMsg) + t, err := proto.Marshal(&msg.DataNodeTtMsg) + if err != nil { + return nil, err + } + return t, nil +} + +func (m *DataNodeTtMsg) Unmarshal(input MarshalType) (TsMsg, error) { + msg := datapb.DataNodeTtMsg{} + in, err := ConvertToByteArray(input) + if err != nil { + return nil, err + } + err = proto.Unmarshal(in, &msg) + if err != nil { + return nil, err + } + return &DataNodeTtMsg{ + DataNodeTtMsg: msg, + }, nil +} diff --git a/internal/msgstream/unmarshal.go b/internal/msgstream/unmarshal.go index b1b32e615b..7dc573afb0 100644 --- a/internal/msgstream/unmarshal.go +++ b/internal/msgstream/unmarshal.go @@ -64,6 +64,7 @@ func (pudf *ProtoUDFactory) NewUnmarshalDispatcher() *ProtoUnmarshalDispatcher { queryNodeSegStatsMsg := QueryNodeStatsMsg{} segmentStatisticsMsg := SegmentStatisticsMsg{} loadBalanceSegmentsMsg := LoadBalanceSegmentsMsg{} + dataNodeTtMsg := DataNodeTtMsg{} p := &ProtoUnmarshalDispatcher{} p.TempMap = make(map[commonpb.MsgType]UnmarshalFunc) @@ -82,6 +83,7 @@ func (pudf *ProtoUDFactory) NewUnmarshalDispatcher() *ProtoUnmarshalDispatcher { p.TempMap[commonpb.MsgType_SegmentFlushDone] = flushCompletedMsg.Unmarshal p.TempMap[commonpb.MsgType_SegmentStatistics] = segmentStatisticsMsg.Unmarshal p.TempMap[commonpb.MsgType_LoadBalanceSegments] = loadBalanceSegmentsMsg.Unmarshal + p.TempMap[commonpb.MsgType_DataNodeTt] = dataNodeTtMsg.Unmarshal return p } diff --git a/internal/proto/common.proto b/internal/proto/common.proto index 6088acd06a..052912b298 100644 --- a/internal/proto/common.proto +++ b/internal/proto/common.proto @@ -48,7 +48,8 @@ enum SegmentState { NotExist = 1; Growing = 2; Sealed = 3; - Flushed = 4; + Flushing = 4; + Flushed = 5; } message Status { @@ -135,6 +136,8 @@ enum MsgType { AllocateSegment = 1205; SegmentStatistics = 1206; SegmentFlushDone = 1207; + + DataNodeTt = 1208; } message MsgBase { diff --git a/internal/proto/commonpb/common.pb.go b/internal/proto/commonpb/common.pb.go index 2ade263d66..9a85b62412 100644 --- a/internal/proto/commonpb/common.pb.go +++ b/internal/proto/commonpb/common.pb.go @@ -159,7 +159,8 @@ const ( SegmentState_NotExist SegmentState = 1 SegmentState_Growing SegmentState = 2 SegmentState_Sealed SegmentState = 3 - SegmentState_Flushed SegmentState = 4 + SegmentState_Flushing SegmentState = 4 + SegmentState_Flushed SegmentState = 5 ) var SegmentState_name = map[int32]string{ @@ -167,7 +168,8 @@ var SegmentState_name = map[int32]string{ 1: "NotExist", 2: "Growing", 3: "Sealed", - 4: "Flushed", + 4: "Flushing", + 5: "Flushed", } var SegmentState_value = map[string]int32{ @@ -175,7 +177,8 @@ var SegmentState_value = map[string]int32{ "NotExist": 1, "Growing": 2, "Sealed": 3, - "Flushed": 4, + "Flushing": 4, + "Flushed": 5, } func (x SegmentState) String() string { @@ -246,6 +249,7 @@ const ( MsgType_AllocateSegment MsgType = 1205 MsgType_SegmentStatistics MsgType = 1206 MsgType_SegmentFlushDone MsgType = 1207 + MsgType_DataNodeTt MsgType = 1208 ) var MsgType_name = map[int32]string{ @@ -298,6 +302,7 @@ var MsgType_name = map[int32]string{ 1205: "AllocateSegment", 1206: "SegmentStatistics", 1207: "SegmentFlushDone", + 1208: "DataNodeTt", } var MsgType_value = map[string]int32{ @@ -350,6 +355,7 @@ var MsgType_value = map[string]int32{ "AllocateSegment": 1205, "SegmentStatistics": 1206, "SegmentFlushDone": 1207, + "DataNodeTt": 1208, } func (x MsgType) String() string { @@ -685,84 +691,85 @@ func init() { func init() { proto.RegisterFile("common.proto", fileDescriptor_555bd8c177793206) } var fileDescriptor_555bd8c177793206 = []byte{ - // 1263 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x55, 0xcb, 0x72, 0xdb, 0x36, - 0x17, 0x36, 0x45, 0xc5, 0x32, 0x61, 0xd9, 0x86, 0xe1, 0x4b, 0x9c, 0xfc, 0x9e, 0x7f, 0x32, 0x5e, - 0x65, 0x3c, 0x13, 0xbb, 0x6d, 0xa6, 0xed, 0x2a, 0x8b, 0x58, 0x8c, 0x6d, 0x4d, 0xe2, 0x4b, 0x29, - 0x27, 0xcd, 0x74, 0x93, 0x81, 0xc9, 0x63, 0x09, 0x0d, 0x09, 0xa8, 0x00, 0xe8, 0x58, 0x6f, 0xd1, - 0xe6, 0x1d, 0xba, 0x6b, 0x3b, 0xbd, 0xf7, 0x15, 0x7a, 0x5f, 0xf7, 0x11, 0xba, 0xef, 0x3d, 0xd7, - 0xce, 0x01, 0x29, 0x89, 0x99, 0x49, 0x77, 0x3c, 0xdf, 0xb9, 0xe0, 0xc3, 0x77, 0xce, 0x01, 0x49, - 0x33, 0x56, 0x59, 0xa6, 0xe4, 0x46, 0x5f, 0x2b, 0xab, 0xd8, 0x42, 0x26, 0xd2, 0xd3, 0xdc, 0x14, - 0xd6, 0x46, 0xe1, 0x5a, 0xbb, 0x47, 0x26, 0x3b, 0x96, 0xdb, 0xdc, 0xb0, 0x6b, 0x84, 0x80, 0xd6, - 0x4a, 0xdf, 0x8b, 0x55, 0x02, 0x2b, 0xde, 0x25, 0xef, 0xf2, 0xec, 0x6b, 0xff, 0xdf, 0x78, 0x49, - 0xce, 0xc6, 0x0d, 0x0c, 0x6b, 0xa9, 0x04, 0xa2, 0x00, 0x86, 0x9f, 0x6c, 0x99, 0x4c, 0x6a, 0xe0, - 0x46, 0xc9, 0x95, 0xda, 0x25, 0xef, 0x72, 0x10, 0x95, 0xd6, 0xda, 0x1b, 0xa4, 0x79, 0x13, 0x06, - 0x77, 0x78, 0x9a, 0xc3, 0x21, 0x17, 0x9a, 0x51, 0xe2, 0xdf, 0x87, 0x81, 0xab, 0x1f, 0x44, 0xf8, - 0xc9, 0x16, 0xc9, 0xb9, 0x53, 0x74, 0x97, 0x89, 0x85, 0xb1, 0xb6, 0x4a, 0xea, 0x5b, 0xa9, 0x3a, - 0x1e, 0x7b, 0x31, 0xa3, 0x39, 0xf4, 0x5e, 0x21, 0x8d, 0xeb, 0x49, 0xa2, 0xc1, 0x18, 0x36, 0x4b, - 0x6a, 0xa2, 0x5f, 0xd6, 0xab, 0x89, 0x3e, 0x63, 0xa4, 0xde, 0x57, 0xda, 0xba, 0x6a, 0x7e, 0xe4, - 0xbe, 0xd7, 0x1e, 0x7a, 0xa4, 0xb1, 0x67, 0xba, 0x5b, 0xdc, 0x00, 0x7b, 0x93, 0x4c, 0x65, 0xa6, - 0x7b, 0xcf, 0x0e, 0xfa, 0xc3, 0x5b, 0xae, 0xbe, 0xf4, 0x96, 0x7b, 0xa6, 0x7b, 0x34, 0xe8, 0x43, - 0xd4, 0xc8, 0x8a, 0x0f, 0x64, 0x92, 0x99, 0x6e, 0x3b, 0x2c, 0x2b, 0x17, 0x06, 0x5b, 0x25, 0x81, - 0x15, 0x19, 0x18, 0xcb, 0xb3, 0xfe, 0x8a, 0x7f, 0xc9, 0xbb, 0x5c, 0x8f, 0xc6, 0x00, 0xbb, 0x48, - 0xa6, 0x8c, 0xca, 0x75, 0x0c, 0xed, 0x70, 0xa5, 0xee, 0xd2, 0x46, 0xf6, 0xda, 0x35, 0x12, 0xec, - 0x99, 0xee, 0x2e, 0xf0, 0x04, 0x34, 0x7b, 0x85, 0xd4, 0x8f, 0xb9, 0x29, 0x18, 0x4d, 0xff, 0x37, - 0x23, 0xbc, 0x41, 0xe4, 0x22, 0xd7, 0x3f, 0xac, 0x93, 0x60, 0xd4, 0x09, 0x36, 0x4d, 0x1a, 0x9d, - 0x3c, 0x8e, 0xc1, 0x18, 0x3a, 0xc1, 0x16, 0xc8, 0xdc, 0x6d, 0x09, 0x67, 0x7d, 0x88, 0x2d, 0x24, - 0x2e, 0x86, 0x7a, 0x6c, 0x9e, 0xcc, 0xb4, 0x94, 0x94, 0x10, 0xdb, 0x6d, 0x2e, 0x52, 0x48, 0x68, - 0x8d, 0x2d, 0x12, 0x7a, 0x08, 0x3a, 0x13, 0xc6, 0x08, 0x25, 0x43, 0x90, 0x02, 0x12, 0xea, 0xb3, - 0xf3, 0x64, 0xa1, 0xa5, 0xd2, 0x14, 0x62, 0x2b, 0x94, 0xdc, 0x57, 0xf6, 0xc6, 0x99, 0x30, 0xd6, - 0xd0, 0x3a, 0x96, 0x6d, 0xa7, 0x29, 0x74, 0x79, 0x7a, 0x5d, 0x77, 0xf3, 0x0c, 0xa4, 0xa5, 0xe7, - 0xb0, 0x46, 0x09, 0x86, 0x22, 0x03, 0x89, 0x95, 0x68, 0xa3, 0x82, 0xb6, 0x65, 0x02, 0x67, 0xa8, - 0x1f, 0x9d, 0x62, 0x17, 0xc8, 0x52, 0x89, 0x56, 0x0e, 0xe0, 0x19, 0xd0, 0x80, 0xcd, 0x91, 0xe9, - 0xd2, 0x75, 0x74, 0x70, 0x78, 0x93, 0x92, 0x4a, 0x85, 0x48, 0x3d, 0x88, 0x20, 0x56, 0x3a, 0xa1, - 0xd3, 0x15, 0x0a, 0x77, 0x20, 0xb6, 0x4a, 0xb7, 0x43, 0xda, 0x44, 0xc2, 0x25, 0xd8, 0x01, 0xae, - 0xe3, 0x5e, 0x04, 0x26, 0x4f, 0x2d, 0x9d, 0x61, 0x94, 0x34, 0xb7, 0x45, 0x0a, 0xfb, 0xca, 0x6e, - 0xab, 0x5c, 0x26, 0x74, 0x96, 0xcd, 0x12, 0xb2, 0x07, 0x96, 0x97, 0x0a, 0xcc, 0xe1, 0xb1, 0x2d, - 0x1e, 0xf7, 0xa0, 0x04, 0x28, 0x5b, 0x26, 0xac, 0xc5, 0xa5, 0x54, 0xb6, 0xa5, 0x81, 0x5b, 0xd8, - 0x56, 0x69, 0x02, 0x9a, 0xce, 0x23, 0x9d, 0x17, 0x70, 0x91, 0x02, 0x65, 0xe3, 0xe8, 0x10, 0x52, - 0x18, 0x45, 0x2f, 0x8c, 0xa3, 0x4b, 0x1c, 0xa3, 0x17, 0x91, 0xfc, 0x56, 0x2e, 0xd2, 0xc4, 0x49, - 0x52, 0xb4, 0x65, 0x09, 0x39, 0x96, 0xe4, 0xf7, 0x6f, 0xb5, 0x3b, 0x47, 0x74, 0x99, 0x2d, 0x91, - 0xf9, 0x12, 0xd9, 0x03, 0xab, 0x45, 0xec, 0xc4, 0x3b, 0x8f, 0x54, 0x0f, 0x72, 0x7b, 0x70, 0xb2, - 0x07, 0x99, 0xd2, 0x03, 0xba, 0x82, 0x0d, 0x75, 0x95, 0x86, 0x2d, 0xa2, 0x17, 0x18, 0x23, 0x33, - 0x61, 0x18, 0xc1, 0x7b, 0x39, 0x18, 0x1b, 0xf1, 0x18, 0xe8, 0xaf, 0x8d, 0xf5, 0xbb, 0x84, 0xb8, - 0x30, 0x5c, 0x73, 0x60, 0x8c, 0xcc, 0x8e, 0xad, 0x7d, 0x25, 0x81, 0x4e, 0xb0, 0x26, 0x99, 0xba, - 0x2d, 0x85, 0x31, 0x39, 0x24, 0xd4, 0x43, 0x89, 0xda, 0xf2, 0x50, 0xab, 0x2e, 0x6e, 0x17, 0xad, - 0xa1, 0x77, 0x5b, 0x48, 0x61, 0x7a, 0x6e, 0x38, 0x08, 0x99, 0x2c, 0xb5, 0xaa, 0xaf, 0xdf, 0x25, - 0xcd, 0x0e, 0x74, 0x71, 0x0e, 0x8a, 0xda, 0x8b, 0x84, 0x56, 0xed, 0x71, 0xf5, 0x11, 0x43, 0x0f, - 0xe7, 0x74, 0x47, 0xab, 0x07, 0x42, 0x76, 0x69, 0x0d, 0x8b, 0x75, 0x80, 0xa7, 0xae, 0xf0, 0x34, - 0x69, 0x6c, 0xa7, 0xb9, 0x3b, 0xa5, 0xbe, 0xfe, 0x5b, 0xc3, 0xed, 0xab, 0x5b, 0xbb, 0x19, 0x12, - 0xdc, 0x96, 0x09, 0x9c, 0x08, 0x09, 0x09, 0x9d, 0x70, 0xd2, 0xba, 0x16, 0x8c, 0x47, 0x88, 0x26, - 0x78, 0xad, 0x50, 0xab, 0x7e, 0x05, 0x03, 0xd4, 0x67, 0x97, 0x9b, 0x0a, 0x74, 0x82, 0xfd, 0x0a, - 0xc1, 0xc4, 0x5a, 0x1c, 0x57, 0xd3, 0xbb, 0xd8, 0x99, 0x4e, 0x4f, 0x3d, 0x18, 0x63, 0x86, 0xf6, - 0xf0, 0xa4, 0x1d, 0xb0, 0x9d, 0x81, 0xb1, 0x90, 0xb5, 0x94, 0x3c, 0x11, 0x5d, 0x43, 0x05, 0x9e, - 0x74, 0x4b, 0xf1, 0xa4, 0x92, 0xfe, 0x2e, 0x76, 0x2c, 0x82, 0x14, 0xb8, 0xa9, 0x56, 0xbd, 0xcf, - 0x16, 0xc9, 0x5c, 0x41, 0xf5, 0x90, 0x6b, 0x2b, 0x1c, 0xf8, 0xad, 0xe7, 0x7a, 0xa4, 0x55, 0x7f, - 0x8c, 0x7d, 0x87, 0xbb, 0xd9, 0xdc, 0xe5, 0x66, 0x0c, 0x7d, 0xef, 0xb1, 0x65, 0x32, 0x3f, 0xa4, - 0x3a, 0xc6, 0x7f, 0xf0, 0xd8, 0x02, 0x99, 0x45, 0xaa, 0x23, 0xcc, 0xd0, 0x1f, 0x1d, 0x88, 0xa4, - 0x2a, 0xe0, 0x4f, 0xae, 0x42, 0xc9, 0xaa, 0x82, 0xff, 0xec, 0x0e, 0xc3, 0x0a, 0x65, 0xab, 0x0c, - 0x7d, 0xe4, 0x21, 0xd3, 0xe1, 0x61, 0x25, 0x4c, 0x1f, 0xbb, 0x40, 0xac, 0x3a, 0x0a, 0x7c, 0xe2, - 0x02, 0xcb, 0x9a, 0x23, 0xf4, 0xa9, 0x43, 0x77, 0xb9, 0x4c, 0xd4, 0xc9, 0xc9, 0x08, 0x7d, 0xe6, - 0xb1, 0x15, 0xb2, 0x80, 0xe9, 0x5b, 0x3c, 0xe5, 0x32, 0x1e, 0xc7, 0x3f, 0xf7, 0x18, 0x25, 0xd3, - 0x85, 0x30, 0x6e, 0x14, 0xe9, 0x47, 0x35, 0x27, 0x4a, 0x49, 0xa0, 0xc0, 0x3e, 0xae, 0xb1, 0x59, - 0x12, 0xa0, 0x50, 0x85, 0xfd, 0x49, 0x8d, 0x4d, 0x93, 0xc9, 0xb6, 0x34, 0xa0, 0x2d, 0x7d, 0x1f, - 0xc7, 0x65, 0xb2, 0xd8, 0x2d, 0xfa, 0x01, 0x0e, 0xe5, 0x39, 0x37, 0x3b, 0xf4, 0xa1, 0x73, 0x14, - 0xaf, 0x00, 0xfd, 0xdd, 0x77, 0x57, 0xad, 0x3e, 0x09, 0x7f, 0xf8, 0x78, 0xd2, 0x0e, 0xd8, 0xf1, - 0x0e, 0xd0, 0x3f, 0x7d, 0x76, 0x91, 0x2c, 0x0d, 0x31, 0xb7, 0xa0, 0xa3, 0xe9, 0xff, 0xcb, 0x67, - 0xab, 0xe4, 0xfc, 0x0e, 0xd8, 0x71, 0x5f, 0x31, 0x49, 0x18, 0x2b, 0x62, 0x43, 0xff, 0xf6, 0xd9, - 0xff, 0xc8, 0xf2, 0x0e, 0xd8, 0x91, 0xbe, 0x15, 0xe7, 0x3f, 0x3e, 0x9b, 0x21, 0x53, 0x11, 0x6e, - 0x30, 0x9c, 0x02, 0x7d, 0xe4, 0x63, 0x93, 0x86, 0x66, 0x49, 0xe7, 0xb1, 0x8f, 0xd2, 0xbd, 0xcd, - 0x6d, 0xdc, 0x0b, 0xb3, 0x56, 0x8f, 0x4b, 0x09, 0xa9, 0xa1, 0x4f, 0x7c, 0xb6, 0x44, 0x68, 0x04, - 0x99, 0x3a, 0x85, 0x0a, 0xfc, 0x14, 0x5f, 0x66, 0xe6, 0x82, 0xdf, 0xca, 0x41, 0x0f, 0x46, 0x8e, - 0x67, 0x3e, 0x4a, 0x5d, 0xc4, 0xbf, 0xe8, 0x79, 0xee, 0xa3, 0xd4, 0xa5, 0xf2, 0x6d, 0x79, 0xa2, - 0xe8, 0x2f, 0x75, 0x64, 0x75, 0x24, 0x32, 0x38, 0x12, 0xf1, 0x7d, 0xfa, 0x69, 0x80, 0xac, 0x5c, - 0xd2, 0xbe, 0x4a, 0x00, 0xe9, 0x1b, 0xfa, 0x59, 0x80, 0xd2, 0x63, 0xeb, 0x0a, 0xe9, 0x3f, 0x77, - 0x76, 0xf9, 0xaa, 0xb4, 0x43, 0xfa, 0x05, 0xbe, 0xd6, 0xa4, 0xb4, 0x8f, 0x3a, 0x07, 0xf4, 0xcb, - 0x00, 0xaf, 0x71, 0x3d, 0x4d, 0x55, 0xcc, 0xed, 0x68, 0x80, 0xbe, 0x0a, 0x70, 0x02, 0x2b, 0x0f, - 0x42, 0x29, 0xcc, 0xd7, 0x01, 0x5e, 0xaf, 0xc4, 0x5d, 0xdb, 0x42, 0x7c, 0x28, 0xbe, 0x09, 0xd6, - 0xd7, 0x48, 0x23, 0x34, 0xa9, 0x5b, 0xfa, 0x06, 0xf1, 0x43, 0x93, 0xd2, 0x09, 0x7c, 0x8d, 0xb6, - 0x94, 0x4a, 0x6f, 0x9c, 0xf5, 0xf5, 0x9d, 0x57, 0xa9, 0xb7, 0xf5, 0xfa, 0x3b, 0x57, 0xbb, 0xc2, - 0xf6, 0xf2, 0x63, 0xfc, 0x29, 0x6e, 0x16, 0x7f, 0xc9, 0x2b, 0x42, 0x95, 0x5f, 0x9b, 0x42, 0x5a, - 0xd0, 0x92, 0xa7, 0x9b, 0xee, 0xc7, 0xb9, 0x59, 0xfc, 0x38, 0xfb, 0xc7, 0xc7, 0x93, 0xce, 0xbe, - 0xfa, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x16, 0x2d, 0x7a, 0x6b, 0x12, 0x09, 0x00, 0x00, + // 1280 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, } diff --git a/internal/proto/data_service.proto b/internal/proto/data_service.proto index 80d9ea4670..5cb1f5496f 100644 --- a/internal/proto/data_service.proto +++ b/internal/proto/data_service.proto @@ -288,3 +288,9 @@ message GetRecoveryInfoRequest { int64 partitionID = 3; } +message DataNodeTtMsg { + common.MsgBase base =1; + string channel_name = 2; + uint64 timestamp = 3; +} + diff --git a/internal/proto/datapb/data_service.pb.go b/internal/proto/datapb/data_service.pb.go index 6b08e673ad..b240225e49 100644 --- a/internal/proto/datapb/data_service.pb.go +++ b/internal/proto/datapb/data_service.pb.go @@ -2277,6 +2277,61 @@ func (m *GetRecoveryInfoRequest) GetPartitionID() int64 { return 0 } +type DataNodeTtMsg struct { + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + ChannelName string `protobuf:"bytes,2,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DataNodeTtMsg) Reset() { *m = DataNodeTtMsg{} } +func (m *DataNodeTtMsg) String() string { return proto.CompactTextString(m) } +func (*DataNodeTtMsg) ProtoMessage() {} +func (*DataNodeTtMsg) Descriptor() ([]byte, []int) { + return fileDescriptor_3385cd32ad6cfe64, []int{41} +} + +func (m *DataNodeTtMsg) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DataNodeTtMsg.Unmarshal(m, b) +} +func (m *DataNodeTtMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DataNodeTtMsg.Marshal(b, m, deterministic) +} +func (m *DataNodeTtMsg) XXX_Merge(src proto.Message) { + xxx_messageInfo_DataNodeTtMsg.Merge(m, src) +} +func (m *DataNodeTtMsg) XXX_Size() int { + return xxx_messageInfo_DataNodeTtMsg.Size(m) +} +func (m *DataNodeTtMsg) XXX_DiscardUnknown() { + xxx_messageInfo_DataNodeTtMsg.DiscardUnknown(m) +} + +var xxx_messageInfo_DataNodeTtMsg proto.InternalMessageInfo + +func (m *DataNodeTtMsg) GetBase() *commonpb.MsgBase { + if m != nil { + return m.Base + } + return nil +} + +func (m *DataNodeTtMsg) GetChannelName() string { + if m != nil { + return m.ChannelName + } + return "" +} + +func (m *DataNodeTtMsg) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + func init() { proto.RegisterType((*RegisterNodeRequest)(nil), "milvus.proto.data.RegisterNodeRequest") proto.RegisterType((*RegisterNodeResponse)(nil), "milvus.proto.data.RegisterNodeResponse") @@ -2319,136 +2374,139 @@ func init() { proto.RegisterType((*FieldBinlog)(nil), "milvus.proto.data.FieldBinlog") proto.RegisterType((*GetRecoveryInfoResponse)(nil), "milvus.proto.data.GetRecoveryInfoResponse") proto.RegisterType((*GetRecoveryInfoRequest)(nil), "milvus.proto.data.GetRecoveryInfoRequest") + proto.RegisterType((*DataNodeTtMsg)(nil), "milvus.proto.data.DataNodeTtMsg") } func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad6cfe64) } var fileDescriptor_3385cd32ad6cfe64 = []byte{ - // 1975 bytes of a gzipped FileDescriptorProto + // 2003 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0xdb, 0x6f, 0x1b, 0x59, 0x19, 0xcf, 0xd8, 0xb9, 0xf9, 0xf3, 0xd8, 0x69, 0x4f, 0x43, 0xd6, 0xb8, 0x6d, 0x9a, 0xce, 0xb2, 0xdd, 0x6c, 0x57, 0x24, 0x5b, 0x17, 0x71, 0x51, 0x05, 0xa8, 0xa9, 0xb7, 0xc1, 0xa2, 0x29, 0xe1, - 0x64, 0xd9, 0x95, 0x58, 0x21, 0x6b, 0xe2, 0x39, 0x71, 0x86, 0xcc, 0xc5, 0x3b, 0x67, 0xdc, 0xa6, - 0x4f, 0x45, 0xc0, 0x03, 0x42, 0x88, 0x85, 0xff, 0x00, 0x90, 0x90, 0x90, 0xe0, 0x81, 0x67, 0xde, - 0xf9, 0x17, 0xf8, 0x7b, 0xd0, 0xb9, 0xcc, 0xfd, 0xd8, 0x9e, 0x38, 0xed, 0xe6, 0xcd, 0xe7, 0xf8, - 0xbb, 0x9d, 0xef, 0xf2, 0x3b, 0xdf, 0xf9, 0x06, 0x90, 0x65, 0x86, 0x66, 0x9f, 0x92, 0xe0, 0x85, - 0x3d, 0x20, 0x3b, 0xa3, 0xc0, 0x0f, 0x7d, 0x74, 0xdd, 0xb5, 0x9d, 0x17, 0x63, 0x2a, 0x56, 0x3b, - 0x8c, 0xa0, 0xad, 0x0f, 0x7c, 0xd7, 0xf5, 0x3d, 0xb1, 0xd5, 0x6e, 0xda, 0x5e, 0x48, 0x02, 0xcf, - 0x74, 0xe4, 0x5a, 0x4f, 0x33, 0xb4, 0x75, 0x3a, 0x38, 0x25, 0xae, 0x29, 0x56, 0xc6, 0x6b, 0xb8, - 0x81, 0xc9, 0xd0, 0xa6, 0x21, 0x09, 0x9e, 0xfb, 0x16, 0xc1, 0xe4, 0x8b, 0x31, 0xa1, 0x21, 0xfa, - 0x08, 0x16, 0x8f, 0x4d, 0x4a, 0x5a, 0xda, 0x96, 0xb6, 0x5d, 0xef, 0xdc, 0xda, 0xc9, 0xa8, 0x94, - 0xca, 0x0e, 0xe8, 0x70, 0xcf, 0xa4, 0x04, 0x73, 0x4a, 0xf4, 0x6d, 0x58, 0x31, 0x2d, 0x2b, 0x20, - 0x94, 0xb6, 0x2a, 0x53, 0x98, 0x1e, 0x0b, 0x1a, 0x1c, 0x11, 0x1b, 0x5f, 0x6a, 0xb0, 0x9e, 0xb5, + 0xa4, 0xec, 0x4a, 0xac, 0x90, 0x35, 0xf1, 0x9c, 0x38, 0x43, 0xe6, 0xe2, 0x9d, 0x33, 0x6e, 0xd3, + 0xa7, 0xa2, 0x85, 0x07, 0x84, 0x10, 0x85, 0xff, 0x00, 0x90, 0x90, 0x90, 0xe0, 0x81, 0x67, 0xde, + 0xf9, 0x17, 0xf8, 0x7b, 0x56, 0xe7, 0x32, 0xf7, 0x63, 0x7b, 0xea, 0xb4, 0xca, 0x9b, 0xcf, 0xf1, + 0x77, 0x3b, 0xdf, 0xe5, 0x77, 0xbe, 0xf3, 0x0d, 0x20, 0xcb, 0x0c, 0xcd, 0x3e, 0x25, 0xc1, 0x73, + 0x7b, 0x40, 0x76, 0x46, 0x81, 0x1f, 0xfa, 0xe8, 0xaa, 0x6b, 0x3b, 0xcf, 0xc7, 0x54, 0xac, 0x76, + 0x18, 0x41, 0x5b, 0x1f, 0xf8, 0xae, 0xeb, 0x7b, 0x62, 0xab, 0xdd, 0xb4, 0xbd, 0x90, 0x04, 0x9e, + 0xe9, 0xc8, 0xb5, 0x9e, 0x66, 0x68, 0xeb, 0x74, 0x70, 0x4a, 0x5c, 0x53, 0xac, 0x8c, 0x57, 0x70, + 0x0d, 0x93, 0xa1, 0x4d, 0x43, 0x12, 0x3c, 0xf5, 0x2d, 0x82, 0xc9, 0x97, 0x63, 0x42, 0x43, 0xf4, + 0x09, 0x2c, 0x1e, 0x9b, 0x94, 0xb4, 0xb4, 0x2d, 0x6d, 0xbb, 0xde, 0xb9, 0xb1, 0x93, 0x51, 0x29, + 0x95, 0x1d, 0xd0, 0xe1, 0x9e, 0x49, 0x09, 0xe6, 0x94, 0xe8, 0xbb, 0xb0, 0x62, 0x5a, 0x56, 0x40, + 0x28, 0x6d, 0x55, 0xa6, 0x30, 0x3d, 0x14, 0x34, 0x38, 0x22, 0x36, 0x5e, 0x6b, 0xb0, 0x9e, 0xb5, 0x80, 0x8e, 0x7c, 0x8f, 0x12, 0xb4, 0x07, 0x75, 0xdb, 0xb3, 0xc3, 0xfe, 0xc8, 0x0c, 0x4c, 0x97, - 0x4a, 0x4b, 0xee, 0x66, 0x85, 0xc6, 0x07, 0xed, 0x79, 0x76, 0x78, 0xc8, 0x09, 0x31, 0xd8, 0xf1, - 0x6f, 0xf4, 0x10, 0x96, 0x69, 0x68, 0x86, 0xe3, 0xc8, 0xa6, 0x9b, 0x4a, 0x9b, 0x8e, 0x38, 0x09, - 0x96, 0xa4, 0xc6, 0x39, 0xe8, 0x4f, 0x9d, 0x31, 0x3d, 0x9d, 0xdf, 0x17, 0x08, 0x16, 0xad, 0xe3, + 0x4a, 0x4b, 0x6e, 0x67, 0x85, 0xc6, 0x07, 0xed, 0x79, 0x76, 0x78, 0xc8, 0x09, 0x31, 0xd8, 0xf1, + 0x6f, 0x74, 0x1f, 0x96, 0x69, 0x68, 0x86, 0xe3, 0xc8, 0xa6, 0xeb, 0x4a, 0x9b, 0x8e, 0x38, 0x09, + 0x96, 0xa4, 0xc6, 0x39, 0xe8, 0x8f, 0x9d, 0x31, 0x3d, 0x9d, 0xdf, 0x17, 0x08, 0x16, 0xad, 0xe3, 0x5e, 0x97, 0x2b, 0xad, 0x62, 0xfe, 0x1b, 0x19, 0xa0, 0x0f, 0x7c, 0xc7, 0x21, 0x83, 0xd0, 0xf6, - 0xbd, 0x5e, 0xb7, 0xb5, 0xc8, 0xff, 0xcb, 0xec, 0x19, 0x7f, 0xd6, 0xe0, 0xda, 0x11, 0x19, 0xba, + 0xbd, 0x5e, 0xb7, 0xb5, 0xc8, 0xff, 0xcb, 0xec, 0x19, 0x7f, 0xd1, 0xe0, 0xca, 0x11, 0x19, 0xba, 0xc4, 0x0b, 0x7b, 0xdd, 0x48, 0xfd, 0x3a, 0x2c, 0x0d, 0xfc, 0xb1, 0x17, 0x72, 0xfd, 0x0d, 0x2c, - 0x16, 0xe8, 0x2e, 0xe8, 0x83, 0x53, 0xd3, 0xf3, 0x88, 0xd3, 0xf7, 0x4c, 0x97, 0x70, 0x55, 0x35, - 0x5c, 0x97, 0x7b, 0xcf, 0x4d, 0x97, 0x14, 0x34, 0x56, 0x8b, 0x1a, 0xd1, 0x16, 0xd4, 0x47, 0x66, - 0x10, 0xda, 0x19, 0xa3, 0xd2, 0x5b, 0xc6, 0x5f, 0x34, 0xd8, 0x78, 0x4c, 0xa9, 0x3d, 0xf4, 0x0a, - 0x96, 0x6d, 0xc0, 0xb2, 0xe7, 0x5b, 0xa4, 0xd7, 0xe5, 0xa6, 0x55, 0xb1, 0x5c, 0xa1, 0x9b, 0x50, - 0x1b, 0x11, 0x12, 0xf4, 0x03, 0xdf, 0x89, 0x0c, 0x5b, 0x65, 0x1b, 0xd8, 0x77, 0x08, 0xfa, 0x29, - 0x5c, 0xa7, 0x39, 0x41, 0xb4, 0x55, 0xdd, 0xaa, 0x6e, 0xd7, 0x3b, 0xef, 0xee, 0x14, 0x32, 0x7b, - 0x27, 0xaf, 0x14, 0x17, 0xb9, 0x8d, 0x5f, 0x55, 0xe0, 0x46, 0x4c, 0x27, 0x6c, 0x65, 0xbf, 0x99, + 0x16, 0xe8, 0x36, 0xe8, 0x83, 0x53, 0xd3, 0xf3, 0x88, 0xd3, 0xf7, 0x4c, 0x97, 0x70, 0x55, 0x35, + 0x5c, 0x97, 0x7b, 0x4f, 0x4d, 0x97, 0x14, 0x34, 0x56, 0x8b, 0x1a, 0xd1, 0x16, 0xd4, 0x47, 0x66, + 0x10, 0xda, 0x19, 0xa3, 0xd2, 0x5b, 0xc6, 0x5f, 0x35, 0xd8, 0x78, 0x48, 0xa9, 0x3d, 0xf4, 0x0a, + 0x96, 0x6d, 0xc0, 0xb2, 0xe7, 0x5b, 0xa4, 0xd7, 0xe5, 0xa6, 0x55, 0xb1, 0x5c, 0xa1, 0xeb, 0x50, + 0x1b, 0x11, 0x12, 0xf4, 0x03, 0xdf, 0x89, 0x0c, 0x5b, 0x65, 0x1b, 0xd8, 0x77, 0x08, 0xfa, 0x39, + 0x5c, 0xa5, 0x39, 0x41, 0xb4, 0x55, 0xdd, 0xaa, 0x6e, 0xd7, 0x3b, 0xef, 0xef, 0x14, 0x32, 0x7b, + 0x27, 0xaf, 0x14, 0x17, 0xb9, 0x8d, 0xdf, 0x54, 0xe0, 0x5a, 0x4c, 0x27, 0x6c, 0x65, 0xbf, 0x99, 0xe7, 0x28, 0x19, 0xc6, 0xe6, 0x89, 0x45, 0x19, 0xcf, 0xc5, 0x2e, 0xaf, 0xa6, 0x5d, 0x5e, 0x22, - 0x82, 0x79, 0x7f, 0x2e, 0x15, 0xfc, 0x89, 0xee, 0x40, 0x9d, 0x9c, 0x8f, 0xec, 0x80, 0xf4, 0x43, - 0xdb, 0x25, 0xad, 0xe5, 0x2d, 0x6d, 0x7b, 0x11, 0x83, 0xd8, 0xfa, 0xc4, 0x76, 0x49, 0x2a, 0x67, - 0x57, 0xca, 0xe7, 0xec, 0xdf, 0x34, 0x78, 0xa7, 0x10, 0x25, 0x59, 0x48, 0x18, 0xae, 0xf1, 0x93, - 0x27, 0x9e, 0x61, 0xd5, 0xc4, 0x1c, 0x7e, 0x6f, 0x9a, 0xc3, 0x13, 0x72, 0x5c, 0xe0, 0x9f, 0xaf, - 0xb0, 0xfe, 0xaa, 0xc1, 0x8d, 0xa3, 0x53, 0xff, 0xa5, 0x54, 0x41, 0xe7, 0x2f, 0xb0, 0x7c, 0x28, - 0x2a, 0xb3, 0x43, 0x51, 0x2d, 0x86, 0x22, 0x2a, 0xd3, 0xc5, 0xa4, 0x4c, 0x8d, 0x33, 0x58, 0xcf, - 0x9a, 0x28, 0x9d, 0xb8, 0x09, 0x10, 0x27, 0x9e, 0x70, 0x5f, 0x15, 0xa7, 0x76, 0xe6, 0x73, 0xc8, - 0x19, 0xbc, 0xb3, 0x4f, 0x42, 0xa9, 0x8b, 0xfd, 0x47, 0x2e, 0xe1, 0x93, 0xac, 0x85, 0x95, 0xbc, - 0x85, 0xc6, 0xbf, 0x2b, 0x31, 0xb8, 0x70, 0x55, 0x3d, 0xef, 0xc4, 0x47, 0xb7, 0xa0, 0x16, 0x93, - 0xc8, 0x32, 0x49, 0x36, 0xd0, 0x77, 0x60, 0x89, 0x59, 0x2a, 0x6a, 0xa4, 0x99, 0x07, 0xdf, 0xe8, - 0x4c, 0x29, 0x99, 0x58, 0xd0, 0xa3, 0x1e, 0x34, 0x69, 0x68, 0x06, 0x61, 0x7f, 0xe4, 0x53, 0xee, - 0x6d, 0xee, 0xfe, 0x7a, 0xc7, 0x98, 0x00, 0xdf, 0x07, 0x74, 0x78, 0x28, 0x29, 0x71, 0x83, 0x73, - 0x46, 0x4b, 0xf4, 0x31, 0xe8, 0xc4, 0xb3, 0x12, 0x41, 0x8b, 0xa5, 0x05, 0xd5, 0x89, 0x67, 0xc5, - 0x62, 0x92, 0xf8, 0x2c, 0x95, 0x8f, 0xcf, 0x1f, 0x34, 0x68, 0x15, 0x03, 0x24, 0x33, 0x22, 0x91, - 0xa8, 0x95, 0x96, 0x88, 0x1e, 0x09, 0x26, 0x22, 0x02, 0x34, 0x15, 0xf2, 0xe2, 0x20, 0x61, 0xc9, - 0x62, 0xd8, 0xf0, 0xb5, 0xc4, 0x1a, 0xfe, 0xcf, 0x5b, 0x4b, 0x96, 0xdf, 0x68, 0xb0, 0x91, 0xd7, - 0x75, 0x99, 0x73, 0x7f, 0x0b, 0x96, 0x6c, 0xef, 0xc4, 0x8f, 0x8e, 0xbd, 0x39, 0x05, 0x78, 0x98, - 0x2e, 0x41, 0x6c, 0xb8, 0x70, 0x73, 0x9f, 0x84, 0x3d, 0x8f, 0x92, 0x20, 0xdc, 0xb3, 0x3d, 0xc7, - 0x1f, 0x1e, 0x9a, 0xe1, 0xe9, 0x25, 0x6a, 0x24, 0x93, 0xee, 0x95, 0x5c, 0xba, 0x1b, 0xff, 0xd0, - 0xe0, 0x96, 0x5a, 0x9f, 0x3c, 0x7a, 0x1b, 0x56, 0x4f, 0x6c, 0xe2, 0x58, 0x09, 0x04, 0xc4, 0x6b, - 0x56, 0x2b, 0x23, 0x46, 0x2c, 0x4f, 0x38, 0xa9, 0x51, 0x39, 0x0a, 0x03, 0xdb, 0x1b, 0x3e, 0xb3, - 0x69, 0x88, 0x05, 0x7d, 0xca, 0x9f, 0xd5, 0xf2, 0x99, 0xf9, 0x5b, 0x91, 0x99, 0xc2, 0xd4, 0x27, - 0xe2, 0xea, 0xa2, 0x6f, 0xb7, 0x61, 0x51, 0xb4, 0x0f, 0xc6, 0xef, 0x35, 0xd8, 0xdc, 0x27, 0xe1, - 0x93, 0x78, 0x8f, 0x99, 0x69, 0xd3, 0xd0, 0x1e, 0x5c, 0x81, 0x31, 0x5f, 0x6a, 0x70, 0x67, 0xa2, - 0x31, 0x32, 0x82, 0x12, 0xd1, 0xa2, 0x0b, 0x50, 0x8d, 0x68, 0x3f, 0x26, 0xaf, 0x3e, 0x35, 0x9d, - 0x31, 0x39, 0x34, 0xed, 0x40, 0x20, 0xda, 0x9c, 0xf8, 0xfe, 0x4f, 0x0d, 0x6e, 0xef, 0x13, 0xd6, - 0x8c, 0x8a, 0x3b, 0xe7, 0x0a, 0xbd, 0x53, 0xa2, 0xd3, 0xfb, 0xa3, 0x08, 0xa6, 0xd2, 0xda, 0x2b, - 0x71, 0xdf, 0x26, 0x2f, 0xc7, 0x14, 0x2e, 0xc8, 0x44, 0x97, 0xce, 0x33, 0x7c, 0xd8, 0xf8, 0xcc, - 0x0c, 0x07, 0xa7, 0x5d, 0xf7, 0xf2, 0x15, 0xf0, 0x2e, 0x34, 0xd2, 0x5d, 0xa1, 0x28, 0xe3, 0x1a, - 0xd6, 0x53, 0x6d, 0x21, 0x65, 0xbd, 0xf0, 0x3a, 0x7f, 0x1a, 0x5c, 0xbe, 0x83, 0x99, 0x37, 0x8c, - 0x59, 0xe0, 0x5e, 0x2c, 0x00, 0xf7, 0x39, 0x80, 0x34, 0xee, 0x80, 0x0e, 0xe7, 0xb0, 0xeb, 0xbb, - 0xb0, 0x22, 0xa5, 0xc9, 0x48, 0xcd, 0x82, 0xea, 0x88, 0xdc, 0x38, 0x82, 0x0d, 0xb9, 0xff, 0x94, - 0x61, 0xa2, 0xc0, 0xcf, 0x03, 0x12, 0x9a, 0xa8, 0x05, 0x2b, 0x12, 0x26, 0x65, 0x8b, 0x11, 0x2d, - 0x59, 0x33, 0x7c, 0xcc, 0xe9, 0xfa, 0x0c, 0x0b, 0x65, 0x2b, 0x0e, 0xc7, 0x31, 0xf4, 0x1a, 0xbf, - 0x80, 0x46, 0xb7, 0xfb, 0x2c, 0x25, 0xeb, 0x1e, 0xac, 0x59, 0x96, 0xd3, 0x4f, 0x73, 0x69, 0x9c, - 0xab, 0x61, 0x59, 0x4e, 0x82, 0xd9, 0xe8, 0x1b, 0xd0, 0x0c, 0x69, 0xbf, 0x28, 0x5c, 0x0f, 0x69, - 0x42, 0x65, 0x1c, 0x40, 0x93, 0x1b, 0xcb, 0x83, 0x3a, 0xc3, 0xd6, 0xbb, 0xa0, 0xa7, 0xc4, 0x45, - 0x09, 0x52, 0x4f, 0x8c, 0xa5, 0x0c, 0x0e, 0xa3, 0x16, 0x2b, 0x91, 0x38, 0xbd, 0xc5, 0xba, 0x0d, - 0x60, 0xd3, 0xfe, 0x09, 0xa3, 0x26, 0x16, 0xb7, 0x71, 0x15, 0xd7, 0x6c, 0xfa, 0x54, 0x6c, 0xa0, - 0xef, 0xc1, 0x32, 0xd7, 0xcf, 0xda, 0x16, 0x45, 0xc5, 0xf1, 0x68, 0x64, 0x4f, 0x80, 0x25, 0x83, - 0xf1, 0x33, 0xd0, 0xbb, 0xdd, 0x67, 0x89, 0x1d, 0xf9, 0xec, 0xd2, 0x14, 0xd9, 0x55, 0xe2, 0x8c, - 0xaf, 0xa1, 0x99, 0x20, 0x2c, 0xef, 0x21, 0x9b, 0x50, 0x89, 0xc5, 0x55, 0x7a, 0x5d, 0xf4, 0x7d, - 0x58, 0x16, 0x23, 0x06, 0x99, 0x41, 0xef, 0x65, 0x6d, 0x96, 0xe3, 0x87, 0x14, 0x4c, 0xf3, 0x0d, - 0x2c, 0x99, 0x58, 0x86, 0xc7, 0xa8, 0x24, 0x5e, 0x86, 0x55, 0x9c, 0xda, 0x31, 0xfe, 0x53, 0x85, - 0x7a, 0x2a, 0x01, 0x0b, 0xea, 0xdf, 0xcc, 0xdb, 0xe0, 0x3d, 0x68, 0xda, 0xfc, 0x72, 0xed, 0x4b, - 0x04, 0xe0, 0x88, 0x59, 0xc3, 0x0d, 0x3b, 0x7d, 0xe5, 0xa2, 0xaf, 0xc3, 0xaa, 0x37, 0x76, 0xfb, - 0x81, 0xff, 0x92, 0xca, 0xc7, 0xde, 0x8a, 0x37, 0x76, 0xb1, 0xff, 0x92, 0x26, 0xcd, 0xf3, 0xf2, - 0xa5, 0x9b, 0xe7, 0x95, 0x37, 0xd5, 0x3c, 0xaf, 0xce, 0xd7, 0x3c, 0x6f, 0x42, 0xdd, 0x35, 0xcf, - 0xd9, 0x29, 0xfb, 0xde, 0xd8, 0x6d, 0xd5, 0x44, 0x12, 0xbb, 0xe6, 0x39, 0xf6, 0x5f, 0x3e, 0x1f, - 0xbb, 0x68, 0x1b, 0xae, 0x39, 0x26, 0x0d, 0xfb, 0xe9, 0x87, 0x2d, 0xf0, 0x87, 0x6d, 0x93, 0xed, - 0x7f, 0x1c, 0x3f, 0x6e, 0x8d, 0x87, 0x50, 0xef, 0x75, 0x3b, 0x2c, 0x93, 0x58, 0x0b, 0x54, 0x88, - 0xdd, 0x3a, 0x2c, 0x1d, 0xa6, 0x12, 0x4f, 0x2c, 0x18, 0xec, 0xea, 0x91, 0x2d, 0xec, 0x52, 0x51, - 0x78, 0x48, 0x7b, 0x53, 0x1e, 0xaa, 0xcc, 0xe5, 0x21, 0xe3, 0x5f, 0x55, 0xd8, 0x38, 0x32, 0x5f, - 0x90, 0xb7, 0xdf, 0xa5, 0x96, 0xba, 0x25, 0x9e, 0xc1, 0x75, 0x8e, 0x02, 0x9d, 0x94, 0x3d, 0xf2, - 0xe5, 0xa4, 0xc2, 0xf3, 0x54, 0x48, 0x70, 0x91, 0x11, 0xfd, 0x08, 0x9a, 0x19, 0x70, 0x8d, 0xc0, - 0x68, 0x4b, 0x21, 0x2a, 0x83, 0xd6, 0x38, 0xc7, 0x87, 0xf6, 0x40, 0xb7, 0x5c, 0x27, 0xf1, 0xf6, - 0x32, 0x37, 0xe9, 0x8e, 0x42, 0x4e, 0x3a, 0xde, 0xb8, 0x6e, 0xb9, 0x4e, 0x1c, 0x31, 0x26, 0xc3, - 0x72, 0xf2, 0xc5, 0x51, 0x42, 0x86, 0x15, 0xcb, 0x60, 0x40, 0x0d, 0x4f, 0x4e, 0xc9, 0xe0, 0xec, - 0xd0, 0xb7, 0xbd, 0x70, 0x06, 0x44, 0xff, 0x00, 0x56, 0xe7, 0x48, 0x8f, 0x98, 0x87, 0x55, 0x0f, - 0xc3, 0x08, 0xff, 0x44, 0xc0, 0x84, 0x88, 0x57, 0xcd, 0x1b, 0xbb, 0x3f, 0x39, 0x61, 0x40, 0x61, - 0xfc, 0xae, 0x02, 0xfa, 0xa7, 0x12, 0x65, 0x38, 0xa2, 0x95, 0x41, 0xea, 0x2d, 0x48, 0x4f, 0xac, - 0x54, 0x43, 0xac, 0x7d, 0x68, 0x50, 0x42, 0xce, 0xe6, 0x79, 0x82, 0xeb, 0x8c, 0x31, 0x76, 0xf8, - 0x0f, 0x99, 0xaa, 0xc8, 0x57, 0xa2, 0xe7, 0xa8, 0x77, 0x6e, 0x2b, 0xfc, 0x9d, 0x78, 0x14, 0xa7, - 0x39, 0xd0, 0x36, 0xac, 0xc9, 0x0b, 0x2e, 0xea, 0x9b, 0x78, 0x02, 0x55, 0x71, 0x7e, 0xdb, 0x08, - 0xa0, 0x29, 0x7f, 0x8b, 0xac, 0xa1, 0x33, 0x42, 0xb3, 0x07, 0xfa, 0x49, 0xd2, 0x6c, 0x4c, 0x7b, - 0x5d, 0xa6, 0x7a, 0x12, 0x9c, 0xe1, 0x31, 0x1e, 0x43, 0x3d, 0xf5, 0xe7, 0x94, 0x06, 0xa0, 0x05, - 0x2b, 0xc7, 0x29, 0x3d, 0x35, 0x1c, 0x2d, 0x8d, 0xff, 0x6a, 0x7c, 0x90, 0x83, 0xc9, 0xc0, 0x7f, - 0x41, 0x82, 0x57, 0x99, 0xe7, 0xf2, 0xc5, 0xcb, 0xff, 0x11, 0xac, 0xca, 0x38, 0x46, 0x07, 0x52, - 0x25, 0x77, 0x3a, 0x63, 0x70, 0xcc, 0x80, 0x1e, 0x25, 0x46, 0x56, 0x27, 0x76, 0x0c, 0x59, 0x1f, - 0x27, 0xe7, 0xf8, 0x93, 0x78, 0xf5, 0x67, 0xcf, 0x71, 0xa5, 0x33, 0xba, 0xce, 0xff, 0x1a, 0x50, - 0xef, 0x9a, 0xa1, 0x79, 0x24, 0x3e, 0x81, 0xa0, 0x11, 0x20, 0xfe, 0xc6, 0x73, 0x47, 0xbe, 0x17, - 0xcf, 0x64, 0xd0, 0x47, 0x13, 0x92, 0xba, 0x48, 0x2a, 0xcf, 0xd3, 0xbe, 0x37, 0x81, 0x23, 0x47, - 0x6e, 0x2c, 0x20, 0x97, 0x6b, 0x64, 0xb7, 0xd7, 0x27, 0xf6, 0xe0, 0x2c, 0xba, 0xf8, 0xa7, 0x68, - 0xcc, 0x91, 0x46, 0x1a, 0x73, 0xa3, 0x1e, 0xb9, 0x10, 0xf3, 0x80, 0x28, 0x5b, 0x8c, 0x05, 0xf4, - 0x05, 0xac, 0xb3, 0x47, 0x4f, 0xfc, 0xf6, 0x8a, 0x14, 0x76, 0x26, 0x2b, 0x2c, 0x10, 0x5f, 0x50, - 0xa5, 0x09, 0x7a, 0xfa, 0x0b, 0x0c, 0x52, 0x8d, 0x85, 0x15, 0x1f, 0x89, 0xda, 0xef, 0xcf, 0xa4, - 0x8b, 0x55, 0xec, 0xc3, 0x12, 0xef, 0x44, 0x91, 0x2a, 0x95, 0xd3, 0x5f, 0x5b, 0xda, 0xd3, 0x5e, - 0x86, 0xc6, 0x02, 0xfa, 0x25, 0xac, 0xe5, 0xe6, 0xdc, 0xe8, 0x03, 0x85, 0x48, 0xf5, 0x17, 0x8b, - 0xf6, 0xfd, 0x32, 0xa4, 0x69, 0xbf, 0xa4, 0x67, 0xc1, 0x4a, 0xbf, 0x28, 0xe6, 0xd9, 0x4a, 0xbf, - 0xa8, 0x86, 0xca, 0xc6, 0x02, 0x1a, 0x42, 0x33, 0xfb, 0xc4, 0x45, 0xdb, 0x0a, 0x66, 0xe5, 0xd4, - 0xaf, 0xfd, 0x41, 0x09, 0xca, 0x58, 0x91, 0x0b, 0xd7, 0xf2, 0x93, 0x4c, 0x74, 0x7f, 0xaa, 0x80, - 0x6c, 0xbd, 0x7c, 0x58, 0x8a, 0x36, 0x56, 0xf7, 0x8a, 0x67, 0x71, 0x61, 0x92, 0x86, 0x76, 0xd4, - 0x62, 0x26, 0x8d, 0xf8, 0xda, 0xbb, 0xa5, 0xe9, 0x63, 0xd5, 0x04, 0xae, 0x17, 0x26, 0x63, 0xe8, - 0xc3, 0x69, 0x72, 0x72, 0xd3, 0x83, 0xf6, 0xec, 0xd9, 0x9d, 0xb1, 0x80, 0x7e, 0x2d, 0x30, 0x5f, - 0x35, 0x6d, 0x42, 0x0f, 0xd4, 0xda, 0xa6, 0x8c, 0xc9, 0xda, 0x9d, 0x8b, 0xb0, 0xc4, 0x67, 0x7d, - 0xcd, 0xf1, 0x5a, 0x31, 0xb1, 0xc9, 0xe3, 0x53, 0x24, 0x6f, 0xf2, 0x28, 0xaa, 0xfd, 0xe0, 0x02, - 0x1c, 0xb1, 0x01, 0x7e, 0x7e, 0x24, 0x1d, 0xc1, 0xd5, 0xee, 0xcc, 0xe4, 0x9c, 0x0f, 0xab, 0x3e, - 0x87, 0xb5, 0x5c, 0x9f, 0xad, 0xac, 0x7f, 0x75, 0x2f, 0x5e, 0x02, 0x5c, 0x72, 0xd7, 0x1f, 0x9a, - 0x50, 0x64, 0x8a, 0x2b, 0xb2, 0x7d, 0xbf, 0x0c, 0x69, 0x74, 0x90, 0xce, 0xdf, 0xab, 0xb0, 0xca, - 0x2e, 0x36, 0x8e, 0xb8, 0x5f, 0xfd, 0xad, 0x76, 0x05, 0xd7, 0xcc, 0xe7, 0xb0, 0x96, 0x1b, 0xd7, - 0x29, 0xbd, 0xab, 0x1e, 0xe9, 0xcd, 0x0a, 0xdd, 0x67, 0xd0, 0xc8, 0x4c, 0xe6, 0xd0, 0xfb, 0x93, - 0x2e, 0x9a, 0x3c, 0x5a, 0x4f, 0x17, 0xbc, 0xf7, 0xf0, 0xe7, 0x0f, 0x86, 0x76, 0x78, 0x3a, 0x3e, - 0x66, 0xff, 0xec, 0x0a, 0xd2, 0x6f, 0xda, 0xbe, 0xfc, 0xb5, 0x1b, 0x39, 0x68, 0x97, 0x73, 0xef, - 0x32, 0x35, 0xa3, 0xe3, 0xe3, 0x65, 0xbe, 0x7a, 0xf8, 0xff, 0x00, 0x00, 0x00, 0xff, 0xff, 0xf7, - 0x87, 0x24, 0x0f, 0xbf, 0x21, 0x00, 0x00, + 0x82, 0x79, 0x7f, 0x2e, 0x15, 0xfc, 0x89, 0x6e, 0x41, 0x9d, 0x9c, 0x8f, 0xec, 0x80, 0xf4, 0x43, + 0xdb, 0x25, 0xad, 0xe5, 0x2d, 0x6d, 0x7b, 0x11, 0x83, 0xd8, 0x7a, 0x66, 0xbb, 0x24, 0x95, 0xb3, + 0x2b, 0xe5, 0x73, 0xf6, 0xef, 0x1a, 0xbc, 0x57, 0x88, 0x92, 0x2c, 0x24, 0x0c, 0x57, 0xf8, 0xc9, + 0x13, 0xcf, 0xb0, 0x6a, 0x62, 0x0e, 0xbf, 0x33, 0xcd, 0xe1, 0x09, 0x39, 0x2e, 0xf0, 0xcf, 0x57, + 0x58, 0x7f, 0xd3, 0xe0, 0xda, 0xd1, 0xa9, 0xff, 0x42, 0xaa, 0xa0, 0xf3, 0x17, 0x58, 0x3e, 0x14, + 0x95, 0xd9, 0xa1, 0xa8, 0x16, 0x43, 0x11, 0x95, 0xe9, 0x62, 0x52, 0xa6, 0xc6, 0x19, 0xac, 0x67, + 0x4d, 0x94, 0x4e, 0xdc, 0x04, 0x88, 0x13, 0x4f, 0xb8, 0xaf, 0x8a, 0x53, 0x3b, 0xf3, 0x39, 0xe4, + 0x0c, 0xde, 0xdb, 0x27, 0xa1, 0xd4, 0xc5, 0xfe, 0x23, 0x17, 0xf0, 0x49, 0xd6, 0xc2, 0x4a, 0xde, + 0x42, 0xe3, 0x3f, 0x95, 0x18, 0x5c, 0xb8, 0xaa, 0x9e, 0x77, 0xe2, 0xa3, 0x1b, 0x50, 0x8b, 0x49, + 0x64, 0x99, 0x24, 0x1b, 0xe8, 0x7b, 0xb0, 0xc4, 0x2c, 0x15, 0x35, 0xd2, 0xcc, 0x83, 0x6f, 0x74, + 0xa6, 0x94, 0x4c, 0x2c, 0xe8, 0x51, 0x0f, 0x9a, 0x34, 0x34, 0x83, 0xb0, 0x3f, 0xf2, 0x29, 0xf7, + 0x36, 0x77, 0x7f, 0xbd, 0x63, 0x4c, 0x80, 0xef, 0x03, 0x3a, 0x3c, 0x94, 0x94, 0xb8, 0xc1, 0x39, + 0xa3, 0x25, 0xfa, 0x14, 0x74, 0xe2, 0x59, 0x89, 0xa0, 0xc5, 0xd2, 0x82, 0xea, 0xc4, 0xb3, 0x62, + 0x31, 0x49, 0x7c, 0x96, 0xca, 0xc7, 0xe7, 0x8f, 0x1a, 0xb4, 0x8a, 0x01, 0x92, 0x19, 0x91, 0x48, + 0xd4, 0x4a, 0x4b, 0x44, 0x0f, 0x04, 0x13, 0x11, 0x01, 0x9a, 0x0a, 0x79, 0x71, 0x90, 0xb0, 0x64, + 0x31, 0x6c, 0xf8, 0x46, 0x62, 0x0d, 0xff, 0xe7, 0x9d, 0x25, 0xcb, 0x6f, 0x35, 0xd8, 0xc8, 0xeb, + 0xba, 0xc8, 0xb9, 0xbf, 0x03, 0x4b, 0xb6, 0x77, 0xe2, 0x47, 0xc7, 0xde, 0x9c, 0x02, 0x3c, 0x4c, + 0x97, 0x20, 0x36, 0x5c, 0xb8, 0xbe, 0x4f, 0xc2, 0x9e, 0x47, 0x49, 0x10, 0xee, 0xd9, 0x9e, 0xe3, + 0x0f, 0x0f, 0xcd, 0xf0, 0xf4, 0x02, 0x35, 0x92, 0x49, 0xf7, 0x4a, 0x2e, 0xdd, 0x8d, 0x7f, 0x6a, + 0x70, 0x43, 0xad, 0x4f, 0x1e, 0xbd, 0x0d, 0xab, 0x27, 0x36, 0x71, 0xac, 0x04, 0x02, 0xe2, 0x35, + 0xab, 0x95, 0x11, 0x23, 0x96, 0x27, 0x9c, 0xd4, 0xa8, 0x1c, 0x85, 0x81, 0xed, 0x0d, 0x9f, 0xd8, + 0x34, 0xc4, 0x82, 0x3e, 0xe5, 0xcf, 0x6a, 0xf9, 0xcc, 0xfc, 0x9d, 0xc8, 0x4c, 0x61, 0xea, 0x23, + 0x71, 0x75, 0xd1, 0x77, 0xdb, 0xb0, 0x28, 0xda, 0x07, 0xe3, 0x0f, 0x1a, 0x6c, 0xee, 0x93, 0xf0, + 0x51, 0xbc, 0xc7, 0xcc, 0xb4, 0x69, 0x68, 0x0f, 0x2e, 0xc1, 0x98, 0xd7, 0x1a, 0xdc, 0x9a, 0x68, + 0x8c, 0x8c, 0xa0, 0x44, 0xb4, 0xe8, 0x02, 0x54, 0x23, 0xda, 0x4f, 0xc9, 0xcb, 0xcf, 0x4c, 0x67, + 0x4c, 0x0e, 0x4d, 0x3b, 0x10, 0x88, 0x36, 0x27, 0xbe, 0xff, 0x4b, 0x83, 0x9b, 0xfb, 0x84, 0x35, + 0xa3, 0xe2, 0xce, 0xb9, 0x44, 0xef, 0x94, 0xe8, 0xf4, 0xfe, 0x24, 0x82, 0xa9, 0xb4, 0xf6, 0x52, + 0xdc, 0xb7, 0xc9, 0xcb, 0x31, 0x85, 0x0b, 0x32, 0xd1, 0xa5, 0xf3, 0x0c, 0x1f, 0x36, 0x3e, 0x37, + 0xc3, 0xc1, 0x69, 0xd7, 0xbd, 0x78, 0x05, 0xbc, 0x0f, 0x8d, 0x74, 0x57, 0x28, 0xca, 0xb8, 0x86, + 0xf5, 0x54, 0x5b, 0x48, 0x59, 0x2f, 0xbc, 0xce, 0x9f, 0x06, 0x17, 0xef, 0x60, 0xe6, 0x0d, 0x63, + 0x16, 0xb8, 0x17, 0x0b, 0xc0, 0x7d, 0x0e, 0x20, 0x8d, 0x3b, 0xa0, 0xc3, 0x39, 0xec, 0xfa, 0x3e, + 0xac, 0x48, 0x69, 0x32, 0x52, 0xb3, 0xa0, 0x3a, 0x22, 0x37, 0x8e, 0x60, 0x43, 0xee, 0x3f, 0x66, + 0x98, 0x28, 0xf0, 0xf3, 0x80, 0x84, 0x26, 0x6a, 0xc1, 0x8a, 0x84, 0x49, 0xd9, 0x62, 0x44, 0x4b, + 0xd6, 0x0c, 0x1f, 0x73, 0xba, 0x3e, 0xc3, 0x42, 0xd9, 0x8a, 0xc3, 0x71, 0x0c, 0xbd, 0xc6, 0xaf, + 0xa0, 0xd1, 0xed, 0x3e, 0x49, 0xc9, 0xba, 0x03, 0x6b, 0x96, 0xe5, 0xf4, 0xd3, 0x5c, 0x1a, 0xe7, + 0x6a, 0x58, 0x96, 0x93, 0x60, 0x36, 0xfa, 0x16, 0x34, 0x43, 0xda, 0x2f, 0x0a, 0xd7, 0x43, 0x9a, + 0x50, 0x19, 0x07, 0xd0, 0xe4, 0xc6, 0xf2, 0xa0, 0xce, 0xb0, 0xf5, 0x36, 0xe8, 0x29, 0x71, 0x51, + 0x82, 0xd4, 0x13, 0x63, 0x29, 0x83, 0xc3, 0xa8, 0xc5, 0x4a, 0x24, 0x4e, 0x6f, 0xb1, 0x6e, 0x02, + 0xd8, 0xb4, 0x7f, 0xc2, 0xa8, 0x89, 0xc5, 0x6d, 0x5c, 0xc5, 0x35, 0x9b, 0x3e, 0x16, 0x1b, 0xe8, + 0x07, 0xb0, 0xcc, 0xf5, 0xb3, 0xb6, 0x45, 0x51, 0x71, 0x3c, 0x1a, 0xd9, 0x13, 0x60, 0xc9, 0x60, + 0xfc, 0x02, 0xf4, 0x6e, 0xf7, 0x49, 0x62, 0x47, 0x3e, 0xbb, 0x34, 0x45, 0x76, 0x95, 0x38, 0xe3, + 0x2b, 0x68, 0x26, 0x08, 0xcb, 0x7b, 0xc8, 0x26, 0x54, 0x62, 0x71, 0x95, 0x5e, 0x17, 0xfd, 0x10, + 0x96, 0xc5, 0x88, 0x41, 0x66, 0xd0, 0x07, 0x59, 0x9b, 0xe5, 0xf8, 0x21, 0x05, 0xd3, 0x7c, 0x03, + 0x4b, 0x26, 0x96, 0xe1, 0x31, 0x2a, 0x89, 0x97, 0x61, 0x15, 0xa7, 0x76, 0x8c, 0xff, 0x56, 0xa1, + 0x9e, 0x4a, 0xc0, 0x82, 0xfa, 0xb7, 0xf3, 0x36, 0xf8, 0x00, 0x9a, 0x36, 0xbf, 0x5c, 0xfb, 0x12, + 0x01, 0x38, 0x62, 0xd6, 0x70, 0xc3, 0x4e, 0x5f, 0xb9, 0xe8, 0x9b, 0xb0, 0xea, 0x8d, 0xdd, 0x7e, + 0xe0, 0xbf, 0xa0, 0xf2, 0xb1, 0xb7, 0xe2, 0x8d, 0x5d, 0xec, 0xbf, 0xa0, 0x49, 0xf3, 0xbc, 0x7c, + 0xe1, 0xe6, 0x79, 0xe5, 0x6d, 0x35, 0xcf, 0xab, 0xf3, 0x35, 0xcf, 0x9b, 0x50, 0x77, 0xcd, 0x73, + 0x76, 0xca, 0xbe, 0x37, 0x76, 0x5b, 0x35, 0x91, 0xc4, 0xae, 0x79, 0x8e, 0xfd, 0x17, 0x4f, 0xc7, + 0x2e, 0xda, 0x86, 0x2b, 0x8e, 0x49, 0xc3, 0x7e, 0xfa, 0x61, 0x0b, 0xfc, 0x61, 0xdb, 0x64, 0xfb, + 0x9f, 0xc6, 0x8f, 0x5b, 0xe3, 0x3e, 0xd4, 0x7b, 0xdd, 0x0e, 0xcb, 0x24, 0xd6, 0x02, 0x15, 0x62, + 0xb7, 0x0e, 0x4b, 0x87, 0xa9, 0xc4, 0x13, 0x0b, 0x06, 0xbb, 0x7a, 0x64, 0x0b, 0xbb, 0x54, 0x14, + 0x1e, 0xd2, 0xde, 0x96, 0x87, 0x2a, 0x73, 0x79, 0xc8, 0xf8, 0x77, 0x15, 0x36, 0x8e, 0xcc, 0xe7, + 0xe4, 0xdd, 0x77, 0xa9, 0xa5, 0x6e, 0x89, 0x27, 0x70, 0x95, 0xa3, 0x40, 0x27, 0x65, 0x8f, 0x7c, + 0x39, 0xa9, 0xf0, 0x3c, 0x15, 0x12, 0x5c, 0x64, 0x44, 0x3f, 0x81, 0x66, 0x06, 0x5c, 0x23, 0x30, + 0xda, 0x52, 0x88, 0xca, 0xa0, 0x35, 0xce, 0xf1, 0xa1, 0x3d, 0xd0, 0x2d, 0xd7, 0x49, 0xbc, 0xbd, + 0xcc, 0x4d, 0xba, 0xa5, 0x90, 0x93, 0x8e, 0x37, 0xae, 0x5b, 0xae, 0x13, 0x47, 0x8c, 0xc9, 0xb0, + 0x9c, 0x7c, 0x71, 0x94, 0x90, 0x61, 0xc5, 0x32, 0x18, 0x50, 0xc3, 0xa3, 0x53, 0x32, 0x38, 0x3b, + 0xf4, 0x6d, 0x2f, 0x9c, 0x01, 0xd1, 0x3f, 0x82, 0xd5, 0x39, 0xd2, 0x23, 0xe6, 0x61, 0xd5, 0xc3, + 0x30, 0xc2, 0x3f, 0x11, 0x30, 0x21, 0xe2, 0x55, 0xf3, 0xc6, 0xee, 0xcf, 0x4e, 0x18, 0x50, 0x18, + 0xbf, 0xaf, 0x80, 0xfe, 0x99, 0x44, 0x19, 0x8e, 0x68, 0x65, 0x90, 0x7a, 0x0b, 0xd2, 0x13, 0x2b, + 0xd5, 0x10, 0x6b, 0x1f, 0x1a, 0x94, 0x90, 0xb3, 0x79, 0x9e, 0xe0, 0x3a, 0x63, 0x8c, 0x1d, 0xfe, + 0x63, 0xa6, 0x2a, 0xf2, 0x95, 0xe8, 0x39, 0xea, 0x9d, 0x9b, 0x0a, 0x7f, 0x27, 0x1e, 0xc5, 0x69, + 0x0e, 0xb4, 0x0d, 0x6b, 0xf2, 0x82, 0x8b, 0xfa, 0x26, 0x9e, 0x40, 0x55, 0x9c, 0xdf, 0x36, 0x02, + 0x68, 0xca, 0xdf, 0x22, 0x6b, 0xe8, 0x8c, 0xd0, 0xec, 0x81, 0x7e, 0x92, 0x34, 0x1b, 0xd3, 0x5e, + 0x97, 0xa9, 0x9e, 0x04, 0x67, 0x78, 0x8c, 0x87, 0x50, 0x4f, 0xfd, 0x39, 0xa5, 0x01, 0x68, 0xc1, + 0xca, 0x71, 0x4a, 0x4f, 0x0d, 0x47, 0x4b, 0xe3, 0x7f, 0x1a, 0x1f, 0xe4, 0x60, 0x32, 0xf0, 0x9f, + 0x93, 0xe0, 0x65, 0xe6, 0xb9, 0xfc, 0xe6, 0xe5, 0xff, 0x00, 0x56, 0x65, 0x1c, 0xa3, 0x03, 0xa9, + 0x92, 0x3b, 0x9d, 0x31, 0x38, 0x66, 0x40, 0x0f, 0x12, 0x23, 0xab, 0x13, 0x3b, 0x86, 0xac, 0x8f, + 0x93, 0x73, 0xfc, 0x59, 0xbc, 0xfa, 0xb3, 0xe7, 0xb8, 0xd4, 0x19, 0x9d, 0xf1, 0x95, 0x06, 0x8d, + 0xae, 0x19, 0x9a, 0x4f, 0x7d, 0x8b, 0x3c, 0x9b, 0xb3, 0xa7, 0x2d, 0x31, 0xf1, 0xbd, 0x01, 0x35, + 0x76, 0x6b, 0xd1, 0xd0, 0x74, 0x47, 0xdc, 0x8c, 0x45, 0x9c, 0x6c, 0x74, 0xfe, 0xdf, 0x80, 0x3a, + 0x33, 0xe2, 0x48, 0x7c, 0x87, 0x41, 0x23, 0x40, 0xfc, 0xa1, 0xe9, 0x8e, 0x7c, 0x2f, 0x1e, 0x0c, + 0xa1, 0x4f, 0x26, 0x54, 0x56, 0x91, 0x54, 0x3a, 0xb5, 0x7d, 0x67, 0x02, 0x47, 0x8e, 0xdc, 0x58, + 0x40, 0x2e, 0xd7, 0xc8, 0xae, 0xd0, 0x67, 0xf6, 0xe0, 0x2c, 0xea, 0x3e, 0xa6, 0x68, 0xcc, 0x91, + 0x46, 0x1a, 0x73, 0xf3, 0x26, 0xb9, 0x10, 0x43, 0x89, 0x28, 0x65, 0x8d, 0x05, 0xf4, 0x25, 0xac, + 0xb3, 0x97, 0x57, 0xfc, 0x00, 0x8c, 0x14, 0x76, 0x26, 0x2b, 0x2c, 0x10, 0xbf, 0xa1, 0x4a, 0x13, + 0xf4, 0xf4, 0x67, 0x20, 0xa4, 0x9a, 0x4d, 0x2b, 0xbe, 0x54, 0xb5, 0x3f, 0x9c, 0x49, 0x17, 0xab, + 0xd8, 0x87, 0x25, 0xde, 0x0e, 0x23, 0x55, 0x3d, 0xa5, 0x3f, 0xf9, 0xb4, 0xa7, 0x3d, 0x4f, 0x8d, + 0x05, 0xf4, 0x6b, 0x58, 0xcb, 0x0d, 0xdb, 0xd1, 0x47, 0x0a, 0x91, 0xea, 0xcf, 0x26, 0xed, 0xbb, + 0x65, 0x48, 0xd3, 0x7e, 0x49, 0x0f, 0xa4, 0x95, 0x7e, 0x51, 0x0c, 0xd5, 0x95, 0x7e, 0x51, 0x4d, + 0xb6, 0x8d, 0x05, 0x34, 0x84, 0x66, 0xf6, 0x9d, 0x8d, 0xb6, 0x15, 0xcc, 0xca, 0xd1, 0x63, 0xfb, + 0xa3, 0x12, 0x94, 0xb1, 0x22, 0x17, 0xae, 0xe4, 0xc7, 0xa9, 0xe8, 0xee, 0x54, 0x01, 0xd9, 0x7a, + 0xf9, 0xb8, 0x14, 0x6d, 0xac, 0xee, 0x25, 0xcf, 0xe2, 0xc2, 0x38, 0x0f, 0xed, 0xa8, 0xc5, 0x4c, + 0x9a, 0x33, 0xb6, 0x77, 0x4b, 0xd3, 0xc7, 0xaa, 0x09, 0x5c, 0x2d, 0x8c, 0xe7, 0xd0, 0xc7, 0xd3, + 0xe4, 0xe4, 0x46, 0x18, 0xed, 0xd9, 0x03, 0x44, 0x63, 0x01, 0x7d, 0x25, 0x2e, 0x1e, 0xd5, 0xc8, + 0x0b, 0xdd, 0x53, 0x6b, 0x9b, 0x32, 0xab, 0x6b, 0x77, 0xde, 0x84, 0x25, 0x3e, 0xeb, 0x2b, 0x7e, + 0x69, 0x28, 0xc6, 0x46, 0x79, 0x7c, 0x8a, 0xe4, 0x4d, 0x9e, 0x87, 0xb5, 0xef, 0xbd, 0x01, 0x47, + 0x6c, 0x80, 0x9f, 0x9f, 0x8b, 0x47, 0x70, 0xb5, 0x3b, 0x33, 0x39, 0xe7, 0xc3, 0xaa, 0x2f, 0x60, + 0x2d, 0xd7, 0xec, 0x2b, 0xeb, 0x5f, 0xfd, 0x20, 0x28, 0x01, 0x2e, 0xb9, 0x3b, 0x18, 0x4d, 0x28, + 0x32, 0xc5, 0x3d, 0xdd, 0xbe, 0x5b, 0x86, 0x34, 0x3a, 0x48, 0xe7, 0x1f, 0x55, 0x58, 0x8d, 0x6e, + 0xd7, 0x4b, 0xb8, 0xd5, 0x2e, 0xe1, 0x9a, 0xf9, 0x02, 0xd6, 0x72, 0x33, 0x43, 0xa5, 0x77, 0xd5, + 0x73, 0xc5, 0x59, 0xa1, 0xfb, 0x1c, 0x1a, 0x99, 0xf1, 0x20, 0xfa, 0x70, 0xd2, 0x45, 0x93, 0x47, + 0xeb, 0xe9, 0x82, 0xf7, 0xee, 0xff, 0xf2, 0xde, 0xd0, 0x0e, 0x4f, 0xc7, 0xc7, 0xec, 0x9f, 0x5d, + 0x41, 0xfa, 0x6d, 0xdb, 0x97, 0xbf, 0x76, 0x23, 0x07, 0xed, 0x72, 0xee, 0x5d, 0xa6, 0x66, 0x74, + 0x7c, 0xbc, 0xcc, 0x57, 0xf7, 0xbf, 0x0e, 0x00, 0x00, 0xff, 0xff, 0xbd, 0xf2, 0x86, 0xb3, 0x44, + 0x22, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used.