Refactor dataservice (#5400)

Pick up grpc calls to grpc_handler.go and refactor code format

Signed-off-by: sunby <bingyi.sun@zilliz.com>
This commit is contained in:
sunby 2021-05-25 15:35:37 +08:00 committed by zhenshan.cao
parent ef7339ceda
commit 3662b3f0e5
16 changed files with 1035 additions and 1034 deletions

View File

@ -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
}

View File

@ -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) {

View File

@ -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)

View File

@ -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
}

View File

@ -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

View File

@ -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)
}

View File

@ -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) {

View File

@ -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)

View File

@ -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
}

View File

@ -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)
}
})
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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 {

View File

@ -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,
}

View File

@ -288,3 +288,9 @@ message GetRecoveryInfoRequest {
int64 partitionID = 3;
}
message DataNodeTtMsg {
common.MsgBase base =1;
string channel_name = 2;
uint64 timestamp = 3;
}

View File

@ -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.