validate grpc model (#825)

* base validate

Signed-off-by: sunwp <244372610@qq.com>
This commit is contained in:
sunwp 2021-12-02 12:06:43 +08:00 committed by Gaius
parent e34eb9607e
commit c1d84fb56d
No known key found for this signature in database
GPG Key ID: 8B4E5D1290FA2FFB
55 changed files with 2715 additions and 808 deletions

View File

@ -24,7 +24,7 @@ import (
// //
// If the fileLength<=0, which means failed to get fileLength // If the fileLength<=0, which means failed to get fileLength
// and then use the DefaultPieceSize. // and then use the DefaultPieceSize.
func ComputePieceSize(length int64) int32 { func ComputePieceSize(length int64) uint32 {
if length <= 0 || length <= 200*1024*1024 { if length <= 0 || length <= 200*1024*1024 {
return config.DefaultPieceSize return config.DefaultPieceSize
} }
@ -34,5 +34,5 @@ func ComputePieceSize(length int64) int32 {
if mpSize > config.DefaultPieceSizeLimit { if mpSize > config.DefaultPieceSizeLimit {
return config.DefaultPieceSizeLimit return config.DefaultPieceSizeLimit
} }
return int32(mpSize) return uint32(mpSize)
} }

View File

@ -21,7 +21,6 @@ import (
"fmt" "fmt"
"strings" "strings"
"github.com/pkg/errors"
"go.opentelemetry.io/otel" "go.opentelemetry.io/otel"
"go.opentelemetry.io/otel/trace" "go.opentelemetry.io/otel/trace"
"google.golang.org/grpc" "google.golang.org/grpc"
@ -36,9 +35,7 @@ import (
"d7y.io/dragonfly/v2/pkg/rpc/base" "d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/cdnsystem" "d7y.io/dragonfly/v2/pkg/rpc/cdnsystem"
cdnserver "d7y.io/dragonfly/v2/pkg/rpc/cdnsystem/server" 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/hostutils"
"d7y.io/dragonfly/v2/pkg/util/net/urlutils"
"d7y.io/dragonfly/v2/pkg/util/stringutils" "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 return svr.Server, nil
} }
func constructRegisterRequest(req *cdnsystem.SeedRequest) (*types.TaskRegisterRequest, error) { func constructRegisterRequest(req *cdnsystem.SeedRequest) *types.TaskRegisterRequest {
if err := checkSeedRequestParams(req); err != nil {
return nil, err
}
meta := req.UrlMeta meta := req.UrlMeta
header := make(map[string]string) header := make(map[string]string)
if meta != nil { if meta != nil {
if !stringutils.IsBlank(meta.Digest) { 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 header["digest"] = meta.Digest
} }
if !stringutils.IsBlank(meta.Range) { if !stringutils.IsBlank(meta.Range) {
@ -88,18 +78,7 @@ func constructRegisterRequest(req *cdnsystem.SeedRequest) (*types.TaskRegisterRe
Digest: header["digest"], Digest: header["digest"],
TaskID: req.TaskId, TaskID: req.TaskId,
Filter: strings.Split(req.UrlMeta.Filter, "&"), 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) { 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) logger.Infof("seeds task %s result success: %t", req.TaskId, err == nil)
}() }()
registerRequest, err := constructRegisterRequest(req) registerRequest := 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
}
// register task // register task
pieceChan, err := css.taskMgr.Register(ctx, registerRequest) pieceChan, err := css.taskMgr.Register(ctx, registerRequest)
if err != nil { if err != nil {
@ -141,7 +115,7 @@ func (css *server) ObtainSeeds(ctx context.Context, req *cdnsystem.SeedRequest,
PeerId: peerID, PeerId: peerID,
HostUuid: idgen.CDNHostID(hostutils.FQDNHostname, int32(css.cfg.ListenPort)), HostUuid: idgen.CDNHostID(hostutils.FQDNHostname, int32(css.cfg.ListenPort)),
PieceInfo: &base.PieceInfo{ PieceInfo: &base.PieceInfo{
PieceNum: piece.PieceNum, PieceNum: int32(piece.PieceNum),
RangeStart: piece.PieceRange.StartIndex, RangeStart: piece.PieceRange.StartIndex,
RangeSize: piece.PieceLen, RangeSize: piece.PieceLen,
PieceMd5: piece.PieceMd5, PieceMd5: piece.PieceMd5,
@ -186,11 +160,6 @@ func (css *server) GetPieceTasks(ctx context.Context, req *base.PieceTaskRequest
} }
}() }()
logger.Infof("get piece tasks: %+v", req) 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) task, err := css.taskMgr.Get(req.TaskId)
if err != nil { if err != nil {
if cdnerrors.IsDataNotFound(err) { if cdnerrors.IsDataNotFound(err) {
@ -214,11 +183,11 @@ func (css *server) GetPieceTasks(ctx context.Context, req *base.PieceTaskRequest
return nil, err return nil, err
} }
pieceInfos := make([]*base.PieceInfo, 0) pieceInfos := make([]*base.PieceInfo, 0)
var count int32 = 0 var count uint32 = 0
for _, piece := range pieces { for _, piece := range pieces {
if piece.PieceNum >= req.StartNum && (count < req.Limit || req.Limit == 0) { if piece.PieceNum >= req.StartNum && (count < req.Limit || req.Limit == 0) {
p := &base.PieceInfo{ p := &base.PieceInfo{
PieceNum: piece.PieceNum, PieceNum: int32(piece.PieceNum),
RangeStart: piece.PieceRange.StartIndex, RangeStart: piece.PieceRange.StartIndex,
RangeSize: piece.PieceLen, RangeSize: piece.PieceLen,
PieceMd5: piece.PieceMd5, PieceMd5: piece.PieceMd5,
@ -241,19 +210,3 @@ func (css *server) GetPieceTasks(ctx context.Context, req *base.PieceTaskRequest
span.SetAttributes(config.AttributePiecePacketResult.String(pp.String())) span.SetAttributes(config.AttributePiecePacketResult.String(pp.String()))
return pp, nil 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
}

View File

@ -23,7 +23,6 @@ import (
"d7y.io/dragonfly/v2/cdn/config" "d7y.io/dragonfly/v2/cdn/config"
"d7y.io/dragonfly/v2/cdn/supervisor" "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/base"
"d7y.io/dragonfly/v2/pkg/rpc/cdnsystem" "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)
}
})
}
}

View File

@ -185,7 +185,7 @@ func (cd *cacheDetector) parseByReadFile(taskID string, metaData *storage.FileMe
var breakPoint uint64 = 0 var breakPoint uint64 = 0
pieceMetaRecords := make([]*storage.PieceMetaRecord, 0, len(tempRecords)) pieceMetaRecords := make([]*storage.PieceMetaRecord, 0, len(tempRecords))
for index := range tempRecords { for index := range tempRecords {
if int32(index) != tempRecords[index].PieceNum { if uint32(index) != tempRecords[index].PieceNum {
break break
} }
// read content // read content

View File

@ -167,8 +167,8 @@ func (cw *cacheWriter) writerPool(ctx context.Context, wg *sync.WaitGroup, routi
start := uint64(p.pieceNum) * uint64(p.pieceSize) start := uint64(p.pieceNum) * uint64(p.pieceSize)
end := start + uint64(pieceLen) - 1 end := start + uint64(pieceLen) - 1
pieceRecord := &storage.PieceMetaRecord{ pieceRecord := &storage.PieceMetaRecord{
PieceNum: p.pieceNum, PieceNum: uint32(p.pieceNum),
PieceLen: int32(pieceLen), PieceLen: uint32(pieceLen),
Md5: digestutils.ToHashString(pieceMd5), Md5: digestutils.ToHashString(pieceMd5),
Range: &rangeutils.Range{ Range: &rangeutils.Range{
StartIndex: start, StartIndex: start,

View File

@ -62,7 +62,7 @@ func (re *reporter) reportPieceMetaRecord(ctx context.Context, taskID string, re
from string) error { from string) error {
// report cache pieces status // report cache pieces status
logger.DownloaderLogger.Info(taskID, logger.DownloaderLogger.Info(taskID,
zap.Int32("pieceNum", record.PieceNum), zap.Uint32("pieceNum", record.PieceNum),
zap.String("md5", record.Md5), zap.String("md5", record.Md5),
zap.String("from", from)) zap.String("from", from))
return re.progress.PublishPiece(ctx, taskID, convertPieceMeta2SeedPiece(record)) 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 { func convertPieceMeta2SeedPiece(record *storage.PieceMetaRecord) *types.SeedPiece {
return &types.SeedPiece{ return &types.SeedPiece{
PieceStyle: record.PieceStyle, PieceStyle: record.PieceStyle,
PieceNum: record.PieceNum, PieceNum: uint32(record.PieceNum),
PieceMd5: record.Md5, PieceMd5: record.Md5,
PieceRange: record.Range, PieceRange: record.Range,
OriginRange: record.OriginRange, OriginRange: record.OriginRange,

View File

@ -97,8 +97,8 @@ type FileMetaData struct {
// PieceMetaRecord meta data of piece // PieceMetaRecord meta data of piece
type PieceMetaRecord struct { type PieceMetaRecord struct {
PieceNum int32 `json:"pieceNum"` // piece Num start from 0 PieceNum uint32 `json:"pieceNum"` // piece Num start from 0
PieceLen int32 `json:"pieceLen"` // 存储到存储介质的真实长度 PieceLen uint32 `json:"pieceLen"` // 存储到存储介质的真实长度
Md5 string `json:"md5"` // for transported piece content不是origin source 的 md5是真是存储到存储介质后的md5为了读取数据文件时方便校验完整性 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 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 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 nil, errors.Wrapf(err, "invalid pieceStyle: %s", fields[5])
} }
return &PieceMetaRecord{ return &PieceMetaRecord{
PieceNum: int32(pieceNum), PieceNum: uint32(pieceNum),
PieceLen: int32(pieceLen), PieceLen: uint32(pieceLen),
Md5: md5, Md5: md5,
Range: pieceRange, Range: pieceRange,
OriginRange: originRange, OriginRange: originRange,

View File

@ -119,7 +119,7 @@ func (tm *Manager) addOrUpdateTask(ctx context.Context, request *types.TaskRegis
// calculate piece size and update the PieceSize and PieceTotal // calculate piece size and update the PieceSize and PieceTotal
if task.PieceSize <= 0 { if task.PieceSize <= 0 {
pieceSize := cdnutil.ComputePieceSize(task.SourceFileLength) pieceSize := cdnutil.ComputePieceSize(task.SourceFileLength)
task.PieceSize = pieceSize task.PieceSize = int32(pieceSize)
} }
if err := tm.taskStore.Add(task.TaskID, task); err != nil { if err := tm.taskStore.Add(task.TaskID, task); err != nil {
return nil, err return nil, err

View File

@ -20,11 +20,11 @@ import "d7y.io/dragonfly/v2/pkg/util/rangeutils"
type SeedPiece struct { type SeedPiece struct {
PieceStyle PieceFormat `json:"piece_style"` // 0: PlainUnspecified PieceStyle PieceFormat `json:"piece_style"` // 0: PlainUnspecified
PieceNum int32 `json:"piece_num"` PieceNum uint32 `json:"piece_num"`
PieceMd5 string `json:"piece_md_5"` PieceMd5 string `json:"piece_md_5"`
PieceRange *rangeutils.Range `json:"piece_range"` PieceRange *rangeutils.Range `json:"piece_range"`
OriginRange *rangeutils.Range `json:"origin_range"` OriginRange *rangeutils.Range `json:"origin_range"`
PieceLen int32 `json:"piece_len"` PieceLen uint32 `json:"piece_len"`
} }
type PieceFormat int8 type PieceFormat int8

View File

@ -89,7 +89,7 @@ type peerTask struct {
md5 string md5 string
contentLength *atomic.Int64 contentLength *atomic.Int64
completedLength *atomic.Int64 completedLength *atomic.Int64
usedTraffic *atomic.Int64 usedTraffic *atomic.Uint64
//sizeScope base.SizeScope //sizeScope base.SizeScope
singlePiece *scheduler.SinglePiece singlePiece *scheduler.SinglePiece
@ -162,11 +162,11 @@ func (pt *peerTask) SetContentLength(i int64) error {
return pt.setContentLengthFunc(i) return pt.setContentLengthFunc(i)
} }
func (pt *peerTask) AddTraffic(n int64) { func (pt *peerTask) AddTraffic(n uint64) {
pt.usedTraffic.Add(n) pt.usedTraffic.Add(n)
} }
func (pt *peerTask) GetTraffic() int64 { func (pt *peerTask) GetTraffic() uint64 {
return pt.usedTraffic.Load() return pt.usedTraffic.Load()
} }
@ -393,11 +393,11 @@ func (pt *peerTask) pullPiecesFromPeers(cleanUnfinishedFunc func()) {
var ( var (
num int32 num int32
ok bool ok bool
limit int32 limit uint32
initialized bool initialized bool
pieceRequestCh chan *DownloadPieceRequest pieceRequestCh chan *DownloadPieceRequest
// keep same size with pt.failedPieceCh for avoiding dead-lock // keep same size with pt.failedPieceCh for avoiding dead-lock
pieceBufferSize = int32(config.DefaultPieceChanSize) pieceBufferSize = uint32(config.DefaultPieceChanSize)
) )
limit = pieceBufferSize limit = pieceBufferSize
loop: loop:
@ -437,7 +437,7 @@ loop:
&base.PieceTaskRequest{ &base.PieceTaskRequest{
TaskId: pt.taskID, TaskId: pt.taskID,
SrcPid: pt.peerID, SrcPid: pt.peerID,
StartNum: num, StartNum: uint32(num),
Limit: limit, 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) pt.contentLength.Store(piecePacket.ContentLength)
if pt.contentLength.Load() > 0 { if pt.contentLength.Load() > 0 {
pt.span.SetAttributes(config.AttributeTaskContentLength.Int64(pt.contentLength.Load())) 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) { if code == base.Code_CDNTaskNotFound && curPeerPacket == pt.peerPacket.Load().(*scheduler.PeerPacket) {
span.AddEvent("retry for CDNTaskNotFound") span.AddEvent("retry for CdnTaskNotFound")
goto retry goto retry
} }
return nil, err return nil, err
@ -875,7 +875,7 @@ func (pt *peerTask) getNextPieceNum(cur int32) int32 {
// double check, re-search not success or not requested pieces // double check, re-search not success or not requested pieces
for i = int32(0); pt.requestedPieces.IsSet(i); i++ { 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 return -1
} }
} }

View File

@ -200,7 +200,7 @@ func newFilePeerTask(ctx context.Context,
schedulerClient: schedulerClient, schedulerClient: schedulerClient,
limiter: limiter, limiter: limiter,
completedLength: atomic.NewInt64(0), completedLength: atomic.NewInt64(0),
usedTraffic: atomic.NewInt64(0), usedTraffic: atomic.NewUint64(0),
SugaredLoggerOnWith: logger.With("peer", request.PeerId, "task", result.TaskId, "component", "filePeerTask"), SugaredLoggerOnWith: logger.With("peer", request.PeerId, "task", result.TaskId, "component", "filePeerTask"),
}, },
} }

View File

@ -51,7 +51,7 @@ func (p *filePeerTaskCallback) Init(pt Task) error {
Destination: p.req.Output, Destination: p.req.Output,
}, },
ContentLength: pt.GetContentLength(), ContentLength: pt.GetContentLength(),
TotalPieces: pt.GetTotalPieces(), TotalPieces: int32(pt.GetTotalPieces()),
PieceMd5Sign: pt.GetPieceMd5Sign(), PieceMd5Sign: pt.GetPieceMd5Sign(),
}) })
if err != nil { if err != nil {
@ -69,7 +69,7 @@ func (p *filePeerTaskCallback) Update(pt Task) error {
TaskID: pt.GetTaskID(), TaskID: pt.GetTaskID(),
}, },
ContentLength: pt.GetContentLength(), ContentLength: pt.GetContentLength(),
TotalPieces: pt.GetTotalPieces(), TotalPieces: int32(pt.GetTotalPieces()),
PieceMd5Sign: pt.GetPieceMd5Sign(), PieceMd5Sign: pt.GetPieceMd5Sign(),
}) })
if err != nil { if err != nil {
@ -90,7 +90,7 @@ func (p *filePeerTaskCallback) Done(pt Task) error {
Destination: p.req.Output, Destination: p.req.Output,
}, },
MetadataOnly: false, MetadataOnly: false,
TotalPieces: pt.GetTotalPieces(), TotalPieces: int32(pt.GetTotalPieces()),
}) })
if e != nil { if e != nil {
return e return e

View File

@ -67,7 +67,7 @@ func TestFilePeerTask_BackSource_WithContentLength(t *testing.T) {
componentsOption{ componentsOption{
taskID: taskID, taskID: taskID,
contentLength: int64(mockContentLength), contentLength: int64(mockContentLength),
pieceSize: int32(pieceSize), pieceSize: uint32(pieceSize),
pieceParallelCount: pieceParallelCount, pieceParallelCount: pieceParallelCount,
}) })
defer storageManager.CleanUp() defer storageManager.CleanUp()
@ -184,7 +184,7 @@ func TestFilePeerTask_BackSource_WithoutContentLength(t *testing.T) {
componentsOption{ componentsOption{
taskID: taskID, taskID: taskID,
contentLength: int64(mockContentLength), contentLength: int64(mockContentLength),
pieceSize: int32(pieceSize), pieceSize: uint32(pieceSize),
pieceParallelCount: pieceParallelCount, pieceParallelCount: pieceParallelCount,
}) })
defer storageManager.CleanUp() defer storageManager.CleanUp()

View File

@ -71,8 +71,8 @@ type Task interface {
// SetContentLength will be called after download completed, when download from source without content length // SetContentLength will be called after download completed, when download from source without content length
SetContentLength(int64) error SetContentLength(int64) error
SetCallback(TaskCallback) SetCallback(TaskCallback)
AddTraffic(int64) AddTraffic(uint64)
GetTraffic() int64 GetTraffic() uint64
SetPieceMd5Sign(string) SetPieceMd5Sign(string)
GetPieceMd5Sign() string GetPieceMd5Sign() string
} }

View File

@ -123,7 +123,7 @@ func (m *MockTask) EXPECT() *MockTaskMockRecorder {
} }
// AddTraffic mocks base method. // AddTraffic mocks base method.
func (m *MockTask) AddTraffic(arg0 int64) { func (m *MockTask) AddTraffic(arg0 uint64) {
m.ctrl.T.Helper() m.ctrl.T.Helper()
m.ctrl.Call(m, "AddTraffic", arg0) m.ctrl.Call(m, "AddTraffic", arg0)
} }
@ -219,10 +219,10 @@ func (mr *MockTaskMockRecorder) GetTotalPieces() *gomock.Call {
} }
// GetTraffic mocks base method. // GetTraffic mocks base method.
func (m *MockTask) GetTraffic() int64 { func (m *MockTask) GetTraffic() uint64 {
m.ctrl.T.Helper() m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "GetTraffic") ret := m.ctrl.Call(m, "GetTraffic")
ret0, _ := ret[0].(int64) ret0, _ := ret[0].(uint64)
return ret0 return ret0
} }

View File

@ -55,7 +55,7 @@ import (
type componentsOption struct { type componentsOption struct {
taskID string taskID string
contentLength int64 contentLength int64
pieceSize int32 pieceSize uint32
pieceParallelCount int32 pieceParallelCount int32
peerPacketDelay []time.Duration peerPacketDelay []time.Duration
} }
@ -67,18 +67,18 @@ func setupPeerTaskManagerComponents(ctrl *gomock.Controller, opt componentsOptio
var daemon = mock_daemon.NewMockDaemonServer(ctrl) 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) { daemon.EXPECT().GetPieceTasks(gomock.Any(), gomock.Any()).AnyTimes().DoAndReturn(func(ctx context.Context, request *base.PieceTaskRequest) (*base.PiecePacket, error) {
var tasks []*base.PieceInfo 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) start := opt.pieceSize * (request.StartNum + i)
if int64(start)+1 > opt.contentLength { if int64(start)+1 > opt.contentLength {
break break
} }
size := opt.pieceSize size := opt.pieceSize
if int64(start+opt.pieceSize) > opt.contentLength { if int64(start+opt.pieceSize) > opt.contentLength {
size = int32(opt.contentLength) - start size = uint32(opt.contentLength) - start
} }
tasks = append(tasks, tasks = append(tasks,
&base.PieceInfo{ &base.PieceInfo{
PieceNum: request.StartNum + i, PieceNum: int32(request.StartNum + i),
RangeStart: uint64(start), RangeStart: uint64(start),
RangeSize: size, RangeSize: size,
PieceMd5: "", PieceMd5: "",
@ -191,7 +191,7 @@ func TestPeerTaskManager_StartFilePeerTask(t *testing.T) {
componentsOption{ componentsOption{
taskID: taskID, taskID: taskID,
contentLength: int64(mockContentLength), contentLength: int64(mockContentLength),
pieceSize: int32(pieceSize), pieceSize: uint32(pieceSize),
pieceParallelCount: pieceParallelCount, pieceParallelCount: pieceParallelCount,
}) })
defer storageManager.CleanUp() defer storageManager.CleanUp()
@ -274,7 +274,7 @@ func TestPeerTaskManager_StartStreamPeerTask(t *testing.T) {
componentsOption{ componentsOption{
taskID: taskID, taskID: taskID,
contentLength: int64(mockContentLength), contentLength: int64(mockContentLength),
pieceSize: int32(pieceSize), pieceSize: uint32(pieceSize),
pieceParallelCount: pieceParallelCount, pieceParallelCount: pieceParallelCount,
}) })
defer storageManager.CleanUp() defer storageManager.CleanUp()
@ -353,7 +353,7 @@ func TestPeerTaskManager_StartStreamPeerTask_BackSource(t *testing.T) {
componentsOption{ componentsOption{
taskID: taskID, taskID: taskID,
contentLength: int64(mockContentLength), contentLength: int64(mockContentLength),
pieceSize: int32(pieceSize), pieceSize: uint32(pieceSize),
pieceParallelCount: pieceParallelCount, pieceParallelCount: pieceParallelCount,
peerPacketDelay: []time.Duration{time.Second}, peerPacketDelay: []time.Duration{time.Second},
}) })

View File

@ -176,7 +176,7 @@ func newStreamPeerTask(ctx context.Context,
schedulerClient: schedulerClient, schedulerClient: schedulerClient,
limiter: limiter, limiter: limiter,
completedLength: atomic.NewInt64(0), completedLength: atomic.NewInt64(0),
usedTraffic: atomic.NewInt64(0), usedTraffic: atomic.NewUint64(0),
SugaredLoggerOnWith: logger.With("peer", request.PeerId, "task", result.TaskId, "component", "streamPeerTask"), SugaredLoggerOnWith: logger.With("peer", request.PeerId, "task", result.TaskId, "component", "streamPeerTask"),
}, },
} }

View File

@ -74,7 +74,7 @@ func setupBackSourcePartialComponents(ctrl *gomock.Controller, testBytes []byte,
if request.StartNum == 0 { if request.StartNum == 0 {
tasks = append(tasks, tasks = append(tasks,
&base.PieceInfo{ &base.PieceInfo{
PieceNum: request.StartNum, PieceNum: int32(request.StartNum),
RangeStart: uint64(0), RangeStart: uint64(0),
RangeSize: opt.pieceSize, RangeSize: opt.pieceSize,
PieceMd5: digestutils.Md5Bytes(testBytes[0:opt.pieceSize]), PieceMd5: digestutils.Md5Bytes(testBytes[0:opt.pieceSize]),
@ -197,7 +197,7 @@ func TestStreamPeerTask_BackSource_Partial_WithContentLength(t *testing.T) {
componentsOption{ componentsOption{
taskID: taskID, taskID: taskID,
contentLength: int64(mockContentLength), contentLength: int64(mockContentLength),
pieceSize: int32(pieceSize), pieceSize: uint32(pieceSize),
pieceParallelCount: pieceParallelCount, pieceParallelCount: pieceParallelCount,
}) })
defer storageManager.CleanUp() defer storageManager.CleanUp()
@ -228,8 +228,8 @@ func TestStreamPeerTask_BackSource_Partial_WithContentLength(t *testing.T) {
calculateDigest: true, calculateDigest: true,
storageManager: storageManager, storageManager: storageManager,
pieceDownloader: downloader, pieceDownloader: downloader,
computePieceSize: func(contentLength int64) int32 { computePieceSize: func(contentLength int64) uint32 {
return int32(pieceSize) return uint32(pieceSize)
}, },
} }
ptm := &peerTaskManager{ ptm := &peerTaskManager{

View File

@ -63,7 +63,7 @@ func TestStreamPeerTask_BackSource_WithContentLength(t *testing.T) {
componentsOption{ componentsOption{
taskID: taskID, taskID: taskID,
contentLength: int64(mockContentLength), contentLength: int64(mockContentLength),
pieceSize: int32(pieceSize), pieceSize: uint32(pieceSize),
pieceParallelCount: pieceParallelCount, pieceParallelCount: pieceParallelCount,
}) })
defer storageManager.CleanUp() defer storageManager.CleanUp()
@ -120,8 +120,8 @@ func TestStreamPeerTask_BackSource_WithContentLength(t *testing.T) {
&pieceManager{ &pieceManager{
storageManager: storageManager, storageManager: storageManager,
pieceDownloader: downloader, pieceDownloader: downloader,
computePieceSize: func(contentLength int64) int32 { computePieceSize: func(contentLength int64) uint32 {
return int32(pieceSize) return uint32(pieceSize)
}, },
}, },
req, req,
@ -169,7 +169,7 @@ func TestStreamPeerTask_BackSource_WithoutContentLength(t *testing.T) {
componentsOption{ componentsOption{
taskID: taskID, taskID: taskID,
contentLength: int64(mockContentLength), contentLength: int64(mockContentLength),
pieceSize: int32(pieceSize), pieceSize: uint32(pieceSize),
pieceParallelCount: pieceParallelCount, pieceParallelCount: pieceParallelCount,
}) })
defer storageManager.CleanUp() defer storageManager.CleanUp()
@ -226,8 +226,8 @@ func TestStreamPeerTask_BackSource_WithoutContentLength(t *testing.T) {
&pieceManager{ &pieceManager{
storageManager: storageManager, storageManager: storageManager,
pieceDownloader: downloader, pieceDownloader: downloader,
computePieceSize: func(contentLength int64) int32 { computePieceSize: func(contentLength int64) uint32 {
return int32(pieceSize) return uint32(pieceSize)
}, },
}, },
req, req,

View File

@ -51,7 +51,7 @@ func TestPieceDownloader_DownloadPiece(t *testing.T) {
taskID string taskID string
pieceRange string pieceRange string
rangeStart uint64 rangeStart uint64
rangeSize int32 rangeSize uint32
targetPieceData []byte targetPieceData []byte
}{ }{
{ {
@ -96,7 +96,7 @@ func TestPieceDownloader_DownloadPiece(t *testing.T) {
taskID: "task-2", taskID: "task-2",
pieceRange: fmt.Sprintf("bytes=512-%d", len(testData)-1), pieceRange: fmt.Sprintf("bytes=512-%d", len(testData)-1),
rangeStart: 512, rangeStart: 512,
rangeSize: int32(len(testData) - 512), rangeSize: uint32(len(testData) - 512),
targetPieceData: testData[512:], targetPieceData: testData[512:],
}, },
{ {

View File

@ -45,7 +45,7 @@ type pieceManager struct {
*rate.Limiter *rate.Limiter
storageManager storage.TaskStorageDriver storageManager storage.TaskStorageDriver
pieceDownloader PieceDownloader pieceDownloader PieceDownloader
computePieceSize func(contentLength int64) int32 computePieceSize func(contentLength int64) uint32
calculateDigest bool calculateDigest bool
} }
@ -148,7 +148,9 @@ func (pm *pieceManager) DownloadPiece(ctx context.Context, pt Task, request *Dow
end = time.Now().UnixNano() end = time.Now().UnixNano()
span.RecordError(err) span.RecordError(err)
span.End() span.End()
pt.AddTraffic(n) if n > 0 {
pt.AddTraffic(uint64(n))
}
if err != nil { if err != nil {
pt.Log().Errorf("put piece to storage failed, piece num: %d, wrote: %d, error: %s", pt.Log().Errorf("put piece to storage failed, piece num: %d, wrote: %d, error: %s",
request.piece.PieceNum, n, err) 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, 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 ( var (
success bool success bool
start = time.Now().UnixNano() start = time.Now().UnixNano()
@ -280,11 +282,13 @@ func (pm *pieceManager) processPieceFromSource(pt Task,
}, },
Reader: reader, Reader: reader,
}) })
if n != int64(size) { if n != int64(size) && n > 0 {
size = int32(n) size = uint32(n)
} }
end = time.Now().UnixNano() end = time.Now().UnixNano()
pt.AddTraffic(n) if n > 0 {
pt.AddTraffic(uint64(n))
}
if err != nil { if err != nil {
pt.Log().Errorf("put piece to storage failed, piece num: %d, wrote: %d, error: %s", pieceNum, n, err) pt.Log().Errorf("put piece to storage failed, piece num: %d, wrote: %d, error: %s", pieceNum, n, err)
return 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 // last piece, piece size maybe 0
if n < int64(size) { if n < int64(size) {
contentLength = int64(pieceNum*pieceSize) + n contentLength = int64(pieceNum)*int64(pieceSize) + n
if err := pm.storageManager.UpdateTask(ctx, if err := pm.storageManager.UpdateTask(ctx,
&storage.UpdateTaskRequest{ &storage.UpdateTaskRequest{
PeerTaskMetaData: storage.PeerTaskMetaData{ PeerTaskMetaData: storage.PeerTaskMetaData{
@ -383,7 +387,7 @@ func (pm *pieceManager) DownloadSource(ctx context.Context, pt Task, request *sc
offset := uint64(pieceNum) * uint64(pieceSize) offset := uint64(pieceNum) * uint64(pieceSize)
// calculate piece size for last piece // calculate piece size for last piece
if contentLength > 0 && int64(offset)+int64(size) > contentLength { if contentLength > 0 && int64(offset)+int64(size) > contentLength {
size = int32(contentLength - int64(offset)) size = uint32(contentLength - int64(offset))
} }
log.Debugf("download piece %d", pieceNum) log.Debugf("download piece %d", pieceNum)

View File

@ -75,7 +75,7 @@ func TestPieceManager_DownloadSource(t *testing.T) {
testCases := []struct { testCases := []struct {
name string name string
pieceSize int32 pieceSize uint32
withContentLength bool withContentLength bool
checkDigest bool checkDigest bool
}{ }{
@ -105,22 +105,22 @@ func TestPieceManager_DownloadSource(t *testing.T) {
}, },
{ {
name: "one pieces with content length case 1", name: "one pieces with content length case 1",
pieceSize: int32(len(testBytes)), pieceSize: uint32(len(testBytes)),
withContentLength: true, withContentLength: true,
}, },
{ {
name: "one pieces without content length case 1", name: "one pieces without content length case 1",
pieceSize: int32(len(testBytes)), pieceSize: uint32(len(testBytes)),
withContentLength: false, withContentLength: false,
}, },
{ {
name: "one pieces with content length case 2", name: "one pieces with content length case 2",
pieceSize: int32(len(testBytes)) + 1, pieceSize: uint32(len(testBytes)) + 1,
withContentLength: true, withContentLength: true,
}, },
{ {
name: "one pieces without content length case 2", name: "one pieces without content length case 2",
pieceSize: int32(len(testBytes)) + 1, pieceSize: uint32(len(testBytes)) + 1,
withContentLength: false, withContentLength: false,
}, },
} }
@ -143,7 +143,7 @@ func TestPieceManager_DownloadSource(t *testing.T) {
func() string { func() string {
return taskID 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( mockPeerTask.EXPECT().ReportPieceResult(gomock.Any()).AnyTimes().DoAndReturn(
func(result *pieceTaskResult) error { func(result *pieceTaskResult) error {
return nil return nil
@ -180,7 +180,7 @@ func TestPieceManager_DownloadSource(t *testing.T) {
pm, err := NewPieceManager(storageManager, pieceDownloadTimeout) pm, err := NewPieceManager(storageManager, pieceDownloadTimeout)
assert.Nil(err) assert.Nil(err)
pm.(*pieceManager).computePieceSize = func(length int64) int32 { pm.(*pieceManager).computePieceSize = func(length int64) uint32 {
return tc.pieceSize return tc.pieceSize
} }

View File

@ -25,6 +25,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/distribution/distribution/v3/uuid"
"github.com/golang/mock/gomock" "github.com/golang/mock/gomock"
"github.com/phayes/freeport" "github.com/phayes/freeport"
testifyassert "github.com/stretchr/testify/assert" testifyassert "github.com/stretchr/testify/assert"
@ -33,12 +34,14 @@ import (
"d7y.io/dragonfly/v2/client/daemon/peer" "d7y.io/dragonfly/v2/client/daemon/peer"
mock_peer "d7y.io/dragonfly/v2/client/daemon/test/mock/peer" mock_peer "d7y.io/dragonfly/v2/client/daemon/test/mock/peer"
mock_storage "d7y.io/dragonfly/v2/client/daemon/test/mock/storage" 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/basic/dfnet"
"d7y.io/dragonfly/v2/pkg/rpc/base" "d7y.io/dragonfly/v2/pkg/rpc/base"
dfdaemongrpc "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon" dfdaemongrpc "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon"
dfclient "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon/client" dfclient "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon/client"
dfdaemonserver "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon/server" dfdaemonserver "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon/server"
"d7y.io/dragonfly/v2/pkg/rpc/scheduler" "d7y.io/dragonfly/v2/pkg/rpc/scheduler"
"d7y.io/dragonfly/v2/pkg/util/net/iputils"
) )
func TestMain(m *testing.M) { func TestMain(m *testing.M) {
@ -100,11 +103,15 @@ func TestDownloadManager_ServeDownload(t *testing.T) {
}) })
assert.Nil(err, "grpc dial should be ok") assert.Nil(err, "grpc dial should be ok")
request := &dfdaemongrpc.DownRequest{ request := &dfdaemongrpc.DownRequest{
Url: "http://localhost/test", Uuid: uuid.Generate().String(),
Output: "./testdata/file1", Url: "http://localhost/test",
Output: "./testdata/file1",
DisableBackSource: false,
UrlMeta: &base.UrlMeta{ UrlMeta: &base.UrlMeta{
Tag: "unit test", Tag: "unit test",
}, },
Pattern: "p2p",
Callsystem: "",
} }
down, err := client.Download(context.Background(), request) down, err := client.Download(context.Background(), request)
assert.Nil(err, "client download grpc call should be ok") assert.Nil(err, "client download grpc call should be ok")
@ -130,16 +137,16 @@ func TestDownloadManager_ServePeer(t *testing.T) {
ctrl := gomock.NewController(t) ctrl := gomock.NewController(t)
defer ctrl.Finish() defer ctrl.Finish()
var maxPieceNum int32 = 10 var maxPieceNum uint32 = 10
mockStorageManger := mock_storage.NewMockManager(ctrl) mockStorageManger := mock_storage.NewMockManager(ctrl)
mockStorageManger.EXPECT().GetPieces(gomock.Any(), gomock.Any()).AnyTimes().DoAndReturn(func(ctx context.Context, req *base.PieceTaskRequest) (*base.PiecePacket, error) { mockStorageManger.EXPECT().GetPieces(gomock.Any(), gomock.Any()).AnyTimes().DoAndReturn(func(ctx context.Context, req *base.PieceTaskRequest) (*base.PiecePacket, error) {
var ( var (
pieces []*base.PieceInfo pieces []*base.PieceInfo
pieceSize = int32(1024) pieceSize = uint32(1024)
) )
for i := req.StartNum; i < req.Limit+req.StartNum && i < maxPieceNum; i++ { for i := req.StartNum; i < req.Limit+req.StartNum && i < maxPieceNum; i++ {
pieces = append(pieces, &base.PieceInfo{ pieces = append(pieces, &base.PieceInfo{
PieceNum: i, PieceNum: int32(i),
RangeStart: uint64(i * pieceSize), RangeStart: uint64(i * pieceSize),
RangeSize: pieceSize, RangeSize: pieceSize,
PieceMd5: "", PieceMd5: "",
@ -191,6 +198,9 @@ func TestDownloadManager_ServePeer(t *testing.T) {
}{ }{
{ {
request: &base.PieceTaskRequest{ request: &base.PieceTaskRequest{
TaskId: idgen.TaskID("http://www.test.com", &base.UrlMeta{}),
SrcPid: idgen.PeerID(iputils.IPv4),
DstPid: idgen.PeerID(iputils.IPv4),
StartNum: 0, StartNum: 0,
Limit: 1, Limit: 1,
}, },
@ -199,6 +209,9 @@ func TestDownloadManager_ServePeer(t *testing.T) {
}, },
{ {
request: &base.PieceTaskRequest{ request: &base.PieceTaskRequest{
TaskId: idgen.TaskID("http://www.test.com", &base.UrlMeta{}),
SrcPid: idgen.PeerID(iputils.IPv4),
DstPid: idgen.PeerID(iputils.IPv4),
StartNum: 0, StartNum: 0,
Limit: 4, Limit: 4,
}, },
@ -207,6 +220,9 @@ func TestDownloadManager_ServePeer(t *testing.T) {
}, },
{ {
request: &base.PieceTaskRequest{ request: &base.PieceTaskRequest{
TaskId: idgen.TaskID("http://www.test.com", &base.UrlMeta{}),
SrcPid: idgen.PeerID(iputils.IPv4),
DstPid: idgen.PeerID(iputils.IPv4),
StartNum: 8, StartNum: 8,
Limit: 1, Limit: 1,
}, },
@ -215,6 +231,9 @@ func TestDownloadManager_ServePeer(t *testing.T) {
}, },
{ {
request: &base.PieceTaskRequest{ request: &base.PieceTaskRequest{
TaskId: idgen.TaskID("http://www.test.com", &base.UrlMeta{}),
SrcPid: idgen.PeerID(iputils.IPv4),
DstPid: idgen.PeerID(iputils.IPv4),
StartNum: 8, StartNum: 8,
Limit: 4, Limit: 4,
}, },

View File

@ -160,7 +160,7 @@ func (t *localTaskStore) UpdateTask(ctx context.Context, req *UpdateTaskRequest)
} }
if req.GenPieceDigest { if req.GenPieceDigest {
var pieceDigests []string 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) pieceDigests = append(pieceDigests, t.Pieces[i].Md5)
} }
@ -184,7 +184,7 @@ func (t *localTaskStore) ValidateDigest(*PeerTaskMetaData) error {
} }
var pieceDigests []string 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) pieceDigests = append(pieceDigests, t.Pieces[i].Md5)
} }
@ -324,16 +324,16 @@ func (t *localTaskStore) GetPieces(ctx context.Context, req *base.PieceTaskReque
t.RLock() t.RLock()
defer t.RUnlock() defer t.RUnlock()
t.touch() 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) t.Errorf("invalid start num: %d", req.StartNum)
return nil, dferrors.ErrInvalidArgument return nil, dferrors.ErrInvalidArgument
} }
for i := int32(0); i < req.Limit; i++ { for i := int32(0); i < int32(req.Limit); i++ {
if piece, ok := t.Pieces[req.StartNum+i]; ok { if piece, ok := t.Pieces[int32(req.StartNum)+i]; ok {
pieces = append(pieces, &base.PieceInfo{ pieces = append(pieces, &base.PieceInfo{
PieceNum: piece.Num, PieceNum: piece.Num,
RangeStart: uint64(piece.Range.Start), RangeStart: uint64(piece.Range.Start),
RangeSize: int32(piece.Range.Length), RangeSize: uint32(piece.Range.Length),
PieceMd5: piece.Md5, PieceMd5: piece.Md5,
PieceOffset: piece.Offset, PieceOffset: piece.Offset,
PieceStyle: piece.Style, PieceStyle: piece.Style,

View File

@ -416,7 +416,7 @@ func (s *storageManager) FindCompletedTask(taskID string) *ReusePeerTask {
TaskID: taskID, TaskID: taskID,
}, },
ContentLength: t.ContentLength, ContentLength: t.ContentLength,
TotalPieces: t.TotalPieces, TotalPieces: int32(t.TotalPieces),
} }
} }
return nil return nil

View File

@ -200,7 +200,7 @@ func newDownRequest(cfg *config.DfgetConfig, hdr map[string]string) *dfdaemon.Do
return &dfdaemon.DownRequest{ return &dfdaemon.DownRequest{
Url: cfg.URL, Url: cfg.URL,
Output: cfg.Output, Output: cfg.Output,
Timeout: int64(cfg.Timeout), Timeout: uint64(cfg.Timeout),
Limit: float64(cfg.RateLimit), Limit: float64(cfg.RateLimit),
DisableBackSource: cfg.DisableBackSource, DisableBackSource: cfg.DisableBackSource,
UrlMeta: &base.UrlMeta{ UrlMeta: &base.UrlMeta{

2
go.mod
View File

@ -35,7 +35,7 @@ require (
github.com/google/go-querystring v1.1.0 // indirect github.com/google/go-querystring v1.1.0 // indirect
github.com/google/uuid v1.2.0 github.com/google/uuid v1.2.0
github.com/gorilla/mux v1.8.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/grpc-ecosystem/go-grpc-prometheus v1.2.0
github.com/jackc/pgproto3/v2 v2.1.1 // indirect github.com/jackc/pgproto3/v2 v2.1.1 // indirect
github.com/jarcoal/httpmock v1.0.8 github.com/jarcoal/httpmock v1.0.8

5
go.sum
View File

@ -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/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs=
github.com/bitly/go-simplejson v0.5.0/go.mod h1:cXHtHw4XUPsvGaxgjIAn8PhEWG9NfngEKAMDJEczWVA= 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/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/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 h1:L/QXpzIa3pOvUGt1D1lA5KjYhPBAN/3iWdP7xeFS9F0=
github.com/bradfitz/gomemcache v0.0.0-20190913173617-a41fca850d0b/go.mod h1:H0wQNHz2YrLsuXOZozoeDmnHXkNCRmMW0gwFWDfEZDA= 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/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 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.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 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho=
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= 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= 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.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 h1:9yCKha/T5XdGtO0q9Q9a6T5NUCsTn/DrBg0D7ufOcFM=
github.com/opencontainers/image-spec v1.0.2/go.mod h1:BtxoFyWECRxE4U/7sNtV5W15zMzWCbyJoFRP3s7yZA0= 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 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs=
github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= 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= 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-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-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-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-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-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=

View File

@ -51,6 +51,6 @@ type GetCDNClustersQuery struct {
} }
type CDNClusterConfig 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"` NetTopology string `yaml:"netTopology" mapstructure:"netTopology" json:"net_topology"`
} }

View File

@ -55,7 +55,7 @@ type SchedulerClusterConfig struct {
} }
type SchedulerClusterClientConfig 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 { type SchedulerClusterScopes struct {

View File

@ -22,6 +22,7 @@
package base package base
import ( import (
_ "github.com/envoyproxy/protoc-gen-validate/validate"
protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl" protoimpl "google.golang.org/protobuf/runtime/protoimpl"
reflect "reflect" 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"` 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"` DstPid string `protobuf:"bytes,3,opt,name=dst_pid,json=dstPid,proto3" json:"dst_pid,omitempty"`
// piece number // piece number
StartNum int32 `protobuf:"varint,4,opt,name=start_num,json=startNum,proto3" json:"start_num,omitempty"` StartNum uint32 `protobuf:"varint,4,opt,name=start_num,json=startNum,proto3" json:"start_num,omitempty"`
// expected piece count // expected piece count, limit = 0 represent request pieces as many shards as possible
Limit int32 `protobuf:"varint,5,opt,name=limit,proto3" json:"limit,omitempty"` Limit uint32 `protobuf:"varint,5,opt,name=limit,proto3" json:"limit,omitempty"`
} }
func (x *PieceTaskRequest) Reset() { func (x *PieceTaskRequest) Reset() {
@ -529,14 +530,14 @@ func (x *PieceTaskRequest) GetDstPid() string {
return "" return ""
} }
func (x *PieceTaskRequest) GetStartNum() int32 { func (x *PieceTaskRequest) GetStartNum() uint32 {
if x != nil { if x != nil {
return x.StartNum return x.StartNum
} }
return 0 return 0
} }
func (x *PieceTaskRequest) GetLimit() int32 { func (x *PieceTaskRequest) GetLimit() uint32 {
if x != nil { if x != nil {
return x.Limit return x.Limit
} }
@ -548,9 +549,10 @@ type PieceInfo struct {
sizeCache protoimpl.SizeCache sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields 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"` 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"` 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"` 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"` 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"` 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 return 0
} }
func (x *PieceInfo) GetRangeSize() int32 { func (x *PieceInfo) GetRangeSize() uint32 {
if x != nil { if x != nil {
return x.RangeSize return x.RangeSize
} }
@ -640,8 +642,9 @@ type PiecePacket struct {
// ip:port // ip:port
DstAddr string `protobuf:"bytes,4,opt,name=dst_addr,json=dstAddr,proto3" json:"dst_addr,omitempty"` 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"` PieceInfos []*PieceInfo `protobuf:"bytes,5,rep,name=piece_infos,json=pieceInfos,proto3" json:"piece_infos,omitempty"`
// total piece count for url // 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"` 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"` ContentLength int64 `protobuf:"varint,7,opt,name=content_length,json=contentLength,proto3" json:"content_length,omitempty"`
// sha256 code of all piece md5 // sha256 code of all piece md5
PieceMd5Sign string `protobuf:"bytes,8,opt,name=piece_md5_sign,json=pieceMd5Sign,proto3" json:"piece_md5_sign,omitempty"` 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{ 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, 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, 0x61, 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x04, 0x62, 0x61, 0x73, 0x65, 0x1a,
0x47, 0x0a, 0x0b, 0x47, 0x72, 0x70, 0x63, 0x44, 0x66, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1e, 0x17, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61,
0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x51, 0x0a, 0x0b, 0x47, 0x72, 0x70, 0x63,
0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x18, 0x44, 0x66, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x28, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18,
0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64,
0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xcf, 0x01, 0x0a, 0x07, 0x55, 0x72, 0x6c, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, 0x01, 0x02, 0x10, 0x01, 0x52, 0x04, 0x63, 0x6f, 0x64,
0x4d, 0x65, 0x74, 0x61, 0x12, 0x16, 0x0a, 0x06, 0x64, 0x69, 0x67, 0x65, 0x73, 0x74, 0x18, 0x01, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01,
0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x93, 0x02, 0x0a, 0x07,
0x74, 0x61, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x74, 0x61, 0x67, 0x12, 0x14, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x3e, 0x0a, 0x06, 0x64, 0x69, 0x67, 0x65, 0x73,
0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x26, 0xfa, 0x42, 0x23, 0x72, 0x21, 0x32, 0x1c,
0x61, 0x6e, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x04, 0x5e, 0x28, 0x6d, 0x64, 0x35, 0x29, 0x7c, 0x28, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x29, 0x3a,
0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x31, 0x0a, 0x06, 0x5b, 0x41, 0x2d, 0x46, 0x61, 0x2d, 0x66, 0x30, 0x2d, 0x39, 0x5d, 0x2b, 0xd0, 0x01, 0x01, 0x52,
0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x62, 0x06, 0x64, 0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x74, 0x61, 0x67, 0x18, 0x02,
0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x74, 0x61, 0x67, 0x12, 0x30, 0x0a, 0x05, 0x72, 0x61, 0x6e,
0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x1a, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x1a, 0xfa, 0x42, 0x17, 0x72, 0x15, 0x32,
0x39, 0x0a, 0x0b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x10, 0x5e, 0x5b, 0x30, 0x2d, 0x39, 0x5d, 0x2b, 0x5c, 0x2d, 0x5e, 0x5b, 0x30, 0x2d, 0x39, 0x5d,
0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x2b, 0xd0, 0x01, 0x01, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x66,
0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c,
0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x63, 0x0a, 0x08, 0x48, 0x6f, 0x74, 0x65, 0x72, 0x12, 0x31, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x05, 0x20,
0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x70, 0x75, 0x5f, 0x72, 0x61, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65,
0x74, 0x69, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x02, 0x52, 0x08, 0x63, 0x70, 0x75, 0x52, 0x61, 0x74, 0x61, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06,
0x74, 0x69, 0x6f, 0x12, 0x1b, 0x0a, 0x09, 0x6d, 0x65, 0x6d, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x1a, 0x39, 0x0a, 0x0b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x52, 0x08, 0x6d, 0x65, 0x6d, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01,
0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x03, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
0x20, 0x01, 0x28, 0x02, 0x52, 0x09, 0x64, 0x69, 0x73, 0x6b, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x22, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38,
0x90, 0x01, 0x0a, 0x10, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x01, 0x22, 0x96, 0x01, 0x0a, 0x08, 0x48, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x2c,
0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x0a, 0x09, 0x63, 0x70, 0x75, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28,
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x02, 0x42, 0x0f, 0xfa, 0x42, 0x0c, 0x0a, 0x0a, 0x1d, 0x00, 0x00, 0x80, 0x3f, 0x2d, 0x00, 0x00,
0x07, 0x73, 0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x00, 0x00, 0x52, 0x08, 0x63, 0x70, 0x75, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x2c, 0x0a, 0x09,
0x73, 0x72, 0x63, 0x50, 0x69, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x6d, 0x65, 0x6d, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x42,
0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x0f, 0xfa, 0x42, 0x0c, 0x0a, 0x0a, 0x1d, 0x00, 0x00, 0x80, 0x3f, 0x2d, 0x00, 0x00, 0x00, 0x00,
0x1b, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x52, 0x08, 0x6d, 0x65, 0x6d, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x2e, 0x0a, 0x0a, 0x64, 0x69,
0x28, 0x05, 0x52, 0x08, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4e, 0x75, 0x6d, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x6b, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x42, 0x0f,
0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0xfa, 0x42, 0x0c, 0x0a, 0x0a, 0x1d, 0x00, 0x00, 0x80, 0x3f, 0x2d, 0x00, 0x00, 0x00, 0x00, 0x52,
0x69, 0x74, 0x22, 0xdb, 0x01, 0x0a, 0x09, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x09, 0x64, 0x69, 0x73, 0x6b, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x22, 0xbd, 0x01, 0x0a, 0x10, 0x50,
0x12, 0x1b, 0x0a, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x01, 0x20, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4e, 0x75, 0x6d, 0x12, 0x1f, 0x0a, 0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
0x0b, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49,
0x28, 0x04, 0x52, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, 0x1d, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x73, 0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01,
0x0a, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x73, 0x72, 0x63,
0x28, 0x05, 0x52, 0x09, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1b, 0x0a, 0x50, 0x69, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03,
0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x6d, 0x64, 0x35, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x64,
0x52, 0x08, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4d, 0x64, 0x35, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x69, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6e,
0x65, 0x63, 0x65, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 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, 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, 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, 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, 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, 0x22, 0x95, 0x02, 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, 0x12, 0x20, 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, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b,
0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20,
0x69, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x04, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x64, 0x73,
0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x64, 0x73, 0x74, 0x41, 0x64, 0x64, 0x72, 0x12, 0x30, 0x0a, 0x74, 0x50, 0x69, 0x64, 0x12, 0x22, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x61, 0x64, 0x64, 0x72,
0x0b, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x05, 0x20, 0x03, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52,
0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x07, 0x64, 0x73, 0x74, 0x41, 0x64, 0x64, 0x72, 0x12, 0x30, 0x0a, 0x0b, 0x70, 0x69, 0x65, 0x63,
0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0f, 0x2e,
0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x18, 0x06, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a,
0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65, 0x63, 0x65, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f,
0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52,
0x74, 0x68, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63,
0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x70, 0x69, 0x65, 0x63, 0x65, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x07, 0x20,
0x5f, 0x6d, 0x64, 0x35, 0x5f, 0x73, 0x69, 0x67, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x6e, 0x67,
0x0c, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4d, 0x64, 0x35, 0x53, 0x69, 0x67, 0x6e, 0x2a, 0xeb, 0x04, 0x74, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x6d, 0x64, 0x35, 0x5f,
0x0a, 0x04, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x58, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x73, 0x69, 0x67, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x69, 0x65, 0x63,
0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x07, 0x53, 0x75, 0x63, 0x65, 0x4d, 0x64, 0x35, 0x53, 0x69, 0x67, 0x6e, 0x2a, 0xeb, 0x04, 0x0a, 0x04, 0x43, 0x6f, 0x64,
0x63, 0x65, 0x73, 0x73, 0x10, 0xc8, 0x01, 0x12, 0x16, 0x0a, 0x11, 0x53, 0x65, 0x72, 0x76, 0x65, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x58, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49,
0x72, 0x55, 0x6e, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x10, 0xf4, 0x03, 0x12, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x07, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x10,
0x13, 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4c, 0x61, 0x63, 0x6b, 0x65, 0xc8, 0x01, 0x12, 0x16, 0x0a, 0x11, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x76,
0x64, 0x10, 0xe8, 0x07, 0x12, 0x0f, 0x0a, 0x0a, 0x42, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x10, 0xf4, 0x03, 0x12, 0x13, 0x0a, 0x0e, 0x52, 0x65,
0x73, 0x74, 0x10, 0xf8, 0x0a, 0x12, 0x15, 0x0a, 0x10, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4c, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0xe8, 0x07, 0x12,
0x6b, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0xfc, 0x0a, 0x12, 0x11, 0x0a, 0x0c, 0x0f, 0x0a, 0x0a, 0x42, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x10, 0xf8, 0x0a,
0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xdc, 0x0b, 0x12, 0x12, 0x15, 0x0a, 0x10, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x4e, 0x6f, 0x74, 0x46,
0x13, 0x0a, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x4f, 0x75, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0xfc, 0x0a, 0x12, 0x11, 0x0a, 0x0c, 0x55, 0x6e, 0x6b, 0x6e, 0x6f,
0x74, 0x10, 0xe0, 0x0b, 0x12, 0x10, 0x0a, 0x0b, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x45, 0x72, 0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xdc, 0x0b, 0x12, 0x13, 0x0a, 0x0e, 0x52, 0x65,
0x72, 0x6f, 0x72, 0x10, 0xa0, 0x1f, 0x12, 0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x4f, 0x75, 0x74, 0x10, 0xe0, 0x0b, 0x12,
0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0x0a, 0x0b, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xa0,
0x10, 0xa1, 0x1f, 0x12, 0x1a, 0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x68, 0x1f, 0x12, 0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65,
0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x10, 0xa2, 0x1f, 0x12, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa1, 0x1f, 0x12, 0x1a,
0x1a, 0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65,
0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x10, 0xa3, 0x1f, 0x12, 0x19, 0x0a, 0x14, 0x43, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x10, 0xa2, 0x1f, 0x12, 0x1a, 0x0a, 0x15, 0x43, 0x6c,
0x6c, 0x69, 0x65, 0x6e, 0x74, 0x57, 0x61, 0x69, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65,
0x61, 0x64, 0x79, 0x10, 0xa4, 0x1f, 0x12, 0x1c, 0x0a, 0x17, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x6c, 0x65, 0x64, 0x10, 0xa3, 0x1f, 0x12, 0x19, 0x0a, 0x14, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74,
0x50, 0x69, 0x65, 0x63, 0x65, 0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x46, 0x61, 0x69, 0x57, 0x61, 0x69, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x61, 0x64, 0x79, 0x10, 0xa4,
0x6c, 0x10, 0xa5, 0x1f, 0x12, 0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x1f, 0x12, 0x1c, 0x0a, 0x17, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa6, 0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa5, 0x1f, 0x12,
0x1f, 0x12, 0x0f, 0x0a, 0x0a, 0x53, 0x63, 0x68, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x88, 0x27, 0x12, 0x18, 0x0a, 0x13, 0x53, 0x63, 0x68, 0x65, 0x64, 0x4e, 0x65, 0x65, 0x64, 0x42, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa6, 0x1f, 0x12, 0x0f, 0x0a, 0x0a,
0x61, 0x63, 0x6b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x10, 0x89, 0x27, 0x12, 0x12, 0x0a, 0x0d, 0x53, 0x63, 0x68, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x88, 0x27, 0x12, 0x18, 0x0a,
0x53, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72, 0x47, 0x6f, 0x6e, 0x65, 0x10, 0x8a, 0x27, 0x13, 0x53, 0x63, 0x68, 0x65, 0x64, 0x4e, 0x65, 0x65, 0x64, 0x42, 0x61, 0x63, 0x6b, 0x53, 0x6f,
0x12, 0x16, 0x0a, 0x11, 0x53, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x6f, 0x74, 0x75, 0x72, 0x63, 0x65, 0x10, 0x89, 0x27, 0x12, 0x12, 0x0a, 0x0d, 0x53, 0x63, 0x68, 0x65, 0x64,
0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x8c, 0x27, 0x12, 0x23, 0x0a, 0x1e, 0x53, 0x63, 0x68, 0x65, 0x50, 0x65, 0x65, 0x72, 0x47, 0x6f, 0x6e, 0x65, 0x10, 0x8a, 0x27, 0x12, 0x16, 0x0a, 0x11, 0x53,
0x64, 0x50, 0x65, 0x65, 0x72, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64,
0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0x8d, 0x27, 0x12, 0x19, 0x0a, 0x10, 0x8c, 0x27, 0x12, 0x23, 0x0a, 0x1e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72,
0x14, 0x53, 0x63, 0x68, 0x65, 0x64, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x65, 0x70, 0x6f, 0x72,
0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x8e, 0x27, 0x12, 0x0d, 0x0a, 0x08, 0x43, 0x44, 0x4e, 0x45, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0x8d, 0x27, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x63, 0x68, 0x65,
0x72, 0x72, 0x6f, 0x72, 0x10, 0xf0, 0x2e, 0x12, 0x18, 0x0a, 0x13, 0x43, 0x44, 0x4e, 0x54, 0x61, 0x64, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x45, 0x72, 0x72, 0x6f, 0x72,
0x73, 0x6b, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x79, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xf1, 0x10, 0x8e, 0x27, 0x12, 0x0d, 0x0a, 0x08, 0x43, 0x44, 0x4e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10,
0x2e, 0x12, 0x18, 0x0a, 0x13, 0x43, 0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b, 0x44, 0x6f, 0x77, 0x6e, 0xf0, 0x2e, 0x12, 0x18, 0x0a, 0x13, 0x43, 0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x67,
0x6c, 0x6f, 0x61, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xf2, 0x2e, 0x12, 0x14, 0x0a, 0x0f, 0x43, 0x69, 0x73, 0x74, 0x72, 0x79, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xf1, 0x2e, 0x12, 0x18, 0x0a, 0x13,
0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x84, 0x43, 0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b, 0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x46,
0x32, 0x12, 0x18, 0x0a, 0x13, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x61, 0x69, 0x6c, 0x10, 0xf2, 0x2e, 0x12, 0x14, 0x0a, 0x0f, 0x43, 0x44, 0x4e, 0x54, 0x61, 0x73,
0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x10, 0xd9, 0x36, 0x2a, 0x17, 0x0a, 0x0a, 0x50, 0x6b, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x84, 0x32, 0x12, 0x18, 0x0a, 0x13,
0x69, 0x65, 0x63, 0x65, 0x53, 0x74, 0x79, 0x6c, 0x65, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x4c, 0x41, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54,
0x49, 0x4e, 0x10, 0x00, 0x2a, 0x2c, 0x0a, 0x09, 0x53, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x79, 0x70, 0x65, 0x10, 0xd9, 0x36, 0x2a, 0x17, 0x0a, 0x0a, 0x50, 0x69, 0x65, 0x63, 0x65, 0x53,
0x65, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x74, 0x79, 0x6c, 0x65, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x4c, 0x41, 0x49, 0x4e, 0x10, 0x00, 0x2a,
0x05, 0x53, 0x4d, 0x41, 0x4c, 0x4c, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x54, 0x49, 0x4e, 0x59, 0x2c, 0x0a, 0x09, 0x53, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06,
0x10, 0x02, 0x42, 0x22, 0x5a, 0x20, 0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x53, 0x4d, 0x41, 0x4c,
0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x4c, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x54, 0x49, 0x4e, 0x59, 0x10, 0x02, 0x42, 0x22, 0x5a,
0x63, 0x2f, 0x62, 0x61, 0x73, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, 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 ( var (

View File

@ -55,7 +55,16 @@ func (m *GrpcDfError) validate(all bool) error {
var errors []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 // no validation rules for Message
@ -156,11 +165,37 @@ func (m *UrlMeta) validate(all bool) error {
var errors []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 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 // no validation rules for Filter
@ -242,6 +277,10 @@ var _ interface {
ErrorName() string ErrorName() string
} = UrlMetaValidationError{} } = 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 // 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 // proto definition for this message. If any rules are violated, the first
// error encountered is returned, or nil if there are no violations. // 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 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 { if len(errors) > 0 {
return HostLoadMultiError(errors) return HostLoadMultiError(errors)
@ -368,15 +434,60 @@ func (m *PieceTaskRequest) validate(all bool) error {
var errors []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 { if len(errors) > 0 {
return PieceTaskRequestMultiError(errors) return PieceTaskRequestMultiError(errors)
@ -479,13 +590,53 @@ func (m *PieceInfo) validate(all bool) error {
// no validation rules for PieceNum // 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 // no validation rules for PieceStyle
@ -565,6 +716,8 @@ var _ interface {
ErrorName() string ErrorName() string
} = PieceInfoValidationError{} } = 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 // 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 // the proto definition for this message. If any rules are violated, the first
// error encountered is returned, or nil if there are no violations. // 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 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() { for idx, item := range m.GetPieceInfos() {
_, _ = idx, item _, _ = idx, item

View File

@ -18,6 +18,8 @@ syntax = "proto3";
package base; package base;
import "validate/validate.proto";
option go_package = "d7y.io/dragonfly/v2/pkg/rpc/base"; option go_package = "d7y.io/dragonfly/v2/pkg/rpc/base";
enum Code{ enum Code{
@ -75,18 +77,18 @@ enum SizeScope{
TINY = 2; TINY = 2;
} }
message GrpcDfError { message GrpcDfError {
Code code = 1; Code code = 1 [(validate.rules).enum.defined_only = true];
string message = 2; string message = 2;
} }
// UrlMeta describes url meta info. // UrlMeta describes url meta info.
message UrlMeta{ message UrlMeta{
// digest checks integrity of url content, for example md5:xxx or sha256:yyy // 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 // url tag identifies different task for same url, conflict with digest
string tag = 2; string tag = 2;
// content range for url // 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 // filter url used to generate task id
string filter = 4; string filter = 4;
// other url header infos // other url header infos
@ -95,40 +97,42 @@ message UrlMeta{
message HostLoad{ message HostLoad{
// cpu usage // cpu usage
float cpu_ratio = 1; float cpu_ratio = 1 [(validate.rules).float = {gte: 0, lte: 1}];
// memory usage // memory usage
float mem_ratio = 2; float mem_ratio = 2 [(validate.rules).float = {gte: 0, lte: 1}];
// disk space usage // disk space usage
float disk_ratio = 3; float disk_ratio = 3 [(validate.rules).float = {gte: 0, lte: 1}];
} }
message PieceTaskRequest{ message PieceTaskRequest{
string task_id = 1; string task_id = 1 [(validate.rules).string.min_len = 1];
string src_pid = 2; string src_pid = 2 [(validate.rules).string.min_len = 1];
string dst_pid = 3; string dst_pid = 3 [(validate.rules).string.min_len = 1];
// piece number // piece number
int32 start_num = 4; uint32 start_num = 4 [(validate.rules).uint32.gte = 0];
// expected piece count // expected piece count, limit = 0 represent request pieces as many shards as possible
int32 limit = 5; uint32 limit = 5 [(validate.rules).uint32.gte = 0];
} }
message PieceInfo{ message PieceInfo{
// piece_num < 0 represent start report piece flag
int32 piece_num = 1; int32 piece_num = 1;
uint64 range_start = 2; uint64 range_start = 2 [(validate.rules).uint64.gte = 0];
int32 range_size = 3; uint32 range_size = 3 [(validate.rules).uint32.gte = 0];
string piece_md5 = 4; 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; uint64 piece_offset = 5 [(validate.rules).uint64.gte = 0];
base.PieceStyle piece_style = 6; base.PieceStyle piece_style = 6;
} }
message PiecePacket{ message PiecePacket{
string task_id = 2; string task_id = 2 [(validate.rules).string.min_len = 1];
string dst_pid = 3; string dst_pid = 3 [(validate.rules).string.min_len = 1];
// ip:port // ip:port
string dst_addr = 4; string dst_addr = 4 [(validate.rules).string.min_len = 1];
repeated PieceInfo piece_infos = 5; 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; int32 total_piece = 6;
// content_length < 0 represent content length is unknown
int64 content_length = 7; int64 content_length = 7;
// sha256 code of all piece md5 // sha256 code of all piece md5
string piece_md5_sign = 8; string piece_md5_sign = 8;

View File

@ -23,6 +23,7 @@ package cdnsystem
import ( import (
base "d7y.io/dragonfly/v2/pkg/rpc/base" base "d7y.io/dragonfly/v2/pkg/rpc/base"
_ "github.com/envoyproxy/protoc-gen-validate/validate"
protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl" protoimpl "google.golang.org/protobuf/runtime/protoimpl"
reflect "reflect" 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"` PieceInfo *base.PieceInfo `protobuf:"bytes,4,opt,name=piece_info,json=pieceInfo,proto3" json:"piece_info,omitempty"`
// whether or not all seeds are downloaded // whether or not all seeds are downloaded
Done bool `protobuf:"varint,5,opt,name=done,proto3" json:"done,omitempty"` 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"` 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"` 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, 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, 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, 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, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x17, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x65, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x22, 0x75, 0x0a, 0x0b, 0x53, 0x65, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
0x6c, 0x12, 0x28, 0x0a, 0x08, 0x75, 0x72, 0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49,
0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x64, 0x12, 0x1a, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x08,
0x74, 0x61, 0x52, 0x07, 0x75, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x22, 0xd8, 0x01, 0x0a, 0x09, 0xfa, 0x42, 0x05, 0x72, 0x03, 0x88, 0x01, 0x01, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x28, 0x0a,
0x50, 0x69, 0x65, 0x63, 0x65, 0x53, 0x65, 0x65, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x08, 0x75, 0x72, 0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x0d, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x07,
0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, 0x75, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x22, 0xea, 0x01, 0x0a, 0x09, 0x50, 0x69, 0x65, 0x63,
0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x55, 0x75, 0x69, 0x64, 0x12, 0x65, 0x53, 0x65, 0x65, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64,
0x2e, 0x0a, 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52,
0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f,
0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x75, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72,
0x12, 0x0a, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x64, 0x02, 0x10, 0x01, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x55, 0x75, 0x69, 0x64, 0x12, 0x2e, 0x0a,
0x6f, 0x6e, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28,
0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e,
0x74, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x2a, 0x0a, 0x11, 0x74, 0x6f, 0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x12, 0x0a,
0x74, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x64, 0x6f, 0x6e,
0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 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,
0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x32, 0x83, 0x01, 0x0a, 0x06, 0x53, 0x65, 0x65, 0x64, 0x65, 0x67, 0x74, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65,
0x72, 0x12, 0x3d, 0x0a, 0x0b, 0x4f, 0x62, 0x74, 0x61, 0x69, 0x6e, 0x53, 0x65, 0x65, 0x64, 0x73, 0x6e, 0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x2a, 0x0a, 0x11, 0x74, 0x6f, 0x74, 0x61,
0x12, 0x16, 0x2e, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x2e, 0x53, 0x65, 0x65, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x07, 0x20,
0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65, 0x63, 0x65, 0x43,
0x73, 0x74, 0x65, 0x6d, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x53, 0x65, 0x65, 0x64, 0x30, 0x01, 0x6f, 0x75, 0x6e, 0x74, 0x32, 0x83, 0x01, 0x0a, 0x06, 0x53, 0x65, 0x65, 0x64, 0x65, 0x72, 0x12,
0x12, 0x3a, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x3d, 0x0a, 0x0b, 0x4f, 0x62, 0x74, 0x61, 0x69, 0x6e, 0x53, 0x65, 0x65, 0x64, 0x73, 0x12, 0x16,
0x73, 0x12, 0x16, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x2e, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x2e, 0x53, 0x65, 0x65, 0x64, 0x52,
0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74,
0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x42, 0x27, 0x5a, 0x25, 0x65, 0x6d, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x53, 0x65, 0x65, 0x64, 0x30, 0x01, 0x12, 0x3a,
0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x73, 0x12,
0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x63, 0x64, 0x6e, 0x73, 0x16, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b,
0x79, 0x73, 0x74, 0x65, 0x6d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, 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 ( var (

View File

@ -34,18 +34,79 @@ var (
) )
// Validate checks the field values on SeedRequest with the rules defined in // 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 // the proto definition for this message. If any rules are violated, the first
// is returned. // error encountered is returned, or nil if there are no violations.
func (m *SeedRequest) Validate() error { 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 { if m == nil {
return 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 { if err := v.Validate(); err != nil {
return SeedRequestValidationError{ return SeedRequestValidationError{
field: "UrlMeta", field: "UrlMeta",
@ -55,9 +116,28 @@ func (m *SeedRequest) Validate() error {
} }
} }
if len(errors) > 0 {
return SeedRequestMultiError(errors)
}
return nil 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 // SeedRequestValidationError is the validation error returned by
// SeedRequest.Validate if the designated constraints aren't met. // SeedRequest.Validate if the designated constraints aren't met.
type SeedRequestValidationError struct { type SeedRequestValidationError struct {
@ -113,17 +193,69 @@ var _ interface {
} = SeedRequestValidationError{} } = SeedRequestValidationError{}
// Validate checks the field values on PieceSeed with the rules defined in the // 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 { 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 { if m == nil {
return 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 { if err := v.Validate(); err != nil {
return PieceSeedValidationError{ return PieceSeedValidationError{
field: "PieceInfo", field: "PieceInfo",
@ -139,9 +271,28 @@ func (m *PieceSeed) Validate() error {
// no validation rules for TotalPieceCount // no validation rules for TotalPieceCount
if len(errors) > 0 {
return PieceSeedMultiError(errors)
}
return nil 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 // PieceSeedValidationError is the validation error returned by
// PieceSeed.Validate if the designated constraints aren't met. // PieceSeed.Validate if the designated constraints aren't met.
type PieceSeedValidationError struct { type PieceSeedValidationError struct {

View File

@ -19,12 +19,13 @@ syntax = "proto3";
package cdnsystem; package cdnsystem;
import "pkg/rpc/base/base.proto"; import "pkg/rpc/base/base.proto";
import "validate/validate.proto";
option go_package = "d7y.io/dragonfly/v2/pkg/rpc/cdnsystem"; option go_package = "d7y.io/dragonfly/v2/pkg/rpc/cdnsystem";
message SeedRequest{ message SeedRequest{
string task_id = 1; string task_id = 1 [(validate.rules).string.min_len = 1];
string url = 2; string url = 2 [(validate.rules).string.uri = true];
base.UrlMeta url_meta = 3; base.UrlMeta url_meta = 3;
} }
@ -32,16 +33,16 @@ message SeedRequest{
// check piece md5, md5s sign and total content length // check piece md5, md5s sign and total content length
message PieceSeed{ message PieceSeed{
// peer id for cdn node, need suffix with _CDN // 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 // cdn node host uuid
string host_uuid = 3; string host_uuid = 3 [(validate.rules).string.min_len = 1];
base.PieceInfo piece_info = 4; base.PieceInfo piece_info = 4;
// whether or not all seeds are downloaded // whether or not all seeds are downloaded
bool done = 5; 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; int64 content_length = 6;
// total piece count // total piece count, -1 represents task is downloading or failed
int32 total_piece_count = 7; int32 total_piece_count = 7;
} }

View File

@ -12,7 +12,6 @@ import (
// This is a compile-time assertion to ensure that this generated file // This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against. // is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7 const _ = grpc.SupportPackageIsVersion7
// SeederClient is the client API for Seeder service. // 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) { 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 { if err != nil {
return nil, err return nil, err
} }
@ -105,7 +104,7 @@ type UnsafeSeederServer interface {
} }
func RegisterSeederServer(s grpc.ServiceRegistrar, srv SeederServer) { 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 { 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) return interceptor(ctx, in, info, handler)
} }
// Seeder_ServiceDesc is the grpc.ServiceDesc for Seeder service. var _Seeder_serviceDesc = grpc.ServiceDesc{
// 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{
ServiceName: "cdnsystem.Seeder", ServiceName: "cdnsystem.Seeder",
HandlerType: (*SeederServer)(nil), HandlerType: (*SeederServer)(nil),
Methods: []grpc.MethodDesc{ Methods: []grpc.MethodDesc{

View File

@ -23,6 +23,7 @@ package dfdaemon
import ( import (
base "d7y.io/dragonfly/v2/pkg/rpc/base" base "d7y.io/dragonfly/v2/pkg/rpc/base"
_ "github.com/envoyproxy/protoc-gen-validate/validate"
protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl" protoimpl "google.golang.org/protobuf/runtime/protoimpl"
emptypb "google.golang.org/protobuf/types/known/emptypb" 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 // 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"` Output string `protobuf:"bytes,3,opt,name=output,proto3" json:"output,omitempty"`
// timeout duration // 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 // rate limit in bytes per second
Limit float64 `protobuf:"fixed64,5,opt,name=limit,proto3" json:"limit,omitempty"` 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"` 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"` 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"` Pattern string `protobuf:"bytes,8,opt,name=pattern,proto3" json:"pattern,omitempty"`
// call system // call system
Callsystem string `protobuf:"bytes,9,opt,name=callsystem,proto3" json:"callsystem,omitempty"` Callsystem string `protobuf:"bytes,9,opt,name=callsystem,proto3" json:"callsystem,omitempty"`
@ -118,7 +119,7 @@ func (x *DownRequest) GetOutput() string {
return "" return ""
} }
func (x *DownRequest) GetTimeout() int64 { func (x *DownRequest) GetTimeout() uint64 {
if x != nil { if x != nil {
return x.Timeout 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, 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, 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, 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, 0x6f, 0x1a, 0x17, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2f, 0x76, 0x61, 0x6c, 0x69,
0x74, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x85, 0x03, 0x0a, 0x0b, 0x44,
0x04, 0x75, 0x75, 0x69, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x04, 0x75, 0x75,
0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x72, 0x03, 0xb0,
0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x01, 0x01, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x12, 0x1a, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18,
0x18, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x72, 0x03, 0x88, 0x01, 0x01, 0x52,
0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x1f, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x18, 0x03,
0x69, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x6f,
0x2e, 0x0a, 0x13, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x5f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x21, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x64, 0x69, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x32, 0x02, 0x28, 0x00, 0x52,
0x73, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x24, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69,
0x28, 0x0a, 0x08, 0x75, 0x72, 0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x01, 0x42, 0x0e, 0xfa, 0x42, 0x0b, 0x12, 0x09, 0x29, 0x00,
0x0b, 0x32, 0x0d, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x2e,
0x52, 0x07, 0x75, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x74, 0x0a, 0x13, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x5f, 0x73,
0x74, 0x65, 0x72, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x61, 0x74, 0x74, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x64, 0x69, 0x73,
0x65, 0x72, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x61, 0x6c, 0x6c, 0x73, 0x79, 0x73, 0x74, 0x65, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x28,
0x6d, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x61, 0x6c, 0x6c, 0x73, 0x79, 0x73, 0x0a, 0x08, 0x75, 0x72, 0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b,
0x74, 0x65, 0x6d, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x32, 0x0d, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x52,
0x52, 0x03, 0x75, 0x69, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x67, 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x07, 0x75, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x34, 0x0a, 0x07, 0x70, 0x61, 0x74, 0x74,
0x28, 0x03, 0x52, 0x03, 0x67, 0x69, 0x64, 0x22, 0x7d, 0x0a, 0x0a, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x72, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x42, 0x1a, 0xfa, 0x42, 0x17, 0x72, 0x15,
0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x52, 0x03, 0x70, 0x32, 0x70, 0x52, 0x03, 0x63, 0x64, 0x6e, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72,
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x63, 0x65, 0xd0, 0x01, 0x01, 0x52, 0x07, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, 0x1e,
0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x0a, 0x63, 0x61, 0x6c, 0x6c, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x18, 0x09, 0x20, 0x01,
0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x61, 0x6c, 0x6c, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x12, 0x10,
0x65, 0x74, 0x65, 0x64, 0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x75, 0x69, 0x64,
0x04, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x4c, 0x65, 0x6e, 0x67, 0x12, 0x10, 0x0a, 0x03, 0x67, 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x67,
0x74, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x69, 0x64, 0x22, 0x98, 0x01, 0x0a, 0x0a, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c,
0x52, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x32, 0xbe, 0x01, 0x0a, 0x06, 0x44, 0x61, 0x65, 0x6d, 0x6f, 0x74, 0x12, 0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01,
0x6e, 0x12, 0x39, 0x0a, 0x08, 0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x15, 0x2e, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73,
0x64, 0x66, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x6b, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x64, 0x66, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x70,
0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x30, 0x01, 0x12, 0x3a, 0x0a, 0x0d, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x32, 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74,
0x47, 0x65, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x73, 0x12, 0x16, 0x2e, 0x65, 0x64, 0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x42,
0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x07, 0xfa, 0x42, 0x04, 0x32, 0x02, 0x28, 0x00, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x74, 0x65, 0x64, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x6f, 0x6e,
0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x12, 0x3d, 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x32, 0xbe, 0x01,
0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x0a, 0x06, 0x44, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x12, 0x39, 0x0a, 0x08, 0x44, 0x6f, 0x77, 0x6e,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x15, 0x2e, 0x64, 0x66, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e,
0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x64, 0x66,
0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x26, 0x5a, 0x24, 0x64, 0x37, 0x79, 0x2e, 0x69, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c,
0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x74, 0x30, 0x01, 0x12, 0x3a, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54,
0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x64, 0x66, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x62, 0x61, 0x73, 0x6b, 0x73, 0x12, 0x16, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63,
0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, 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 ( var (

View File

@ -33,27 +33,119 @@ var (
_ = anypb.Any{} _ = 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 // 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 // the proto definition for this message. If any rules are violated, the first
// is returned. // error encountered is returned, or nil if there are no violations.
func (m *DownRequest) Validate() error { 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 { if m == nil {
return 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 // 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 { if err := v.Validate(); err != nil {
return DownRequestValidationError{ return DownRequestValidationError{
field: "UrlMeta", 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 // no validation rules for Callsystem
@ -71,9 +176,36 @@ func (m *DownRequest) Validate() error {
// no validation rules for Gid // no validation rules for Gid
if len(errors) > 0 {
return DownRequestMultiError(errors)
}
return nil 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 // DownRequestValidationError is the validation error returned by
// DownRequest.Validate if the designated constraints aren't met. // DownRequest.Validate if the designated constraints aren't met.
type DownRequestValidationError struct { type DownRequestValidationError struct {
@ -128,24 +260,91 @@ var _ interface {
ErrorName() string ErrorName() string
} = DownRequestValidationError{} } = DownRequestValidationError{}
var _DownRequest_Pattern_InLookup = map[string]struct{}{
"p2p": {},
"cdn": {},
"source": {},
}
// Validate checks the field values on DownResult with the rules defined in the // 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 { 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 { if m == nil {
return 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 // no validation rules for Done
if len(errors) > 0 {
return DownResultMultiError(errors)
}
return nil 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 // DownResultValidationError is the validation error returned by
// DownResult.Validate if the designated constraints aren't met. // DownResult.Validate if the designated constraints aren't met.
type DownResultValidationError struct { type DownResultValidationError struct {

View File

@ -20,25 +20,26 @@ package dfdaemon;
import "pkg/rpc/base/base.proto"; import "pkg/rpc/base/base.proto";
import "google/protobuf/empty.proto"; import "google/protobuf/empty.proto";
import "validate/validate.proto";
option go_package = "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon"; option go_package = "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon";
message DownRequest{ message DownRequest{
// identify one downloading, the framework will fill it automatically // 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 // 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, // pieces will be written to output path directly,
// at the same time, dfdaemon workspace also makes soft link to the output // 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 // timeout duration
int64 timeout = 4; uint64 timeout = 4 [(validate.rules).uint64.gte = 0];
// rate limit in bytes per second // rate limit in bytes per second
double limit = 5; double limit = 5 [(validate.rules).double.gte = 0];
bool disable_back_source = 6; bool disable_back_source = 6;
base.UrlMeta url_meta = 7; base.UrlMeta url_meta = 7;
// p2p/cdn/source // p2p/cdn/source, default is p2p
string pattern = 8; string pattern = 8 [(validate.rules).string = {in:["p2p", "cdn", "source"], ignore_empty:true}];
// call system // call system
string callsystem = 9; string callsystem = 9;
// user id // user id
@ -48,9 +49,9 @@ message DownRequest{
} }
message DownResult{ message DownResult{
string task_id = 2; string task_id = 2 [(validate.rules).string.min_len = 1];
string peer_id = 3; string peer_id = 3 [(validate.rules).string.min_len = 1];
uint64 completed_length = 4; uint64 completed_length = 4 [(validate.rules).uint64.gte = 0];
bool done = 5; bool done = 5;
} }

View File

@ -13,7 +13,6 @@ import (
// This is a compile-time assertion to ensure that this generated file // This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against. // is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7 const _ = grpc.SupportPackageIsVersion7
// DaemonClient is the client API for Daemon service. // 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) { 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 { if err != nil {
return nil, err return nil, err
} }
@ -122,7 +121,7 @@ type UnsafeDaemonServer interface {
} }
func RegisterDaemonServer(s grpc.ServiceRegistrar, srv DaemonServer) { 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 { 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) return interceptor(ctx, in, info, handler)
} }
// Daemon_ServiceDesc is the grpc.ServiceDesc for Daemon service. var _Daemon_serviceDesc = grpc.ServiceDesc{
// 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{
ServiceName: "dfdaemon.Daemon", ServiceName: "dfdaemon.Daemon",
HandlerType: (*DaemonServer)(nil), HandlerType: (*DaemonServer)(nil),
Methods: []grpc.MethodDesc{ Methods: []grpc.MethodDesc{

View File

@ -22,13 +22,12 @@
package manager package manager
import ( import (
reflect "reflect"
sync "sync"
_ "github.com/envoyproxy/protoc-gen-validate/validate" _ "github.com/envoyproxy/protoc-gen-validate/validate"
protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl" protoimpl "google.golang.org/protobuf/runtime/protoimpl"
emptypb "google.golang.org/protobuf/types/known/emptypb" emptypb "google.golang.org/protobuf/types/known/emptypb"
reflect "reflect"
sync "sync"
) )
const ( const (

File diff suppressed because it is too large Load Diff

View File

@ -12,7 +12,6 @@ import (
// This is a compile-time assertion to ensure that this generated file // This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against. // is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7 const _ = grpc.SupportPackageIsVersion7
// ManagerClient is the client API for Manager service. // 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) { 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 { if err != nil {
return nil, err return nil, err
} }
@ -171,7 +170,7 @@ type UnsafeManagerServer interface {
} }
func RegisterManagerServer(s grpc.ServiceRegistrar, srv ManagerServer) { 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) { 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 return m, nil
} }
// Manager_ServiceDesc is the grpc.ServiceDesc for Manager service. var _Manager_serviceDesc = grpc.ServiceDesc{
// 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{
ServiceName: "manager.Manager", ServiceName: "manager.Manager",
HandlerType: (*ManagerServer)(nil), HandlerType: (*ManagerServer)(nil),
Methods: []grpc.MethodDesc{ Methods: []grpc.MethodDesc{

View File

@ -23,6 +23,7 @@ package scheduler
import ( import (
base "d7y.io/dragonfly/v2/pkg/rpc/base" base "d7y.io/dragonfly/v2/pkg/rpc/base"
_ "github.com/envoyproxy/protoc-gen-validate/validate"
protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl" protoimpl "google.golang.org/protobuf/runtime/protoimpl"
emptypb "google.golang.org/protobuf/types/known/emptypb" 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"` Code base.Code `protobuf:"varint,8,opt,name=code,proto3,enum=base.Code" json:"code,omitempty"`
// current host resource usage // current host resource usage
HostLoad *base.HostLoad `protobuf:"bytes,9,opt,name=host_load,json=hostLoad,proto3" json:"host_load,omitempty"` 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"` 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) // total content length(byte)
ContentLength int64 `protobuf:"varint,7,opt,name=content_length,json=contentLength,proto3" json:"content_length,omitempty"` ContentLength int64 `protobuf:"varint,7,opt,name=content_length,json=contentLength,proto3" json:"content_length,omitempty"`
// total network traffic(byte) // 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 // total time(millisecond) consumed
Cost uint32 `protobuf:"varint,9,opt,name=cost,proto3" json:"cost,omitempty"` Cost uint32 `protobuf:"varint,9,opt,name=cost,proto3" json:"cost,omitempty"`
// whether peer downloading file is successfully // whether peer downloading file is successfully
Success bool `protobuf:"varint,10,opt,name=success,proto3" json:"success,omitempty"` Success bool `protobuf:"varint,10,opt,name=success,proto3" json:"success,omitempty"`
// result code // result code
Code base.Code `protobuf:"varint,11,opt,name=code,proto3,enum=base.Code" json:"code,omitempty"` 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"` // -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() { func (x *PeerResult) Reset() {
@ -742,7 +744,7 @@ func (x *PeerResult) GetContentLength() int64 {
return 0 return 0
} }
func (x *PeerResult) GetTraffic() int64 { func (x *PeerResult) GetTraffic() uint64 {
if x != nil { if x != nil {
return x.Traffic 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, 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, 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, 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, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x17, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, 0x65, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x72, 0x6f, 0x74, 0x6f, 0x22, 0xe8, 0x01, 0x0a, 0x0f, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xfb, 0x01,
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x28, 0x0a, 0x08, 0x75, 0x72, 0x0a, 0x0f, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x62, 0x74, 0x12, 0x1a, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x08,
0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x07, 0x75, 0x72, 0x6c, 0xfa, 0x42, 0x05, 0x72, 0x03, 0x88, 0x01, 0x01, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x28, 0x0a,
0x4d, 0x65, 0x74, 0x61, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x08, 0x75, 0x72, 0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x0d, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x07,
0x09, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x75, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f,
0x32, 0x13, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10,
0x72, 0x48, 0x6f, 0x73, 0x74, 0x52, 0x08, 0x70, 0x65, 0x65, 0x72, 0x48, 0x6f, 0x73, 0x74, 0x12, 0x01, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x09, 0x70, 0x65, 0x65,
0x2b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x05, 0x20, 0x01, 0x72, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73,
0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x48, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x48, 0x6f, 0x73,
0x61, 0x64, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x52, 0x08, 0x70, 0x65, 0x65, 0x72, 0x48, 0x6f, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x09, 0x68,
0x69, 0x73, 0x5f, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x18, 0x06, 0x20, 0x01, 0x6f, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e,
0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x22, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x48, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x52, 0x08,
0xcd, 0x01, 0x0a, 0x0e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x68, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x6d,
0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b,
0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x2e, 0x0a, 0x0a, 0x73, 0x69, 0x73, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x22, 0xe0, 0x01, 0x0a, 0x0e,
0x69, 0x7a, 0x65, 0x5f, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x20,
0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x53, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42,
0x52, 0x09, 0x73, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x3b, 0x0a, 0x0c, 0x73, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64,
0x69, 0x6e, 0x67, 0x6c, 0x65, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x12, 0x38, 0x0a, 0x0a, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x03,
0x0b, 0x32, 0x16, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x53, 0x69, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x53, 0x69, 0x7a, 0x65,
0x6e, 0x67, 0x6c, 0x65, 0x50, 0x69, 0x65, 0x63, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x69, 0x6e, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, 0x01, 0x02, 0x10, 0x01, 0x52,
0x67, 0x6c, 0x65, 0x50, 0x69, 0x65, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0d, 0x70, 0x69, 0x65, 0x63, 0x09, 0x73, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x3b, 0x0a, 0x0c, 0x73, 0x69,
0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x6e, 0x67, 0x6c, 0x65, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b,
0x00, 0x52, 0x0c, 0x70, 0x69, 0x65, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x42, 0x32, 0x16, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x53, 0x69, 0x6e,
0x0e, 0x0a, 0x0c, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x22, 0x67, 0x6c, 0x65, 0x50, 0x69, 0x65, 0x63, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x69, 0x6e, 0x67,
0x71, 0x0a, 0x0b, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x50, 0x69, 0x65, 0x63, 0x65, 0x12, 0x17, 0x6c, 0x65, 0x50, 0x69, 0x65, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0d, 0x70, 0x69, 0x65, 0x63, 0x65,
0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00,
0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x61, 0x52, 0x0c, 0x70, 0x69, 0x65, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x42, 0x0e,
0x64, 0x64, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x64, 0x73, 0x74, 0x41, 0x64, 0x0a, 0x0c, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x22, 0x83,
0x64, 0x72, 0x12, 0x2e, 0x0a, 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x01, 0x0a, 0x0b, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x50, 0x69, 0x65, 0x63, 0x65, 0x12, 0x20,
0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42,
0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64,
0x66, 0x6f, 0x22, 0xfd, 0x01, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x48, 0x6f, 0x73, 0x74, 0x12, 0x12, 0x22, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x02, 0x20, 0x01,
0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x07, 0x64, 0x73, 0x74,
0x75, 0x69, 0x64, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x41, 0x64, 0x64, 0x72, 0x12, 0x2e, 0x0a, 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e,
0x02, 0x69, 0x70, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x70, 0x63, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x66, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e,
0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x72, 0x70, 0x63, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1b, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65,
0x0a, 0x09, 0x64, 0x6f, 0x77, 0x6e, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0xb5, 0x02, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x48, 0x6f, 0x73,
0x05, 0x52, 0x08, 0x64, 0x6f, 0x77, 0x6e, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x68, 0x74, 0x12, 0x1c, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42,
0x6f, 0x73, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x08, 0xfa, 0x42, 0x05, 0x72, 0x03, 0xb0, 0x01, 0x01, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x12,
0x68, 0x6f, 0x73, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x63, 0x75, 0x17, 0x0a, 0x02, 0x69, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04,
0x72, 0x69, 0x74, 0x79, 0x5f, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x72, 0x02, 0x70, 0x01, 0x52, 0x02, 0x69, 0x70, 0x12, 0x27, 0x0a, 0x08, 0x72, 0x70, 0x63, 0x5f,
0x09, 0x52, 0x0e, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x42, 0x0c, 0xfa, 0x42, 0x09, 0x1a,
0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x07, 0x10, 0xff, 0xff, 0x03, 0x28, 0x80, 0x08, 0x52, 0x07, 0x72, 0x70, 0x63, 0x50, 0x6f, 0x72,
0x01, 0x28, 0x09, 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x74, 0x12, 0x29, 0x0a, 0x09, 0x64, 0x6f, 0x77, 0x6e, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x04,
0x03, 0x69, 0x64, 0x63, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x63, 0x12, 0x20, 0x01, 0x28, 0x05, 0x42, 0x0c, 0xfa, 0x42, 0x09, 0x1a, 0x07, 0x10, 0xff, 0xff, 0x03, 0x28,
0x21, 0x0a, 0x0c, 0x6e, 0x65, 0x74, 0x5f, 0x74, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x18, 0x80, 0x08, 0x52, 0x08, 0x64, 0x6f, 0x77, 0x6e, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x24, 0x0a, 0x09,
0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x42,
0x67, 0x79, 0x22, 0xd0, 0x02, 0x0a, 0x0b, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x68, 0x01, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x4e, 0x61,
0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, 0x64,
0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x73, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x65, 0x63,
0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x72, 0x75, 0x72, 0x69, 0x74, 0x79, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x6c,
0x63, 0x50, 0x69, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6c,
0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x2e, 0x0a, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x63, 0x18, 0x08,
0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x63, 0x12, 0x21, 0x0a, 0x0c, 0x6e, 0x65, 0x74,
0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x5f, 0x74, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52,
0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x0a, 0x0b, 0x6e, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x22, 0xec, 0x02, 0x0a,
0x0a, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x20, 0x0a, 0x07,
0x04, 0x52, 0x09, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa,
0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x20,
0x65, 0x6e, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x0a, 0x07, 0x73, 0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42,
0x73, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x73, 0x72, 0x63, 0x50, 0x69, 0x64,
0x73, 0x12, 0x1e, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28,
0x0a, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x09, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x2e, 0x0a, 0x0a, 0x70, 0x69, 0x65,
0x65, 0x12, 0x2b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x48, 0x6f, 0x73, 0x74, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09,
0x4c, 0x6f, 0x61, 0x64, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x25, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x65, 0x67,
0x0a, 0x0e, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x62,
0x18, 0x0a, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x5f,
0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xd3, 0x02, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x50, 0x61, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x65, 0x6e, 0x64, 0x54,
0x63, 0x6b, 0x65, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x69, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x07,
0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x28, 0x0a,
0x07, 0x73, 0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61,
0x73, 0x72, 0x63, 0x50, 0x69, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, 0x01, 0x02, 0x10,
0x65, 0x6c, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x01, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x2b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f,
0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x3b, 0x0a, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x62, 0x61, 0x73,
0x09, 0x6d, 0x61, 0x69, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 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, 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, 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, 0x52, 0x0a, 0x73, 0x74, 0x65, 0x61, 0x6c, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x28, 0x0a, 0x04,
0x65, 0x61, 0x6c, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61, 0x73,
0x1e, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, 0x01, 0x02, 0x10, 0x01,
0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x1a, 0x6e, 0x0a, 0x08, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65,
0x0a, 0x73, 0x74, 0x65, 0x61, 0x6c, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x1e, 0x0a, 0x04, 0x63, 0x65, 0x72, 0x12, 0x17, 0x0a, 0x02, 0x69, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07,
0x6f, 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61, 0x73, 0x65, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x70, 0x01, 0x52, 0x02, 0x69, 0x70, 0x12, 0x27, 0x0a, 0x08, 0x72,
0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x1a, 0x4e, 0x0a, 0x08, 0x44, 0x70, 0x63, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x42, 0x0c, 0xfa,
0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x70, 0x18, 0x01, 0x20, 0x42, 0x09, 0x1a, 0x07, 0x10, 0xff, 0xff, 0x03, 0x28, 0x80, 0x08, 0x52, 0x07, 0x72, 0x70, 0x63,
0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x70, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x70, 0x63, 0x5f, 0x70, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18,
0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x72, 0x70, 0x63, 0x50, 0x6f, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06,
0x72, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x22, 0x8c, 0x03, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x52,
0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x22, 0xdd, 0x02, 0x0a, 0x0a, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64,
0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52,
0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f,
0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10,
0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x15, 0x0a, 0x06, 0x01, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1e, 0x0a, 0x06, 0x73, 0x72, 0x63,
0x73, 0x72, 0x63, 0x5f, 0x69, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x72, 0x5f, 0x69, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02,
0x63, 0x49, 0x70, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, 0x70, 0x01, 0x52, 0x05, 0x73, 0x72, 0x63, 0x49, 0x70, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x63,
0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x65, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x04, 0x20, 0x01,
0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x44, 0x6f, 0x6d, 0x61,
0x69, 0x64, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x63, 0x12, 0x10, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52,
0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 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, 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, 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, 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, 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, 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, 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, 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, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x82, 0x01,
0x2a, 0x0a, 0x11, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x63, 0x02, 0x10, 0x01, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x74, 0x6f, 0x74,
0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0c,
0x6c, 0x50, 0x69, 0x65, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x3e, 0x0a, 0x0a, 0x50, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65, 0x63, 0x65,
0x65, 0x65, 0x72, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x50, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x72,
0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x67, 0x65, 0x74, 0x12, 0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01,
0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74,
0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x32, 0x9d, 0x02, 0x0a, 0x09, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64,
0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x10, 0x52, 0x65, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52,
0x69, 0x73, 0x74, 0x65, 0x72, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x1a, 0x2e, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x32, 0x9d, 0x02, 0x0a, 0x09, 0x53, 0x63, 0x68, 0x65,
0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x10, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65,
0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x72, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x1a, 0x2e, 0x73, 0x63, 0x68, 0x65,
0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65,
0x73, 0x75, 0x6c, 0x74, 0x12, 0x46, 0x0a, 0x11, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65,
0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74,
0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x12, 0x46, 0x0a, 0x11, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52,
0x74, 0x1a, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65,
0x65, 0x72, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x28, 0x01, 0x30, 0x01, 0x12, 0x41, 0x0a, 0x10, 0x72, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x1a, 0x15, 0x2e,
0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x50, 0x61,
0x12, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x63, 0x6b, 0x65, 0x74, 0x28, 0x01, 0x30, 0x01, 0x12, 0x41, 0x0a, 0x10, 0x52, 0x65, 0x70, 0x6f,
0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x72, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x15, 0x2e, 0x73,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73,
0x3a, 0x0a, 0x09, 0x4c, 0x65, 0x61, 0x76, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x15, 0x2e, 0x73, 0x75, 0x6c, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f,
0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x72, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x3a, 0x0a, 0x09, 0x4c,
0x67, 0x65, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x65, 0x61, 0x76, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64,
0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x27, 0x5a, 0x25, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x1a,
0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x27, 0x5a, 0x25, 0x64, 0x37, 0x79, 0x2e, 0x69,
0x75, 0x6c, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, 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 ( var (

File diff suppressed because it is too large Load Diff

View File

@ -19,17 +19,18 @@ syntax = "proto3";
package scheduler; package scheduler;
import "pkg/rpc/base/base.proto"; import "pkg/rpc/base/base.proto";
import "validate/validate.proto";
import "google/protobuf/empty.proto"; import "google/protobuf/empty.proto";
option go_package = "d7y.io/dragonfly/v2/pkg/rpc/scheduler"; option go_package = "d7y.io/dragonfly/v2/pkg/rpc/scheduler";
message PeerTaskRequest{ message PeerTaskRequest{
// universal resource locator for different kind of storage // universal resource locator for different kind of storage
string url = 1; string url = 1 [(validate.rules).string.uri = true];
// url meta info // url meta info
base.UrlMeta url_meta = 2; base.UrlMeta url_meta = 2;
// peer's id and must be global uniqueness // 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 // peer host info
PeerHost peer_host = 4; PeerHost peer_host = 4;
// current host load // current host load
@ -40,9 +41,9 @@ message PeerTaskRequest{
message RegisterResult{ message RegisterResult{
// task id // task id
string task_id = 2; string task_id = 2 [(validate.rules).string.min_len = 1];
// file content length scope for the url // 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 // download the only piece directly for small or tiny file
oneof direct_piece{ oneof direct_piece{
// for small file // for small file
@ -54,24 +55,24 @@ message RegisterResult{
message SinglePiece{ message SinglePiece{
// destination peer id // destination peer id
string dst_pid = 1; string dst_pid = 1 [(validate.rules).string.min_len = 1];
// download address(ip:port) // download address(ip:port)
string dst_addr = 2; string dst_addr = 2 [(validate.rules).string.min_len = 1];
// one piece info // one piece info
base.PieceInfo piece_info = 3; base.PieceInfo piece_info = 3;
} }
message PeerHost{ message PeerHost{
// each time the daemon starts, it will generate a different uuid // 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 // peer host ip
string ip = 2; string ip = 2 [(validate.rules).string.ip = true];
// rpc service port for peer // 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 // piece downloading port for peer
int32 down_port = 4; int32 down_port = 4 [(validate.rules).int32 = {gte: 1024, lt: 65535}];
// peer host name // peer host name
string host_name = 5; string host_name = 5 [(validate.rules).string.hostname = true];
// security isolation domain for network // security isolation domain for network
string security_domain = 6; string security_domain = 6;
// location path: area|country|province|city|... // location path: area|country|province|city|...
@ -84,9 +85,9 @@ message PeerHost{
message PieceResult{ message PieceResult{
// task id // task id
string task_id = 1; string task_id = 1 [(validate.rules).string.min_len = 1];
// source peer id // source peer id
string src_pid = 2; string src_pid = 2 [(validate.rules).string.min_len = 1];
// dest peer id // dest peer id
string dst_pid = 3; string dst_pid = 3;
// piece info // piece info
@ -98,10 +99,10 @@ message PieceResult{
// whether the piece downloading is successfully // whether the piece downloading is successfully
bool success = 7; bool success = 7;
// result code // result code
base.Code code = 8; base.Code code = 8 [(validate.rules).enum = {defined_only:true}];
// current host resource usage // current host resource usage
base.HostLoad host_load = 9; base.HostLoad host_load = 9;
// currently completed piece count // currently completed piece count, -1 represent download failed
int32 finished_count = 10; int32 finished_count = 10;
} }
@ -109,48 +110,49 @@ message PeerPacket{
message DestPeer{ message DestPeer{
// dest ip // dest ip
string ip = 1; string ip = 1 [(validate.rules).string.ip = true];
// rpc service port for dest peer // rpc service port for dest peer
int32 rpc_port = 2; int32 rpc_port = 2 [(validate.rules).int32 = {gte: 1024, lt: 65535}];
// dest peer id // 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 // source peer id
string src_pid = 3; string src_pid = 3 [(validate.rules).string.min_len = 1];
// concurrent downloading count from main peer // concurrent downloading count from main peer
int32 parallel_count = 4; int32 parallel_count = 4 [(validate.rules).int32.gte = 1];
DestPeer main_peer = 5; DestPeer main_peer = 5;
repeated DestPeer steal_peers = 6; repeated DestPeer steal_peers = 6;
// result code // result code
base.Code code = 7; base.Code code = 7 [(validate.rules).enum.defined_only = true];
} }
message PeerResult{ message PeerResult{
string task_id = 1; string task_id = 1 [(validate.rules).string.min_len = 1];
string peer_id = 2; string peer_id = 2 [(validate.rules).string.min_len = 1];
string src_ip = 3; string src_ip = 3 [(validate.rules).string.ip = true];
string security_domain = 4; string security_domain = 4;
string idc = 5; string idc = 5;
string url = 6; string url = 6 [(validate.rules).string.uri = true];
// total content length(byte) // total content length(byte)
int64 content_length = 7; int64 content_length = 7;
// total network traffic(byte) // total network traffic(byte)
int64 traffic = 8; uint64 traffic = 8;
// total time(millisecond) consumed // total time(millisecond) consumed
uint32 cost = 9; uint32 cost = 9;
// whether peer downloading file is successfully // whether peer downloading file is successfully
bool success = 10; bool success = 10;
// result code // 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; int32 total_piece_count = 12;
} }
message PeerTarget{ message PeerTarget{
string task_id = 1; string task_id = 1 [(validate.rules).string.min_len = 1];
string peer_id = 2; string peer_id = 2 [(validate.rules).string.min_len = 1];
} }
// Scheduler System RPC Service // Scheduler System RPC Service

View File

@ -12,7 +12,6 @@ import (
// This is a compile-time assertion to ensure that this generated file // This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against. // is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7 const _ = grpc.SupportPackageIsVersion7
// SchedulerClient is the client API for Scheduler service. // 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) { 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 { if err != nil {
return nil, err return nil, err
} }
@ -140,7 +139,7 @@ type UnsafeSchedulerServer interface {
} }
func RegisterSchedulerServer(s grpc.ServiceRegistrar, srv SchedulerServer) { 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) { 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) return interceptor(ctx, in, info, handler)
} }
// Scheduler_ServiceDesc is the grpc.ServiceDesc for Scheduler service. var _Scheduler_serviceDesc = grpc.ServiceDesc{
// 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{
ServiceName: "scheduler.Scheduler", ServiceName: "scheduler.Scheduler",
HandlerType: (*SchedulerServer)(nil), HandlerType: (*SchedulerServer)(nil),
Methods: []grpc.MethodDesc{ Methods: []grpc.MethodDesc{

View File

@ -103,7 +103,7 @@ func (p *proxy) ReportPeerResult(ctx context.Context, req *scheduler.PeerResult)
zap.String("SchedulerIP", iputils.IPv4), zap.String("SchedulerIP", iputils.IPv4),
zap.String("SchedulerHostname", hostutils.FQDNHostname), zap.String("SchedulerHostname", hostutils.FQDNHostname),
zap.Int64("ContentLength", req.ContentLength), zap.Int64("ContentLength", req.ContentLength),
zap.Int64("Traffic", req.Traffic), zap.Uint64("Traffic", req.Traffic),
zap.Uint32("Cost", req.Cost), zap.Uint32("Cost", req.Cost),
zap.Int32("Code", int32(req.Code)), zap.Int32("Code", int32(req.Code)),
) )

View File

@ -22,13 +22,16 @@ import (
grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware" grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
grpc_zap "github.com/grpc-ecosystem/go-grpc-middleware/logging/zap" 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" grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/keepalive" "google.golang.org/grpc/keepalive"
"google.golang.org/grpc/status" "google.golang.org/grpc/status"
"d7y.io/dragonfly/v2/internal/dferrors" "d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog" logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/base/common" "d7y.io/dragonfly/v2/pkg/rpc/base/common"
) )
@ -46,11 +49,13 @@ var DefaultServerOptions = []grpc.ServerOption{
streamServerInterceptor, streamServerInterceptor,
grpc_prometheus.StreamServerInterceptor, grpc_prometheus.StreamServerInterceptor,
grpc_zap.StreamServerInterceptor(logger.GrpcLogger.Desugar()), grpc_zap.StreamServerInterceptor(logger.GrpcLogger.Desugar()),
grpc_validator.StreamServerInterceptor(),
)), )),
grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer( grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(
unaryServerInterceptor, unaryServerInterceptor,
grpc_prometheus.UnaryServerInterceptor, grpc_prometheus.UnaryServerInterceptor,
grpc_zap.UnaryServerInterceptor(logger.GrpcLogger.Desugar()), 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) err = convertServerError(err)
logger.GrpcLogger.Errorf("do stream server error: %v for method: %s", err, info.FullMethod) logger.GrpcLogger.Errorf("do stream server error: %v for method: %s", err, info.FullMethod)
} }
return err return err
} }
@ -75,6 +79,9 @@ func unaryServerInterceptor(ctx context.Context, req interface{}, info *grpc.Una
} }
func convertServerError(err error) error { 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 { if v, ok := err.(*dferrors.DfError); ok {
logger.GrpcLogger.Errorf(v.Message) logger.GrpcLogger.Errorf(v.Message)
if s, e := status.Convert(err).WithDetails(common.NewGrpcDfError(v.Code, v.Message)); e == nil { if s, e := status.Convert(err).WithDetails(common.NewGrpcDfError(v.Code, v.Message)); e == nil {

View File

@ -43,8 +43,8 @@ type factor struct {
idc string idc string
location string location string
netTopology string netTopology string
totalUploadLoad int32 totalUploadLoad uint32
currentUploadLoad int32 currentUploadLoad uint32
finishedPieceCount int32 finishedPieceCount int32
hostUUID string hostUUID string
taskPieceCount int32 taskPieceCount int32

View File

@ -226,7 +226,7 @@ func (s *SchedulerService) RegisterPeerTask(req *schedulerRPC.PeerTaskRequest, t
var options []supervisor.HostOption var options []supervisor.HostOption
if clientConfig, ok := s.dynconfig.GetSchedulerClusterClientConfig(); ok { if clientConfig, ok := s.dynconfig.GetSchedulerClusterClientConfig(); ok {
options = []supervisor.HostOption{ options = []supervisor.HostOption{
supervisor.WithTotalUploadLoad(int32(clientConfig.LoadLimit)), supervisor.WithTotalUploadLoad(clientConfig.LoadLimit),
} }
} }

View File

@ -31,8 +31,6 @@ import (
"d7y.io/dragonfly/v2/pkg/rpc/base" "d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/scheduler" "d7y.io/dragonfly/v2/pkg/rpc/scheduler"
schedulerserver "d7y.io/dragonfly/v2/pkg/rpc/scheduler/server" 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/config"
"d7y.io/dragonfly/v2/scheduler/core" "d7y.io/dragonfly/v2/scheduler/core"
"d7y.io/dragonfly/v2/scheduler/supervisor" "d7y.io/dragonfly/v2/scheduler/supervisor"
@ -65,12 +63,6 @@ func (s *server) RegisterPeerTask(ctx context.Context, request *scheduler.PeerTa
defer span.End() defer span.End()
logger.Debugf("register peer task, req: %+v", request) logger.Debugf("register peer task, req: %+v", request)
resp = new(scheduler.RegisterResult) 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) taskID := idgen.TaskID(request.Url, request.UrlMeta)
span.SetAttributes(config.AttributeTaskID.String(taskID)) 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) 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 { func getTaskSizeScope(task *supervisor.Task) base.SizeScope {
if task.IsSuccess() { if task.IsSuccess() {
if task.ContentLength.Load() <= supervisor.TinyFileSize { if task.ContentLength.Load() <= supervisor.TinyFileSize {

View File

@ -313,7 +313,7 @@ func cdnsToHosts(cdns []*config.CDN) map[string]*Host {
if config, ok := cdn.GetCDNClusterConfig(); ok { if config, ok := cdn.GetCDNClusterConfig(); ok {
options = []HostOption{ options = []HostOption{
WithNetTopology(config.NetTopology), WithNetTopology(config.NetTopology),
WithTotalUploadLoad(int32(config.LoadLimit)), WithTotalUploadLoad(config.LoadLimit),
} }
} }

View File

@ -68,7 +68,7 @@ func (m *hostManager) Delete(key string) {
type HostOption func(rt *Host) *Host type HostOption func(rt *Host) *Host
func WithTotalUploadLoad(load int32) HostOption { func WithTotalUploadLoad(load uint32) HostOption {
return func(h *Host) *Host { return func(h *Host) *Host {
h.TotalUploadLoad = load h.TotalUploadLoad = load
return h return h
@ -106,9 +106,9 @@ type Host struct {
// Example: switch|router|... // Example: switch|router|...
NetTopology string NetTopology string
// TODO TotalUploadLoad currentUploadLoad decided by real time client report host info // TODO TotalUploadLoad currentUploadLoad decided by real time client report host info
TotalUploadLoad int32 TotalUploadLoad uint32
// CurrentUploadLoad is current upload load number // CurrentUploadLoad is current upload load number
CurrentUploadLoad atomic.Int32 CurrentUploadLoad atomic.Uint32
// peers info map // peers info map
peers *sync.Map peers *sync.Map
// host logger // host logger
@ -175,7 +175,7 @@ func (h *Host) GetPeersLen() int {
} }
func (h *Host) GetFreeUploadLoad() int32 { func (h *Host) GetFreeUploadLoad() int32 {
return h.TotalUploadLoad - h.CurrentUploadLoad.Load() return int32(h.TotalUploadLoad - h.CurrentUploadLoad.Load())
} }
func (h *Host) Log() *logger.SugaredLoggerOnWith { func (h *Host) Log() *logger.SugaredLoggerOnWith {