validate grpc model (#825)
* base validate Signed-off-by: sunwp <244372610@qq.com>
This commit is contained in:
parent
e34eb9607e
commit
c1d84fb56d
|
|
@ -24,7 +24,7 @@ import (
|
|||
//
|
||||
// If the fileLength<=0, which means failed to get fileLength
|
||||
// and then use the DefaultPieceSize.
|
||||
func ComputePieceSize(length int64) int32 {
|
||||
func ComputePieceSize(length int64) uint32 {
|
||||
if length <= 0 || length <= 200*1024*1024 {
|
||||
return config.DefaultPieceSize
|
||||
}
|
||||
|
|
@ -34,5 +34,5 @@ func ComputePieceSize(length int64) int32 {
|
|||
if mpSize > config.DefaultPieceSizeLimit {
|
||||
return config.DefaultPieceSizeLimit
|
||||
}
|
||||
return int32(mpSize)
|
||||
return uint32(mpSize)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -21,7 +21,6 @@ import (
|
|||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"go.opentelemetry.io/otel"
|
||||
"go.opentelemetry.io/otel/trace"
|
||||
"google.golang.org/grpc"
|
||||
|
|
@ -36,9 +35,7 @@ import (
|
|||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/cdnsystem"
|
||||
cdnserver "d7y.io/dragonfly/v2/pkg/rpc/cdnsystem/server"
|
||||
"d7y.io/dragonfly/v2/pkg/util/digestutils"
|
||||
"d7y.io/dragonfly/v2/pkg/util/hostutils"
|
||||
"d7y.io/dragonfly/v2/pkg/util/net/urlutils"
|
||||
"d7y.io/dragonfly/v2/pkg/util/stringutils"
|
||||
)
|
||||
|
||||
|
|
@ -61,18 +58,11 @@ func New(cfg *config.Config, taskMgr supervisor.SeedTaskMgr, opts ...grpc.Server
|
|||
return svr.Server, nil
|
||||
}
|
||||
|
||||
func constructRegisterRequest(req *cdnsystem.SeedRequest) (*types.TaskRegisterRequest, error) {
|
||||
if err := checkSeedRequestParams(req); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
func constructRegisterRequest(req *cdnsystem.SeedRequest) *types.TaskRegisterRequest {
|
||||
meta := req.UrlMeta
|
||||
header := make(map[string]string)
|
||||
if meta != nil {
|
||||
if !stringutils.IsBlank(meta.Digest) {
|
||||
digest := digestutils.Parse(meta.Digest)
|
||||
if _, ok := digestutils.Algorithms[digest[0]]; !ok {
|
||||
return nil, errors.Errorf("unsupported digest algorithm")
|
||||
}
|
||||
header["digest"] = meta.Digest
|
||||
}
|
||||
if !stringutils.IsBlank(meta.Range) {
|
||||
|
|
@ -88,18 +78,7 @@ func constructRegisterRequest(req *cdnsystem.SeedRequest) (*types.TaskRegisterRe
|
|||
Digest: header["digest"],
|
||||
TaskID: req.TaskId,
|
||||
Filter: strings.Split(req.UrlMeta.Filter, "&"),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// checkSeedRequestParams check the params of SeedRequest.
|
||||
func checkSeedRequestParams(req *cdnsystem.SeedRequest) error {
|
||||
if !urlutils.IsValidURL(req.Url) {
|
||||
return errors.Errorf("resource url: %s is invalid", req.Url)
|
||||
}
|
||||
if stringutils.IsBlank(req.TaskId) {
|
||||
return errors.New("taskId is empty")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (css *server) ObtainSeeds(ctx context.Context, req *cdnsystem.SeedRequest, psc chan<- *cdnsystem.PieceSeed) (err error) {
|
||||
|
|
@ -117,12 +96,7 @@ func (css *server) ObtainSeeds(ctx context.Context, req *cdnsystem.SeedRequest,
|
|||
}
|
||||
logger.Infof("seeds task %s result success: %t", req.TaskId, err == nil)
|
||||
}()
|
||||
registerRequest, err := constructRegisterRequest(req)
|
||||
if err != nil {
|
||||
err = dferrors.Newf(base.Code_BadRequest, "bad seed request for task(%s): %v", req.TaskId, err)
|
||||
span.RecordError(err)
|
||||
return err
|
||||
}
|
||||
registerRequest := constructRegisterRequest(req)
|
||||
// register task
|
||||
pieceChan, err := css.taskMgr.Register(ctx, registerRequest)
|
||||
if err != nil {
|
||||
|
|
@ -141,7 +115,7 @@ func (css *server) ObtainSeeds(ctx context.Context, req *cdnsystem.SeedRequest,
|
|||
PeerId: peerID,
|
||||
HostUuid: idgen.CDNHostID(hostutils.FQDNHostname, int32(css.cfg.ListenPort)),
|
||||
PieceInfo: &base.PieceInfo{
|
||||
PieceNum: piece.PieceNum,
|
||||
PieceNum: int32(piece.PieceNum),
|
||||
RangeStart: piece.PieceRange.StartIndex,
|
||||
RangeSize: piece.PieceLen,
|
||||
PieceMd5: piece.PieceMd5,
|
||||
|
|
@ -186,11 +160,6 @@ func (css *server) GetPieceTasks(ctx context.Context, req *base.PieceTaskRequest
|
|||
}
|
||||
}()
|
||||
logger.Infof("get piece tasks: %+v", req)
|
||||
if err := checkPieceTasksRequestParams(req); err != nil {
|
||||
err = dferrors.Newf(base.Code_BadRequest, "failed to validate seed request for task(%s): %v", req.TaskId, err)
|
||||
span.RecordError(err)
|
||||
return nil, err
|
||||
}
|
||||
task, err := css.taskMgr.Get(req.TaskId)
|
||||
if err != nil {
|
||||
if cdnerrors.IsDataNotFound(err) {
|
||||
|
|
@ -214,11 +183,11 @@ func (css *server) GetPieceTasks(ctx context.Context, req *base.PieceTaskRequest
|
|||
return nil, err
|
||||
}
|
||||
pieceInfos := make([]*base.PieceInfo, 0)
|
||||
var count int32 = 0
|
||||
var count uint32 = 0
|
||||
for _, piece := range pieces {
|
||||
if piece.PieceNum >= req.StartNum && (count < req.Limit || req.Limit == 0) {
|
||||
p := &base.PieceInfo{
|
||||
PieceNum: piece.PieceNum,
|
||||
PieceNum: int32(piece.PieceNum),
|
||||
RangeStart: piece.PieceRange.StartIndex,
|
||||
RangeSize: piece.PieceLen,
|
||||
PieceMd5: piece.PieceMd5,
|
||||
|
|
@ -241,19 +210,3 @@ func (css *server) GetPieceTasks(ctx context.Context, req *base.PieceTaskRequest
|
|||
span.SetAttributes(config.AttributePiecePacketResult.String(pp.String()))
|
||||
return pp, nil
|
||||
}
|
||||
|
||||
func checkPieceTasksRequestParams(req *base.PieceTaskRequest) error {
|
||||
if stringutils.IsBlank(req.TaskId) {
|
||||
return errors.Wrap(cdnerrors.ErrInvalidValue, "taskId is nil")
|
||||
}
|
||||
if stringutils.IsBlank(req.SrcPid) {
|
||||
return errors.Wrapf(cdnerrors.ErrInvalidValue, "src peer id is nil")
|
||||
}
|
||||
if req.StartNum < 0 {
|
||||
return errors.Wrapf(cdnerrors.ErrInvalidValue, "invalid starNum %d", req.StartNum)
|
||||
}
|
||||
if req.Limit < 0 {
|
||||
return errors.Wrapf(cdnerrors.ErrInvalidValue, "invalid limit %d", req.Limit)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
|
|
|||
|
|
@ -23,7 +23,6 @@ import (
|
|||
|
||||
"d7y.io/dragonfly/v2/cdn/config"
|
||||
"d7y.io/dragonfly/v2/cdn/supervisor"
|
||||
"d7y.io/dragonfly/v2/cdn/types"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/cdnsystem"
|
||||
)
|
||||
|
|
@ -121,69 +120,3 @@ func TestNewCdnSeedServer(t *testing.T) {
|
|||
})
|
||||
}
|
||||
}
|
||||
|
||||
func Test_checkPieceTasksRequestParams(t *testing.T) {
|
||||
type args struct {
|
||||
req *base.PieceTaskRequest
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
args args
|
||||
wantErr bool
|
||||
}{
|
||||
// TODO: Add test cases.
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
if err := checkPieceTasksRequestParams(tt.args.req); (err != nil) != tt.wantErr {
|
||||
t.Errorf("checkPieceTasksRequestParams() error = %v, wantErr %v", err, tt.wantErr)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func Test_checkSeedRequestParams(t *testing.T) {
|
||||
type args struct {
|
||||
req *cdnsystem.SeedRequest
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
args args
|
||||
wantErr bool
|
||||
}{
|
||||
// TODO: Add test cases.
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
if err := checkSeedRequestParams(tt.args.req); (err != nil) != tt.wantErr {
|
||||
t.Errorf("checkSeedRequestParams() error = %v, wantErr %v", err, tt.wantErr)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func Test_constructRegisterRequest(t *testing.T) {
|
||||
type args struct {
|
||||
req *cdnsystem.SeedRequest
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
args args
|
||||
want *types.TaskRegisterRequest
|
||||
wantErr bool
|
||||
}{
|
||||
// TODO: Add test cases.
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
got, err := constructRegisterRequest(tt.args.req)
|
||||
if (err != nil) != tt.wantErr {
|
||||
t.Errorf("constructRegisterRequest() error = %v, wantErr %v", err, tt.wantErr)
|
||||
return
|
||||
}
|
||||
if !reflect.DeepEqual(got, tt.want) {
|
||||
t.Errorf("constructRegisterRequest() got = %v, want %v", got, tt.want)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -185,7 +185,7 @@ func (cd *cacheDetector) parseByReadFile(taskID string, metaData *storage.FileMe
|
|||
var breakPoint uint64 = 0
|
||||
pieceMetaRecords := make([]*storage.PieceMetaRecord, 0, len(tempRecords))
|
||||
for index := range tempRecords {
|
||||
if int32(index) != tempRecords[index].PieceNum {
|
||||
if uint32(index) != tempRecords[index].PieceNum {
|
||||
break
|
||||
}
|
||||
// read content
|
||||
|
|
|
|||
|
|
@ -167,8 +167,8 @@ func (cw *cacheWriter) writerPool(ctx context.Context, wg *sync.WaitGroup, routi
|
|||
start := uint64(p.pieceNum) * uint64(p.pieceSize)
|
||||
end := start + uint64(pieceLen) - 1
|
||||
pieceRecord := &storage.PieceMetaRecord{
|
||||
PieceNum: p.pieceNum,
|
||||
PieceLen: int32(pieceLen),
|
||||
PieceNum: uint32(p.pieceNum),
|
||||
PieceLen: uint32(pieceLen),
|
||||
Md5: digestutils.ToHashString(pieceMd5),
|
||||
Range: &rangeutils.Range{
|
||||
StartIndex: start,
|
||||
|
|
|
|||
|
|
@ -62,7 +62,7 @@ func (re *reporter) reportPieceMetaRecord(ctx context.Context, taskID string, re
|
|||
from string) error {
|
||||
// report cache pieces status
|
||||
logger.DownloaderLogger.Info(taskID,
|
||||
zap.Int32("pieceNum", record.PieceNum),
|
||||
zap.Uint32("pieceNum", record.PieceNum),
|
||||
zap.String("md5", record.Md5),
|
||||
zap.String("from", from))
|
||||
return re.progress.PublishPiece(ctx, taskID, convertPieceMeta2SeedPiece(record))
|
||||
|
|
@ -74,7 +74,7 @@ func (re *reporter) reportPieceMetaRecord(ctx context.Context, taskID string, re
|
|||
func convertPieceMeta2SeedPiece(record *storage.PieceMetaRecord) *types.SeedPiece {
|
||||
return &types.SeedPiece{
|
||||
PieceStyle: record.PieceStyle,
|
||||
PieceNum: record.PieceNum,
|
||||
PieceNum: uint32(record.PieceNum),
|
||||
PieceMd5: record.Md5,
|
||||
PieceRange: record.Range,
|
||||
OriginRange: record.OriginRange,
|
||||
|
|
|
|||
|
|
@ -97,8 +97,8 @@ type FileMetaData struct {
|
|||
|
||||
// PieceMetaRecord meta data of piece
|
||||
type PieceMetaRecord struct {
|
||||
PieceNum int32 `json:"pieceNum"` // piece Num start from 0
|
||||
PieceLen int32 `json:"pieceLen"` // 存储到存储介质的真实长度
|
||||
PieceNum uint32 `json:"pieceNum"` // piece Num start from 0
|
||||
PieceLen uint32 `json:"pieceLen"` // 存储到存储介质的真实长度
|
||||
Md5 string `json:"md5"` // for transported piece content,不是origin source 的 md5,是真是存储到存储介质后的md5(为了读取数据文件时方便校验完整性)
|
||||
Range *rangeutils.Range `json:"range"` // 下载存储到磁盘的range,不是origin source的range.提供给客户端发送下载请求,for transported piece content
|
||||
OriginRange *rangeutils.Range `json:"originRange"` // piece's real offset in the file
|
||||
|
|
@ -141,8 +141,8 @@ func ParsePieceMetaRecord(value string) (record *PieceMetaRecord, err error) {
|
|||
return nil, errors.Wrapf(err, "invalid pieceStyle: %s", fields[5])
|
||||
}
|
||||
return &PieceMetaRecord{
|
||||
PieceNum: int32(pieceNum),
|
||||
PieceLen: int32(pieceLen),
|
||||
PieceNum: uint32(pieceNum),
|
||||
PieceLen: uint32(pieceLen),
|
||||
Md5: md5,
|
||||
Range: pieceRange,
|
||||
OriginRange: originRange,
|
||||
|
|
|
|||
|
|
@ -119,7 +119,7 @@ func (tm *Manager) addOrUpdateTask(ctx context.Context, request *types.TaskRegis
|
|||
// calculate piece size and update the PieceSize and PieceTotal
|
||||
if task.PieceSize <= 0 {
|
||||
pieceSize := cdnutil.ComputePieceSize(task.SourceFileLength)
|
||||
task.PieceSize = pieceSize
|
||||
task.PieceSize = int32(pieceSize)
|
||||
}
|
||||
if err := tm.taskStore.Add(task.TaskID, task); err != nil {
|
||||
return nil, err
|
||||
|
|
|
|||
|
|
@ -20,11 +20,11 @@ import "d7y.io/dragonfly/v2/pkg/util/rangeutils"
|
|||
|
||||
type SeedPiece struct {
|
||||
PieceStyle PieceFormat `json:"piece_style"` // 0: PlainUnspecified
|
||||
PieceNum int32 `json:"piece_num"`
|
||||
PieceNum uint32 `json:"piece_num"`
|
||||
PieceMd5 string `json:"piece_md_5"`
|
||||
PieceRange *rangeutils.Range `json:"piece_range"`
|
||||
OriginRange *rangeutils.Range `json:"origin_range"`
|
||||
PieceLen int32 `json:"piece_len"`
|
||||
PieceLen uint32 `json:"piece_len"`
|
||||
}
|
||||
|
||||
type PieceFormat int8
|
||||
|
|
|
|||
|
|
@ -89,7 +89,7 @@ type peerTask struct {
|
|||
md5 string
|
||||
contentLength *atomic.Int64
|
||||
completedLength *atomic.Int64
|
||||
usedTraffic *atomic.Int64
|
||||
usedTraffic *atomic.Uint64
|
||||
|
||||
//sizeScope base.SizeScope
|
||||
singlePiece *scheduler.SinglePiece
|
||||
|
|
@ -162,11 +162,11 @@ func (pt *peerTask) SetContentLength(i int64) error {
|
|||
return pt.setContentLengthFunc(i)
|
||||
}
|
||||
|
||||
func (pt *peerTask) AddTraffic(n int64) {
|
||||
func (pt *peerTask) AddTraffic(n uint64) {
|
||||
pt.usedTraffic.Add(n)
|
||||
}
|
||||
|
||||
func (pt *peerTask) GetTraffic() int64 {
|
||||
func (pt *peerTask) GetTraffic() uint64 {
|
||||
return pt.usedTraffic.Load()
|
||||
}
|
||||
|
||||
|
|
@ -393,11 +393,11 @@ func (pt *peerTask) pullPiecesFromPeers(cleanUnfinishedFunc func()) {
|
|||
var (
|
||||
num int32
|
||||
ok bool
|
||||
limit int32
|
||||
limit uint32
|
||||
initialized bool
|
||||
pieceRequestCh chan *DownloadPieceRequest
|
||||
// keep same size with pt.failedPieceCh for avoiding dead-lock
|
||||
pieceBufferSize = int32(config.DefaultPieceChanSize)
|
||||
pieceBufferSize = uint32(config.DefaultPieceChanSize)
|
||||
)
|
||||
limit = pieceBufferSize
|
||||
loop:
|
||||
|
|
@ -437,7 +437,7 @@ loop:
|
|||
&base.PieceTaskRequest{
|
||||
TaskId: pt.taskID,
|
||||
SrcPid: pt.peerID,
|
||||
StartNum: num,
|
||||
StartNum: uint32(num),
|
||||
Limit: limit,
|
||||
})
|
||||
|
||||
|
|
@ -492,7 +492,7 @@ loop:
|
|||
}
|
||||
}
|
||||
|
||||
func (pt *peerTask) init(piecePacket *base.PiecePacket, pieceBufferSize int32) (chan *DownloadPieceRequest, bool) {
|
||||
func (pt *peerTask) init(piecePacket *base.PiecePacket, pieceBufferSize uint32) (chan *DownloadPieceRequest, bool) {
|
||||
pt.contentLength.Store(piecePacket.ContentLength)
|
||||
if pt.contentLength.Load() > 0 {
|
||||
pt.span.SetAttributes(config.AttributeTaskContentLength.Int64(pt.contentLength.Load()))
|
||||
|
|
@ -796,7 +796,7 @@ retry:
|
|||
}
|
||||
|
||||
if code == base.Code_CDNTaskNotFound && curPeerPacket == pt.peerPacket.Load().(*scheduler.PeerPacket) {
|
||||
span.AddEvent("retry for CDNTaskNotFound")
|
||||
span.AddEvent("retry for CdnTaskNotFound")
|
||||
goto retry
|
||||
}
|
||||
return nil, err
|
||||
|
|
@ -875,7 +875,7 @@ func (pt *peerTask) getNextPieceNum(cur int32) int32 {
|
|||
// double check, re-search not success or not requested pieces
|
||||
for i = int32(0); pt.requestedPieces.IsSet(i); i++ {
|
||||
}
|
||||
if pt.totalPiece > 0 && i >= pt.totalPiece {
|
||||
if pt.totalPiece > 0 && i >= int32(pt.totalPiece) {
|
||||
return -1
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -200,7 +200,7 @@ func newFilePeerTask(ctx context.Context,
|
|||
schedulerClient: schedulerClient,
|
||||
limiter: limiter,
|
||||
completedLength: atomic.NewInt64(0),
|
||||
usedTraffic: atomic.NewInt64(0),
|
||||
usedTraffic: atomic.NewUint64(0),
|
||||
SugaredLoggerOnWith: logger.With("peer", request.PeerId, "task", result.TaskId, "component", "filePeerTask"),
|
||||
},
|
||||
}
|
||||
|
|
|
|||
|
|
@ -51,7 +51,7 @@ func (p *filePeerTaskCallback) Init(pt Task) error {
|
|||
Destination: p.req.Output,
|
||||
},
|
||||
ContentLength: pt.GetContentLength(),
|
||||
TotalPieces: pt.GetTotalPieces(),
|
||||
TotalPieces: int32(pt.GetTotalPieces()),
|
||||
PieceMd5Sign: pt.GetPieceMd5Sign(),
|
||||
})
|
||||
if err != nil {
|
||||
|
|
@ -69,7 +69,7 @@ func (p *filePeerTaskCallback) Update(pt Task) error {
|
|||
TaskID: pt.GetTaskID(),
|
||||
},
|
||||
ContentLength: pt.GetContentLength(),
|
||||
TotalPieces: pt.GetTotalPieces(),
|
||||
TotalPieces: int32(pt.GetTotalPieces()),
|
||||
PieceMd5Sign: pt.GetPieceMd5Sign(),
|
||||
})
|
||||
if err != nil {
|
||||
|
|
@ -90,7 +90,7 @@ func (p *filePeerTaskCallback) Done(pt Task) error {
|
|||
Destination: p.req.Output,
|
||||
},
|
||||
MetadataOnly: false,
|
||||
TotalPieces: pt.GetTotalPieces(),
|
||||
TotalPieces: int32(pt.GetTotalPieces()),
|
||||
})
|
||||
if e != nil {
|
||||
return e
|
||||
|
|
|
|||
|
|
@ -67,7 +67,7 @@ func TestFilePeerTask_BackSource_WithContentLength(t *testing.T) {
|
|||
componentsOption{
|
||||
taskID: taskID,
|
||||
contentLength: int64(mockContentLength),
|
||||
pieceSize: int32(pieceSize),
|
||||
pieceSize: uint32(pieceSize),
|
||||
pieceParallelCount: pieceParallelCount,
|
||||
})
|
||||
defer storageManager.CleanUp()
|
||||
|
|
@ -184,7 +184,7 @@ func TestFilePeerTask_BackSource_WithoutContentLength(t *testing.T) {
|
|||
componentsOption{
|
||||
taskID: taskID,
|
||||
contentLength: int64(mockContentLength),
|
||||
pieceSize: int32(pieceSize),
|
||||
pieceSize: uint32(pieceSize),
|
||||
pieceParallelCount: pieceParallelCount,
|
||||
})
|
||||
defer storageManager.CleanUp()
|
||||
|
|
|
|||
|
|
@ -71,8 +71,8 @@ type Task interface {
|
|||
// SetContentLength will be called after download completed, when download from source without content length
|
||||
SetContentLength(int64) error
|
||||
SetCallback(TaskCallback)
|
||||
AddTraffic(int64)
|
||||
GetTraffic() int64
|
||||
AddTraffic(uint64)
|
||||
GetTraffic() uint64
|
||||
SetPieceMd5Sign(string)
|
||||
GetPieceMd5Sign() string
|
||||
}
|
||||
|
|
|
|||
|
|
@ -123,7 +123,7 @@ func (m *MockTask) EXPECT() *MockTaskMockRecorder {
|
|||
}
|
||||
|
||||
// AddTraffic mocks base method.
|
||||
func (m *MockTask) AddTraffic(arg0 int64) {
|
||||
func (m *MockTask) AddTraffic(arg0 uint64) {
|
||||
m.ctrl.T.Helper()
|
||||
m.ctrl.Call(m, "AddTraffic", arg0)
|
||||
}
|
||||
|
|
@ -219,10 +219,10 @@ func (mr *MockTaskMockRecorder) GetTotalPieces() *gomock.Call {
|
|||
}
|
||||
|
||||
// GetTraffic mocks base method.
|
||||
func (m *MockTask) GetTraffic() int64 {
|
||||
func (m *MockTask) GetTraffic() uint64 {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "GetTraffic")
|
||||
ret0, _ := ret[0].(int64)
|
||||
ret0, _ := ret[0].(uint64)
|
||||
return ret0
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -55,7 +55,7 @@ import (
|
|||
type componentsOption struct {
|
||||
taskID string
|
||||
contentLength int64
|
||||
pieceSize int32
|
||||
pieceSize uint32
|
||||
pieceParallelCount int32
|
||||
peerPacketDelay []time.Duration
|
||||
}
|
||||
|
|
@ -67,18 +67,18 @@ func setupPeerTaskManagerComponents(ctrl *gomock.Controller, opt componentsOptio
|
|||
var daemon = mock_daemon.NewMockDaemonServer(ctrl)
|
||||
daemon.EXPECT().GetPieceTasks(gomock.Any(), gomock.Any()).AnyTimes().DoAndReturn(func(ctx context.Context, request *base.PieceTaskRequest) (*base.PiecePacket, error) {
|
||||
var tasks []*base.PieceInfo
|
||||
for i := int32(0); i < request.Limit; i++ {
|
||||
for i := uint32(0); i < request.Limit; i++ {
|
||||
start := opt.pieceSize * (request.StartNum + i)
|
||||
if int64(start)+1 > opt.contentLength {
|
||||
break
|
||||
}
|
||||
size := opt.pieceSize
|
||||
if int64(start+opt.pieceSize) > opt.contentLength {
|
||||
size = int32(opt.contentLength) - start
|
||||
size = uint32(opt.contentLength) - start
|
||||
}
|
||||
tasks = append(tasks,
|
||||
&base.PieceInfo{
|
||||
PieceNum: request.StartNum + i,
|
||||
PieceNum: int32(request.StartNum + i),
|
||||
RangeStart: uint64(start),
|
||||
RangeSize: size,
|
||||
PieceMd5: "",
|
||||
|
|
@ -191,7 +191,7 @@ func TestPeerTaskManager_StartFilePeerTask(t *testing.T) {
|
|||
componentsOption{
|
||||
taskID: taskID,
|
||||
contentLength: int64(mockContentLength),
|
||||
pieceSize: int32(pieceSize),
|
||||
pieceSize: uint32(pieceSize),
|
||||
pieceParallelCount: pieceParallelCount,
|
||||
})
|
||||
defer storageManager.CleanUp()
|
||||
|
|
@ -274,7 +274,7 @@ func TestPeerTaskManager_StartStreamPeerTask(t *testing.T) {
|
|||
componentsOption{
|
||||
taskID: taskID,
|
||||
contentLength: int64(mockContentLength),
|
||||
pieceSize: int32(pieceSize),
|
||||
pieceSize: uint32(pieceSize),
|
||||
pieceParallelCount: pieceParallelCount,
|
||||
})
|
||||
defer storageManager.CleanUp()
|
||||
|
|
@ -353,7 +353,7 @@ func TestPeerTaskManager_StartStreamPeerTask_BackSource(t *testing.T) {
|
|||
componentsOption{
|
||||
taskID: taskID,
|
||||
contentLength: int64(mockContentLength),
|
||||
pieceSize: int32(pieceSize),
|
||||
pieceSize: uint32(pieceSize),
|
||||
pieceParallelCount: pieceParallelCount,
|
||||
peerPacketDelay: []time.Duration{time.Second},
|
||||
})
|
||||
|
|
|
|||
|
|
@ -176,7 +176,7 @@ func newStreamPeerTask(ctx context.Context,
|
|||
schedulerClient: schedulerClient,
|
||||
limiter: limiter,
|
||||
completedLength: atomic.NewInt64(0),
|
||||
usedTraffic: atomic.NewInt64(0),
|
||||
usedTraffic: atomic.NewUint64(0),
|
||||
SugaredLoggerOnWith: logger.With("peer", request.PeerId, "task", result.TaskId, "component", "streamPeerTask"),
|
||||
},
|
||||
}
|
||||
|
|
|
|||
|
|
@ -74,7 +74,7 @@ func setupBackSourcePartialComponents(ctrl *gomock.Controller, testBytes []byte,
|
|||
if request.StartNum == 0 {
|
||||
tasks = append(tasks,
|
||||
&base.PieceInfo{
|
||||
PieceNum: request.StartNum,
|
||||
PieceNum: int32(request.StartNum),
|
||||
RangeStart: uint64(0),
|
||||
RangeSize: opt.pieceSize,
|
||||
PieceMd5: digestutils.Md5Bytes(testBytes[0:opt.pieceSize]),
|
||||
|
|
@ -197,7 +197,7 @@ func TestStreamPeerTask_BackSource_Partial_WithContentLength(t *testing.T) {
|
|||
componentsOption{
|
||||
taskID: taskID,
|
||||
contentLength: int64(mockContentLength),
|
||||
pieceSize: int32(pieceSize),
|
||||
pieceSize: uint32(pieceSize),
|
||||
pieceParallelCount: pieceParallelCount,
|
||||
})
|
||||
defer storageManager.CleanUp()
|
||||
|
|
@ -228,8 +228,8 @@ func TestStreamPeerTask_BackSource_Partial_WithContentLength(t *testing.T) {
|
|||
calculateDigest: true,
|
||||
storageManager: storageManager,
|
||||
pieceDownloader: downloader,
|
||||
computePieceSize: func(contentLength int64) int32 {
|
||||
return int32(pieceSize)
|
||||
computePieceSize: func(contentLength int64) uint32 {
|
||||
return uint32(pieceSize)
|
||||
},
|
||||
}
|
||||
ptm := &peerTaskManager{
|
||||
|
|
|
|||
|
|
@ -63,7 +63,7 @@ func TestStreamPeerTask_BackSource_WithContentLength(t *testing.T) {
|
|||
componentsOption{
|
||||
taskID: taskID,
|
||||
contentLength: int64(mockContentLength),
|
||||
pieceSize: int32(pieceSize),
|
||||
pieceSize: uint32(pieceSize),
|
||||
pieceParallelCount: pieceParallelCount,
|
||||
})
|
||||
defer storageManager.CleanUp()
|
||||
|
|
@ -120,8 +120,8 @@ func TestStreamPeerTask_BackSource_WithContentLength(t *testing.T) {
|
|||
&pieceManager{
|
||||
storageManager: storageManager,
|
||||
pieceDownloader: downloader,
|
||||
computePieceSize: func(contentLength int64) int32 {
|
||||
return int32(pieceSize)
|
||||
computePieceSize: func(contentLength int64) uint32 {
|
||||
return uint32(pieceSize)
|
||||
},
|
||||
},
|
||||
req,
|
||||
|
|
@ -169,7 +169,7 @@ func TestStreamPeerTask_BackSource_WithoutContentLength(t *testing.T) {
|
|||
componentsOption{
|
||||
taskID: taskID,
|
||||
contentLength: int64(mockContentLength),
|
||||
pieceSize: int32(pieceSize),
|
||||
pieceSize: uint32(pieceSize),
|
||||
pieceParallelCount: pieceParallelCount,
|
||||
})
|
||||
defer storageManager.CleanUp()
|
||||
|
|
@ -226,8 +226,8 @@ func TestStreamPeerTask_BackSource_WithoutContentLength(t *testing.T) {
|
|||
&pieceManager{
|
||||
storageManager: storageManager,
|
||||
pieceDownloader: downloader,
|
||||
computePieceSize: func(contentLength int64) int32 {
|
||||
return int32(pieceSize)
|
||||
computePieceSize: func(contentLength int64) uint32 {
|
||||
return uint32(pieceSize)
|
||||
},
|
||||
},
|
||||
req,
|
||||
|
|
|
|||
|
|
@ -51,7 +51,7 @@ func TestPieceDownloader_DownloadPiece(t *testing.T) {
|
|||
taskID string
|
||||
pieceRange string
|
||||
rangeStart uint64
|
||||
rangeSize int32
|
||||
rangeSize uint32
|
||||
targetPieceData []byte
|
||||
}{
|
||||
{
|
||||
|
|
@ -96,7 +96,7 @@ func TestPieceDownloader_DownloadPiece(t *testing.T) {
|
|||
taskID: "task-2",
|
||||
pieceRange: fmt.Sprintf("bytes=512-%d", len(testData)-1),
|
||||
rangeStart: 512,
|
||||
rangeSize: int32(len(testData) - 512),
|
||||
rangeSize: uint32(len(testData) - 512),
|
||||
targetPieceData: testData[512:],
|
||||
},
|
||||
{
|
||||
|
|
|
|||
|
|
@ -45,7 +45,7 @@ type pieceManager struct {
|
|||
*rate.Limiter
|
||||
storageManager storage.TaskStorageDriver
|
||||
pieceDownloader PieceDownloader
|
||||
computePieceSize func(contentLength int64) int32
|
||||
computePieceSize func(contentLength int64) uint32
|
||||
|
||||
calculateDigest bool
|
||||
}
|
||||
|
|
@ -148,7 +148,9 @@ func (pm *pieceManager) DownloadPiece(ctx context.Context, pt Task, request *Dow
|
|||
end = time.Now().UnixNano()
|
||||
span.RecordError(err)
|
||||
span.End()
|
||||
pt.AddTraffic(n)
|
||||
if n > 0 {
|
||||
pt.AddTraffic(uint64(n))
|
||||
}
|
||||
if err != nil {
|
||||
pt.Log().Errorf("put piece to storage failed, piece num: %d, wrote: %d, error: %s",
|
||||
request.piece.PieceNum, n, err)
|
||||
|
|
@ -211,7 +213,7 @@ func (pm *pieceManager) ReadPiece(ctx context.Context, req *storage.ReadPieceReq
|
|||
}
|
||||
|
||||
func (pm *pieceManager) processPieceFromSource(pt Task,
|
||||
reader io.Reader, contentLength int64, pieceNum int32, pieceOffset uint64, pieceSize int32) (int64, error) {
|
||||
reader io.Reader, contentLength int64, pieceNum int32, pieceOffset uint64, pieceSize uint32) (int64, error) {
|
||||
var (
|
||||
success bool
|
||||
start = time.Now().UnixNano()
|
||||
|
|
@ -280,11 +282,13 @@ func (pm *pieceManager) processPieceFromSource(pt Task,
|
|||
},
|
||||
Reader: reader,
|
||||
})
|
||||
if n != int64(size) {
|
||||
size = int32(n)
|
||||
if n != int64(size) && n > 0 {
|
||||
size = uint32(n)
|
||||
}
|
||||
end = time.Now().UnixNano()
|
||||
pt.AddTraffic(n)
|
||||
if n > 0 {
|
||||
pt.AddTraffic(uint64(n))
|
||||
}
|
||||
if err != nil {
|
||||
pt.Log().Errorf("put piece to storage failed, piece num: %d, wrote: %d, error: %s", pieceNum, n, err)
|
||||
return n, err
|
||||
|
|
@ -358,7 +362,7 @@ func (pm *pieceManager) DownloadSource(ctx context.Context, pt Task, request *sc
|
|||
}
|
||||
// last piece, piece size maybe 0
|
||||
if n < int64(size) {
|
||||
contentLength = int64(pieceNum*pieceSize) + n
|
||||
contentLength = int64(pieceNum)*int64(pieceSize) + n
|
||||
if err := pm.storageManager.UpdateTask(ctx,
|
||||
&storage.UpdateTaskRequest{
|
||||
PeerTaskMetaData: storage.PeerTaskMetaData{
|
||||
|
|
@ -383,7 +387,7 @@ func (pm *pieceManager) DownloadSource(ctx context.Context, pt Task, request *sc
|
|||
offset := uint64(pieceNum) * uint64(pieceSize)
|
||||
// calculate piece size for last piece
|
||||
if contentLength > 0 && int64(offset)+int64(size) > contentLength {
|
||||
size = int32(contentLength - int64(offset))
|
||||
size = uint32(contentLength - int64(offset))
|
||||
}
|
||||
|
||||
log.Debugf("download piece %d", pieceNum)
|
||||
|
|
|
|||
|
|
@ -75,7 +75,7 @@ func TestPieceManager_DownloadSource(t *testing.T) {
|
|||
|
||||
testCases := []struct {
|
||||
name string
|
||||
pieceSize int32
|
||||
pieceSize uint32
|
||||
withContentLength bool
|
||||
checkDigest bool
|
||||
}{
|
||||
|
|
@ -105,22 +105,22 @@ func TestPieceManager_DownloadSource(t *testing.T) {
|
|||
},
|
||||
{
|
||||
name: "one pieces with content length case 1",
|
||||
pieceSize: int32(len(testBytes)),
|
||||
pieceSize: uint32(len(testBytes)),
|
||||
withContentLength: true,
|
||||
},
|
||||
{
|
||||
name: "one pieces without content length case 1",
|
||||
pieceSize: int32(len(testBytes)),
|
||||
pieceSize: uint32(len(testBytes)),
|
||||
withContentLength: false,
|
||||
},
|
||||
{
|
||||
name: "one pieces with content length case 2",
|
||||
pieceSize: int32(len(testBytes)) + 1,
|
||||
pieceSize: uint32(len(testBytes)) + 1,
|
||||
withContentLength: true,
|
||||
},
|
||||
{
|
||||
name: "one pieces without content length case 2",
|
||||
pieceSize: int32(len(testBytes)) + 1,
|
||||
pieceSize: uint32(len(testBytes)) + 1,
|
||||
withContentLength: false,
|
||||
},
|
||||
}
|
||||
|
|
@ -143,7 +143,7 @@ func TestPieceManager_DownloadSource(t *testing.T) {
|
|||
func() string {
|
||||
return taskID
|
||||
})
|
||||
mockPeerTask.EXPECT().AddTraffic(gomock.Any()).AnyTimes().DoAndReturn(func(int642 int64) {})
|
||||
mockPeerTask.EXPECT().AddTraffic(gomock.Any()).AnyTimes().DoAndReturn(func(int642 uint64) {})
|
||||
mockPeerTask.EXPECT().ReportPieceResult(gomock.Any()).AnyTimes().DoAndReturn(
|
||||
func(result *pieceTaskResult) error {
|
||||
return nil
|
||||
|
|
@ -180,7 +180,7 @@ func TestPieceManager_DownloadSource(t *testing.T) {
|
|||
|
||||
pm, err := NewPieceManager(storageManager, pieceDownloadTimeout)
|
||||
assert.Nil(err)
|
||||
pm.(*pieceManager).computePieceSize = func(length int64) int32 {
|
||||
pm.(*pieceManager).computePieceSize = func(length int64) uint32 {
|
||||
return tc.pieceSize
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -25,6 +25,7 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/distribution/distribution/v3/uuid"
|
||||
"github.com/golang/mock/gomock"
|
||||
"github.com/phayes/freeport"
|
||||
testifyassert "github.com/stretchr/testify/assert"
|
||||
|
|
@ -33,12 +34,14 @@ import (
|
|||
"d7y.io/dragonfly/v2/client/daemon/peer"
|
||||
mock_peer "d7y.io/dragonfly/v2/client/daemon/test/mock/peer"
|
||||
mock_storage "d7y.io/dragonfly/v2/client/daemon/test/mock/storage"
|
||||
"d7y.io/dragonfly/v2/internal/idgen"
|
||||
"d7y.io/dragonfly/v2/pkg/basic/dfnet"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
dfdaemongrpc "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon"
|
||||
dfclient "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon/client"
|
||||
dfdaemonserver "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon/server"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
|
||||
"d7y.io/dragonfly/v2/pkg/util/net/iputils"
|
||||
)
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
|
|
@ -100,11 +103,15 @@ func TestDownloadManager_ServeDownload(t *testing.T) {
|
|||
})
|
||||
assert.Nil(err, "grpc dial should be ok")
|
||||
request := &dfdaemongrpc.DownRequest{
|
||||
Url: "http://localhost/test",
|
||||
Output: "./testdata/file1",
|
||||
Uuid: uuid.Generate().String(),
|
||||
Url: "http://localhost/test",
|
||||
Output: "./testdata/file1",
|
||||
DisableBackSource: false,
|
||||
UrlMeta: &base.UrlMeta{
|
||||
Tag: "unit test",
|
||||
},
|
||||
Pattern: "p2p",
|
||||
Callsystem: "",
|
||||
}
|
||||
down, err := client.Download(context.Background(), request)
|
||||
assert.Nil(err, "client download grpc call should be ok")
|
||||
|
|
@ -130,16 +137,16 @@ func TestDownloadManager_ServePeer(t *testing.T) {
|
|||
ctrl := gomock.NewController(t)
|
||||
defer ctrl.Finish()
|
||||
|
||||
var maxPieceNum int32 = 10
|
||||
var maxPieceNum uint32 = 10
|
||||
mockStorageManger := mock_storage.NewMockManager(ctrl)
|
||||
mockStorageManger.EXPECT().GetPieces(gomock.Any(), gomock.Any()).AnyTimes().DoAndReturn(func(ctx context.Context, req *base.PieceTaskRequest) (*base.PiecePacket, error) {
|
||||
var (
|
||||
pieces []*base.PieceInfo
|
||||
pieceSize = int32(1024)
|
||||
pieceSize = uint32(1024)
|
||||
)
|
||||
for i := req.StartNum; i < req.Limit+req.StartNum && i < maxPieceNum; i++ {
|
||||
pieces = append(pieces, &base.PieceInfo{
|
||||
PieceNum: i,
|
||||
PieceNum: int32(i),
|
||||
RangeStart: uint64(i * pieceSize),
|
||||
RangeSize: pieceSize,
|
||||
PieceMd5: "",
|
||||
|
|
@ -191,6 +198,9 @@ func TestDownloadManager_ServePeer(t *testing.T) {
|
|||
}{
|
||||
{
|
||||
request: &base.PieceTaskRequest{
|
||||
TaskId: idgen.TaskID("http://www.test.com", &base.UrlMeta{}),
|
||||
SrcPid: idgen.PeerID(iputils.IPv4),
|
||||
DstPid: idgen.PeerID(iputils.IPv4),
|
||||
StartNum: 0,
|
||||
Limit: 1,
|
||||
},
|
||||
|
|
@ -199,6 +209,9 @@ func TestDownloadManager_ServePeer(t *testing.T) {
|
|||
},
|
||||
{
|
||||
request: &base.PieceTaskRequest{
|
||||
TaskId: idgen.TaskID("http://www.test.com", &base.UrlMeta{}),
|
||||
SrcPid: idgen.PeerID(iputils.IPv4),
|
||||
DstPid: idgen.PeerID(iputils.IPv4),
|
||||
StartNum: 0,
|
||||
Limit: 4,
|
||||
},
|
||||
|
|
@ -207,6 +220,9 @@ func TestDownloadManager_ServePeer(t *testing.T) {
|
|||
},
|
||||
{
|
||||
request: &base.PieceTaskRequest{
|
||||
TaskId: idgen.TaskID("http://www.test.com", &base.UrlMeta{}),
|
||||
SrcPid: idgen.PeerID(iputils.IPv4),
|
||||
DstPid: idgen.PeerID(iputils.IPv4),
|
||||
StartNum: 8,
|
||||
Limit: 1,
|
||||
},
|
||||
|
|
@ -215,6 +231,9 @@ func TestDownloadManager_ServePeer(t *testing.T) {
|
|||
},
|
||||
{
|
||||
request: &base.PieceTaskRequest{
|
||||
TaskId: idgen.TaskID("http://www.test.com", &base.UrlMeta{}),
|
||||
SrcPid: idgen.PeerID(iputils.IPv4),
|
||||
DstPid: idgen.PeerID(iputils.IPv4),
|
||||
StartNum: 8,
|
||||
Limit: 4,
|
||||
},
|
||||
|
|
|
|||
|
|
@ -160,7 +160,7 @@ func (t *localTaskStore) UpdateTask(ctx context.Context, req *UpdateTaskRequest)
|
|||
}
|
||||
if req.GenPieceDigest {
|
||||
var pieceDigests []string
|
||||
for i := int32(0); i < t.TotalPieces; i++ {
|
||||
for i := int32(0); i < int32(t.TotalPieces); i++ {
|
||||
pieceDigests = append(pieceDigests, t.Pieces[i].Md5)
|
||||
}
|
||||
|
||||
|
|
@ -184,7 +184,7 @@ func (t *localTaskStore) ValidateDigest(*PeerTaskMetaData) error {
|
|||
}
|
||||
|
||||
var pieceDigests []string
|
||||
for i := int32(0); i < t.TotalPieces; i++ {
|
||||
for i := int32(0); i < int32(t.TotalPieces); i++ {
|
||||
pieceDigests = append(pieceDigests, t.Pieces[i].Md5)
|
||||
}
|
||||
|
||||
|
|
@ -324,16 +324,16 @@ func (t *localTaskStore) GetPieces(ctx context.Context, req *base.PieceTaskReque
|
|||
t.RLock()
|
||||
defer t.RUnlock()
|
||||
t.touch()
|
||||
if t.TotalPieces > 0 && req.StartNum >= t.TotalPieces {
|
||||
if t.TotalPieces > 0 && int32(req.StartNum) >= t.TotalPieces {
|
||||
t.Errorf("invalid start num: %d", req.StartNum)
|
||||
return nil, dferrors.ErrInvalidArgument
|
||||
}
|
||||
for i := int32(0); i < req.Limit; i++ {
|
||||
if piece, ok := t.Pieces[req.StartNum+i]; ok {
|
||||
for i := int32(0); i < int32(req.Limit); i++ {
|
||||
if piece, ok := t.Pieces[int32(req.StartNum)+i]; ok {
|
||||
pieces = append(pieces, &base.PieceInfo{
|
||||
PieceNum: piece.Num,
|
||||
RangeStart: uint64(piece.Range.Start),
|
||||
RangeSize: int32(piece.Range.Length),
|
||||
RangeSize: uint32(piece.Range.Length),
|
||||
PieceMd5: piece.Md5,
|
||||
PieceOffset: piece.Offset,
|
||||
PieceStyle: piece.Style,
|
||||
|
|
|
|||
|
|
@ -416,7 +416,7 @@ func (s *storageManager) FindCompletedTask(taskID string) *ReusePeerTask {
|
|||
TaskID: taskID,
|
||||
},
|
||||
ContentLength: t.ContentLength,
|
||||
TotalPieces: t.TotalPieces,
|
||||
TotalPieces: int32(t.TotalPieces),
|
||||
}
|
||||
}
|
||||
return nil
|
||||
|
|
|
|||
|
|
@ -200,7 +200,7 @@ func newDownRequest(cfg *config.DfgetConfig, hdr map[string]string) *dfdaemon.Do
|
|||
return &dfdaemon.DownRequest{
|
||||
Url: cfg.URL,
|
||||
Output: cfg.Output,
|
||||
Timeout: int64(cfg.Timeout),
|
||||
Timeout: uint64(cfg.Timeout),
|
||||
Limit: float64(cfg.RateLimit),
|
||||
DisableBackSource: cfg.DisableBackSource,
|
||||
UrlMeta: &base.UrlMeta{
|
||||
|
|
|
|||
2
go.mod
2
go.mod
|
|
@ -35,7 +35,7 @@ require (
|
|||
github.com/google/go-querystring v1.1.0 // indirect
|
||||
github.com/google/uuid v1.2.0
|
||||
github.com/gorilla/mux v1.8.0
|
||||
github.com/grpc-ecosystem/go-grpc-middleware v1.0.0
|
||||
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0
|
||||
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0
|
||||
github.com/jackc/pgproto3/v2 v2.1.1 // indirect
|
||||
github.com/jarcoal/httpmock v1.0.8
|
||||
|
|
|
|||
5
go.sum
5
go.sum
|
|
@ -95,6 +95,7 @@ github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6r
|
|||
github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs=
|
||||
github.com/bitly/go-simplejson v0.5.0/go.mod h1:cXHtHw4XUPsvGaxgjIAn8PhEWG9NfngEKAMDJEczWVA=
|
||||
github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84=
|
||||
github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4YnC6+E63dPcxHo2sUxDIu8g3QgEJdRY=
|
||||
github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4=
|
||||
github.com/bradfitz/gomemcache v0.0.0-20190913173617-a41fca850d0b h1:L/QXpzIa3pOvUGt1D1lA5KjYhPBAN/3iWdP7xeFS9F0=
|
||||
github.com/bradfitz/gomemcache v0.0.0-20190913173617-a41fca850d0b/go.mod h1:H0wQNHz2YrLsuXOZozoeDmnHXkNCRmMW0gwFWDfEZDA=
|
||||
|
|
@ -393,6 +394,8 @@ github.com/gorilla/sessions v1.2.0/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/z
|
|||
github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
|
||||
github.com/grpc-ecosystem/go-grpc-middleware v1.0.0 h1:Iju5GlWwrvL6UBg4zJJt3btmonfrMlCDdsejg4CZE7c=
|
||||
github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs=
|
||||
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw=
|
||||
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y=
|
||||
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho=
|
||||
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk=
|
||||
github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
|
||||
|
|
@ -657,6 +660,7 @@ github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3I
|
|||
github.com/opencontainers/image-spec v1.0.1/go.mod h1:BtxoFyWECRxE4U/7sNtV5W15zMzWCbyJoFRP3s7yZA0=
|
||||
github.com/opencontainers/image-spec v1.0.2 h1:9yCKha/T5XdGtO0q9Q9a6T5NUCsTn/DrBg0D7ufOcFM=
|
||||
github.com/opencontainers/image-spec v1.0.2/go.mod h1:BtxoFyWECRxE4U/7sNtV5W15zMzWCbyJoFRP3s7yZA0=
|
||||
github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
|
||||
github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs=
|
||||
github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc=
|
||||
github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
|
||||
|
|
@ -1248,6 +1252,7 @@ google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfG
|
|||
google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
|
||||
google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
|
||||
google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
|
||||
google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
|
||||
google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
|
||||
google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
|
||||
google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
|
||||
|
|
|
|||
|
|
@ -51,6 +51,6 @@ type GetCDNClustersQuery struct {
|
|||
}
|
||||
|
||||
type CDNClusterConfig struct {
|
||||
LoadLimit uint `yaml:"loadLimit" mapstructure:"loadLimit" json:"load_limit" binding:"omitempty,gte=1,lte=5000"`
|
||||
LoadLimit uint32 `yaml:"loadLimit" mapstructure:"loadLimit" json:"load_limit" binding:"omitempty,gte=1,lte=5000"`
|
||||
NetTopology string `yaml:"netTopology" mapstructure:"netTopology" json:"net_topology"`
|
||||
}
|
||||
|
|
|
|||
|
|
@ -55,7 +55,7 @@ type SchedulerClusterConfig struct {
|
|||
}
|
||||
|
||||
type SchedulerClusterClientConfig struct {
|
||||
LoadLimit uint `yaml:"loadLimit" mapstructure:"loadLimit" json:"load_limit" binding:"omitempty,gte=1,lte=5000"`
|
||||
LoadLimit uint32 `yaml:"loadLimit" mapstructure:"loadLimit" json:"load_limit" binding:"omitempty,gte=1,lte=5000"`
|
||||
}
|
||||
|
||||
type SchedulerClusterScopes struct {
|
||||
|
|
|
|||
|
|
@ -22,6 +22,7 @@
|
|||
package base
|
||||
|
||||
import (
|
||||
_ "github.com/envoyproxy/protoc-gen-validate/validate"
|
||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||
reflect "reflect"
|
||||
|
|
@ -471,9 +472,9 @@ type PieceTaskRequest struct {
|
|||
SrcPid string `protobuf:"bytes,2,opt,name=src_pid,json=srcPid,proto3" json:"src_pid,omitempty"`
|
||||
DstPid string `protobuf:"bytes,3,opt,name=dst_pid,json=dstPid,proto3" json:"dst_pid,omitempty"`
|
||||
// piece number
|
||||
StartNum int32 `protobuf:"varint,4,opt,name=start_num,json=startNum,proto3" json:"start_num,omitempty"`
|
||||
// expected piece count
|
||||
Limit int32 `protobuf:"varint,5,opt,name=limit,proto3" json:"limit,omitempty"`
|
||||
StartNum uint32 `protobuf:"varint,4,opt,name=start_num,json=startNum,proto3" json:"start_num,omitempty"`
|
||||
// expected piece count, limit = 0 represent request pieces as many shards as possible
|
||||
Limit uint32 `protobuf:"varint,5,opt,name=limit,proto3" json:"limit,omitempty"`
|
||||
}
|
||||
|
||||
func (x *PieceTaskRequest) Reset() {
|
||||
|
|
@ -529,14 +530,14 @@ func (x *PieceTaskRequest) GetDstPid() string {
|
|||
return ""
|
||||
}
|
||||
|
||||
func (x *PieceTaskRequest) GetStartNum() int32 {
|
||||
func (x *PieceTaskRequest) GetStartNum() uint32 {
|
||||
if x != nil {
|
||||
return x.StartNum
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *PieceTaskRequest) GetLimit() int32 {
|
||||
func (x *PieceTaskRequest) GetLimit() uint32 {
|
||||
if x != nil {
|
||||
return x.Limit
|
||||
}
|
||||
|
|
@ -548,9 +549,10 @@ type PieceInfo struct {
|
|||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
// piece_num < 0 represent start report piece flag
|
||||
PieceNum int32 `protobuf:"varint,1,opt,name=piece_num,json=pieceNum,proto3" json:"piece_num,omitempty"`
|
||||
RangeStart uint64 `protobuf:"varint,2,opt,name=range_start,json=rangeStart,proto3" json:"range_start,omitempty"`
|
||||
RangeSize int32 `protobuf:"varint,3,opt,name=range_size,json=rangeSize,proto3" json:"range_size,omitempty"`
|
||||
RangeSize uint32 `protobuf:"varint,3,opt,name=range_size,json=rangeSize,proto3" json:"range_size,omitempty"`
|
||||
PieceMd5 string `protobuf:"bytes,4,opt,name=piece_md5,json=pieceMd5,proto3" json:"piece_md5,omitempty"`
|
||||
PieceOffset uint64 `protobuf:"varint,5,opt,name=piece_offset,json=pieceOffset,proto3" json:"piece_offset,omitempty"`
|
||||
PieceStyle PieceStyle `protobuf:"varint,6,opt,name=piece_style,json=pieceStyle,proto3,enum=base.PieceStyle" json:"piece_style,omitempty"`
|
||||
|
|
@ -602,7 +604,7 @@ func (x *PieceInfo) GetRangeStart() uint64 {
|
|||
return 0
|
||||
}
|
||||
|
||||
func (x *PieceInfo) GetRangeSize() int32 {
|
||||
func (x *PieceInfo) GetRangeSize() uint32 {
|
||||
if x != nil {
|
||||
return x.RangeSize
|
||||
}
|
||||
|
|
@ -640,8 +642,9 @@ type PiecePacket struct {
|
|||
// ip:port
|
||||
DstAddr string `protobuf:"bytes,4,opt,name=dst_addr,json=dstAddr,proto3" json:"dst_addr,omitempty"`
|
||||
PieceInfos []*PieceInfo `protobuf:"bytes,5,rep,name=piece_infos,json=pieceInfos,proto3" json:"piece_infos,omitempty"`
|
||||
// total piece count for url
|
||||
TotalPiece int32 `protobuf:"varint,6,opt,name=total_piece,json=totalPiece,proto3" json:"total_piece,omitempty"`
|
||||
// total piece count for url, total_piece represent total piece is unknown
|
||||
TotalPiece int32 `protobuf:"varint,6,opt,name=total_piece,json=totalPiece,proto3" json:"total_piece,omitempty"`
|
||||
// content_length < 0 represent content length is unknown
|
||||
ContentLength int64 `protobuf:"varint,7,opt,name=content_length,json=contentLength,proto3" json:"content_length,omitempty"`
|
||||
// sha256 code of all piece md5
|
||||
PieceMd5Sign string `protobuf:"bytes,8,opt,name=piece_md5_sign,json=pieceMd5Sign,proto3" json:"piece_md5_sign,omitempty"`
|
||||
|
|
@ -732,116 +735,136 @@ var File_pkg_rpc_base_base_proto protoreflect.FileDescriptor
|
|||
|
||||
var file_pkg_rpc_base_base_proto_rawDesc = []byte{
|
||||
0x0a, 0x17, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x62, 0x61, 0x73, 0x65, 0x2f, 0x62,
|
||||
0x61, 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x04, 0x62, 0x61, 0x73, 0x65, 0x22,
|
||||
0x47, 0x0a, 0x0b, 0x47, 0x72, 0x70, 0x63, 0x44, 0x66, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1e,
|
||||
0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62,
|
||||
0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x18,
|
||||
0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xcf, 0x01, 0x0a, 0x07, 0x55, 0x72, 0x6c,
|
||||
0x4d, 0x65, 0x74, 0x61, 0x12, 0x16, 0x0a, 0x06, 0x64, 0x69, 0x67, 0x65, 0x73, 0x74, 0x18, 0x01,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03,
|
||||
0x74, 0x61, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x74, 0x61, 0x67, 0x12, 0x14,
|
||||
0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72,
|
||||
0x61, 0x6e, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x04,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x31, 0x0a, 0x06,
|
||||
0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x62,
|
||||
0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x48, 0x65, 0x61, 0x64,
|
||||
0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x1a,
|
||||
0x39, 0x0a, 0x0b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10,
|
||||
0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79,
|
||||
0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x63, 0x0a, 0x08, 0x48, 0x6f,
|
||||
0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x70, 0x75, 0x5f, 0x72, 0x61,
|
||||
0x74, 0x69, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x02, 0x52, 0x08, 0x63, 0x70, 0x75, 0x52, 0x61,
|
||||
0x74, 0x69, 0x6f, 0x12, 0x1b, 0x0a, 0x09, 0x6d, 0x65, 0x6d, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x52, 0x08, 0x6d, 0x65, 0x6d, 0x52, 0x61, 0x74, 0x69, 0x6f,
|
||||
0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x03,
|
||||
0x20, 0x01, 0x28, 0x02, 0x52, 0x09, 0x64, 0x69, 0x73, 0x6b, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x22,
|
||||
0x90, 0x01, 0x0a, 0x10, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71,
|
||||
0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a,
|
||||
0x07, 0x73, 0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06,
|
||||
0x73, 0x72, 0x63, 0x50, 0x69, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69,
|
||||
0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12,
|
||||
0x1b, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01,
|
||||
0x28, 0x05, 0x52, 0x08, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4e, 0x75, 0x6d, 0x12, 0x14, 0x0a, 0x05,
|
||||
0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x6c, 0x69, 0x6d,
|
||||
0x69, 0x74, 0x22, 0xdb, 0x01, 0x0a, 0x09, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f,
|
||||
0x12, 0x1b, 0x0a, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4e, 0x75, 0x6d, 0x12, 0x1f, 0x0a,
|
||||
0x0b, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01,
|
||||
0x28, 0x04, 0x52, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, 0x1d,
|
||||
0x0a, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01,
|
||||
0x28, 0x05, 0x52, 0x09, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1b, 0x0a,
|
||||
0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x6d, 0x64, 0x35, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09,
|
||||
0x52, 0x08, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4d, 0x64, 0x35, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x69,
|
||||
0x65, 0x63, 0x65, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04,
|
||||
0x61, 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x04, 0x62, 0x61, 0x73, 0x65, 0x1a,
|
||||
0x17, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61,
|
||||
0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x51, 0x0a, 0x0b, 0x47, 0x72, 0x70, 0x63,
|
||||
0x44, 0x66, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x28, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64,
|
||||
0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, 0x01, 0x02, 0x10, 0x01, 0x52, 0x04, 0x63, 0x6f, 0x64,
|
||||
0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x93, 0x02, 0x0a, 0x07,
|
||||
0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x3e, 0x0a, 0x06, 0x64, 0x69, 0x67, 0x65, 0x73,
|
||||
0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x26, 0xfa, 0x42, 0x23, 0x72, 0x21, 0x32, 0x1c,
|
||||
0x5e, 0x28, 0x6d, 0x64, 0x35, 0x29, 0x7c, 0x28, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x29, 0x3a,
|
||||
0x5b, 0x41, 0x2d, 0x46, 0x61, 0x2d, 0x66, 0x30, 0x2d, 0x39, 0x5d, 0x2b, 0xd0, 0x01, 0x01, 0x52,
|
||||
0x06, 0x64, 0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x74, 0x61, 0x67, 0x18, 0x02,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x74, 0x61, 0x67, 0x12, 0x30, 0x0a, 0x05, 0x72, 0x61, 0x6e,
|
||||
0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x1a, 0xfa, 0x42, 0x17, 0x72, 0x15, 0x32,
|
||||
0x10, 0x5e, 0x5b, 0x30, 0x2d, 0x39, 0x5d, 0x2b, 0x5c, 0x2d, 0x5e, 0x5b, 0x30, 0x2d, 0x39, 0x5d,
|
||||
0x2b, 0xd0, 0x01, 0x01, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x66,
|
||||
0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c,
|
||||
0x74, 0x65, 0x72, 0x12, 0x31, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x05, 0x20,
|
||||
0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65,
|
||||
0x74, 0x61, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06,
|
||||
0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x1a, 0x39, 0x0a, 0x0b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
|
||||
0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38,
|
||||
0x01, 0x22, 0x96, 0x01, 0x0a, 0x08, 0x48, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x2c,
|
||||
0x0a, 0x09, 0x63, 0x70, 0x75, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28,
|
||||
0x02, 0x42, 0x0f, 0xfa, 0x42, 0x0c, 0x0a, 0x0a, 0x1d, 0x00, 0x00, 0x80, 0x3f, 0x2d, 0x00, 0x00,
|
||||
0x00, 0x00, 0x52, 0x08, 0x63, 0x70, 0x75, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x2c, 0x0a, 0x09,
|
||||
0x6d, 0x65, 0x6d, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x42,
|
||||
0x0f, 0xfa, 0x42, 0x0c, 0x0a, 0x0a, 0x1d, 0x00, 0x00, 0x80, 0x3f, 0x2d, 0x00, 0x00, 0x00, 0x00,
|
||||
0x52, 0x08, 0x6d, 0x65, 0x6d, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x2e, 0x0a, 0x0a, 0x64, 0x69,
|
||||
0x73, 0x6b, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x42, 0x0f,
|
||||
0xfa, 0x42, 0x0c, 0x0a, 0x0a, 0x1d, 0x00, 0x00, 0x80, 0x3f, 0x2d, 0x00, 0x00, 0x00, 0x00, 0x52,
|
||||
0x09, 0x64, 0x69, 0x73, 0x6b, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x22, 0xbd, 0x01, 0x0a, 0x10, 0x50,
|
||||
0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
|
||||
0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
|
||||
0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49,
|
||||
0x64, 0x12, 0x20, 0x0a, 0x07, 0x73, 0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01,
|
||||
0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x73, 0x72, 0x63,
|
||||
0x50, 0x69, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03,
|
||||
0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x64,
|
||||
0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6e,
|
||||
0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x2a, 0x02, 0x28,
|
||||
0x00, 0x52, 0x08, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4e, 0x75, 0x6d, 0x12, 0x1d, 0x0a, 0x05, 0x6c,
|
||||
0x69, 0x6d, 0x69, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x2a,
|
||||
0x02, 0x28, 0x00, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x22, 0xb3, 0x02, 0x0a, 0x09, 0x50,
|
||||
0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1b, 0x0a, 0x09, 0x70, 0x69, 0x65, 0x63,
|
||||
0x65, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x69, 0x65,
|
||||
0x63, 0x65, 0x4e, 0x75, 0x6d, 0x12, 0x28, 0x0a, 0x0b, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73,
|
||||
0x74, 0x61, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x32,
|
||||
0x02, 0x28, 0x00, 0x52, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12,
|
||||
0x26, 0x0a, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20,
|
||||
0x01, 0x28, 0x0d, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x2a, 0x02, 0x28, 0x00, 0x52, 0x09, 0x72, 0x61,
|
||||
0x6e, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x58, 0x0a, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65,
|
||||
0x5f, 0x6d, 0x64, 0x35, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x42, 0x3b, 0xfa, 0x42, 0x38, 0x72,
|
||||
0x36, 0x32, 0x31, 0x28, 0x5b, 0x61, 0x2d, 0x66, 0x5c, 0x64, 0x5d, 0x7b, 0x33, 0x32, 0x7d, 0x7c,
|
||||
0x5b, 0x41, 0x2d, 0x46, 0x5c, 0x64, 0x5d, 0x7b, 0x33, 0x32, 0x7d, 0x7c, 0x5b, 0x61, 0x2d, 0x66,
|
||||
0x5c, 0x64, 0x5d, 0x7b, 0x31, 0x36, 0x7d, 0x7c, 0x5b, 0x41, 0x2d, 0x46, 0x5c, 0x64, 0x5d, 0x7b,
|
||||
0x31, 0x36, 0x7d, 0x29, 0xd0, 0x01, 0x01, 0x52, 0x08, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4d, 0x64,
|
||||
0x35, 0x12, 0x2a, 0x0a, 0x0c, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65,
|
||||
0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x32, 0x02, 0x28, 0x00,
|
||||
0x52, 0x0b, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x31, 0x0a,
|
||||
0x0b, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x73, 0x74, 0x79, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01,
|
||||
0x28, 0x0e, 0x32, 0x10, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x53,
|
||||
0x74, 0x79, 0x6c, 0x65, 0x52, 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x53, 0x74, 0x79, 0x6c, 0x65,
|
||||
0x22, 0xfa, 0x01, 0x0a, 0x0b, 0x50, 0x69, 0x65, 0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74,
|
||||
0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x73, 0x74,
|
||||
0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50,
|
||||
0x69, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x04,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x64, 0x73, 0x74, 0x41, 0x64, 0x64, 0x72, 0x12, 0x30, 0x0a,
|
||||
0x0b, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x05, 0x20, 0x03,
|
||||
0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49,
|
||||
0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12,
|
||||
0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x18, 0x06,
|
||||
0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65, 0x63, 0x65,
|
||||
0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x65, 0x6e, 0x67,
|
||||
0x74, 0x68, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e,
|
||||
0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x70, 0x69, 0x65, 0x63, 0x65,
|
||||
0x5f, 0x6d, 0x64, 0x35, 0x5f, 0x73, 0x69, 0x67, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x0c, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4d, 0x64, 0x35, 0x53, 0x69, 0x67, 0x6e, 0x2a, 0xeb, 0x04,
|
||||
0x0a, 0x04, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x58, 0x5f, 0x55, 0x4e, 0x53, 0x50,
|
||||
0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x07, 0x53, 0x75, 0x63,
|
||||
0x63, 0x65, 0x73, 0x73, 0x10, 0xc8, 0x01, 0x12, 0x16, 0x0a, 0x11, 0x53, 0x65, 0x72, 0x76, 0x65,
|
||||
0x72, 0x55, 0x6e, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x10, 0xf4, 0x03, 0x12,
|
||||
0x13, 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4c, 0x61, 0x63, 0x6b, 0x65,
|
||||
0x64, 0x10, 0xe8, 0x07, 0x12, 0x0f, 0x0a, 0x0a, 0x42, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65,
|
||||
0x73, 0x74, 0x10, 0xf8, 0x0a, 0x12, 0x15, 0x0a, 0x10, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73,
|
||||
0x6b, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0xfc, 0x0a, 0x12, 0x11, 0x0a, 0x0c,
|
||||
0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xdc, 0x0b, 0x12,
|
||||
0x13, 0x0a, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x4f, 0x75,
|
||||
0x74, 0x10, 0xe0, 0x0b, 0x12, 0x10, 0x0a, 0x0b, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x45, 0x72,
|
||||
0x72, 0x6f, 0x72, 0x10, 0xa0, 0x1f, 0x12, 0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74,
|
||||
0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x46, 0x61, 0x69, 0x6c,
|
||||
0x10, 0xa1, 0x1f, 0x12, 0x1a, 0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x68,
|
||||
0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x10, 0xa2, 0x1f, 0x12,
|
||||
0x1a, 0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74,
|
||||
0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x10, 0xa3, 0x1f, 0x12, 0x19, 0x0a, 0x14, 0x43,
|
||||
0x6c, 0x69, 0x65, 0x6e, 0x74, 0x57, 0x61, 0x69, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65,
|
||||
0x61, 0x64, 0x79, 0x10, 0xa4, 0x1f, 0x12, 0x1c, 0x0a, 0x17, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74,
|
||||
0x50, 0x69, 0x65, 0x63, 0x65, 0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x46, 0x61, 0x69,
|
||||
0x6c, 0x10, 0xa5, 0x1f, 0x12, 0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x52, 0x65,
|
||||
0x71, 0x75, 0x65, 0x73, 0x74, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa6,
|
||||
0x1f, 0x12, 0x0f, 0x0a, 0x0a, 0x53, 0x63, 0x68, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10,
|
||||
0x88, 0x27, 0x12, 0x18, 0x0a, 0x13, 0x53, 0x63, 0x68, 0x65, 0x64, 0x4e, 0x65, 0x65, 0x64, 0x42,
|
||||
0x61, 0x63, 0x6b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x10, 0x89, 0x27, 0x12, 0x12, 0x0a, 0x0d,
|
||||
0x53, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72, 0x47, 0x6f, 0x6e, 0x65, 0x10, 0x8a, 0x27,
|
||||
0x12, 0x16, 0x0a, 0x11, 0x53, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x6f, 0x74,
|
||||
0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x8c, 0x27, 0x12, 0x23, 0x0a, 0x1e, 0x53, 0x63, 0x68, 0x65,
|
||||
0x64, 0x50, 0x65, 0x65, 0x72, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74,
|
||||
0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0x8d, 0x27, 0x12, 0x19, 0x0a,
|
||||
0x14, 0x53, 0x63, 0x68, 0x65, 0x64, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73,
|
||||
0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x8e, 0x27, 0x12, 0x0d, 0x0a, 0x08, 0x43, 0x44, 0x4e, 0x45,
|
||||
0x72, 0x72, 0x6f, 0x72, 0x10, 0xf0, 0x2e, 0x12, 0x18, 0x0a, 0x13, 0x43, 0x44, 0x4e, 0x54, 0x61,
|
||||
0x73, 0x6b, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x79, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xf1,
|
||||
0x2e, 0x12, 0x18, 0x0a, 0x13, 0x43, 0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b, 0x44, 0x6f, 0x77, 0x6e,
|
||||
0x6c, 0x6f, 0x61, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xf2, 0x2e, 0x12, 0x14, 0x0a, 0x0f, 0x43,
|
||||
0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x84,
|
||||
0x32, 0x12, 0x18, 0x0a, 0x13, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x52, 0x65, 0x73, 0x6f,
|
||||
0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x10, 0xd9, 0x36, 0x2a, 0x17, 0x0a, 0x0a, 0x50,
|
||||
0x69, 0x65, 0x63, 0x65, 0x53, 0x74, 0x79, 0x6c, 0x65, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x4c, 0x41,
|
||||
0x49, 0x4e, 0x10, 0x00, 0x2a, 0x2c, 0x0a, 0x09, 0x53, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70,
|
||||
0x65, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a,
|
||||
0x05, 0x53, 0x4d, 0x41, 0x4c, 0x4c, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x54, 0x49, 0x4e, 0x59,
|
||||
0x10, 0x02, 0x42, 0x22, 0x5a, 0x20, 0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61,
|
||||
0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70,
|
||||
0x63, 0x2f, 0x62, 0x61, 0x73, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x22, 0x95, 0x02, 0x0a, 0x0b, 0x50, 0x69, 0x65, 0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74,
|
||||
0x12, 0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28,
|
||||
0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b,
|
||||
0x49, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20,
|
||||
0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x64, 0x73,
|
||||
0x74, 0x50, 0x69, 0x64, 0x12, 0x22, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x61, 0x64, 0x64, 0x72,
|
||||
0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52,
|
||||
0x07, 0x64, 0x73, 0x74, 0x41, 0x64, 0x64, 0x72, 0x12, 0x30, 0x0a, 0x0b, 0x70, 0x69, 0x65, 0x63,
|
||||
0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0f, 0x2e,
|
||||
0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a,
|
||||
0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f,
|
||||
0x74, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52,
|
||||
0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63,
|
||||
0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x07, 0x20,
|
||||
0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x6e, 0x67,
|
||||
0x74, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x6d, 0x64, 0x35, 0x5f,
|
||||
0x73, 0x69, 0x67, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x69, 0x65, 0x63,
|
||||
0x65, 0x4d, 0x64, 0x35, 0x53, 0x69, 0x67, 0x6e, 0x2a, 0xeb, 0x04, 0x0a, 0x04, 0x43, 0x6f, 0x64,
|
||||
0x65, 0x12, 0x11, 0x0a, 0x0d, 0x58, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49,
|
||||
0x45, 0x44, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x07, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x10,
|
||||
0xc8, 0x01, 0x12, 0x16, 0x0a, 0x11, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x76,
|
||||
0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x10, 0xf4, 0x03, 0x12, 0x13, 0x0a, 0x0e, 0x52, 0x65,
|
||||
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4c, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0xe8, 0x07, 0x12,
|
||||
0x0f, 0x0a, 0x0a, 0x42, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x10, 0xf8, 0x0a,
|
||||
0x12, 0x15, 0x0a, 0x10, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x4e, 0x6f, 0x74, 0x46,
|
||||
0x6f, 0x75, 0x6e, 0x64, 0x10, 0xfc, 0x0a, 0x12, 0x11, 0x0a, 0x0c, 0x55, 0x6e, 0x6b, 0x6e, 0x6f,
|
||||
0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xdc, 0x0b, 0x12, 0x13, 0x0a, 0x0e, 0x52, 0x65,
|
||||
0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x4f, 0x75, 0x74, 0x10, 0xe0, 0x0b, 0x12,
|
||||
0x10, 0x0a, 0x0b, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xa0,
|
||||
0x1f, 0x12, 0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65,
|
||||
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa1, 0x1f, 0x12, 0x1a,
|
||||
0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65,
|
||||
0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x10, 0xa2, 0x1f, 0x12, 0x1a, 0x0a, 0x15, 0x43, 0x6c,
|
||||
0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65,
|
||||
0x6c, 0x65, 0x64, 0x10, 0xa3, 0x1f, 0x12, 0x19, 0x0a, 0x14, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74,
|
||||
0x57, 0x61, 0x69, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x61, 0x64, 0x79, 0x10, 0xa4,
|
||||
0x1f, 0x12, 0x1c, 0x0a, 0x17, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65,
|
||||
0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa5, 0x1f, 0x12,
|
||||
0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x4c, 0x69, 0x6d, 0x69, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa6, 0x1f, 0x12, 0x0f, 0x0a, 0x0a,
|
||||
0x53, 0x63, 0x68, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x88, 0x27, 0x12, 0x18, 0x0a,
|
||||
0x13, 0x53, 0x63, 0x68, 0x65, 0x64, 0x4e, 0x65, 0x65, 0x64, 0x42, 0x61, 0x63, 0x6b, 0x53, 0x6f,
|
||||
0x75, 0x72, 0x63, 0x65, 0x10, 0x89, 0x27, 0x12, 0x12, 0x0a, 0x0d, 0x53, 0x63, 0x68, 0x65, 0x64,
|
||||
0x50, 0x65, 0x65, 0x72, 0x47, 0x6f, 0x6e, 0x65, 0x10, 0x8a, 0x27, 0x12, 0x16, 0x0a, 0x11, 0x53,
|
||||
0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64,
|
||||
0x10, 0x8c, 0x27, 0x12, 0x23, 0x0a, 0x1e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72,
|
||||
0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x65, 0x70, 0x6f, 0x72,
|
||||
0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0x8d, 0x27, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x63, 0x68, 0x65,
|
||||
0x64, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x45, 0x72, 0x72, 0x6f, 0x72,
|
||||
0x10, 0x8e, 0x27, 0x12, 0x0d, 0x0a, 0x08, 0x43, 0x44, 0x4e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10,
|
||||
0xf0, 0x2e, 0x12, 0x18, 0x0a, 0x13, 0x43, 0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x67,
|
||||
0x69, 0x73, 0x74, 0x72, 0x79, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xf1, 0x2e, 0x12, 0x18, 0x0a, 0x13,
|
||||
0x43, 0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b, 0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x46,
|
||||
0x61, 0x69, 0x6c, 0x10, 0xf2, 0x2e, 0x12, 0x14, 0x0a, 0x0f, 0x43, 0x44, 0x4e, 0x54, 0x61, 0x73,
|
||||
0x6b, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x84, 0x32, 0x12, 0x18, 0x0a, 0x13,
|
||||
0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54,
|
||||
0x79, 0x70, 0x65, 0x10, 0xd9, 0x36, 0x2a, 0x17, 0x0a, 0x0a, 0x50, 0x69, 0x65, 0x63, 0x65, 0x53,
|
||||
0x74, 0x79, 0x6c, 0x65, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x4c, 0x41, 0x49, 0x4e, 0x10, 0x00, 0x2a,
|
||||
0x2c, 0x0a, 0x09, 0x53, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06,
|
||||
0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x53, 0x4d, 0x41, 0x4c,
|
||||
0x4c, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x54, 0x49, 0x4e, 0x59, 0x10, 0x02, 0x42, 0x22, 0x5a,
|
||||
0x20, 0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c,
|
||||
0x79, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x62, 0x61, 0x73,
|
||||
0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
|
|
|
|||
|
|
@ -55,7 +55,16 @@ func (m *GrpcDfError) validate(all bool) error {
|
|||
|
||||
var errors []error
|
||||
|
||||
// no validation rules for Code
|
||||
if _, ok := Code_name[int32(m.GetCode())]; !ok {
|
||||
err := GrpcDfErrorValidationError{
|
||||
field: "Code",
|
||||
reason: "value must be one of the defined enum values",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for Message
|
||||
|
||||
|
|
@ -156,11 +165,37 @@ func (m *UrlMeta) validate(all bool) error {
|
|||
|
||||
var errors []error
|
||||
|
||||
// no validation rules for Digest
|
||||
if m.GetDigest() != "" {
|
||||
|
||||
if !_UrlMeta_Digest_Pattern.MatchString(m.GetDigest()) {
|
||||
err := UrlMetaValidationError{
|
||||
field: "Digest",
|
||||
reason: "value does not match regex pattern \"^(md5)|(sha256):[A-Fa-f0-9]+\"",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// no validation rules for Tag
|
||||
|
||||
// no validation rules for Range
|
||||
if m.GetRange() != "" {
|
||||
|
||||
if !_UrlMeta_Range_Pattern.MatchString(m.GetRange()) {
|
||||
err := UrlMetaValidationError{
|
||||
field: "Range",
|
||||
reason: "value does not match regex pattern \"^[0-9]+\\\\-^[0-9]+\"",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// no validation rules for Filter
|
||||
|
||||
|
|
@ -242,6 +277,10 @@ var _ interface {
|
|||
ErrorName() string
|
||||
} = UrlMetaValidationError{}
|
||||
|
||||
var _UrlMeta_Digest_Pattern = regexp.MustCompile("^(md5)|(sha256):[A-Fa-f0-9]+")
|
||||
|
||||
var _UrlMeta_Range_Pattern = regexp.MustCompile("^[0-9]+\\-^[0-9]+")
|
||||
|
||||
// Validate checks the field values on HostLoad with the rules defined in the
|
||||
// proto definition for this message. If any rules are violated, the first
|
||||
// error encountered is returned, or nil if there are no violations.
|
||||
|
|
@ -264,11 +303,38 @@ func (m *HostLoad) validate(all bool) error {
|
|||
|
||||
var errors []error
|
||||
|
||||
// no validation rules for CpuRatio
|
||||
if val := m.GetCpuRatio(); val < 0 || val > 1 {
|
||||
err := HostLoadValidationError{
|
||||
field: "CpuRatio",
|
||||
reason: "value must be inside range [0, 1]",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for MemRatio
|
||||
if val := m.GetMemRatio(); val < 0 || val > 1 {
|
||||
err := HostLoadValidationError{
|
||||
field: "MemRatio",
|
||||
reason: "value must be inside range [0, 1]",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for DiskRatio
|
||||
if val := m.GetDiskRatio(); val < 0 || val > 1 {
|
||||
err := HostLoadValidationError{
|
||||
field: "DiskRatio",
|
||||
reason: "value must be inside range [0, 1]",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
if len(errors) > 0 {
|
||||
return HostLoadMultiError(errors)
|
||||
|
|
@ -368,15 +434,60 @@ func (m *PieceTaskRequest) validate(all bool) error {
|
|||
|
||||
var errors []error
|
||||
|
||||
// no validation rules for TaskId
|
||||
if utf8.RuneCountInString(m.GetTaskId()) < 1 {
|
||||
err := PieceTaskRequestValidationError{
|
||||
field: "TaskId",
|
||||
reason: "value length must be at least 1 runes",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for SrcPid
|
||||
if utf8.RuneCountInString(m.GetSrcPid()) < 1 {
|
||||
err := PieceTaskRequestValidationError{
|
||||
field: "SrcPid",
|
||||
reason: "value length must be at least 1 runes",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for DstPid
|
||||
if utf8.RuneCountInString(m.GetDstPid()) < 1 {
|
||||
err := PieceTaskRequestValidationError{
|
||||
field: "DstPid",
|
||||
reason: "value length must be at least 1 runes",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for StartNum
|
||||
if m.GetStartNum() < 0 {
|
||||
err := PieceTaskRequestValidationError{
|
||||
field: "StartNum",
|
||||
reason: "value must be greater than or equal to 0",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for Limit
|
||||
if m.GetLimit() < 0 {
|
||||
err := PieceTaskRequestValidationError{
|
||||
field: "Limit",
|
||||
reason: "value must be greater than or equal to 0",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
if len(errors) > 0 {
|
||||
return PieceTaskRequestMultiError(errors)
|
||||
|
|
@ -479,13 +590,53 @@ func (m *PieceInfo) validate(all bool) error {
|
|||
|
||||
// no validation rules for PieceNum
|
||||
|
||||
// no validation rules for RangeStart
|
||||
if m.GetRangeStart() < 0 {
|
||||
err := PieceInfoValidationError{
|
||||
field: "RangeStart",
|
||||
reason: "value must be greater than or equal to 0",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for RangeSize
|
||||
if m.GetRangeSize() < 0 {
|
||||
err := PieceInfoValidationError{
|
||||
field: "RangeSize",
|
||||
reason: "value must be greater than or equal to 0",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for PieceMd5
|
||||
if m.GetPieceMd5() != "" {
|
||||
|
||||
// no validation rules for PieceOffset
|
||||
if !_PieceInfo_PieceMd5_Pattern.MatchString(m.GetPieceMd5()) {
|
||||
err := PieceInfoValidationError{
|
||||
field: "PieceMd5",
|
||||
reason: "value does not match regex pattern \"([a-f\\\\d]{32}|[A-F\\\\d]{32}|[a-f\\\\d]{16}|[A-F\\\\d]{16})\"",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
if m.GetPieceOffset() < 0 {
|
||||
err := PieceInfoValidationError{
|
||||
field: "PieceOffset",
|
||||
reason: "value must be greater than or equal to 0",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for PieceStyle
|
||||
|
||||
|
|
@ -565,6 +716,8 @@ var _ interface {
|
|||
ErrorName() string
|
||||
} = PieceInfoValidationError{}
|
||||
|
||||
var _PieceInfo_PieceMd5_Pattern = regexp.MustCompile("([a-f\\d]{32}|[A-F\\d]{32}|[a-f\\d]{16}|[A-F\\d]{16})")
|
||||
|
||||
// Validate checks the field values on PiecePacket with the rules defined in
|
||||
// the proto definition for this message. If any rules are violated, the first
|
||||
// error encountered is returned, or nil if there are no violations.
|
||||
|
|
@ -587,11 +740,38 @@ func (m *PiecePacket) validate(all bool) error {
|
|||
|
||||
var errors []error
|
||||
|
||||
// no validation rules for TaskId
|
||||
if utf8.RuneCountInString(m.GetTaskId()) < 1 {
|
||||
err := PiecePacketValidationError{
|
||||
field: "TaskId",
|
||||
reason: "value length must be at least 1 runes",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for DstPid
|
||||
if utf8.RuneCountInString(m.GetDstPid()) < 1 {
|
||||
err := PiecePacketValidationError{
|
||||
field: "DstPid",
|
||||
reason: "value length must be at least 1 runes",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for DstAddr
|
||||
if utf8.RuneCountInString(m.GetDstAddr()) < 1 {
|
||||
err := PiecePacketValidationError{
|
||||
field: "DstAddr",
|
||||
reason: "value length must be at least 1 runes",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
for idx, item := range m.GetPieceInfos() {
|
||||
_, _ = idx, item
|
||||
|
|
|
|||
|
|
@ -18,6 +18,8 @@ syntax = "proto3";
|
|||
|
||||
package base;
|
||||
|
||||
import "validate/validate.proto";
|
||||
|
||||
option go_package = "d7y.io/dragonfly/v2/pkg/rpc/base";
|
||||
|
||||
enum Code{
|
||||
|
|
@ -75,18 +77,18 @@ enum SizeScope{
|
|||
TINY = 2;
|
||||
}
|
||||
message GrpcDfError {
|
||||
Code code = 1;
|
||||
Code code = 1 [(validate.rules).enum.defined_only = true];
|
||||
string message = 2;
|
||||
}
|
||||
|
||||
// UrlMeta describes url meta info.
|
||||
message UrlMeta{
|
||||
// digest checks integrity of url content, for example md5:xxx or sha256:yyy
|
||||
string digest = 1;
|
||||
string digest = 1 [(validate.rules).string = {pattern: "^(md5)|(sha256):[A-Fa-f0-9]+", ignore_empty:true}];
|
||||
// url tag identifies different task for same url, conflict with digest
|
||||
string tag = 2;
|
||||
// content range for url
|
||||
string range = 3;
|
||||
string range = 3 [(validate.rules).string = {pattern: "^[0-9]+\\-^[0-9]+", ignore_empty:true}];
|
||||
// filter url used to generate task id
|
||||
string filter = 4;
|
||||
// other url header infos
|
||||
|
|
@ -95,40 +97,42 @@ message UrlMeta{
|
|||
|
||||
message HostLoad{
|
||||
// cpu usage
|
||||
float cpu_ratio = 1;
|
||||
float cpu_ratio = 1 [(validate.rules).float = {gte: 0, lte: 1}];
|
||||
// memory usage
|
||||
float mem_ratio = 2;
|
||||
float mem_ratio = 2 [(validate.rules).float = {gte: 0, lte: 1}];
|
||||
// disk space usage
|
||||
float disk_ratio = 3;
|
||||
float disk_ratio = 3 [(validate.rules).float = {gte: 0, lte: 1}];
|
||||
}
|
||||
|
||||
message PieceTaskRequest{
|
||||
string task_id = 1;
|
||||
string src_pid = 2;
|
||||
string dst_pid = 3;
|
||||
string task_id = 1 [(validate.rules).string.min_len = 1];
|
||||
string src_pid = 2 [(validate.rules).string.min_len = 1];
|
||||
string dst_pid = 3 [(validate.rules).string.min_len = 1];
|
||||
// piece number
|
||||
int32 start_num = 4;
|
||||
// expected piece count
|
||||
int32 limit = 5;
|
||||
uint32 start_num = 4 [(validate.rules).uint32.gte = 0];
|
||||
// expected piece count, limit = 0 represent request pieces as many shards as possible
|
||||
uint32 limit = 5 [(validate.rules).uint32.gte = 0];
|
||||
}
|
||||
|
||||
message PieceInfo{
|
||||
// piece_num < 0 represent start report piece flag
|
||||
int32 piece_num = 1;
|
||||
uint64 range_start = 2;
|
||||
int32 range_size = 3;
|
||||
string piece_md5 = 4;
|
||||
uint64 piece_offset = 5;
|
||||
uint64 range_start = 2 [(validate.rules).uint64.gte = 0];
|
||||
uint32 range_size = 3 [(validate.rules).uint32.gte = 0];
|
||||
string piece_md5 = 4 [(validate.rules).string = {pattern:"([a-f\\d]{32}|[A-F\\d]{32}|[a-f\\d]{16}|[A-F\\d]{16})", ignore_empty:true}];
|
||||
uint64 piece_offset = 5 [(validate.rules).uint64.gte = 0];
|
||||
base.PieceStyle piece_style = 6;
|
||||
}
|
||||
|
||||
message PiecePacket{
|
||||
string task_id = 2;
|
||||
string dst_pid = 3;
|
||||
string task_id = 2 [(validate.rules).string.min_len = 1];
|
||||
string dst_pid = 3 [(validate.rules).string.min_len = 1];
|
||||
// ip:port
|
||||
string dst_addr = 4;
|
||||
string dst_addr = 4 [(validate.rules).string.min_len = 1];
|
||||
repeated PieceInfo piece_infos = 5;
|
||||
// total piece count for url
|
||||
// total piece count for url, total_piece represent total piece is unknown
|
||||
int32 total_piece = 6;
|
||||
// content_length < 0 represent content length is unknown
|
||||
int64 content_length = 7;
|
||||
// sha256 code of all piece md5
|
||||
string piece_md5_sign = 8;
|
||||
|
|
|
|||
|
|
@ -23,6 +23,7 @@ package cdnsystem
|
|||
|
||||
import (
|
||||
base "d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
_ "github.com/envoyproxy/protoc-gen-validate/validate"
|
||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||
reflect "reflect"
|
||||
|
|
@ -113,9 +114,9 @@ type PieceSeed struct {
|
|||
PieceInfo *base.PieceInfo `protobuf:"bytes,4,opt,name=piece_info,json=pieceInfo,proto3" json:"piece_info,omitempty"`
|
||||
// whether or not all seeds are downloaded
|
||||
Done bool `protobuf:"varint,5,opt,name=done,proto3" json:"done,omitempty"`
|
||||
// content total length for the url
|
||||
// content total length for the url, content_length < 0 represent content length is unknown
|
||||
ContentLength int64 `protobuf:"varint,6,opt,name=content_length,json=contentLength,proto3" json:"content_length,omitempty"`
|
||||
// total piece count
|
||||
// total piece count, -1 represents task is downloading or failed
|
||||
TotalPieceCount int32 `protobuf:"varint,7,opt,name=total_piece_count,json=totalPieceCount,proto3" json:"total_piece_count,omitempty"`
|
||||
}
|
||||
|
||||
|
|
@ -200,38 +201,42 @@ var file_pkg_rpc_cdnsystem_cdnsystem_proto_rawDesc = []byte{
|
|||
0x74, 0x65, 0x6d, 0x2f, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x2e, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x12, 0x09, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x1a, 0x17,
|
||||
0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x62, 0x61, 0x73, 0x65, 0x2f, 0x62, 0x61, 0x73,
|
||||
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x62, 0x0a, 0x0b, 0x53, 0x65, 0x65, 0x64, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69,
|
||||
0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12,
|
||||
0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72,
|
||||
0x6c, 0x12, 0x28, 0x0a, 0x08, 0x75, 0x72, 0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20,
|
||||
0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65,
|
||||
0x74, 0x61, 0x52, 0x07, 0x75, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x22, 0xd8, 0x01, 0x0a, 0x09,
|
||||
0x50, 0x69, 0x65, 0x63, 0x65, 0x53, 0x65, 0x65, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65,
|
||||
0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72,
|
||||
0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18,
|
||||
0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x55, 0x75, 0x69, 0x64, 0x12,
|
||||
0x2e, 0x0a, 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20,
|
||||
0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65,
|
||||
0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12,
|
||||
0x12, 0x0a, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x64,
|
||||
0x6f, 0x6e, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c,
|
||||
0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e,
|
||||
0x74, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x2a, 0x0a, 0x11, 0x74, 0x6f,
|
||||
0x74, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18,
|
||||
0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65, 0x63,
|
||||
0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x32, 0x83, 0x01, 0x0a, 0x06, 0x53, 0x65, 0x65, 0x64, 0x65,
|
||||
0x72, 0x12, 0x3d, 0x0a, 0x0b, 0x4f, 0x62, 0x74, 0x61, 0x69, 0x6e, 0x53, 0x65, 0x65, 0x64, 0x73,
|
||||
0x12, 0x16, 0x2e, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x2e, 0x53, 0x65, 0x65,
|
||||
0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x63, 0x64, 0x6e, 0x73, 0x79,
|
||||
0x73, 0x74, 0x65, 0x6d, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x53, 0x65, 0x65, 0x64, 0x30, 0x01,
|
||||
0x12, 0x3a, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b,
|
||||
0x73, 0x12, 0x16, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61,
|
||||
0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x62, 0x61, 0x73, 0x65,
|
||||
0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x42, 0x27, 0x5a, 0x25,
|
||||
0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79,
|
||||
0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x63, 0x64, 0x6e, 0x73,
|
||||
0x79, 0x73, 0x74, 0x65, 0x6d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x17, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74,
|
||||
0x65, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x22, 0x75, 0x0a, 0x0b, 0x53, 0x65, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
|
||||
0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
|
||||
0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49,
|
||||
0x64, 0x12, 0x1a, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x08,
|
||||
0xfa, 0x42, 0x05, 0x72, 0x03, 0x88, 0x01, 0x01, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x28, 0x0a,
|
||||
0x08, 0x75, 0x72, 0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32,
|
||||
0x0d, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x07,
|
||||
0x75, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x22, 0xea, 0x01, 0x0a, 0x09, 0x50, 0x69, 0x65, 0x63,
|
||||
0x65, 0x53, 0x65, 0x65, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52,
|
||||
0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f,
|
||||
0x75, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72,
|
||||
0x02, 0x10, 0x01, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x55, 0x75, 0x69, 0x64, 0x12, 0x2e, 0x0a,
|
||||
0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28,
|
||||
0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e,
|
||||
0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x12, 0x0a,
|
||||
0x04, 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x64, 0x6f, 0x6e,
|
||||
0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x65, 0x6e,
|
||||
0x67, 0x74, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65,
|
||||
0x6e, 0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x2a, 0x0a, 0x11, 0x74, 0x6f, 0x74, 0x61,
|
||||
0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x07, 0x20,
|
||||
0x01, 0x28, 0x05, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65, 0x63, 0x65, 0x43,
|
||||
0x6f, 0x75, 0x6e, 0x74, 0x32, 0x83, 0x01, 0x0a, 0x06, 0x53, 0x65, 0x65, 0x64, 0x65, 0x72, 0x12,
|
||||
0x3d, 0x0a, 0x0b, 0x4f, 0x62, 0x74, 0x61, 0x69, 0x6e, 0x53, 0x65, 0x65, 0x64, 0x73, 0x12, 0x16,
|
||||
0x2e, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x2e, 0x53, 0x65, 0x65, 0x64, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74,
|
||||
0x65, 0x6d, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x53, 0x65, 0x65, 0x64, 0x30, 0x01, 0x12, 0x3a,
|
||||
0x0a, 0x0d, 0x47, 0x65, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x73, 0x12,
|
||||
0x16, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b,
|
||||
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50,
|
||||
0x69, 0x65, 0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x42, 0x27, 0x5a, 0x25, 0x64, 0x37,
|
||||
0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f, 0x76,
|
||||
0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73,
|
||||
0x74, 0x65, 0x6d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
|
|
|
|||
|
|
@ -34,18 +34,79 @@ var (
|
|||
)
|
||||
|
||||
// Validate checks the field values on SeedRequest with the rules defined in
|
||||
// the proto definition for this message. If any rules are violated, an error
|
||||
// is returned.
|
||||
// the proto definition for this message. If any rules are violated, the first
|
||||
// error encountered is returned, or nil if there are no violations.
|
||||
func (m *SeedRequest) Validate() error {
|
||||
return m.validate(false)
|
||||
}
|
||||
|
||||
// ValidateAll checks the field values on SeedRequest with the rules defined in
|
||||
// the proto definition for this message. If any rules are violated, the
|
||||
// result is a list of violation errors wrapped in SeedRequestMultiError, or
|
||||
// nil if none found.
|
||||
func (m *SeedRequest) ValidateAll() error {
|
||||
return m.validate(true)
|
||||
}
|
||||
|
||||
func (m *SeedRequest) validate(all bool) error {
|
||||
if m == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// no validation rules for TaskId
|
||||
var errors []error
|
||||
|
||||
// no validation rules for Url
|
||||
if utf8.RuneCountInString(m.GetTaskId()) < 1 {
|
||||
err := SeedRequestValidationError{
|
||||
field: "TaskId",
|
||||
reason: "value length must be at least 1 runes",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok {
|
||||
if uri, err := url.Parse(m.GetUrl()); err != nil {
|
||||
err = SeedRequestValidationError{
|
||||
field: "Url",
|
||||
reason: "value must be a valid URI",
|
||||
cause: err,
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
} else if !uri.IsAbs() {
|
||||
err := SeedRequestValidationError{
|
||||
field: "Url",
|
||||
reason: "value must be absolute",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
if all {
|
||||
switch v := interface{}(m.GetUrlMeta()).(type) {
|
||||
case interface{ ValidateAll() error }:
|
||||
if err := v.ValidateAll(); err != nil {
|
||||
errors = append(errors, SeedRequestValidationError{
|
||||
field: "UrlMeta",
|
||||
reason: "embedded message failed validation",
|
||||
cause: err,
|
||||
})
|
||||
}
|
||||
case interface{ Validate() error }:
|
||||
if err := v.Validate(); err != nil {
|
||||
errors = append(errors, SeedRequestValidationError{
|
||||
field: "UrlMeta",
|
||||
reason: "embedded message failed validation",
|
||||
cause: err,
|
||||
})
|
||||
}
|
||||
}
|
||||
} else if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok {
|
||||
if err := v.Validate(); err != nil {
|
||||
return SeedRequestValidationError{
|
||||
field: "UrlMeta",
|
||||
|
|
@ -55,9 +116,28 @@ func (m *SeedRequest) Validate() error {
|
|||
}
|
||||
}
|
||||
|
||||
if len(errors) > 0 {
|
||||
return SeedRequestMultiError(errors)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// SeedRequestMultiError is an error wrapping multiple validation errors
|
||||
// returned by SeedRequest.ValidateAll() if the designated constraints aren't met.
|
||||
type SeedRequestMultiError []error
|
||||
|
||||
// Error returns a concatenation of all the error messages it wraps.
|
||||
func (m SeedRequestMultiError) Error() string {
|
||||
var msgs []string
|
||||
for _, err := range m {
|
||||
msgs = append(msgs, err.Error())
|
||||
}
|
||||
return strings.Join(msgs, "; ")
|
||||
}
|
||||
|
||||
// AllErrors returns a list of validation violation errors.
|
||||
func (m SeedRequestMultiError) AllErrors() []error { return m }
|
||||
|
||||
// SeedRequestValidationError is the validation error returned by
|
||||
// SeedRequest.Validate if the designated constraints aren't met.
|
||||
type SeedRequestValidationError struct {
|
||||
|
|
@ -113,17 +193,69 @@ var _ interface {
|
|||
} = SeedRequestValidationError{}
|
||||
|
||||
// Validate checks the field values on PieceSeed with the rules defined in the
|
||||
// proto definition for this message. If any rules are violated, an error is returned.
|
||||
// proto definition for this message. If any rules are violated, the first
|
||||
// error encountered is returned, or nil if there are no violations.
|
||||
func (m *PieceSeed) Validate() error {
|
||||
return m.validate(false)
|
||||
}
|
||||
|
||||
// ValidateAll checks the field values on PieceSeed with the rules defined in
|
||||
// the proto definition for this message. If any rules are violated, the
|
||||
// result is a list of violation errors wrapped in PieceSeedMultiError, or nil
|
||||
// if none found.
|
||||
func (m *PieceSeed) ValidateAll() error {
|
||||
return m.validate(true)
|
||||
}
|
||||
|
||||
func (m *PieceSeed) validate(all bool) error {
|
||||
if m == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// no validation rules for PeerId
|
||||
var errors []error
|
||||
|
||||
// no validation rules for HostUuid
|
||||
if utf8.RuneCountInString(m.GetPeerId()) < 1 {
|
||||
err := PieceSeedValidationError{
|
||||
field: "PeerId",
|
||||
reason: "value length must be at least 1 runes",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
if v, ok := interface{}(m.GetPieceInfo()).(interface{ Validate() error }); ok {
|
||||
if utf8.RuneCountInString(m.GetHostUuid()) < 1 {
|
||||
err := PieceSeedValidationError{
|
||||
field: "HostUuid",
|
||||
reason: "value length must be at least 1 runes",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
if all {
|
||||
switch v := interface{}(m.GetPieceInfo()).(type) {
|
||||
case interface{ ValidateAll() error }:
|
||||
if err := v.ValidateAll(); err != nil {
|
||||
errors = append(errors, PieceSeedValidationError{
|
||||
field: "PieceInfo",
|
||||
reason: "embedded message failed validation",
|
||||
cause: err,
|
||||
})
|
||||
}
|
||||
case interface{ Validate() error }:
|
||||
if err := v.Validate(); err != nil {
|
||||
errors = append(errors, PieceSeedValidationError{
|
||||
field: "PieceInfo",
|
||||
reason: "embedded message failed validation",
|
||||
cause: err,
|
||||
})
|
||||
}
|
||||
}
|
||||
} else if v, ok := interface{}(m.GetPieceInfo()).(interface{ Validate() error }); ok {
|
||||
if err := v.Validate(); err != nil {
|
||||
return PieceSeedValidationError{
|
||||
field: "PieceInfo",
|
||||
|
|
@ -139,9 +271,28 @@ func (m *PieceSeed) Validate() error {
|
|||
|
||||
// no validation rules for TotalPieceCount
|
||||
|
||||
if len(errors) > 0 {
|
||||
return PieceSeedMultiError(errors)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// PieceSeedMultiError is an error wrapping multiple validation errors returned
|
||||
// by PieceSeed.ValidateAll() if the designated constraints aren't met.
|
||||
type PieceSeedMultiError []error
|
||||
|
||||
// Error returns a concatenation of all the error messages it wraps.
|
||||
func (m PieceSeedMultiError) Error() string {
|
||||
var msgs []string
|
||||
for _, err := range m {
|
||||
msgs = append(msgs, err.Error())
|
||||
}
|
||||
return strings.Join(msgs, "; ")
|
||||
}
|
||||
|
||||
// AllErrors returns a list of validation violation errors.
|
||||
func (m PieceSeedMultiError) AllErrors() []error { return m }
|
||||
|
||||
// PieceSeedValidationError is the validation error returned by
|
||||
// PieceSeed.Validate if the designated constraints aren't met.
|
||||
type PieceSeedValidationError struct {
|
||||
|
|
|
|||
|
|
@ -19,12 +19,13 @@ syntax = "proto3";
|
|||
package cdnsystem;
|
||||
|
||||
import "pkg/rpc/base/base.proto";
|
||||
import "validate/validate.proto";
|
||||
|
||||
option go_package = "d7y.io/dragonfly/v2/pkg/rpc/cdnsystem";
|
||||
|
||||
message SeedRequest{
|
||||
string task_id = 1;
|
||||
string url = 2;
|
||||
string task_id = 1 [(validate.rules).string.min_len = 1];
|
||||
string url = 2 [(validate.rules).string.uri = true];
|
||||
base.UrlMeta url_meta = 3;
|
||||
}
|
||||
|
||||
|
|
@ -32,16 +33,16 @@ message SeedRequest{
|
|||
// check piece md5, md5s sign and total content length
|
||||
message PieceSeed{
|
||||
// peer id for cdn node, need suffix with _CDN
|
||||
string peer_id = 2;
|
||||
string peer_id = 2 [(validate.rules).string.min_len = 1];
|
||||
// cdn node host uuid
|
||||
string host_uuid = 3;
|
||||
string host_uuid = 3 [(validate.rules).string.min_len = 1];
|
||||
base.PieceInfo piece_info = 4;
|
||||
|
||||
// whether or not all seeds are downloaded
|
||||
bool done = 5;
|
||||
// content total length for the url
|
||||
// content total length for the url, content_length < 0 represent content length is unknown
|
||||
int64 content_length = 6;
|
||||
// total piece count
|
||||
// total piece count, -1 represents task is downloading or failed
|
||||
int32 total_piece_count = 7;
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -12,7 +12,6 @@ import (
|
|||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the grpc package it is being compiled against.
|
||||
// Requires gRPC-Go v1.32.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion7
|
||||
|
||||
// SeederClient is the client API for Seeder service.
|
||||
|
|
@ -34,7 +33,7 @@ func NewSeederClient(cc grpc.ClientConnInterface) SeederClient {
|
|||
}
|
||||
|
||||
func (c *seederClient) ObtainSeeds(ctx context.Context, in *SeedRequest, opts ...grpc.CallOption) (Seeder_ObtainSeedsClient, error) {
|
||||
stream, err := c.cc.NewStream(ctx, &Seeder_ServiceDesc.Streams[0], "/cdnsystem.Seeder/ObtainSeeds", opts...)
|
||||
stream, err := c.cc.NewStream(ctx, &_Seeder_serviceDesc.Streams[0], "/cdnsystem.Seeder/ObtainSeeds", opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
@ -105,7 +104,7 @@ type UnsafeSeederServer interface {
|
|||
}
|
||||
|
||||
func RegisterSeederServer(s grpc.ServiceRegistrar, srv SeederServer) {
|
||||
s.RegisterService(&Seeder_ServiceDesc, srv)
|
||||
s.RegisterService(&_Seeder_serviceDesc, srv)
|
||||
}
|
||||
|
||||
func _Seeder_ObtainSeeds_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
|
|
@ -147,10 +146,7 @@ func _Seeder_GetPieceTasks_Handler(srv interface{}, ctx context.Context, dec fun
|
|||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
// Seeder_ServiceDesc is the grpc.ServiceDesc for Seeder service.
|
||||
// It's only intended for direct use with grpc.RegisterService,
|
||||
// and not to be introspected or modified (even as a copy)
|
||||
var Seeder_ServiceDesc = grpc.ServiceDesc{
|
||||
var _Seeder_serviceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "cdnsystem.Seeder",
|
||||
HandlerType: (*SeederServer)(nil),
|
||||
Methods: []grpc.MethodDesc{
|
||||
|
|
|
|||
|
|
@ -23,6 +23,7 @@ package dfdaemon
|
|||
|
||||
import (
|
||||
base "d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
_ "github.com/envoyproxy/protoc-gen-validate/validate"
|
||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||
emptypb "google.golang.org/protobuf/types/known/emptypb"
|
||||
|
|
@ -50,12 +51,12 @@ type DownRequest struct {
|
|||
// at the same time, dfdaemon workspace also makes soft link to the output
|
||||
Output string `protobuf:"bytes,3,opt,name=output,proto3" json:"output,omitempty"`
|
||||
// timeout duration
|
||||
Timeout int64 `protobuf:"varint,4,opt,name=timeout,proto3" json:"timeout,omitempty"`
|
||||
Timeout uint64 `protobuf:"varint,4,opt,name=timeout,proto3" json:"timeout,omitempty"`
|
||||
// rate limit in bytes per second
|
||||
Limit float64 `protobuf:"fixed64,5,opt,name=limit,proto3" json:"limit,omitempty"`
|
||||
DisableBackSource bool `protobuf:"varint,6,opt,name=disable_back_source,json=disableBackSource,proto3" json:"disable_back_source,omitempty"`
|
||||
UrlMeta *base.UrlMeta `protobuf:"bytes,7,opt,name=url_meta,json=urlMeta,proto3" json:"url_meta,omitempty"`
|
||||
// p2p/cdn/source
|
||||
// p2p/cdn/source, default is p2p
|
||||
Pattern string `protobuf:"bytes,8,opt,name=pattern,proto3" json:"pattern,omitempty"`
|
||||
// call system
|
||||
Callsystem string `protobuf:"bytes,9,opt,name=callsystem,proto3" json:"callsystem,omitempty"`
|
||||
|
|
@ -118,7 +119,7 @@ func (x *DownRequest) GetOutput() string {
|
|||
return ""
|
||||
}
|
||||
|
||||
func (x *DownRequest) GetTimeout() int64 {
|
||||
func (x *DownRequest) GetTimeout() uint64 {
|
||||
if x != nil {
|
||||
return x.Timeout
|
||||
}
|
||||
|
|
@ -254,49 +255,57 @@ var file_pkg_rpc_dfdaemon_dfdaemon_proto_rawDesc = []byte{
|
|||
0x2f, 0x72, 0x70, 0x63, 0x2f, 0x62, 0x61, 0x73, 0x65, 0x2f, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x22, 0xb3, 0x02, 0x0a, 0x0b, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
|
||||
0x74, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x04, 0x75, 0x75, 0x69, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75,
|
||||
0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12,
|
||||
0x18, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03,
|
||||
0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d,
|
||||
0x69, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x12,
|
||||
0x2e, 0x0a, 0x13, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x5f,
|
||||
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x64, 0x69,
|
||||
0x73, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12,
|
||||
0x28, 0x0a, 0x08, 0x75, 0x72, 0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28,
|
||||
0x0b, 0x32, 0x0d, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61,
|
||||
0x52, 0x07, 0x75, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x74,
|
||||
0x74, 0x65, 0x72, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x61, 0x74, 0x74,
|
||||
0x65, 0x72, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x61, 0x6c, 0x6c, 0x73, 0x79, 0x73, 0x74, 0x65,
|
||||
0x6d, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x61, 0x6c, 0x6c, 0x73, 0x79, 0x73,
|
||||
0x74, 0x65, 0x6d, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03,
|
||||
0x52, 0x03, 0x75, 0x69, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x67, 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01,
|
||||
0x28, 0x03, 0x52, 0x03, 0x67, 0x69, 0x64, 0x22, 0x7d, 0x0a, 0x0a, 0x44, 0x6f, 0x77, 0x6e, 0x52,
|
||||
0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17,
|
||||
0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x6c,
|
||||
0x65, 0x74, 0x65, 0x64, 0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28,
|
||||
0x04, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x4c, 0x65, 0x6e, 0x67,
|
||||
0x74, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08,
|
||||
0x52, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x32, 0xbe, 0x01, 0x0a, 0x06, 0x44, 0x61, 0x65, 0x6d, 0x6f,
|
||||
0x6e, 0x12, 0x39, 0x0a, 0x08, 0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x15, 0x2e,
|
||||
0x64, 0x66, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71,
|
||||
0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x64, 0x66, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e,
|
||||
0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x30, 0x01, 0x12, 0x3a, 0x0a, 0x0d,
|
||||
0x47, 0x65, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x73, 0x12, 0x16, 0x2e,
|
||||
0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65,
|
||||
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65,
|
||||
0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x12, 0x3d, 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63,
|
||||
0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
|
||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a,
|
||||
0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
|
||||
0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x26, 0x5a, 0x24, 0x64, 0x37, 0x79, 0x2e, 0x69,
|
||||
0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f, 0x76, 0x32, 0x2f, 0x70,
|
||||
0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x64, 0x66, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x62,
|
||||
0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x6f, 0x1a, 0x17, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2f, 0x76, 0x61, 0x6c, 0x69,
|
||||
0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x85, 0x03, 0x0a, 0x0b, 0x44,
|
||||
0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x04, 0x75, 0x75,
|
||||
0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x72, 0x03, 0xb0,
|
||||
0x01, 0x01, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x12, 0x1a, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18,
|
||||
0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x72, 0x03, 0x88, 0x01, 0x01, 0x52,
|
||||
0x03, 0x75, 0x72, 0x6c, 0x12, 0x1f, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x18, 0x03,
|
||||
0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x6f,
|
||||
0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x21, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74,
|
||||
0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x32, 0x02, 0x28, 0x00, 0x52,
|
||||
0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x24, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69,
|
||||
0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x01, 0x42, 0x0e, 0xfa, 0x42, 0x0b, 0x12, 0x09, 0x29, 0x00,
|
||||
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x2e,
|
||||
0x0a, 0x13, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x5f, 0x73,
|
||||
0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x64, 0x69, 0x73,
|
||||
0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x28,
|
||||
0x0a, 0x08, 0x75, 0x72, 0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x0d, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x52,
|
||||
0x07, 0x75, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x34, 0x0a, 0x07, 0x70, 0x61, 0x74, 0x74,
|
||||
0x65, 0x72, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x42, 0x1a, 0xfa, 0x42, 0x17, 0x72, 0x15,
|
||||
0x52, 0x03, 0x70, 0x32, 0x70, 0x52, 0x03, 0x63, 0x64, 0x6e, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72,
|
||||
0x63, 0x65, 0xd0, 0x01, 0x01, 0x52, 0x07, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, 0x1e,
|
||||
0x0a, 0x0a, 0x63, 0x61, 0x6c, 0x6c, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x18, 0x09, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x0a, 0x63, 0x61, 0x6c, 0x6c, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x12, 0x10,
|
||||
0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x75, 0x69, 0x64,
|
||||
0x12, 0x10, 0x0a, 0x03, 0x67, 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x67,
|
||||
0x69, 0x64, 0x22, 0x98, 0x01, 0x0a, 0x0a, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c,
|
||||
0x74, 0x12, 0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01,
|
||||
0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73,
|
||||
0x6b, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03,
|
||||
0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x70,
|
||||
0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x32, 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74,
|
||||
0x65, 0x64, 0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x42,
|
||||
0x07, 0xfa, 0x42, 0x04, 0x32, 0x02, 0x28, 0x00, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65,
|
||||
0x74, 0x65, 0x64, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x6f, 0x6e,
|
||||
0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x32, 0xbe, 0x01,
|
||||
0x0a, 0x06, 0x44, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x12, 0x39, 0x0a, 0x08, 0x44, 0x6f, 0x77, 0x6e,
|
||||
0x6c, 0x6f, 0x61, 0x64, 0x12, 0x15, 0x2e, 0x64, 0x66, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e,
|
||||
0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x64, 0x66,
|
||||
0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c,
|
||||
0x74, 0x30, 0x01, 0x12, 0x3a, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54,
|
||||
0x61, 0x73, 0x6b, 0x73, 0x12, 0x16, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63,
|
||||
0x65, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x62,
|
||||
0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x12,
|
||||
0x3d, 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x16,
|
||||
0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66,
|
||||
0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x26,
|
||||
0x5a, 0x24, 0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66,
|
||||
0x6c, 0x79, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x64, 0x66,
|
||||
0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
|
|
|
|||
|
|
@ -33,27 +33,119 @@ var (
|
|||
_ = anypb.Any{}
|
||||
)
|
||||
|
||||
// define the regex for a UUID once up-front
|
||||
var _dfdaemon_uuidPattern = regexp.MustCompile("^[0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}$")
|
||||
|
||||
// Validate checks the field values on DownRequest with the rules defined in
|
||||
// the proto definition for this message. If any rules are violated, an error
|
||||
// is returned.
|
||||
// the proto definition for this message. If any rules are violated, the first
|
||||
// error encountered is returned, or nil if there are no violations.
|
||||
func (m *DownRequest) Validate() error {
|
||||
return m.validate(false)
|
||||
}
|
||||
|
||||
// ValidateAll checks the field values on DownRequest with the rules defined in
|
||||
// the proto definition for this message. If any rules are violated, the
|
||||
// result is a list of violation errors wrapped in DownRequestMultiError, or
|
||||
// nil if none found.
|
||||
func (m *DownRequest) ValidateAll() error {
|
||||
return m.validate(true)
|
||||
}
|
||||
|
||||
func (m *DownRequest) validate(all bool) error {
|
||||
if m == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// no validation rules for Uuid
|
||||
var errors []error
|
||||
|
||||
// no validation rules for Url
|
||||
if err := m._validateUuid(m.GetUuid()); err != nil {
|
||||
err = DownRequestValidationError{
|
||||
field: "Uuid",
|
||||
reason: "value must be a valid UUID",
|
||||
cause: err,
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for Output
|
||||
if uri, err := url.Parse(m.GetUrl()); err != nil {
|
||||
err = DownRequestValidationError{
|
||||
field: "Url",
|
||||
reason: "value must be a valid URI",
|
||||
cause: err,
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
} else if !uri.IsAbs() {
|
||||
err := DownRequestValidationError{
|
||||
field: "Url",
|
||||
reason: "value must be absolute",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for Timeout
|
||||
if utf8.RuneCountInString(m.GetOutput()) < 1 {
|
||||
err := DownRequestValidationError{
|
||||
field: "Output",
|
||||
reason: "value length must be at least 1 runes",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for Limit
|
||||
if m.GetTimeout() < 0 {
|
||||
err := DownRequestValidationError{
|
||||
field: "Timeout",
|
||||
reason: "value must be greater than or equal to 0",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
if m.GetLimit() < 0 {
|
||||
err := DownRequestValidationError{
|
||||
field: "Limit",
|
||||
reason: "value must be greater than or equal to 0",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for DisableBackSource
|
||||
|
||||
if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok {
|
||||
if all {
|
||||
switch v := interface{}(m.GetUrlMeta()).(type) {
|
||||
case interface{ ValidateAll() error }:
|
||||
if err := v.ValidateAll(); err != nil {
|
||||
errors = append(errors, DownRequestValidationError{
|
||||
field: "UrlMeta",
|
||||
reason: "embedded message failed validation",
|
||||
cause: err,
|
||||
})
|
||||
}
|
||||
case interface{ Validate() error }:
|
||||
if err := v.Validate(); err != nil {
|
||||
errors = append(errors, DownRequestValidationError{
|
||||
field: "UrlMeta",
|
||||
reason: "embedded message failed validation",
|
||||
cause: err,
|
||||
})
|
||||
}
|
||||
}
|
||||
} else if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok {
|
||||
if err := v.Validate(); err != nil {
|
||||
return DownRequestValidationError{
|
||||
field: "UrlMeta",
|
||||
|
|
@ -63,7 +155,20 @@ func (m *DownRequest) Validate() error {
|
|||
}
|
||||
}
|
||||
|
||||
// no validation rules for Pattern
|
||||
if m.GetPattern() != "" {
|
||||
|
||||
if _, ok := _DownRequest_Pattern_InLookup[m.GetPattern()]; !ok {
|
||||
err := DownRequestValidationError{
|
||||
field: "Pattern",
|
||||
reason: "value must be in list [p2p cdn source]",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// no validation rules for Callsystem
|
||||
|
||||
|
|
@ -71,9 +176,36 @@ func (m *DownRequest) Validate() error {
|
|||
|
||||
// no validation rules for Gid
|
||||
|
||||
if len(errors) > 0 {
|
||||
return DownRequestMultiError(errors)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *DownRequest) _validateUuid(uuid string) error {
|
||||
if matched := _dfdaemon_uuidPattern.MatchString(uuid); !matched {
|
||||
return errors.New("invalid uuid format")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// DownRequestMultiError is an error wrapping multiple validation errors
|
||||
// returned by DownRequest.ValidateAll() if the designated constraints aren't met.
|
||||
type DownRequestMultiError []error
|
||||
|
||||
// Error returns a concatenation of all the error messages it wraps.
|
||||
func (m DownRequestMultiError) Error() string {
|
||||
var msgs []string
|
||||
for _, err := range m {
|
||||
msgs = append(msgs, err.Error())
|
||||
}
|
||||
return strings.Join(msgs, "; ")
|
||||
}
|
||||
|
||||
// AllErrors returns a list of validation violation errors.
|
||||
func (m DownRequestMultiError) AllErrors() []error { return m }
|
||||
|
||||
// DownRequestValidationError is the validation error returned by
|
||||
// DownRequest.Validate if the designated constraints aren't met.
|
||||
type DownRequestValidationError struct {
|
||||
|
|
@ -128,24 +260,91 @@ var _ interface {
|
|||
ErrorName() string
|
||||
} = DownRequestValidationError{}
|
||||
|
||||
var _DownRequest_Pattern_InLookup = map[string]struct{}{
|
||||
"p2p": {},
|
||||
"cdn": {},
|
||||
"source": {},
|
||||
}
|
||||
|
||||
// Validate checks the field values on DownResult with the rules defined in the
|
||||
// proto definition for this message. If any rules are violated, an error is returned.
|
||||
// proto definition for this message. If any rules are violated, the first
|
||||
// error encountered is returned, or nil if there are no violations.
|
||||
func (m *DownResult) Validate() error {
|
||||
return m.validate(false)
|
||||
}
|
||||
|
||||
// ValidateAll checks the field values on DownResult with the rules defined in
|
||||
// the proto definition for this message. If any rules are violated, the
|
||||
// result is a list of violation errors wrapped in DownResultMultiError, or
|
||||
// nil if none found.
|
||||
func (m *DownResult) ValidateAll() error {
|
||||
return m.validate(true)
|
||||
}
|
||||
|
||||
func (m *DownResult) validate(all bool) error {
|
||||
if m == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// no validation rules for TaskId
|
||||
var errors []error
|
||||
|
||||
// no validation rules for PeerId
|
||||
if utf8.RuneCountInString(m.GetTaskId()) < 1 {
|
||||
err := DownResultValidationError{
|
||||
field: "TaskId",
|
||||
reason: "value length must be at least 1 runes",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for CompletedLength
|
||||
if utf8.RuneCountInString(m.GetPeerId()) < 1 {
|
||||
err := DownResultValidationError{
|
||||
field: "PeerId",
|
||||
reason: "value length must be at least 1 runes",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
if m.GetCompletedLength() < 0 {
|
||||
err := DownResultValidationError{
|
||||
field: "CompletedLength",
|
||||
reason: "value must be greater than or equal to 0",
|
||||
}
|
||||
if !all {
|
||||
return err
|
||||
}
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// no validation rules for Done
|
||||
|
||||
if len(errors) > 0 {
|
||||
return DownResultMultiError(errors)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// DownResultMultiError is an error wrapping multiple validation errors
|
||||
// returned by DownResult.ValidateAll() if the designated constraints aren't met.
|
||||
type DownResultMultiError []error
|
||||
|
||||
// Error returns a concatenation of all the error messages it wraps.
|
||||
func (m DownResultMultiError) Error() string {
|
||||
var msgs []string
|
||||
for _, err := range m {
|
||||
msgs = append(msgs, err.Error())
|
||||
}
|
||||
return strings.Join(msgs, "; ")
|
||||
}
|
||||
|
||||
// AllErrors returns a list of validation violation errors.
|
||||
func (m DownResultMultiError) AllErrors() []error { return m }
|
||||
|
||||
// DownResultValidationError is the validation error returned by
|
||||
// DownResult.Validate if the designated constraints aren't met.
|
||||
type DownResultValidationError struct {
|
||||
|
|
|
|||
|
|
@ -20,25 +20,26 @@ package dfdaemon;
|
|||
|
||||
import "pkg/rpc/base/base.proto";
|
||||
import "google/protobuf/empty.proto";
|
||||
import "validate/validate.proto";
|
||||
|
||||
option go_package = "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon";
|
||||
|
||||
message DownRequest{
|
||||
// identify one downloading, the framework will fill it automatically
|
||||
string uuid = 1;
|
||||
string uuid = 1 [(validate.rules).string.uuid = true];
|
||||
// download file from the url, not only for http
|
||||
string url = 2;
|
||||
string url = 2 [(validate.rules).string.uri = true];
|
||||
// pieces will be written to output path directly,
|
||||
// at the same time, dfdaemon workspace also makes soft link to the output
|
||||
string output = 3;
|
||||
string output = 3 [(validate.rules).string.min_len = 1];
|
||||
// timeout duration
|
||||
int64 timeout = 4;
|
||||
uint64 timeout = 4 [(validate.rules).uint64.gte = 0];
|
||||
// rate limit in bytes per second
|
||||
double limit = 5;
|
||||
double limit = 5 [(validate.rules).double.gte = 0];
|
||||
bool disable_back_source = 6;
|
||||
base.UrlMeta url_meta = 7;
|
||||
// p2p/cdn/source
|
||||
string pattern = 8;
|
||||
// p2p/cdn/source, default is p2p
|
||||
string pattern = 8 [(validate.rules).string = {in:["p2p", "cdn", "source"], ignore_empty:true}];
|
||||
// call system
|
||||
string callsystem = 9;
|
||||
// user id
|
||||
|
|
@ -48,9 +49,9 @@ message DownRequest{
|
|||
}
|
||||
|
||||
message DownResult{
|
||||
string task_id = 2;
|
||||
string peer_id = 3;
|
||||
uint64 completed_length = 4;
|
||||
string task_id = 2 [(validate.rules).string.min_len = 1];
|
||||
string peer_id = 3 [(validate.rules).string.min_len = 1];
|
||||
uint64 completed_length = 4 [(validate.rules).uint64.gte = 0];
|
||||
bool done = 5;
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -13,7 +13,6 @@ import (
|
|||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the grpc package it is being compiled against.
|
||||
// Requires gRPC-Go v1.32.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion7
|
||||
|
||||
// DaemonClient is the client API for Daemon service.
|
||||
|
|
@ -37,7 +36,7 @@ func NewDaemonClient(cc grpc.ClientConnInterface) DaemonClient {
|
|||
}
|
||||
|
||||
func (c *daemonClient) Download(ctx context.Context, in *DownRequest, opts ...grpc.CallOption) (Daemon_DownloadClient, error) {
|
||||
stream, err := c.cc.NewStream(ctx, &Daemon_ServiceDesc.Streams[0], "/dfdaemon.Daemon/Download", opts...)
|
||||
stream, err := c.cc.NewStream(ctx, &_Daemon_serviceDesc.Streams[0], "/dfdaemon.Daemon/Download", opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
@ -122,7 +121,7 @@ type UnsafeDaemonServer interface {
|
|||
}
|
||||
|
||||
func RegisterDaemonServer(s grpc.ServiceRegistrar, srv DaemonServer) {
|
||||
s.RegisterService(&Daemon_ServiceDesc, srv)
|
||||
s.RegisterService(&_Daemon_serviceDesc, srv)
|
||||
}
|
||||
|
||||
func _Daemon_Download_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
|
|
@ -182,10 +181,7 @@ func _Daemon_CheckHealth_Handler(srv interface{}, ctx context.Context, dec func(
|
|||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
// Daemon_ServiceDesc is the grpc.ServiceDesc for Daemon service.
|
||||
// It's only intended for direct use with grpc.RegisterService,
|
||||
// and not to be introspected or modified (even as a copy)
|
||||
var Daemon_ServiceDesc = grpc.ServiceDesc{
|
||||
var _Daemon_serviceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "dfdaemon.Daemon",
|
||||
HandlerType: (*DaemonServer)(nil),
|
||||
Methods: []grpc.MethodDesc{
|
||||
|
|
|
|||
|
|
@ -22,13 +22,12 @@
|
|||
package manager
|
||||
|
||||
import (
|
||||
reflect "reflect"
|
||||
sync "sync"
|
||||
|
||||
_ "github.com/envoyproxy/protoc-gen-validate/validate"
|
||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||
emptypb "google.golang.org/protobuf/types/known/emptypb"
|
||||
reflect "reflect"
|
||||
sync "sync"
|
||||
)
|
||||
|
||||
const (
|
||||
|
|
|
|||
File diff suppressed because it is too large
Load Diff
|
|
@ -12,7 +12,6 @@ import (
|
|||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the grpc package it is being compiled against.
|
||||
// Requires gRPC-Go v1.32.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion7
|
||||
|
||||
// ManagerClient is the client API for Manager service.
|
||||
|
|
@ -87,7 +86,7 @@ func (c *managerClient) ListSchedulers(ctx context.Context, in *ListSchedulersRe
|
|||
}
|
||||
|
||||
func (c *managerClient) KeepAlive(ctx context.Context, opts ...grpc.CallOption) (Manager_KeepAliveClient, error) {
|
||||
stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[0], "/manager.Manager/KeepAlive", opts...)
|
||||
stream, err := c.cc.NewStream(ctx, &_Manager_serviceDesc.Streams[0], "/manager.Manager/KeepAlive", opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
@ -171,7 +170,7 @@ type UnsafeManagerServer interface {
|
|||
}
|
||||
|
||||
func RegisterManagerServer(s grpc.ServiceRegistrar, srv ManagerServer) {
|
||||
s.RegisterService(&Manager_ServiceDesc, srv)
|
||||
s.RegisterService(&_Manager_serviceDesc, srv)
|
||||
}
|
||||
|
||||
func _Manager_GetCDN_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
|
|
@ -290,10 +289,7 @@ func (x *managerKeepAliveServer) Recv() (*KeepAliveRequest, error) {
|
|||
return m, nil
|
||||
}
|
||||
|
||||
// Manager_ServiceDesc is the grpc.ServiceDesc for Manager service.
|
||||
// It's only intended for direct use with grpc.RegisterService,
|
||||
// and not to be introspected or modified (even as a copy)
|
||||
var Manager_ServiceDesc = grpc.ServiceDesc{
|
||||
var _Manager_serviceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "manager.Manager",
|
||||
HandlerType: (*ManagerServer)(nil),
|
||||
Methods: []grpc.MethodDesc{
|
||||
|
|
|
|||
|
|
@ -23,6 +23,7 @@ package scheduler
|
|||
|
||||
import (
|
||||
base "d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
_ "github.com/envoyproxy/protoc-gen-validate/validate"
|
||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||
emptypb "google.golang.org/protobuf/types/known/emptypb"
|
||||
|
|
@ -441,7 +442,7 @@ type PieceResult struct {
|
|||
Code base.Code `protobuf:"varint,8,opt,name=code,proto3,enum=base.Code" json:"code,omitempty"`
|
||||
// current host resource usage
|
||||
HostLoad *base.HostLoad `protobuf:"bytes,9,opt,name=host_load,json=hostLoad,proto3" json:"host_load,omitempty"`
|
||||
// currently completed piece count
|
||||
// currently completed piece count, -1 represent download failed
|
||||
FinishedCount int32 `protobuf:"varint,10,opt,name=finished_count,json=finishedCount,proto3" json:"finished_count,omitempty"`
|
||||
}
|
||||
|
||||
|
|
@ -651,14 +652,15 @@ type PeerResult struct {
|
|||
// total content length(byte)
|
||||
ContentLength int64 `protobuf:"varint,7,opt,name=content_length,json=contentLength,proto3" json:"content_length,omitempty"`
|
||||
// total network traffic(byte)
|
||||
Traffic int64 `protobuf:"varint,8,opt,name=traffic,proto3" json:"traffic,omitempty"`
|
||||
Traffic uint64 `protobuf:"varint,8,opt,name=traffic,proto3" json:"traffic,omitempty"`
|
||||
// total time(millisecond) consumed
|
||||
Cost uint32 `protobuf:"varint,9,opt,name=cost,proto3" json:"cost,omitempty"`
|
||||
// whether peer downloading file is successfully
|
||||
Success bool `protobuf:"varint,10,opt,name=success,proto3" json:"success,omitempty"`
|
||||
// result code
|
||||
Code base.Code `protobuf:"varint,11,opt,name=code,proto3,enum=base.Code" json:"code,omitempty"`
|
||||
TotalPieceCount int32 `protobuf:"varint,12,opt,name=total_piece_count,json=totalPieceCount,proto3" json:"total_piece_count,omitempty"`
|
||||
Code base.Code `protobuf:"varint,11,opt,name=code,proto3,enum=base.Code" json:"code,omitempty"`
|
||||
// -1 represent task is running or download failed
|
||||
TotalPieceCount int32 `protobuf:"varint,12,opt,name=total_piece_count,json=totalPieceCount,proto3" json:"total_piece_count,omitempty"`
|
||||
}
|
||||
|
||||
func (x *PeerResult) Reset() {
|
||||
|
|
@ -742,7 +744,7 @@ func (x *PeerResult) GetContentLength() int64 {
|
|||
return 0
|
||||
}
|
||||
|
||||
func (x *PeerResult) GetTraffic() int64 {
|
||||
func (x *PeerResult) GetTraffic() uint64 {
|
||||
if x != nil {
|
||||
return x.Traffic
|
||||
}
|
||||
|
|
@ -905,149 +907,168 @@ var file_pkg_rpc_scheduler_scheduler_proto_rawDesc = []byte{
|
|||
0x6c, 0x65, 0x72, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x12, 0x09, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x1a, 0x17,
|
||||
0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x62, 0x61, 0x73, 0x65, 0x2f, 0x62, 0x61, 0x73,
|
||||
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x22, 0xe8, 0x01, 0x0a, 0x0f, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73,
|
||||
0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x28, 0x0a, 0x08, 0x75, 0x72,
|
||||
0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x62,
|
||||
0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x07, 0x75, 0x72, 0x6c,
|
||||
0x4d, 0x65, 0x74, 0x61, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18,
|
||||
0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x30, 0x0a,
|
||||
0x09, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x13, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65,
|
||||
0x72, 0x48, 0x6f, 0x73, 0x74, 0x52, 0x08, 0x70, 0x65, 0x65, 0x72, 0x48, 0x6f, 0x73, 0x74, 0x12,
|
||||
0x2b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x05, 0x20, 0x01,
|
||||
0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x48, 0x6f, 0x73, 0x74, 0x4c, 0x6f,
|
||||
0x61, 0x64, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x21, 0x0a, 0x0c,
|
||||
0x69, 0x73, 0x5f, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x18, 0x06, 0x20, 0x01,
|
||||
0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x22,
|
||||
0xcd, 0x01, 0x0a, 0x0e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75,
|
||||
0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x2e, 0x0a, 0x0a, 0x73,
|
||||
0x69, 0x7a, 0x65, 0x5f, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32,
|
||||
0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x53, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65,
|
||||
0x52, 0x09, 0x73, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x3b, 0x0a, 0x0c, 0x73,
|
||||
0x69, 0x6e, 0x67, 0x6c, 0x65, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28,
|
||||
0x0b, 0x32, 0x16, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x53, 0x69,
|
||||
0x6e, 0x67, 0x6c, 0x65, 0x50, 0x69, 0x65, 0x63, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x69, 0x6e,
|
||||
0x67, 0x6c, 0x65, 0x50, 0x69, 0x65, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0d, 0x70, 0x69, 0x65, 0x63,
|
||||
0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x48,
|
||||
0x00, 0x52, 0x0c, 0x70, 0x69, 0x65, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x42,
|
||||
0x0e, 0x0a, 0x0c, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x22,
|
||||
0x71, 0x0a, 0x0b, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x50, 0x69, 0x65, 0x63, 0x65, 0x12, 0x17,
|
||||
0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x61,
|
||||
0x64, 0x64, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x64, 0x73, 0x74, 0x41, 0x64,
|
||||
0x64, 0x72, 0x12, 0x2e, 0x0a, 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f,
|
||||
0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69,
|
||||
0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e,
|
||||
0x66, 0x6f, 0x22, 0xfd, 0x01, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x48, 0x6f, 0x73, 0x74, 0x12,
|
||||
0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75,
|
||||
0x75, 0x69, 0x64, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x02, 0x69, 0x70, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x70, 0x63, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18,
|
||||
0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x72, 0x70, 0x63, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1b,
|
||||
0x0a, 0x09, 0x64, 0x6f, 0x77, 0x6e, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28,
|
||||
0x05, 0x52, 0x08, 0x64, 0x6f, 0x77, 0x6e, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x68,
|
||||
0x6f, 0x73, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,
|
||||
0x68, 0x6f, 0x73, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x63, 0x75,
|
||||
0x72, 0x69, 0x74, 0x79, 0x5f, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x0e, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x44, 0x6f, 0x6d, 0x61, 0x69,
|
||||
0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a,
|
||||
0x03, 0x69, 0x64, 0x63, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x63, 0x12,
|
||||
0x21, 0x0a, 0x0c, 0x6e, 0x65, 0x74, 0x5f, 0x74, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x18,
|
||||
0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f,
|
||||
0x67, 0x79, 0x22, 0xd0, 0x02, 0x0a, 0x0b, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75,
|
||||
0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x73,
|
||||
0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x72,
|
||||
0x63, 0x50, 0x69, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18,
|
||||
0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x2e, 0x0a,
|
||||
0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28,
|
||||
0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e,
|
||||
0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x0a,
|
||||
0x0a, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28,
|
||||
0x04, 0x52, 0x09, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08,
|
||||
0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07,
|
||||
0x65, 0x6e, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65,
|
||||
0x73, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73,
|
||||
0x73, 0x12, 0x1e, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32,
|
||||
0x0a, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64,
|
||||
0x65, 0x12, 0x2b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09,
|
||||
0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x48, 0x6f, 0x73, 0x74,
|
||||
0x4c, 0x6f, 0x61, 0x64, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x25,
|
||||
0x0a, 0x0e, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74,
|
||||
0x18, 0x0a, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64,
|
||||
0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xd3, 0x02, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x50, 0x61,
|
||||
0x63, 0x6b, 0x65, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18,
|
||||
0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a,
|
||||
0x07, 0x73, 0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06,
|
||||
0x73, 0x72, 0x63, 0x50, 0x69, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c,
|
||||
0x65, 0x6c, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d,
|
||||
0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x3b, 0x0a,
|
||||
0x09, 0x6d, 0x61, 0x69, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x17, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74,
|
||||
0x65, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
|
||||
0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xfb, 0x01,
|
||||
0x0a, 0x0f, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
|
||||
0x74, 0x12, 0x1a, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x08,
|
||||
0xfa, 0x42, 0x05, 0x72, 0x03, 0x88, 0x01, 0x01, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x28, 0x0a,
|
||||
0x08, 0x75, 0x72, 0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
|
||||
0x0d, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x07,
|
||||
0x75, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f,
|
||||
0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10,
|
||||
0x01, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x09, 0x70, 0x65, 0x65,
|
||||
0x72, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73,
|
||||
0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x48, 0x6f, 0x73,
|
||||
0x74, 0x52, 0x08, 0x70, 0x65, 0x65, 0x72, 0x48, 0x6f, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x09, 0x68,
|
||||
0x6f, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e,
|
||||
0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x48, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x52, 0x08,
|
||||
0x68, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x6d,
|
||||
0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b,
|
||||
0x69, 0x73, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x22, 0xe0, 0x01, 0x0a, 0x0e,
|
||||
0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x20,
|
||||
0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42,
|
||||
0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64,
|
||||
0x12, 0x38, 0x0a, 0x0a, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x03,
|
||||
0x20, 0x01, 0x28, 0x0e, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x53, 0x69, 0x7a, 0x65,
|
||||
0x53, 0x63, 0x6f, 0x70, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, 0x01, 0x02, 0x10, 0x01, 0x52,
|
||||
0x09, 0x73, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x3b, 0x0a, 0x0c, 0x73, 0x69,
|
||||
0x6e, 0x67, 0x6c, 0x65, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x16, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x53, 0x69, 0x6e,
|
||||
0x67, 0x6c, 0x65, 0x50, 0x69, 0x65, 0x63, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x69, 0x6e, 0x67,
|
||||
0x6c, 0x65, 0x50, 0x69, 0x65, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0d, 0x70, 0x69, 0x65, 0x63, 0x65,
|
||||
0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00,
|
||||
0x52, 0x0c, 0x70, 0x69, 0x65, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x42, 0x0e,
|
||||
0x0a, 0x0c, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x22, 0x83,
|
||||
0x01, 0x0a, 0x0b, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x50, 0x69, 0x65, 0x63, 0x65, 0x12, 0x20,
|
||||
0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42,
|
||||
0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64,
|
||||
0x12, 0x22, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x02, 0x20, 0x01,
|
||||
0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x07, 0x64, 0x73, 0x74,
|
||||
0x41, 0x64, 0x64, 0x72, 0x12, 0x2e, 0x0a, 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e,
|
||||
0x66, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e,
|
||||
0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65,
|
||||
0x49, 0x6e, 0x66, 0x6f, 0x22, 0xb5, 0x02, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x48, 0x6f, 0x73,
|
||||
0x74, 0x12, 0x1c, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42,
|
||||
0x08, 0xfa, 0x42, 0x05, 0x72, 0x03, 0xb0, 0x01, 0x01, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x12,
|
||||
0x17, 0x0a, 0x02, 0x69, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04,
|
||||
0x72, 0x02, 0x70, 0x01, 0x52, 0x02, 0x69, 0x70, 0x12, 0x27, 0x0a, 0x08, 0x72, 0x70, 0x63, 0x5f,
|
||||
0x70, 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x42, 0x0c, 0xfa, 0x42, 0x09, 0x1a,
|
||||
0x07, 0x10, 0xff, 0xff, 0x03, 0x28, 0x80, 0x08, 0x52, 0x07, 0x72, 0x70, 0x63, 0x50, 0x6f, 0x72,
|
||||
0x74, 0x12, 0x29, 0x0a, 0x09, 0x64, 0x6f, 0x77, 0x6e, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x04,
|
||||
0x20, 0x01, 0x28, 0x05, 0x42, 0x0c, 0xfa, 0x42, 0x09, 0x1a, 0x07, 0x10, 0xff, 0xff, 0x03, 0x28,
|
||||
0x80, 0x08, 0x52, 0x08, 0x64, 0x6f, 0x77, 0x6e, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x24, 0x0a, 0x09,
|
||||
0x68, 0x6f, 0x73, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x42,
|
||||
0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x68, 0x01, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x4e, 0x61,
|
||||
0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, 0x64,
|
||||
0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x65, 0x63,
|
||||
0x75, 0x72, 0x69, 0x74, 0x79, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x6c,
|
||||
0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6c,
|
||||
0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x63, 0x18, 0x08,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x63, 0x12, 0x21, 0x0a, 0x0c, 0x6e, 0x65, 0x74,
|
||||
0x5f, 0x74, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x0b, 0x6e, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x22, 0xec, 0x02, 0x0a,
|
||||
0x0b, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x20, 0x0a, 0x07,
|
||||
0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa,
|
||||
0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x20,
|
||||
0x0a, 0x07, 0x73, 0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42,
|
||||
0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x73, 0x72, 0x63, 0x50, 0x69, 0x64,
|
||||
0x12, 0x17, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x2e, 0x0a, 0x0a, 0x70, 0x69, 0x65,
|
||||
0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e,
|
||||
0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09,
|
||||
0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x65, 0x67,
|
||||
0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x62,
|
||||
0x65, 0x67, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x5f,
|
||||
0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x65, 0x6e, 0x64, 0x54,
|
||||
0x69, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x07,
|
||||
0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x28, 0x0a,
|
||||
0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61,
|
||||
0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, 0x01, 0x02, 0x10,
|
||||
0x01, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x2b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f,
|
||||
0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x62, 0x61, 0x73,
|
||||
0x65, 0x2e, 0x48, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74,
|
||||
0x4c, 0x6f, 0x61, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64,
|
||||
0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x66, 0x69,
|
||||
0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x98, 0x03, 0x0a, 0x0a,
|
||||
0x50, 0x65, 0x65, 0x72, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x12, 0x20, 0x0a, 0x07, 0x74, 0x61,
|
||||
0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04,
|
||||
0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x07,
|
||||
0x73, 0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa,
|
||||
0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x73, 0x72, 0x63, 0x50, 0x69, 0x64, 0x12, 0x2e,
|
||||
0x0a, 0x0e, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74,
|
||||
0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x1a, 0x02, 0x28, 0x01, 0x52,
|
||||
0x0d, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x3b,
|
||||
0x0a, 0x09, 0x6d, 0x61, 0x69, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28,
|
||||
0x0b, 0x32, 0x1e, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65,
|
||||
0x65, 0x72, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65, 0x65,
|
||||
0x72, 0x52, 0x08, 0x6d, 0x61, 0x69, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x0b, 0x73,
|
||||
0x74, 0x65, 0x61, 0x6c, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b,
|
||||
0x32, 0x1e, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65,
|
||||
0x72, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72,
|
||||
0x52, 0x08, 0x6d, 0x61, 0x69, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x0b, 0x73, 0x74,
|
||||
0x65, 0x61, 0x6c, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32,
|
||||
0x1e, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72,
|
||||
0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52,
|
||||
0x0a, 0x73, 0x74, 0x65, 0x61, 0x6c, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x1e, 0x0a, 0x04, 0x63,
|
||||
0x6f, 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61, 0x73, 0x65,
|
||||
0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x1a, 0x4e, 0x0a, 0x08, 0x44,
|
||||
0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x70, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x70, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x70, 0x63, 0x5f, 0x70,
|
||||
0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x72, 0x70, 0x63, 0x50, 0x6f,
|
||||
0x72, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x22, 0xdd, 0x02, 0x0a, 0x0a,
|
||||
0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61,
|
||||
0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73,
|
||||
0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x15, 0x0a, 0x06,
|
||||
0x73, 0x72, 0x63, 0x5f, 0x69, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x72,
|
||||
0x63, 0x49, 0x70, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f,
|
||||
0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x65,
|
||||
0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03,
|
||||
0x69, 0x64, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x63, 0x12, 0x10,
|
||||
0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c,
|
||||
0x52, 0x0a, 0x73, 0x74, 0x65, 0x61, 0x6c, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x28, 0x0a, 0x04,
|
||||
0x63, 0x6f, 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61, 0x73,
|
||||
0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, 0x01, 0x02, 0x10, 0x01,
|
||||
0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x1a, 0x6e, 0x0a, 0x08, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65,
|
||||
0x65, 0x72, 0x12, 0x17, 0x0a, 0x02, 0x69, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07,
|
||||
0xfa, 0x42, 0x04, 0x72, 0x02, 0x70, 0x01, 0x52, 0x02, 0x69, 0x70, 0x12, 0x27, 0x0a, 0x08, 0x72,
|
||||
0x70, 0x63, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x42, 0x0c, 0xfa,
|
||||
0x42, 0x09, 0x1a, 0x07, 0x10, 0xff, 0xff, 0x03, 0x28, 0x80, 0x08, 0x52, 0x07, 0x72, 0x70, 0x63,
|
||||
0x50, 0x6f, 0x72, 0x74, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18,
|
||||
0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06,
|
||||
0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x22, 0x8c, 0x03, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x52,
|
||||
0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64,
|
||||
0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52,
|
||||
0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f,
|
||||
0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10,
|
||||
0x01, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1e, 0x0a, 0x06, 0x73, 0x72, 0x63,
|
||||
0x5f, 0x69, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02,
|
||||
0x70, 0x01, 0x52, 0x05, 0x73, 0x72, 0x63, 0x49, 0x70, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x63,
|
||||
0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x04, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x0e, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x44, 0x6f, 0x6d, 0x61,
|
||||
0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x03, 0x69, 0x64, 0x63, 0x12, 0x1a, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28,
|
||||
0x09, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x72, 0x03, 0x88, 0x01, 0x01, 0x52, 0x03, 0x75, 0x72, 0x6c,
|
||||
0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x65, 0x6e, 0x67,
|
||||
0x74, 0x68, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e,
|
||||
0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x72, 0x61, 0x66, 0x66,
|
||||
0x69, 0x63, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x74, 0x72, 0x61, 0x66, 0x66, 0x69,
|
||||
0x69, 0x63, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x74, 0x72, 0x61, 0x66, 0x66, 0x69,
|
||||
0x63, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x73, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0d, 0x52,
|
||||
0x04, 0x63, 0x6f, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73,
|
||||
0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12,
|
||||
0x1e, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e,
|
||||
0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12,
|
||||
0x2a, 0x0a, 0x11, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x63,
|
||||
0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61,
|
||||
0x6c, 0x50, 0x69, 0x65, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x3e, 0x0a, 0x0a, 0x50,
|
||||
0x65, 0x65, 0x72, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73,
|
||||
0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b,
|
||||
0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x32, 0x9d, 0x02, 0x0a, 0x09,
|
||||
0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x10, 0x52, 0x65, 0x67,
|
||||
0x69, 0x73, 0x74, 0x65, 0x72, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x1a, 0x2e,
|
||||
0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61,
|
||||
0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x73, 0x63, 0x68, 0x65,
|
||||
0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65,
|
||||
0x73, 0x75, 0x6c, 0x74, 0x12, 0x46, 0x0a, 0x11, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69,
|
||||
0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x2e, 0x73, 0x63, 0x68, 0x65,
|
||||
0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c,
|
||||
0x74, 0x1a, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65,
|
||||
0x65, 0x72, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x28, 0x01, 0x30, 0x01, 0x12, 0x41, 0x0a, 0x10,
|
||||
0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74,
|
||||
0x12, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65,
|
||||
0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
|
||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12,
|
||||
0x3a, 0x0a, 0x09, 0x4c, 0x65, 0x61, 0x76, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x15, 0x2e, 0x73,
|
||||
0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x72,
|
||||
0x67, 0x65, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x27, 0x5a, 0x25, 0x64,
|
||||
0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f,
|
||||
0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64,
|
||||
0x75, 0x6c, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x28, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e,
|
||||
0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, 0x01,
|
||||
0x02, 0x10, 0x01, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x74, 0x6f, 0x74,
|
||||
0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0c,
|
||||
0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65, 0x63, 0x65,
|
||||
0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x50, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x72,
|
||||
0x67, 0x65, 0x74, 0x12, 0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01,
|
||||
0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74,
|
||||
0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52,
|
||||
0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x32, 0x9d, 0x02, 0x0a, 0x09, 0x53, 0x63, 0x68, 0x65,
|
||||
0x64, 0x75, 0x6c, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x10, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65,
|
||||
0x72, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x1a, 0x2e, 0x73, 0x63, 0x68, 0x65,
|
||||
0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65,
|
||||
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65,
|
||||
0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74,
|
||||
0x12, 0x46, 0x0a, 0x11, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52,
|
||||
0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65,
|
||||
0x72, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x1a, 0x15, 0x2e,
|
||||
0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x50, 0x61,
|
||||
0x63, 0x6b, 0x65, 0x74, 0x28, 0x01, 0x30, 0x01, 0x12, 0x41, 0x0a, 0x10, 0x52, 0x65, 0x70, 0x6f,
|
||||
0x72, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x15, 0x2e, 0x73,
|
||||
0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73,
|
||||
0x75, 0x6c, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x3a, 0x0a, 0x09, 0x4c,
|
||||
0x65, 0x61, 0x76, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64,
|
||||
0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x1a,
|
||||
0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
|
||||
0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x27, 0x5a, 0x25, 0x64, 0x37, 0x79, 0x2e, 0x69,
|
||||
0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f, 0x76, 0x32, 0x2f, 0x70,
|
||||
0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72,
|
||||
0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
|
|
|
|||
File diff suppressed because it is too large
Load Diff
|
|
@ -19,17 +19,18 @@ syntax = "proto3";
|
|||
package scheduler;
|
||||
|
||||
import "pkg/rpc/base/base.proto";
|
||||
import "validate/validate.proto";
|
||||
import "google/protobuf/empty.proto";
|
||||
|
||||
option go_package = "d7y.io/dragonfly/v2/pkg/rpc/scheduler";
|
||||
|
||||
message PeerTaskRequest{
|
||||
// universal resource locator for different kind of storage
|
||||
string url = 1;
|
||||
string url = 1 [(validate.rules).string.uri = true];
|
||||
// url meta info
|
||||
base.UrlMeta url_meta = 2;
|
||||
// peer's id and must be global uniqueness
|
||||
string peer_id = 3;
|
||||
string peer_id = 3 [(validate.rules).string.min_len = 1];
|
||||
// peer host info
|
||||
PeerHost peer_host = 4;
|
||||
// current host load
|
||||
|
|
@ -40,9 +41,9 @@ message PeerTaskRequest{
|
|||
|
||||
message RegisterResult{
|
||||
// task id
|
||||
string task_id = 2;
|
||||
string task_id = 2 [(validate.rules).string.min_len = 1];
|
||||
// file content length scope for the url
|
||||
base.SizeScope size_scope = 3;
|
||||
base.SizeScope size_scope = 3 [(validate.rules).enum.defined_only = true];
|
||||
// download the only piece directly for small or tiny file
|
||||
oneof direct_piece{
|
||||
// for small file
|
||||
|
|
@ -54,24 +55,24 @@ message RegisterResult{
|
|||
|
||||
message SinglePiece{
|
||||
// destination peer id
|
||||
string dst_pid = 1;
|
||||
string dst_pid = 1 [(validate.rules).string.min_len = 1];
|
||||
// download address(ip:port)
|
||||
string dst_addr = 2;
|
||||
string dst_addr = 2 [(validate.rules).string.min_len = 1];
|
||||
// one piece info
|
||||
base.PieceInfo piece_info = 3;
|
||||
}
|
||||
|
||||
message PeerHost{
|
||||
// each time the daemon starts, it will generate a different uuid
|
||||
string uuid = 1;
|
||||
string uuid = 1 [(validate.rules).string.uuid = true];
|
||||
// peer host ip
|
||||
string ip = 2;
|
||||
string ip = 2 [(validate.rules).string.ip = true];
|
||||
// rpc service port for peer
|
||||
int32 rpc_port = 3;
|
||||
int32 rpc_port = 3 [(validate.rules).int32 = {gte: 1024, lt: 65535}];
|
||||
// piece downloading port for peer
|
||||
int32 down_port = 4;
|
||||
int32 down_port = 4 [(validate.rules).int32 = {gte: 1024, lt: 65535}];
|
||||
// peer host name
|
||||
string host_name = 5;
|
||||
string host_name = 5 [(validate.rules).string.hostname = true];
|
||||
// security isolation domain for network
|
||||
string security_domain = 6;
|
||||
// location path: area|country|province|city|...
|
||||
|
|
@ -84,9 +85,9 @@ message PeerHost{
|
|||
|
||||
message PieceResult{
|
||||
// task id
|
||||
string task_id = 1;
|
||||
string task_id = 1 [(validate.rules).string.min_len = 1];
|
||||
// source peer id
|
||||
string src_pid = 2;
|
||||
string src_pid = 2 [(validate.rules).string.min_len = 1];
|
||||
// dest peer id
|
||||
string dst_pid = 3;
|
||||
// piece info
|
||||
|
|
@ -98,10 +99,10 @@ message PieceResult{
|
|||
// whether the piece downloading is successfully
|
||||
bool success = 7;
|
||||
// result code
|
||||
base.Code code = 8;
|
||||
base.Code code = 8 [(validate.rules).enum = {defined_only:true}];
|
||||
// current host resource usage
|
||||
base.HostLoad host_load = 9;
|
||||
// currently completed piece count
|
||||
// currently completed piece count, -1 represent download failed
|
||||
int32 finished_count = 10;
|
||||
}
|
||||
|
||||
|
|
@ -109,48 +110,49 @@ message PeerPacket{
|
|||
|
||||
message DestPeer{
|
||||
// dest ip
|
||||
string ip = 1;
|
||||
string ip = 1 [(validate.rules).string.ip = true];
|
||||
// rpc service port for dest peer
|
||||
int32 rpc_port = 2;
|
||||
int32 rpc_port = 2 [(validate.rules).int32 = {gte: 1024, lt: 65535}];
|
||||
// dest peer id
|
||||
string peer_id = 3;
|
||||
string peer_id = 3 [(validate.rules).string.min_len = 1];
|
||||
}
|
||||
|
||||
string task_id = 2;
|
||||
string task_id = 2 [(validate.rules).string.min_len = 1];
|
||||
// source peer id
|
||||
string src_pid = 3;
|
||||
string src_pid = 3 [(validate.rules).string.min_len = 1];
|
||||
|
||||
// concurrent downloading count from main peer
|
||||
int32 parallel_count = 4;
|
||||
int32 parallel_count = 4 [(validate.rules).int32.gte = 1];
|
||||
DestPeer main_peer = 5;
|
||||
repeated DestPeer steal_peers = 6;
|
||||
// result code
|
||||
base.Code code = 7;
|
||||
base.Code code = 7 [(validate.rules).enum.defined_only = true];
|
||||
}
|
||||
|
||||
message PeerResult{
|
||||
string task_id = 1;
|
||||
string peer_id = 2;
|
||||
string src_ip = 3;
|
||||
string task_id = 1 [(validate.rules).string.min_len = 1];
|
||||
string peer_id = 2 [(validate.rules).string.min_len = 1];
|
||||
string src_ip = 3 [(validate.rules).string.ip = true];
|
||||
string security_domain = 4;
|
||||
string idc = 5;
|
||||
string url = 6;
|
||||
string url = 6 [(validate.rules).string.uri = true];
|
||||
// total content length(byte)
|
||||
int64 content_length = 7;
|
||||
// total network traffic(byte)
|
||||
int64 traffic = 8;
|
||||
uint64 traffic = 8;
|
||||
// total time(millisecond) consumed
|
||||
uint32 cost = 9;
|
||||
// whether peer downloading file is successfully
|
||||
bool success = 10;
|
||||
// result code
|
||||
base.Code code = 11;
|
||||
base.Code code = 11 [(validate.rules).enum = {defined_only:true}];
|
||||
// -1 represent task is running or download failed
|
||||
int32 total_piece_count = 12;
|
||||
}
|
||||
|
||||
message PeerTarget{
|
||||
string task_id = 1;
|
||||
string peer_id = 2;
|
||||
string task_id = 1 [(validate.rules).string.min_len = 1];
|
||||
string peer_id = 2 [(validate.rules).string.min_len = 1];
|
||||
}
|
||||
|
||||
// Scheduler System RPC Service
|
||||
|
|
|
|||
|
|
@ -12,7 +12,6 @@ import (
|
|||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the grpc package it is being compiled against.
|
||||
// Requires gRPC-Go v1.32.0 or later.
|
||||
const _ = grpc.SupportPackageIsVersion7
|
||||
|
||||
// SchedulerClient is the client API for Scheduler service.
|
||||
|
|
@ -49,7 +48,7 @@ func (c *schedulerClient) RegisterPeerTask(ctx context.Context, in *PeerTaskRequ
|
|||
}
|
||||
|
||||
func (c *schedulerClient) ReportPieceResult(ctx context.Context, opts ...grpc.CallOption) (Scheduler_ReportPieceResultClient, error) {
|
||||
stream, err := c.cc.NewStream(ctx, &Scheduler_ServiceDesc.Streams[0], "/scheduler.Scheduler/ReportPieceResult", opts...)
|
||||
stream, err := c.cc.NewStream(ctx, &_Scheduler_serviceDesc.Streams[0], "/scheduler.Scheduler/ReportPieceResult", opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
@ -140,7 +139,7 @@ type UnsafeSchedulerServer interface {
|
|||
}
|
||||
|
||||
func RegisterSchedulerServer(s grpc.ServiceRegistrar, srv SchedulerServer) {
|
||||
s.RegisterService(&Scheduler_ServiceDesc, srv)
|
||||
s.RegisterService(&_Scheduler_serviceDesc, srv)
|
||||
}
|
||||
|
||||
func _Scheduler_RegisterPeerTask_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
|
|
@ -223,10 +222,7 @@ func _Scheduler_LeaveTask_Handler(srv interface{}, ctx context.Context, dec func
|
|||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
// Scheduler_ServiceDesc is the grpc.ServiceDesc for Scheduler service.
|
||||
// It's only intended for direct use with grpc.RegisterService,
|
||||
// and not to be introspected or modified (even as a copy)
|
||||
var Scheduler_ServiceDesc = grpc.ServiceDesc{
|
||||
var _Scheduler_serviceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "scheduler.Scheduler",
|
||||
HandlerType: (*SchedulerServer)(nil),
|
||||
Methods: []grpc.MethodDesc{
|
||||
|
|
|
|||
|
|
@ -103,7 +103,7 @@ func (p *proxy) ReportPeerResult(ctx context.Context, req *scheduler.PeerResult)
|
|||
zap.String("SchedulerIP", iputils.IPv4),
|
||||
zap.String("SchedulerHostname", hostutils.FQDNHostname),
|
||||
zap.Int64("ContentLength", req.ContentLength),
|
||||
zap.Int64("Traffic", req.Traffic),
|
||||
zap.Uint64("Traffic", req.Traffic),
|
||||
zap.Uint32("Cost", req.Cost),
|
||||
zap.Int32("Code", int32(req.Code)),
|
||||
)
|
||||
|
|
|
|||
|
|
@ -22,13 +22,16 @@ import (
|
|||
|
||||
grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
|
||||
grpc_zap "github.com/grpc-ecosystem/go-grpc-middleware/logging/zap"
|
||||
grpc_validator "github.com/grpc-ecosystem/go-grpc-middleware/validator"
|
||||
grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
"google.golang.org/grpc/status"
|
||||
|
||||
"d7y.io/dragonfly/v2/internal/dferrors"
|
||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/base/common"
|
||||
)
|
||||
|
||||
|
|
@ -46,11 +49,13 @@ var DefaultServerOptions = []grpc.ServerOption{
|
|||
streamServerInterceptor,
|
||||
grpc_prometheus.StreamServerInterceptor,
|
||||
grpc_zap.StreamServerInterceptor(logger.GrpcLogger.Desugar()),
|
||||
grpc_validator.StreamServerInterceptor(),
|
||||
)),
|
||||
grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(
|
||||
unaryServerInterceptor,
|
||||
grpc_prometheus.UnaryServerInterceptor,
|
||||
grpc_zap.UnaryServerInterceptor(logger.GrpcLogger.Desugar()),
|
||||
grpc_validator.UnaryServerInterceptor(),
|
||||
)),
|
||||
}
|
||||
|
||||
|
|
@ -60,7 +65,6 @@ func streamServerInterceptor(srv interface{}, ss grpc.ServerStream, info *grpc.S
|
|||
err = convertServerError(err)
|
||||
logger.GrpcLogger.Errorf("do stream server error: %v for method: %s", err, info.FullMethod)
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
|
|
@ -75,6 +79,9 @@ func unaryServerInterceptor(ctx context.Context, req interface{}, info *grpc.Una
|
|||
}
|
||||
|
||||
func convertServerError(err error) error {
|
||||
if status.Code(err) == codes.InvalidArgument {
|
||||
err = dferrors.New(base.Code_BadRequest, err.Error())
|
||||
}
|
||||
if v, ok := err.(*dferrors.DfError); ok {
|
||||
logger.GrpcLogger.Errorf(v.Message)
|
||||
if s, e := status.Convert(err).WithDetails(common.NewGrpcDfError(v.Code, v.Message)); e == nil {
|
||||
|
|
|
|||
|
|
@ -43,8 +43,8 @@ type factor struct {
|
|||
idc string
|
||||
location string
|
||||
netTopology string
|
||||
totalUploadLoad int32
|
||||
currentUploadLoad int32
|
||||
totalUploadLoad uint32
|
||||
currentUploadLoad uint32
|
||||
finishedPieceCount int32
|
||||
hostUUID string
|
||||
taskPieceCount int32
|
||||
|
|
|
|||
|
|
@ -226,7 +226,7 @@ func (s *SchedulerService) RegisterPeerTask(req *schedulerRPC.PeerTaskRequest, t
|
|||
var options []supervisor.HostOption
|
||||
if clientConfig, ok := s.dynconfig.GetSchedulerClusterClientConfig(); ok {
|
||||
options = []supervisor.HostOption{
|
||||
supervisor.WithTotalUploadLoad(int32(clientConfig.LoadLimit)),
|
||||
supervisor.WithTotalUploadLoad(clientConfig.LoadLimit),
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -31,8 +31,6 @@ import (
|
|||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
|
||||
schedulerserver "d7y.io/dragonfly/v2/pkg/rpc/scheduler/server"
|
||||
"d7y.io/dragonfly/v2/pkg/util/net/urlutils"
|
||||
"d7y.io/dragonfly/v2/pkg/util/stringutils"
|
||||
"d7y.io/dragonfly/v2/scheduler/config"
|
||||
"d7y.io/dragonfly/v2/scheduler/core"
|
||||
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
||||
|
|
@ -65,12 +63,6 @@ func (s *server) RegisterPeerTask(ctx context.Context, request *scheduler.PeerTa
|
|||
defer span.End()
|
||||
logger.Debugf("register peer task, req: %+v", request)
|
||||
resp = new(scheduler.RegisterResult)
|
||||
if verifyErr := validateParams(request); verifyErr != nil {
|
||||
err = dferrors.Newf(base.Code_BadRequest, "bad request param: %v", verifyErr)
|
||||
logger.Errorf("register request: %v", err)
|
||||
span.RecordError(err)
|
||||
return
|
||||
}
|
||||
|
||||
taskID := idgen.TaskID(request.Url, request.UrlMeta)
|
||||
span.SetAttributes(config.AttributeTaskID.String(taskID))
|
||||
|
|
@ -215,18 +207,6 @@ func (s *server) LeaveTask(ctx context.Context, target *scheduler.PeerTarget) (e
|
|||
return s.service.HandleLeaveTask(ctx, peer)
|
||||
}
|
||||
|
||||
// validateParams validates the params of scheduler.PeerTaskRequest.
|
||||
func validateParams(req *scheduler.PeerTaskRequest) error {
|
||||
if !urlutils.IsValidURL(req.Url) {
|
||||
return fmt.Errorf("invalid url: %s", req.Url)
|
||||
}
|
||||
|
||||
if stringutils.IsEmpty(req.PeerId) {
|
||||
return fmt.Errorf("empty peerID")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func getTaskSizeScope(task *supervisor.Task) base.SizeScope {
|
||||
if task.IsSuccess() {
|
||||
if task.ContentLength.Load() <= supervisor.TinyFileSize {
|
||||
|
|
|
|||
|
|
@ -313,7 +313,7 @@ func cdnsToHosts(cdns []*config.CDN) map[string]*Host {
|
|||
if config, ok := cdn.GetCDNClusterConfig(); ok {
|
||||
options = []HostOption{
|
||||
WithNetTopology(config.NetTopology),
|
||||
WithTotalUploadLoad(int32(config.LoadLimit)),
|
||||
WithTotalUploadLoad(config.LoadLimit),
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -68,7 +68,7 @@ func (m *hostManager) Delete(key string) {
|
|||
|
||||
type HostOption func(rt *Host) *Host
|
||||
|
||||
func WithTotalUploadLoad(load int32) HostOption {
|
||||
func WithTotalUploadLoad(load uint32) HostOption {
|
||||
return func(h *Host) *Host {
|
||||
h.TotalUploadLoad = load
|
||||
return h
|
||||
|
|
@ -106,9 +106,9 @@ type Host struct {
|
|||
// Example: switch|router|...
|
||||
NetTopology string
|
||||
// TODO TotalUploadLoad currentUploadLoad decided by real time client report host info
|
||||
TotalUploadLoad int32
|
||||
TotalUploadLoad uint32
|
||||
// CurrentUploadLoad is current upload load number
|
||||
CurrentUploadLoad atomic.Int32
|
||||
CurrentUploadLoad atomic.Uint32
|
||||
// peers info map
|
||||
peers *sync.Map
|
||||
// host logger
|
||||
|
|
@ -175,7 +175,7 @@ func (h *Host) GetPeersLen() int {
|
|||
}
|
||||
|
||||
func (h *Host) GetFreeUploadLoad() int32 {
|
||||
return h.TotalUploadLoad - h.CurrentUploadLoad.Load()
|
||||
return int32(h.TotalUploadLoad - h.CurrentUploadLoad.Load())
|
||||
}
|
||||
|
||||
func (h *Host) Log() *logger.SugaredLoggerOnWith {
|
||||
|
|
|
|||
Loading…
Reference in New Issue