提交 3662b3f0 编写于 作者: S sunby 提交者: zhenshan.cao

Refactor dataservice (#5400)

Pick up grpc calls to grpc_handler.go and refactor code format
Signed-off-by: Nsunby <bingyi.sun@zilliz.com>
上级 ef7339ce
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
}
......@@ -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) {
......
......@@ -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)
......
......@@ -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
}
......
......@@ -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
......
// 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)
}
......
此差异已折叠。
......@@ -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)
......
// 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
}
// 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)
}
})
}
......@@ -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
}
......@@ -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
}
......
......@@ -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 {
......
......@@ -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,
}
......@@ -288,3 +288,9 @@ message GetRecoveryInfoRequest {
int64 partitionID = 3;
}
message DataNodeTtMsg {
common.MsgBase base =1;
string channel_name = 2;
uint64 timestamp = 3;
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册