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
// and then use the DefaultPieceSize.
func ComputePieceSize(length int64) int32 {
func ComputePieceSize(length int64) uint32 {
if length <= 0 || length <= 200*1024*1024 {
return config.DefaultPieceSize
}
@ -34,5 +34,5 @@ func ComputePieceSize(length int64) int32 {
if mpSize > config.DefaultPieceSizeLimit {
return config.DefaultPieceSizeLimit
}
return int32(mpSize)
return uint32(mpSize)
}

View File

@ -21,7 +21,6 @@ import (
"fmt"
"strings"
"github.com/pkg/errors"
"go.opentelemetry.io/otel"
"go.opentelemetry.io/otel/trace"
"google.golang.org/grpc"
@ -36,9 +35,7 @@ import (
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/cdnsystem"
cdnserver "d7y.io/dragonfly/v2/pkg/rpc/cdnsystem/server"
"d7y.io/dragonfly/v2/pkg/util/digestutils"
"d7y.io/dragonfly/v2/pkg/util/hostutils"
"d7y.io/dragonfly/v2/pkg/util/net/urlutils"
"d7y.io/dragonfly/v2/pkg/util/stringutils"
)
@ -61,18 +58,11 @@ func New(cfg *config.Config, taskMgr supervisor.SeedTaskMgr, opts ...grpc.Server
return svr.Server, nil
}
func constructRegisterRequest(req *cdnsystem.SeedRequest) (*types.TaskRegisterRequest, error) {
if err := checkSeedRequestParams(req); err != nil {
return nil, err
}
func constructRegisterRequest(req *cdnsystem.SeedRequest) *types.TaskRegisterRequest {
meta := req.UrlMeta
header := make(map[string]string)
if meta != nil {
if !stringutils.IsBlank(meta.Digest) {
digest := digestutils.Parse(meta.Digest)
if _, ok := digestutils.Algorithms[digest[0]]; !ok {
return nil, errors.Errorf("unsupported digest algorithm")
}
header["digest"] = meta.Digest
}
if !stringutils.IsBlank(meta.Range) {
@ -88,18 +78,7 @@ func constructRegisterRequest(req *cdnsystem.SeedRequest) (*types.TaskRegisterRe
Digest: header["digest"],
TaskID: req.TaskId,
Filter: strings.Split(req.UrlMeta.Filter, "&"),
}, nil
}
// checkSeedRequestParams check the params of SeedRequest.
func checkSeedRequestParams(req *cdnsystem.SeedRequest) error {
if !urlutils.IsValidURL(req.Url) {
return errors.Errorf("resource url: %s is invalid", req.Url)
}
if stringutils.IsBlank(req.TaskId) {
return errors.New("taskId is empty")
}
return nil
}
func (css *server) ObtainSeeds(ctx context.Context, req *cdnsystem.SeedRequest, psc chan<- *cdnsystem.PieceSeed) (err error) {
@ -117,12 +96,7 @@ func (css *server) ObtainSeeds(ctx context.Context, req *cdnsystem.SeedRequest,
}
logger.Infof("seeds task %s result success: %t", req.TaskId, err == nil)
}()
registerRequest, err := constructRegisterRequest(req)
if err != nil {
err = dferrors.Newf(base.Code_BadRequest, "bad seed request for task(%s): %v", req.TaskId, err)
span.RecordError(err)
return err
}
registerRequest := constructRegisterRequest(req)
// register task
pieceChan, err := css.taskMgr.Register(ctx, registerRequest)
if err != nil {
@ -141,7 +115,7 @@ func (css *server) ObtainSeeds(ctx context.Context, req *cdnsystem.SeedRequest,
PeerId: peerID,
HostUuid: idgen.CDNHostID(hostutils.FQDNHostname, int32(css.cfg.ListenPort)),
PieceInfo: &base.PieceInfo{
PieceNum: piece.PieceNum,
PieceNum: int32(piece.PieceNum),
RangeStart: piece.PieceRange.StartIndex,
RangeSize: piece.PieceLen,
PieceMd5: piece.PieceMd5,
@ -186,11 +160,6 @@ func (css *server) GetPieceTasks(ctx context.Context, req *base.PieceTaskRequest
}
}()
logger.Infof("get piece tasks: %+v", req)
if err := checkPieceTasksRequestParams(req); err != nil {
err = dferrors.Newf(base.Code_BadRequest, "failed to validate seed request for task(%s): %v", req.TaskId, err)
span.RecordError(err)
return nil, err
}
task, err := css.taskMgr.Get(req.TaskId)
if err != nil {
if cdnerrors.IsDataNotFound(err) {
@ -214,11 +183,11 @@ func (css *server) GetPieceTasks(ctx context.Context, req *base.PieceTaskRequest
return nil, err
}
pieceInfos := make([]*base.PieceInfo, 0)
var count int32 = 0
var count uint32 = 0
for _, piece := range pieces {
if piece.PieceNum >= req.StartNum && (count < req.Limit || req.Limit == 0) {
p := &base.PieceInfo{
PieceNum: piece.PieceNum,
PieceNum: int32(piece.PieceNum),
RangeStart: piece.PieceRange.StartIndex,
RangeSize: piece.PieceLen,
PieceMd5: piece.PieceMd5,
@ -241,19 +210,3 @@ func (css *server) GetPieceTasks(ctx context.Context, req *base.PieceTaskRequest
span.SetAttributes(config.AttributePiecePacketResult.String(pp.String()))
return pp, nil
}
func checkPieceTasksRequestParams(req *base.PieceTaskRequest) error {
if stringutils.IsBlank(req.TaskId) {
return errors.Wrap(cdnerrors.ErrInvalidValue, "taskId is nil")
}
if stringutils.IsBlank(req.SrcPid) {
return errors.Wrapf(cdnerrors.ErrInvalidValue, "src peer id is nil")
}
if req.StartNum < 0 {
return errors.Wrapf(cdnerrors.ErrInvalidValue, "invalid starNum %d", req.StartNum)
}
if req.Limit < 0 {
return errors.Wrapf(cdnerrors.ErrInvalidValue, "invalid limit %d", req.Limit)
}
return nil
}

View File

@ -23,7 +23,6 @@ import (
"d7y.io/dragonfly/v2/cdn/config"
"d7y.io/dragonfly/v2/cdn/supervisor"
"d7y.io/dragonfly/v2/cdn/types"
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/cdnsystem"
)
@ -121,69 +120,3 @@ func TestNewCdnSeedServer(t *testing.T) {
})
}
}
func Test_checkPieceTasksRequestParams(t *testing.T) {
type args struct {
req *base.PieceTaskRequest
}
tests := []struct {
name string
args args
wantErr bool
}{
// TODO: Add test cases.
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
if err := checkPieceTasksRequestParams(tt.args.req); (err != nil) != tt.wantErr {
t.Errorf("checkPieceTasksRequestParams() error = %v, wantErr %v", err, tt.wantErr)
}
})
}
}
func Test_checkSeedRequestParams(t *testing.T) {
type args struct {
req *cdnsystem.SeedRequest
}
tests := []struct {
name string
args args
wantErr bool
}{
// TODO: Add test cases.
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
if err := checkSeedRequestParams(tt.args.req); (err != nil) != tt.wantErr {
t.Errorf("checkSeedRequestParams() error = %v, wantErr %v", err, tt.wantErr)
}
})
}
}
func Test_constructRegisterRequest(t *testing.T) {
type args struct {
req *cdnsystem.SeedRequest
}
tests := []struct {
name string
args args
want *types.TaskRegisterRequest
wantErr bool
}{
// TODO: Add test cases.
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got, err := constructRegisterRequest(tt.args.req)
if (err != nil) != tt.wantErr {
t.Errorf("constructRegisterRequest() error = %v, wantErr %v", err, tt.wantErr)
return
}
if !reflect.DeepEqual(got, tt.want) {
t.Errorf("constructRegisterRequest() got = %v, want %v", got, tt.want)
}
})
}
}

View File

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

View File

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

View File

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

View File

@ -97,8 +97,8 @@ type FileMetaData struct {
// PieceMetaRecord meta data of piece
type PieceMetaRecord struct {
PieceNum int32 `json:"pieceNum"` // piece Num start from 0
PieceLen int32 `json:"pieceLen"` // 存储到存储介质的真实长度
PieceNum uint32 `json:"pieceNum"` // piece Num start from 0
PieceLen uint32 `json:"pieceLen"` // 存储到存储介质的真实长度
Md5 string `json:"md5"` // for transported piece content不是origin source 的 md5是真是存储到存储介质后的md5为了读取数据文件时方便校验完整性
Range *rangeutils.Range `json:"range"` // 下载存储到磁盘的range不是origin source的range.提供给客户端发送下载请求,for transported piece content
OriginRange *rangeutils.Range `json:"originRange"` // piece's real offset in the file
@ -141,8 +141,8 @@ func ParsePieceMetaRecord(value string) (record *PieceMetaRecord, err error) {
return nil, errors.Wrapf(err, "invalid pieceStyle: %s", fields[5])
}
return &PieceMetaRecord{
PieceNum: int32(pieceNum),
PieceLen: int32(pieceLen),
PieceNum: uint32(pieceNum),
PieceLen: uint32(pieceLen),
Md5: md5,
Range: pieceRange,
OriginRange: originRange,

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
if task.PieceSize <= 0 {
pieceSize := cdnutil.ComputePieceSize(task.SourceFileLength)
task.PieceSize = pieceSize
task.PieceSize = int32(pieceSize)
}
if err := tm.taskStore.Add(task.TaskID, task); err != nil {
return nil, err

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

2
go.mod
View File

@ -35,7 +35,7 @@ require (
github.com/google/go-querystring v1.1.0 // indirect
github.com/google/uuid v1.2.0
github.com/gorilla/mux v1.8.0
github.com/grpc-ecosystem/go-grpc-middleware v1.0.0
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0
github.com/jackc/pgproto3/v2 v2.1.1 // indirect
github.com/jarcoal/httpmock v1.0.8

5
go.sum
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/bitly/go-simplejson v0.5.0/go.mod h1:cXHtHw4XUPsvGaxgjIAn8PhEWG9NfngEKAMDJEczWVA=
github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84=
github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4YnC6+E63dPcxHo2sUxDIu8g3QgEJdRY=
github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4=
github.com/bradfitz/gomemcache v0.0.0-20190913173617-a41fca850d0b h1:L/QXpzIa3pOvUGt1D1lA5KjYhPBAN/3iWdP7xeFS9F0=
github.com/bradfitz/gomemcache v0.0.0-20190913173617-a41fca850d0b/go.mod h1:H0wQNHz2YrLsuXOZozoeDmnHXkNCRmMW0gwFWDfEZDA=
@ -393,6 +394,8 @@ github.com/gorilla/sessions v1.2.0/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/z
github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
github.com/grpc-ecosystem/go-grpc-middleware v1.0.0 h1:Iju5GlWwrvL6UBg4zJJt3btmonfrMlCDdsejg4CZE7c=
github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs=
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw=
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y=
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho=
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk=
github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
@ -657,6 +660,7 @@ github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3I
github.com/opencontainers/image-spec v1.0.1/go.mod h1:BtxoFyWECRxE4U/7sNtV5W15zMzWCbyJoFRP3s7yZA0=
github.com/opencontainers/image-spec v1.0.2 h1:9yCKha/T5XdGtO0q9Q9a6T5NUCsTn/DrBg0D7ufOcFM=
github.com/opencontainers/image-spec v1.0.2/go.mod h1:BtxoFyWECRxE4U/7sNtV5W15zMzWCbyJoFRP3s7yZA0=
github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs=
github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc=
github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
@ -1248,6 +1252,7 @@ google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfG
google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=

View File

@ -51,6 +51,6 @@ type GetCDNClustersQuery struct {
}
type CDNClusterConfig struct {
LoadLimit uint `yaml:"loadLimit" mapstructure:"loadLimit" json:"load_limit" binding:"omitempty,gte=1,lte=5000"`
LoadLimit uint32 `yaml:"loadLimit" mapstructure:"loadLimit" json:"load_limit" binding:"omitempty,gte=1,lte=5000"`
NetTopology string `yaml:"netTopology" mapstructure:"netTopology" json:"net_topology"`
}

View File

@ -55,7 +55,7 @@ type SchedulerClusterConfig struct {
}
type SchedulerClusterClientConfig struct {
LoadLimit uint `yaml:"loadLimit" mapstructure:"loadLimit" json:"load_limit" binding:"omitempty,gte=1,lte=5000"`
LoadLimit uint32 `yaml:"loadLimit" mapstructure:"loadLimit" json:"load_limit" binding:"omitempty,gte=1,lte=5000"`
}
type SchedulerClusterScopes struct {

View File

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

View File

@ -55,7 +55,16 @@ func (m *GrpcDfError) validate(all bool) error {
var errors []error
// no validation rules for Code
if _, ok := Code_name[int32(m.GetCode())]; !ok {
err := GrpcDfErrorValidationError{
field: "Code",
reason: "value must be one of the defined enum values",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for Message
@ -156,11 +165,37 @@ func (m *UrlMeta) validate(all bool) error {
var errors []error
// no validation rules for Digest
if m.GetDigest() != "" {
if !_UrlMeta_Digest_Pattern.MatchString(m.GetDigest()) {
err := UrlMetaValidationError{
field: "Digest",
reason: "value does not match regex pattern \"^(md5)|(sha256):[A-Fa-f0-9]+\"",
}
if !all {
return err
}
errors = append(errors, err)
}
}
// no validation rules for Tag
// no validation rules for Range
if m.GetRange() != "" {
if !_UrlMeta_Range_Pattern.MatchString(m.GetRange()) {
err := UrlMetaValidationError{
field: "Range",
reason: "value does not match regex pattern \"^[0-9]+\\\\-^[0-9]+\"",
}
if !all {
return err
}
errors = append(errors, err)
}
}
// no validation rules for Filter
@ -242,6 +277,10 @@ var _ interface {
ErrorName() string
} = UrlMetaValidationError{}
var _UrlMeta_Digest_Pattern = regexp.MustCompile("^(md5)|(sha256):[A-Fa-f0-9]+")
var _UrlMeta_Range_Pattern = regexp.MustCompile("^[0-9]+\\-^[0-9]+")
// Validate checks the field values on HostLoad with the rules defined in the
// proto definition for this message. If any rules are violated, the first
// error encountered is returned, or nil if there are no violations.
@ -264,11 +303,38 @@ func (m *HostLoad) validate(all bool) error {
var errors []error
// no validation rules for CpuRatio
if val := m.GetCpuRatio(); val < 0 || val > 1 {
err := HostLoadValidationError{
field: "CpuRatio",
reason: "value must be inside range [0, 1]",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for MemRatio
if val := m.GetMemRatio(); val < 0 || val > 1 {
err := HostLoadValidationError{
field: "MemRatio",
reason: "value must be inside range [0, 1]",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for DiskRatio
if val := m.GetDiskRatio(); val < 0 || val > 1 {
err := HostLoadValidationError{
field: "DiskRatio",
reason: "value must be inside range [0, 1]",
}
if !all {
return err
}
errors = append(errors, err)
}
if len(errors) > 0 {
return HostLoadMultiError(errors)
@ -368,15 +434,60 @@ func (m *PieceTaskRequest) validate(all bool) error {
var errors []error
// no validation rules for TaskId
if utf8.RuneCountInString(m.GetTaskId()) < 1 {
err := PieceTaskRequestValidationError{
field: "TaskId",
reason: "value length must be at least 1 runes",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for SrcPid
if utf8.RuneCountInString(m.GetSrcPid()) < 1 {
err := PieceTaskRequestValidationError{
field: "SrcPid",
reason: "value length must be at least 1 runes",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for DstPid
if utf8.RuneCountInString(m.GetDstPid()) < 1 {
err := PieceTaskRequestValidationError{
field: "DstPid",
reason: "value length must be at least 1 runes",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for StartNum
if m.GetStartNum() < 0 {
err := PieceTaskRequestValidationError{
field: "StartNum",
reason: "value must be greater than or equal to 0",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for Limit
if m.GetLimit() < 0 {
err := PieceTaskRequestValidationError{
field: "Limit",
reason: "value must be greater than or equal to 0",
}
if !all {
return err
}
errors = append(errors, err)
}
if len(errors) > 0 {
return PieceTaskRequestMultiError(errors)
@ -479,13 +590,53 @@ func (m *PieceInfo) validate(all bool) error {
// no validation rules for PieceNum
// no validation rules for RangeStart
if m.GetRangeStart() < 0 {
err := PieceInfoValidationError{
field: "RangeStart",
reason: "value must be greater than or equal to 0",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for RangeSize
if m.GetRangeSize() < 0 {
err := PieceInfoValidationError{
field: "RangeSize",
reason: "value must be greater than or equal to 0",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for PieceMd5
if m.GetPieceMd5() != "" {
// no validation rules for PieceOffset
if !_PieceInfo_PieceMd5_Pattern.MatchString(m.GetPieceMd5()) {
err := PieceInfoValidationError{
field: "PieceMd5",
reason: "value does not match regex pattern \"([a-f\\\\d]{32}|[A-F\\\\d]{32}|[a-f\\\\d]{16}|[A-F\\\\d]{16})\"",
}
if !all {
return err
}
errors = append(errors, err)
}
}
if m.GetPieceOffset() < 0 {
err := PieceInfoValidationError{
field: "PieceOffset",
reason: "value must be greater than or equal to 0",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for PieceStyle
@ -565,6 +716,8 @@ var _ interface {
ErrorName() string
} = PieceInfoValidationError{}
var _PieceInfo_PieceMd5_Pattern = regexp.MustCompile("([a-f\\d]{32}|[A-F\\d]{32}|[a-f\\d]{16}|[A-F\\d]{16})")
// Validate checks the field values on PiecePacket with the rules defined in
// the proto definition for this message. If any rules are violated, the first
// error encountered is returned, or nil if there are no violations.
@ -587,11 +740,38 @@ func (m *PiecePacket) validate(all bool) error {
var errors []error
// no validation rules for TaskId
if utf8.RuneCountInString(m.GetTaskId()) < 1 {
err := PiecePacketValidationError{
field: "TaskId",
reason: "value length must be at least 1 runes",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for DstPid
if utf8.RuneCountInString(m.GetDstPid()) < 1 {
err := PiecePacketValidationError{
field: "DstPid",
reason: "value length must be at least 1 runes",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for DstAddr
if utf8.RuneCountInString(m.GetDstAddr()) < 1 {
err := PiecePacketValidationError{
field: "DstAddr",
reason: "value length must be at least 1 runes",
}
if !all {
return err
}
errors = append(errors, err)
}
for idx, item := range m.GetPieceInfos() {
_, _ = idx, item

View File

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

View File

@ -23,6 +23,7 @@ package cdnsystem
import (
base "d7y.io/dragonfly/v2/pkg/rpc/base"
_ "github.com/envoyproxy/protoc-gen-validate/validate"
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
reflect "reflect"
@ -113,9 +114,9 @@ type PieceSeed struct {
PieceInfo *base.PieceInfo `protobuf:"bytes,4,opt,name=piece_info,json=pieceInfo,proto3" json:"piece_info,omitempty"`
// whether or not all seeds are downloaded
Done bool `protobuf:"varint,5,opt,name=done,proto3" json:"done,omitempty"`
// content total length for the url
// content total length for the url, content_length < 0 represent content length is unknown
ContentLength int64 `protobuf:"varint,6,opt,name=content_length,json=contentLength,proto3" json:"content_length,omitempty"`
// total piece count
// total piece count, -1 represents task is downloading or failed
TotalPieceCount int32 `protobuf:"varint,7,opt,name=total_piece_count,json=totalPieceCount,proto3" json:"total_piece_count,omitempty"`
}
@ -200,38 +201,42 @@ var file_pkg_rpc_cdnsystem_cdnsystem_proto_rawDesc = []byte{
0x74, 0x65, 0x6d, 0x2f, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x12, 0x09, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x1a, 0x17,
0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x62, 0x61, 0x73, 0x65, 0x2f, 0x62, 0x61, 0x73,
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x62, 0x0a, 0x0b, 0x53, 0x65, 0x65, 0x64, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69,
0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12,
0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72,
0x6c, 0x12, 0x28, 0x0a, 0x08, 0x75, 0x72, 0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20,
0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65,
0x74, 0x61, 0x52, 0x07, 0x75, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x22, 0xd8, 0x01, 0x0a, 0x09,
0x50, 0x69, 0x65, 0x63, 0x65, 0x53, 0x65, 0x65, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65,
0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72,
0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18,
0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x55, 0x75, 0x69, 0x64, 0x12,
0x2e, 0x0a, 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20,
0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65,
0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12,
0x12, 0x0a, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x64,
0x6f, 0x6e, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c,
0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e,
0x74, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x2a, 0x0a, 0x11, 0x74, 0x6f,
0x74, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18,
0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65, 0x63,
0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x32, 0x83, 0x01, 0x0a, 0x06, 0x53, 0x65, 0x65, 0x64, 0x65,
0x72, 0x12, 0x3d, 0x0a, 0x0b, 0x4f, 0x62, 0x74, 0x61, 0x69, 0x6e, 0x53, 0x65, 0x65, 0x64, 0x73,
0x12, 0x16, 0x2e, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x2e, 0x53, 0x65, 0x65,
0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x63, 0x64, 0x6e, 0x73, 0x79,
0x73, 0x74, 0x65, 0x6d, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x53, 0x65, 0x65, 0x64, 0x30, 0x01,
0x12, 0x3a, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b,
0x73, 0x12, 0x16, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61,
0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x62, 0x61, 0x73, 0x65,
0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x42, 0x27, 0x5a, 0x25,
0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79,
0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x63, 0x64, 0x6e, 0x73,
0x79, 0x73, 0x74, 0x65, 0x6d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x17, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74,
0x65, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x22, 0x75, 0x0a, 0x0b, 0x53, 0x65, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49,
0x64, 0x12, 0x1a, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x08,
0xfa, 0x42, 0x05, 0x72, 0x03, 0x88, 0x01, 0x01, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x28, 0x0a,
0x08, 0x75, 0x72, 0x6c, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x0d, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x55, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x07,
0x75, 0x72, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x22, 0xea, 0x01, 0x0a, 0x09, 0x50, 0x69, 0x65, 0x63,
0x65, 0x53, 0x65, 0x65, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64,
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52,
0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f,
0x75, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72,
0x02, 0x10, 0x01, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x55, 0x75, 0x69, 0x64, 0x12, 0x2e, 0x0a,
0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28,
0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e,
0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x12, 0x0a,
0x04, 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x64, 0x6f, 0x6e,
0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x65, 0x6e,
0x67, 0x74, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65,
0x6e, 0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x2a, 0x0a, 0x11, 0x74, 0x6f, 0x74, 0x61,
0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x07, 0x20,
0x01, 0x28, 0x05, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65, 0x63, 0x65, 0x43,
0x6f, 0x75, 0x6e, 0x74, 0x32, 0x83, 0x01, 0x0a, 0x06, 0x53, 0x65, 0x65, 0x64, 0x65, 0x72, 0x12,
0x3d, 0x0a, 0x0b, 0x4f, 0x62, 0x74, 0x61, 0x69, 0x6e, 0x53, 0x65, 0x65, 0x64, 0x73, 0x12, 0x16,
0x2e, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x2e, 0x53, 0x65, 0x65, 0x64, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73, 0x74,
0x65, 0x6d, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x53, 0x65, 0x65, 0x64, 0x30, 0x01, 0x12, 0x3a,
0x0a, 0x0d, 0x47, 0x65, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x73, 0x12,
0x16, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x54, 0x61, 0x73, 0x6b,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50,
0x69, 0x65, 0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x42, 0x27, 0x5a, 0x25, 0x64, 0x37,
0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f, 0x76,
0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x63, 0x64, 0x6e, 0x73, 0x79, 0x73,
0x74, 0x65, 0x6d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (

View File

@ -34,18 +34,79 @@ var (
)
// Validate checks the field values on SeedRequest with the rules defined in
// the proto definition for this message. If any rules are violated, an error
// is returned.
// the proto definition for this message. If any rules are violated, the first
// error encountered is returned, or nil if there are no violations.
func (m *SeedRequest) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on SeedRequest with the rules defined in
// the proto definition for this message. If any rules are violated, the
// result is a list of violation errors wrapped in SeedRequestMultiError, or
// nil if none found.
func (m *SeedRequest) ValidateAll() error {
return m.validate(true)
}
func (m *SeedRequest) validate(all bool) error {
if m == nil {
return nil
}
// no validation rules for TaskId
var errors []error
// no validation rules for Url
if utf8.RuneCountInString(m.GetTaskId()) < 1 {
err := SeedRequestValidationError{
field: "TaskId",
reason: "value length must be at least 1 runes",
}
if !all {
return err
}
errors = append(errors, err)
}
if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok {
if uri, err := url.Parse(m.GetUrl()); err != nil {
err = SeedRequestValidationError{
field: "Url",
reason: "value must be a valid URI",
cause: err,
}
if !all {
return err
}
errors = append(errors, err)
} else if !uri.IsAbs() {
err := SeedRequestValidationError{
field: "Url",
reason: "value must be absolute",
}
if !all {
return err
}
errors = append(errors, err)
}
if all {
switch v := interface{}(m.GetUrlMeta()).(type) {
case interface{ ValidateAll() error }:
if err := v.ValidateAll(); err != nil {
errors = append(errors, SeedRequestValidationError{
field: "UrlMeta",
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, SeedRequestValidationError{
field: "UrlMeta",
reason: "embedded message failed validation",
cause: err,
})
}
}
} else if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok {
if err := v.Validate(); err != nil {
return SeedRequestValidationError{
field: "UrlMeta",
@ -55,9 +116,28 @@ func (m *SeedRequest) Validate() error {
}
}
if len(errors) > 0 {
return SeedRequestMultiError(errors)
}
return nil
}
// SeedRequestMultiError is an error wrapping multiple validation errors
// returned by SeedRequest.ValidateAll() if the designated constraints aren't met.
type SeedRequestMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m SeedRequestMultiError) Error() string {
var msgs []string
for _, err := range m {
msgs = append(msgs, err.Error())
}
return strings.Join(msgs, "; ")
}
// AllErrors returns a list of validation violation errors.
func (m SeedRequestMultiError) AllErrors() []error { return m }
// SeedRequestValidationError is the validation error returned by
// SeedRequest.Validate if the designated constraints aren't met.
type SeedRequestValidationError struct {
@ -113,17 +193,69 @@ var _ interface {
} = SeedRequestValidationError{}
// Validate checks the field values on PieceSeed with the rules defined in the
// proto definition for this message. If any rules are violated, an error is returned.
// proto definition for this message. If any rules are violated, the first
// error encountered is returned, or nil if there are no violations.
func (m *PieceSeed) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on PieceSeed with the rules defined in
// the proto definition for this message. If any rules are violated, the
// result is a list of violation errors wrapped in PieceSeedMultiError, or nil
// if none found.
func (m *PieceSeed) ValidateAll() error {
return m.validate(true)
}
func (m *PieceSeed) validate(all bool) error {
if m == nil {
return nil
}
// no validation rules for PeerId
var errors []error
// no validation rules for HostUuid
if utf8.RuneCountInString(m.GetPeerId()) < 1 {
err := PieceSeedValidationError{
field: "PeerId",
reason: "value length must be at least 1 runes",
}
if !all {
return err
}
errors = append(errors, err)
}
if v, ok := interface{}(m.GetPieceInfo()).(interface{ Validate() error }); ok {
if utf8.RuneCountInString(m.GetHostUuid()) < 1 {
err := PieceSeedValidationError{
field: "HostUuid",
reason: "value length must be at least 1 runes",
}
if !all {
return err
}
errors = append(errors, err)
}
if all {
switch v := interface{}(m.GetPieceInfo()).(type) {
case interface{ ValidateAll() error }:
if err := v.ValidateAll(); err != nil {
errors = append(errors, PieceSeedValidationError{
field: "PieceInfo",
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, PieceSeedValidationError{
field: "PieceInfo",
reason: "embedded message failed validation",
cause: err,
})
}
}
} else if v, ok := interface{}(m.GetPieceInfo()).(interface{ Validate() error }); ok {
if err := v.Validate(); err != nil {
return PieceSeedValidationError{
field: "PieceInfo",
@ -139,9 +271,28 @@ func (m *PieceSeed) Validate() error {
// no validation rules for TotalPieceCount
if len(errors) > 0 {
return PieceSeedMultiError(errors)
}
return nil
}
// PieceSeedMultiError is an error wrapping multiple validation errors returned
// by PieceSeed.ValidateAll() if the designated constraints aren't met.
type PieceSeedMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m PieceSeedMultiError) Error() string {
var msgs []string
for _, err := range m {
msgs = append(msgs, err.Error())
}
return strings.Join(msgs, "; ")
}
// AllErrors returns a list of validation violation errors.
func (m PieceSeedMultiError) AllErrors() []error { return m }
// PieceSeedValidationError is the validation error returned by
// PieceSeed.Validate if the designated constraints aren't met.
type PieceSeedValidationError struct {

View File

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

View File

@ -12,7 +12,6 @@ import (
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7
// SeederClient is the client API for Seeder service.
@ -34,7 +33,7 @@ func NewSeederClient(cc grpc.ClientConnInterface) SeederClient {
}
func (c *seederClient) ObtainSeeds(ctx context.Context, in *SeedRequest, opts ...grpc.CallOption) (Seeder_ObtainSeedsClient, error) {
stream, err := c.cc.NewStream(ctx, &Seeder_ServiceDesc.Streams[0], "/cdnsystem.Seeder/ObtainSeeds", opts...)
stream, err := c.cc.NewStream(ctx, &_Seeder_serviceDesc.Streams[0], "/cdnsystem.Seeder/ObtainSeeds", opts...)
if err != nil {
return nil, err
}
@ -105,7 +104,7 @@ type UnsafeSeederServer interface {
}
func RegisterSeederServer(s grpc.ServiceRegistrar, srv SeederServer) {
s.RegisterService(&Seeder_ServiceDesc, srv)
s.RegisterService(&_Seeder_serviceDesc, srv)
}
func _Seeder_ObtainSeeds_Handler(srv interface{}, stream grpc.ServerStream) error {
@ -147,10 +146,7 @@ func _Seeder_GetPieceTasks_Handler(srv interface{}, ctx context.Context, dec fun
return interceptor(ctx, in, info, handler)
}
// Seeder_ServiceDesc is the grpc.ServiceDesc for Seeder service.
// It's only intended for direct use with grpc.RegisterService,
// and not to be introspected or modified (even as a copy)
var Seeder_ServiceDesc = grpc.ServiceDesc{
var _Seeder_serviceDesc = grpc.ServiceDesc{
ServiceName: "cdnsystem.Seeder",
HandlerType: (*SeederServer)(nil),
Methods: []grpc.MethodDesc{

View File

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

View File

@ -33,27 +33,119 @@ var (
_ = anypb.Any{}
)
// define the regex for a UUID once up-front
var _dfdaemon_uuidPattern = regexp.MustCompile("^[0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}$")
// Validate checks the field values on DownRequest with the rules defined in
// the proto definition for this message. If any rules are violated, an error
// is returned.
// the proto definition for this message. If any rules are violated, the first
// error encountered is returned, or nil if there are no violations.
func (m *DownRequest) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on DownRequest with the rules defined in
// the proto definition for this message. If any rules are violated, the
// result is a list of violation errors wrapped in DownRequestMultiError, or
// nil if none found.
func (m *DownRequest) ValidateAll() error {
return m.validate(true)
}
func (m *DownRequest) validate(all bool) error {
if m == nil {
return nil
}
// no validation rules for Uuid
var errors []error
// no validation rules for Url
if err := m._validateUuid(m.GetUuid()); err != nil {
err = DownRequestValidationError{
field: "Uuid",
reason: "value must be a valid UUID",
cause: err,
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for Output
if uri, err := url.Parse(m.GetUrl()); err != nil {
err = DownRequestValidationError{
field: "Url",
reason: "value must be a valid URI",
cause: err,
}
if !all {
return err
}
errors = append(errors, err)
} else if !uri.IsAbs() {
err := DownRequestValidationError{
field: "Url",
reason: "value must be absolute",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for Timeout
if utf8.RuneCountInString(m.GetOutput()) < 1 {
err := DownRequestValidationError{
field: "Output",
reason: "value length must be at least 1 runes",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for Limit
if m.GetTimeout() < 0 {
err := DownRequestValidationError{
field: "Timeout",
reason: "value must be greater than or equal to 0",
}
if !all {
return err
}
errors = append(errors, err)
}
if m.GetLimit() < 0 {
err := DownRequestValidationError{
field: "Limit",
reason: "value must be greater than or equal to 0",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for DisableBackSource
if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok {
if all {
switch v := interface{}(m.GetUrlMeta()).(type) {
case interface{ ValidateAll() error }:
if err := v.ValidateAll(); err != nil {
errors = append(errors, DownRequestValidationError{
field: "UrlMeta",
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, DownRequestValidationError{
field: "UrlMeta",
reason: "embedded message failed validation",
cause: err,
})
}
}
} else if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok {
if err := v.Validate(); err != nil {
return DownRequestValidationError{
field: "UrlMeta",
@ -63,7 +155,20 @@ func (m *DownRequest) Validate() error {
}
}
// no validation rules for Pattern
if m.GetPattern() != "" {
if _, ok := _DownRequest_Pattern_InLookup[m.GetPattern()]; !ok {
err := DownRequestValidationError{
field: "Pattern",
reason: "value must be in list [p2p cdn source]",
}
if !all {
return err
}
errors = append(errors, err)
}
}
// no validation rules for Callsystem
@ -71,9 +176,36 @@ func (m *DownRequest) Validate() error {
// no validation rules for Gid
if len(errors) > 0 {
return DownRequestMultiError(errors)
}
return nil
}
func (m *DownRequest) _validateUuid(uuid string) error {
if matched := _dfdaemon_uuidPattern.MatchString(uuid); !matched {
return errors.New("invalid uuid format")
}
return nil
}
// DownRequestMultiError is an error wrapping multiple validation errors
// returned by DownRequest.ValidateAll() if the designated constraints aren't met.
type DownRequestMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m DownRequestMultiError) Error() string {
var msgs []string
for _, err := range m {
msgs = append(msgs, err.Error())
}
return strings.Join(msgs, "; ")
}
// AllErrors returns a list of validation violation errors.
func (m DownRequestMultiError) AllErrors() []error { return m }
// DownRequestValidationError is the validation error returned by
// DownRequest.Validate if the designated constraints aren't met.
type DownRequestValidationError struct {
@ -128,24 +260,91 @@ var _ interface {
ErrorName() string
} = DownRequestValidationError{}
var _DownRequest_Pattern_InLookup = map[string]struct{}{
"p2p": {},
"cdn": {},
"source": {},
}
// Validate checks the field values on DownResult with the rules defined in the
// proto definition for this message. If any rules are violated, an error is returned.
// proto definition for this message. If any rules are violated, the first
// error encountered is returned, or nil if there are no violations.
func (m *DownResult) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on DownResult with the rules defined in
// the proto definition for this message. If any rules are violated, the
// result is a list of violation errors wrapped in DownResultMultiError, or
// nil if none found.
func (m *DownResult) ValidateAll() error {
return m.validate(true)
}
func (m *DownResult) validate(all bool) error {
if m == nil {
return nil
}
// no validation rules for TaskId
var errors []error
// no validation rules for PeerId
if utf8.RuneCountInString(m.GetTaskId()) < 1 {
err := DownResultValidationError{
field: "TaskId",
reason: "value length must be at least 1 runes",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for CompletedLength
if utf8.RuneCountInString(m.GetPeerId()) < 1 {
err := DownResultValidationError{
field: "PeerId",
reason: "value length must be at least 1 runes",
}
if !all {
return err
}
errors = append(errors, err)
}
if m.GetCompletedLength() < 0 {
err := DownResultValidationError{
field: "CompletedLength",
reason: "value must be greater than or equal to 0",
}
if !all {
return err
}
errors = append(errors, err)
}
// no validation rules for Done
if len(errors) > 0 {
return DownResultMultiError(errors)
}
return nil
}
// DownResultMultiError is an error wrapping multiple validation errors
// returned by DownResult.ValidateAll() if the designated constraints aren't met.
type DownResultMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m DownResultMultiError) Error() string {
var msgs []string
for _, err := range m {
msgs = append(msgs, err.Error())
}
return strings.Join(msgs, "; ")
}
// AllErrors returns a list of validation violation errors.
func (m DownResultMultiError) AllErrors() []error { return m }
// DownResultValidationError is the validation error returned by
// DownResult.Validate if the designated constraints aren't met.
type DownResultValidationError struct {

View File

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

View File

@ -13,7 +13,6 @@ import (
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7
// DaemonClient is the client API for Daemon service.
@ -37,7 +36,7 @@ func NewDaemonClient(cc grpc.ClientConnInterface) DaemonClient {
}
func (c *daemonClient) Download(ctx context.Context, in *DownRequest, opts ...grpc.CallOption) (Daemon_DownloadClient, error) {
stream, err := c.cc.NewStream(ctx, &Daemon_ServiceDesc.Streams[0], "/dfdaemon.Daemon/Download", opts...)
stream, err := c.cc.NewStream(ctx, &_Daemon_serviceDesc.Streams[0], "/dfdaemon.Daemon/Download", opts...)
if err != nil {
return nil, err
}
@ -122,7 +121,7 @@ type UnsafeDaemonServer interface {
}
func RegisterDaemonServer(s grpc.ServiceRegistrar, srv DaemonServer) {
s.RegisterService(&Daemon_ServiceDesc, srv)
s.RegisterService(&_Daemon_serviceDesc, srv)
}
func _Daemon_Download_Handler(srv interface{}, stream grpc.ServerStream) error {
@ -182,10 +181,7 @@ func _Daemon_CheckHealth_Handler(srv interface{}, ctx context.Context, dec func(
return interceptor(ctx, in, info, handler)
}
// Daemon_ServiceDesc is the grpc.ServiceDesc for Daemon service.
// It's only intended for direct use with grpc.RegisterService,
// and not to be introspected or modified (even as a copy)
var Daemon_ServiceDesc = grpc.ServiceDesc{
var _Daemon_serviceDesc = grpc.ServiceDesc{
ServiceName: "dfdaemon.Daemon",
HandlerType: (*DaemonServer)(nil),
Methods: []grpc.MethodDesc{

View File

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

File diff suppressed because it is too large Load Diff

View File

@ -12,7 +12,6 @@ import (
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7
// ManagerClient is the client API for Manager service.
@ -87,7 +86,7 @@ func (c *managerClient) ListSchedulers(ctx context.Context, in *ListSchedulersRe
}
func (c *managerClient) KeepAlive(ctx context.Context, opts ...grpc.CallOption) (Manager_KeepAliveClient, error) {
stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[0], "/manager.Manager/KeepAlive", opts...)
stream, err := c.cc.NewStream(ctx, &_Manager_serviceDesc.Streams[0], "/manager.Manager/KeepAlive", opts...)
if err != nil {
return nil, err
}
@ -171,7 +170,7 @@ type UnsafeManagerServer interface {
}
func RegisterManagerServer(s grpc.ServiceRegistrar, srv ManagerServer) {
s.RegisterService(&Manager_ServiceDesc, srv)
s.RegisterService(&_Manager_serviceDesc, srv)
}
func _Manager_GetCDN_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
@ -290,10 +289,7 @@ func (x *managerKeepAliveServer) Recv() (*KeepAliveRequest, error) {
return m, nil
}
// Manager_ServiceDesc is the grpc.ServiceDesc for Manager service.
// It's only intended for direct use with grpc.RegisterService,
// and not to be introspected or modified (even as a copy)
var Manager_ServiceDesc = grpc.ServiceDesc{
var _Manager_serviceDesc = grpc.ServiceDesc{
ServiceName: "manager.Manager",
HandlerType: (*ManagerServer)(nil),
Methods: []grpc.MethodDesc{

View File

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

File diff suppressed because it is too large Load Diff

View File

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

View File

@ -12,7 +12,6 @@ import (
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7
// SchedulerClient is the client API for Scheduler service.
@ -49,7 +48,7 @@ func (c *schedulerClient) RegisterPeerTask(ctx context.Context, in *PeerTaskRequ
}
func (c *schedulerClient) ReportPieceResult(ctx context.Context, opts ...grpc.CallOption) (Scheduler_ReportPieceResultClient, error) {
stream, err := c.cc.NewStream(ctx, &Scheduler_ServiceDesc.Streams[0], "/scheduler.Scheduler/ReportPieceResult", opts...)
stream, err := c.cc.NewStream(ctx, &_Scheduler_serviceDesc.Streams[0], "/scheduler.Scheduler/ReportPieceResult", opts...)
if err != nil {
return nil, err
}
@ -140,7 +139,7 @@ type UnsafeSchedulerServer interface {
}
func RegisterSchedulerServer(s grpc.ServiceRegistrar, srv SchedulerServer) {
s.RegisterService(&Scheduler_ServiceDesc, srv)
s.RegisterService(&_Scheduler_serviceDesc, srv)
}
func _Scheduler_RegisterPeerTask_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
@ -223,10 +222,7 @@ func _Scheduler_LeaveTask_Handler(srv interface{}, ctx context.Context, dec func
return interceptor(ctx, in, info, handler)
}
// Scheduler_ServiceDesc is the grpc.ServiceDesc for Scheduler service.
// It's only intended for direct use with grpc.RegisterService,
// and not to be introspected or modified (even as a copy)
var Scheduler_ServiceDesc = grpc.ServiceDesc{
var _Scheduler_serviceDesc = grpc.ServiceDesc{
ServiceName: "scheduler.Scheduler",
HandlerType: (*SchedulerServer)(nil),
Methods: []grpc.MethodDesc{

View File

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

View File

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

View File

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

View File

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

View File

@ -31,8 +31,6 @@ import (
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
schedulerserver "d7y.io/dragonfly/v2/pkg/rpc/scheduler/server"
"d7y.io/dragonfly/v2/pkg/util/net/urlutils"
"d7y.io/dragonfly/v2/pkg/util/stringutils"
"d7y.io/dragonfly/v2/scheduler/config"
"d7y.io/dragonfly/v2/scheduler/core"
"d7y.io/dragonfly/v2/scheduler/supervisor"
@ -65,12 +63,6 @@ func (s *server) RegisterPeerTask(ctx context.Context, request *scheduler.PeerTa
defer span.End()
logger.Debugf("register peer task, req: %+v", request)
resp = new(scheduler.RegisterResult)
if verifyErr := validateParams(request); verifyErr != nil {
err = dferrors.Newf(base.Code_BadRequest, "bad request param: %v", verifyErr)
logger.Errorf("register request: %v", err)
span.RecordError(err)
return
}
taskID := idgen.TaskID(request.Url, request.UrlMeta)
span.SetAttributes(config.AttributeTaskID.String(taskID))
@ -215,18 +207,6 @@ func (s *server) LeaveTask(ctx context.Context, target *scheduler.PeerTarget) (e
return s.service.HandleLeaveTask(ctx, peer)
}
// validateParams validates the params of scheduler.PeerTaskRequest.
func validateParams(req *scheduler.PeerTaskRequest) error {
if !urlutils.IsValidURL(req.Url) {
return fmt.Errorf("invalid url: %s", req.Url)
}
if stringutils.IsEmpty(req.PeerId) {
return fmt.Errorf("empty peerID")
}
return nil
}
func getTaskSizeScope(task *supervisor.Task) base.SizeScope {
if task.IsSuccess() {
if task.ContentLength.Load() <= supervisor.TinyFileSize {

View File

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

View File

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