feat: client back source (#579)

* feat: client back source

Signed-off-by: santong <244372610@qq.com>
This commit is contained in:
sunwp 2021-09-06 10:28:07 +08:00 committed by Gaius
parent 3c917604d9
commit 60bee061d2
No known key found for this signature in database
GPG Key ID: 8B4E5D1290FA2FFB
52 changed files with 2676 additions and 809 deletions

View File

@ -207,6 +207,7 @@ func (css *CdnSeedServer) GetPieceTasks(ctx context.Context, req *base.PieceTask
logger.WithTaskID(req.TaskId).Errorf("failed to get piece tasks, req=%+v: %v", req, err)
}
}()
logger.Infof("get piece tasks: %+v", req)
if err := checkPieceTasksRequestParams(req); err != nil {
err = dferrors.Newf(dfcodes.BadRequest, "failed to validate seed request for task(%s): %v", req.TaskId, err)
span.RecordError(err)

View File

@ -68,6 +68,7 @@ type peerTask struct {
backSourceFunc func()
reportPieceResultFunc func(result *pieceTaskResult) error
setContentLengthFunc func(i int64) error
setTotalPiecesFunc func(i int32)
request *scheduler.PeerTaskRequest
@ -172,6 +173,10 @@ func (pt *peerTask) GetTotalPieces() int32 {
return pt.totalPiece
}
func (pt *peerTask) SetTotalPieces(i int32) {
pt.setTotalPiecesFunc(i)
}
func (pt *peerTask) Context() context.Context {
return pt.ctx
}
@ -369,6 +374,7 @@ func (pt *peerTask) pullPiecesFromPeers(cleanUnfinishedFunc func()) {
}()
if !pt.waitFirstPeerPacket() {
// TODO 如果是客户端直接回源,这里不应该在输出错误日志
pt.Errorf("wait first peer packet error")
return
}
@ -650,7 +656,7 @@ func (pt *peerTask) downloadPieceWorker(id int32, pti Task, requests chan *Downl
TaskId: pt.GetTaskID(),
SrcPid: pt.GetPeerID(),
DstPid: request.DstPid,
PieceNum: request.piece.PieceNum,
PieceInfo: request.piece,
Success: false,
Code: dfcodes.ClientRequestLimitFail,
HostLoad: nil,
@ -761,6 +767,7 @@ retry:
TaskId: pt.taskID,
SrcPid: pt.peerID,
DstPid: peer.PeerId,
PieceInfo: &base.PieceInfo{},
Success: false,
Code: code,
HostLoad: nil,
@ -805,6 +812,7 @@ func (pt *peerTask) getPieceTasks(span trace.Span, curPeerPacket *scheduler.Peer
TaskId: pt.taskID,
SrcPid: pt.peerID,
DstPid: peer.PeerId,
PieceInfo: &base.PieceInfo{},
Success: false,
Code: dfcodes.ClientWaitPieceReady,
HostLoad: nil,

View File

@ -208,6 +208,7 @@ func newFilePeerTask(ctx context.Context,
// bind func that base peer task did not implement
pt.backSourceFunc = pt.backSource
pt.setContentLengthFunc = pt.SetContentLength
pt.setTotalPiecesFunc = pt.SetTotalPieces
pt.reportPieceResultFunc = pt.ReportPieceResult
return ctx, pt, nil, nil
}
@ -237,19 +238,19 @@ func (pt *filePeerTask) ReportPieceResult(result *pieceTaskResult) error {
if !result.pieceResult.Success {
result.pieceResult.FinishedCount = pt.readyPieces.Settled()
_ = pt.peerPacketStream.Send(result.pieceResult)
pt.failedPieceCh <- result.pieceResult.PieceNum
pt.failedPieceCh <- result.pieceResult.PieceInfo.PieceNum
pt.Errorf("%d download failed, retry later", result.piece.PieceNum)
return nil
}
pt.lock.Lock()
if pt.readyPieces.IsSet(result.pieceResult.PieceNum) {
if pt.readyPieces.IsSet(result.pieceResult.PieceInfo.PieceNum) {
pt.lock.Unlock()
pt.Warnf("piece %d is already reported, skipped", result.pieceResult.PieceNum)
pt.Warnf("piece %d is already reported, skipped", result.pieceResult.PieceInfo.PieceNum)
return nil
}
// mark piece processed
pt.readyPieces.Set(result.pieceResult.PieceNum)
pt.readyPieces.Set(result.pieceResult.PieceInfo.PieceNum)
pt.completedLength.Add(int64(result.piece.RangeSize))
pt.lock.Unlock()
@ -420,6 +421,7 @@ func (pt *filePeerTask) cleanUnfinished() {
})
}
// TODO SetContentLength 需要和pt.finish解绑以便在下载进度处可以看到文件长度
func (pt *filePeerTask) SetContentLength(i int64) error {
pt.contentLength.Store(i)
if !pt.isCompleted() {
@ -429,6 +431,10 @@ func (pt *filePeerTask) SetContentLength(i int64) error {
return pt.finish()
}
func (pt *filePeerTask) SetTotalPieces(i int32) {
pt.totalPiece = i
}
func (pt *filePeerTask) backSource() {
defer pt.cleanUnfinished()
if pt.disableBackSource {
@ -436,6 +442,13 @@ func (pt *filePeerTask) backSource() {
pt.failedReason = reasonBackSourceDisabled
return
}
_ = pt.callback.Init(pt)
if peerPacketStream, err := pt.schedulerClient.ReportPieceResult(pt.ctx, pt.taskID, pt.request); err != nil {
logger.Errorf("step 2: peer %s report piece failed: err", pt.request.PeerId, err)
} else {
pt.peerPacketStream = peerPacketStream
}
logger.Infof("step 2: start report peer %s back source piece result", pt.request.PeerId)
err := pt.pieceManager.DownloadSource(pt.ctx, pt, pt.request)
if err != nil {
pt.Errorf("download from source error: %s", err)
@ -444,5 +457,4 @@ func (pt *filePeerTask) backSource() {
}
pt.Infof("download from source ok")
_ = pt.finish()
return
}

View File

@ -17,7 +17,6 @@
package peer
import (
"context"
"time"
"d7y.io/dragonfly/v2/client/daemon/storage"
@ -92,18 +91,19 @@ func (p *filePeerTaskCallback) Done(pt Task) error {
return e
}
p.ptm.PeerTaskDone(p.req.PeerId)
err := p.pt.schedulerClient.ReportPeerResult(context.Background(), &scheduler.PeerResult{
TaskId: pt.GetTaskID(),
PeerId: pt.GetPeerID(),
SrcIp: p.ptm.host.Ip,
SecurityDomain: p.ptm.host.SecurityDomain,
Idc: p.ptm.host.Idc,
Url: p.req.Url,
ContentLength: pt.GetContentLength(),
Traffic: pt.GetTraffic(),
Cost: uint32(cost),
Success: true,
Code: dfcodes.Success,
err := p.pt.schedulerClient.ReportPeerResult(p.pt.ctx, &scheduler.PeerResult{
TaskId: pt.GetTaskID(),
PeerId: pt.GetPeerID(),
SrcIp: p.ptm.host.Ip,
SecurityDomain: p.ptm.host.SecurityDomain,
Idc: p.ptm.host.Idc,
Url: p.req.Url,
ContentLength: pt.GetContentLength(),
Traffic: pt.GetTraffic(),
TotalPieceCount: pt.GetTotalPieces(),
Cost: uint32(cost),
Success: true,
Code: dfcodes.Success,
})
if err != nil {
pt.Log().Errorf("step 3: report successful peer result, error: %v", err)
@ -117,18 +117,19 @@ func (p *filePeerTaskCallback) Fail(pt Task, code base.Code, reason string) erro
p.ptm.PeerTaskDone(p.req.PeerId)
var end = time.Now()
pt.Log().Errorf("file peer task failed, code: %d, reason: %s", code, reason)
err := p.pt.schedulerClient.ReportPeerResult(context.Background(), &scheduler.PeerResult{
TaskId: pt.GetTaskID(),
PeerId: pt.GetPeerID(),
SrcIp: p.ptm.host.Ip,
SecurityDomain: p.ptm.host.SecurityDomain,
Idc: p.ptm.host.Idc,
Url: p.req.Url,
ContentLength: pt.GetContentLength(),
Traffic: pt.GetTraffic(),
Cost: uint32(end.Sub(p.start).Milliseconds()),
Success: false,
Code: code,
err := p.pt.schedulerClient.ReportPeerResult(p.pt.ctx, &scheduler.PeerResult{
TaskId: pt.GetTaskID(),
PeerId: pt.GetPeerID(),
SrcIp: p.ptm.host.Ip,
SecurityDomain: p.ptm.host.SecurityDomain,
Idc: p.ptm.host.Idc,
Url: p.req.Url,
ContentLength: pt.GetContentLength(),
Traffic: pt.GetTraffic(),
TotalPieceCount: p.pt.totalPiece,
Cost: uint32(end.Sub(p.start).Milliseconds()),
Success: false,
Code: code,
})
if err != nil {
pt.Log().Errorf("step 3: report fail peer result, error: %v", err)

View File

@ -66,6 +66,7 @@ type Task interface {
GetPeerID() string
GetTaskID() string
GetTotalPieces() int32
SetTotalPieces(int32)
GetContentLength() int64
// SetContentLength will be called after download completed, when download from source without content length
SetContentLength(int64) error

View File

@ -1,5 +1,5 @@
// Code generated by MockGen. DO NOT EDIT.
// Source: peertask_manager.go
// Source: client/daemon/peer/peertask_manager.go
// Package peer is a generated GoMock package.
package peer
@ -272,6 +272,18 @@ func (mr *MockTaskMockRecorder) SetContentLength(arg0 interface{}) *gomock.Call
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetContentLength", reflect.TypeOf((*MockTask)(nil).SetContentLength), arg0)
}
// SetTotalPieces mocks base method.
func (m *MockTask) SetTotalPieces(arg0 int32) {
m.ctrl.T.Helper()
m.ctrl.Call(m, "SetTotalPieces", arg0)
}
// SetTotalPieces indicates an expected call of SetTotalPieces.
func (mr *MockTaskMockRecorder) SetTotalPieces(arg0 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetTotalPieces", reflect.TypeOf((*MockTask)(nil).SetTotalPieces), arg0)
}
// MockTaskCallback is a mock of TaskCallback interface.
type MockTaskCallback struct {
ctrl *gomock.Controller

View File

@ -184,6 +184,7 @@ func newStreamPeerTask(ctx context.Context,
// bind func that base peer task did not implement
pt.backSourceFunc = pt.backSource
pt.setContentLengthFunc = pt.SetContentLength
pt.setTotalPiecesFunc = pt.SetTotalPieces
pt.reportPieceResultFunc = pt.ReportPieceResult
return ctx, pt, nil, nil
}
@ -193,18 +194,18 @@ func (s *streamPeerTask) ReportPieceResult(result *pieceTaskResult) error {
// retry failed piece
if !result.pieceResult.Success {
_ = s.peerPacketStream.Send(result.pieceResult)
s.failedPieceCh <- result.pieceResult.PieceNum
s.failedPieceCh <- result.pieceResult.PieceInfo.PieceNum
return nil
}
s.lock.Lock()
if s.readyPieces.IsSet(result.pieceResult.PieceNum) {
if s.readyPieces.IsSet(result.pieceResult.PieceInfo.PieceNum) {
s.lock.Unlock()
s.Warnf("piece %d is already reported, skipped", result.pieceResult.PieceNum)
s.Warnf("piece %d is already reported, skipped", result.pieceResult.PieceInfo.PieceNum)
return nil
}
// mark piece processed
s.readyPieces.Set(result.pieceResult.PieceNum)
s.readyPieces.Set(result.pieceResult.PieceInfo.PieceNum)
s.completedLength.Add(int64(result.piece.RangeSize))
s.lock.Unlock()
@ -416,6 +417,10 @@ func (s *streamPeerTask) SetContentLength(i int64) error {
return s.finish()
}
func (s *streamPeerTask) SetTotalPieces(i int32) {
s.totalPiece = i
}
func (s *streamPeerTask) writeTo(w io.Writer, pieceNum int32) (int64, error) {
pr, pc, err := s.pieceManager.ReadPiece(s.ctx, &storage.ReadPieceRequest{
PeerTaskMetaData: storage.PeerTaskMetaData{
@ -440,6 +445,12 @@ func (s *streamPeerTask) writeTo(w io.Writer, pieceNum int32) (int64, error) {
func (s *streamPeerTask) backSource() {
s.contentLength.Store(-1)
_ = s.callback.Init(s)
if peerPacketStream, err := s.schedulerClient.ReportPieceResult(s.ctx, s.taskID, s.request); err != nil {
logger.Errorf("step 2: peer %s report piece failed: err", s.request.PeerId, err)
} else {
s.peerPacketStream = peerPacketStream
}
logger.Infof("step 2: start report peer %s back source piece result", s.request.PeerId)
err := s.pieceManager.DownloadSource(s.ctx, s, s.request)
if err != nil {
s.Errorf("download from source error: %s", err)

View File

@ -17,7 +17,6 @@
package peer
import (
"context"
"time"
"d7y.io/dragonfly/v2/client/daemon/storage"
@ -90,18 +89,19 @@ func (p *streamPeerTaskCallback) Done(pt Task) error {
return e
}
p.ptm.PeerTaskDone(p.req.PeerId)
err := p.pt.schedulerClient.ReportPeerResult(context.Background(), &scheduler.PeerResult{
TaskId: pt.GetTaskID(),
PeerId: pt.GetPeerID(),
SrcIp: p.ptm.host.Ip,
SecurityDomain: p.ptm.host.SecurityDomain,
Idc: p.ptm.host.Idc,
Url: p.req.Url,
ContentLength: pt.GetContentLength(),
Traffic: pt.GetTraffic(),
Cost: uint32(cost),
Success: true,
Code: dfcodes.Success,
err := p.pt.schedulerClient.ReportPeerResult(p.pt.ctx, &scheduler.PeerResult{
TaskId: pt.GetTaskID(),
PeerId: pt.GetPeerID(),
SrcIp: p.ptm.host.Ip,
SecurityDomain: p.ptm.host.SecurityDomain,
Idc: p.ptm.host.Idc,
Url: p.req.Url,
ContentLength: pt.GetContentLength(),
Traffic: pt.GetTraffic(),
TotalPieceCount: p.pt.totalPiece,
Cost: uint32(cost),
Success: true,
Code: dfcodes.Success,
})
if err != nil {
pt.Log().Errorf("step 3: report successful peer result, error: %v", err)
@ -115,18 +115,19 @@ func (p *streamPeerTaskCallback) Fail(pt Task, code base.Code, reason string) er
p.ptm.PeerTaskDone(p.req.PeerId)
var end = time.Now()
pt.Log().Errorf("stream peer task failed, code: %d, reason: %s", code, reason)
err := p.pt.schedulerClient.ReportPeerResult(context.Background(), &scheduler.PeerResult{
TaskId: pt.GetTaskID(),
PeerId: pt.GetPeerID(),
SrcIp: p.ptm.host.Ip,
SecurityDomain: p.ptm.host.SecurityDomain,
Idc: p.ptm.host.Idc,
Url: p.req.Url,
ContentLength: pt.GetContentLength(),
Traffic: pt.GetTraffic(),
Cost: uint32(end.Sub(p.start).Milliseconds()),
Success: false,
Code: code,
err := p.pt.schedulerClient.ReportPeerResult(p.pt.ctx, &scheduler.PeerResult{
TaskId: pt.GetTaskID(),
PeerId: pt.GetPeerID(),
SrcIp: p.ptm.host.Ip,
SecurityDomain: p.ptm.host.SecurityDomain,
Idc: p.ptm.host.Idc,
Url: p.req.Url,
ContentLength: pt.GetContentLength(),
Traffic: pt.GetTraffic(),
TotalPieceCount: p.pt.totalPiece,
Cost: uint32(end.Sub(p.start).Milliseconds()),
Success: false,
Code: code,
})
if err != nil {
pt.Log().Errorf("step 3: report fail peer result, error: %v", err)

View File

@ -173,13 +173,14 @@ func (pm *pieceManager) pushSuccessResult(peerTask Task, dstPid string, piece *b
TaskId: peerTask.GetTaskID(),
SrcPid: peerTask.GetPeerID(),
DstPid: dstPid,
PieceNum: piece.PieceNum,
PieceInfo: piece,
BeginTime: uint64(start),
EndTime: uint64(end),
Success: true,
Code: dfcodes.Success,
HostLoad: nil, // TODO(jim): update host load
FinishedCount: 0, // update by peer task
HostLoad: nil, // TODO(jim): update host load
FinishedCount: piece.PieceNum + 1, // update by peer task
// TODO range_start, range_size, piece_md5, piece_offset, piece_style
},
err: nil,
})
@ -196,7 +197,7 @@ func (pm *pieceManager) pushFailResult(peerTask Task, dstPid string, piece *base
TaskId: peerTask.GetTaskID(),
SrcPid: peerTask.GetPeerID(),
DstPid: dstPid,
PieceNum: piece.PieceNum,
PieceInfo: piece,
BeginTime: uint64(start),
EndTime: uint64(end),
Success: false,
@ -371,10 +372,11 @@ func (pm *pieceManager) DownloadSource(ctx context.Context, pt Task, request *sc
},
ContentLength: contentLength,
})
pt.SetTotalPieces(pieceNum + 1)
return pt.SetContentLength(contentLength)
}
}
// unreachable code
//unreachable code
//return nil
}
@ -398,6 +400,8 @@ func (pm *pieceManager) DownloadSource(ctx context.Context, pt Task, request *sc
return storage.ErrShortRead
}
}
pt.SetTotalPieces(maxPieceNum)
pt.SetContentLength(contentLength)
log.Infof("download from source ok")
return nil
}

View File

@ -130,6 +130,9 @@ func TestPieceManager_DownloadSource(t *testing.T) {
func(arg0 int64) error {
return nil
})
mockPeerTask.EXPECT().SetTotalPieces(gomock.Any()).AnyTimes().DoAndReturn(
func(arg0 int32) {
})
mockPeerTask.EXPECT().GetPeerID().AnyTimes().DoAndReturn(
func() string {
return peerID

View File

@ -35,10 +35,9 @@ var (
// rootCmd represents the base command when called without any subcommands
var rootCmd = &cobra.Command{
Use: "scheduler",
Short: "the scheduler of dragonfly",
Long: `scheduler is a long-running process and is mainly responsible
for deciding which peers transmit blocks to each other.`,
Use: "scheduler",
Short: "the scheduler of dragonfly",
Long: `scheduler is a long-running process and is mainly responsible for deciding which peers transmit blocks to each other.`,
Args: cobra.NoArgs,
DisableAutoGenTag: true,
SilenceUsage: true,

View File

@ -25,7 +25,7 @@ const (
ServerUnavailable base.Code = 500 // framework can not find server node
// common response error 1000-1999
ResourceLacked base.Code = 1000 // client can be migrated to another scheduler
ResourceLacked base.Code = 1000 // client can be migrated to another scheduler/CDN
BadRequest base.Code = 1400
PeerTaskNotFound base.Code = 1404
UnknownError base.Code = 1500
@ -41,16 +41,12 @@ const (
ClientRequestLimitFail base.Code = 4006
// scheduler response error 5000-5999
SchedError base.Code = 5000
/** @deprecated */
SchedError base.Code = 5000
SchedNeedBackSource base.Code = 5001 // client should try to download from source
SchedPeerGone base.Code = 5002 // client should disconnect from scheduler
SchedPeerRegisterFail base.Code = 5003
SchedPeerScheduleFail base.Code = 5004
SchedPeerNotFound base.Code = 5005
SchedPeerPieceResultReportFail base.Code = 5006
SchedCDNSeedFail base.Code = 5007
SchedTaskStatusError base.Code = 5008
SchedPeerNotFound base.Code = 5004 // peer not found in scheduler
SchedPeerPieceResultReportFail base.Code = 5005 // report piece
SchedTaskStatusError base.Code = 5006 // task status is fail
// cdnsystem response error 6000-6999
CdnError base.Code = 6000

View File

@ -30,7 +30,7 @@ var (
ErrEmptyValue = errors.New("empty value")
ErrConvertFailed = errors.New("convert failed")
ErrEndOfStream = errors.New("end of stream")
ErrNoCandidateNode = errors.New("candidate server node not found")
ErrNoCandidateNode = errors.New("no candidate server node")
)
func IsEndOfStream(err error) bool {

View File

@ -15,13 +15,14 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.26.0
// protoc v3.15.8
// protoc-gen-go v1.25.0
// protoc v3.17.3
// source: pkg/rpc/base/base.proto
package base
import (
proto "github.com/golang/protobuf/proto"
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
reflect "reflect"
@ -35,6 +36,10 @@ const (
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
)
// This is a compile-time assertion that a sufficiently up-to-date version
// of the legacy proto package is being used.
const _ = proto.ProtoPackageIsVersion4
type Code int32
const (

View File

@ -34,20 +34,53 @@ var (
)
// Validate checks the field values on GrpcDfError 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 *GrpcDfError) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on GrpcDfError 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 GrpcDfErrorMultiError, or
// nil if none found.
func (m *GrpcDfError) ValidateAll() error {
return m.validate(true)
}
func (m *GrpcDfError) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for Code
// no validation rules for Message
if len(errors) > 0 {
return GrpcDfErrorMultiError(errors)
}
return nil
}
// GrpcDfErrorMultiError is an error wrapping multiple validation errors
// returned by GrpcDfError.ValidateAll() if the designated constraints aren't met.
type GrpcDfErrorMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m GrpcDfErrorMultiError) 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 GrpcDfErrorMultiError) AllErrors() []error { return m }
// GrpcDfErrorValidationError is the validation error returned by
// GrpcDfError.Validate if the designated constraints aren't met.
type GrpcDfErrorValidationError struct {
@ -103,12 +136,26 @@ var _ interface {
} = GrpcDfErrorValidationError{}
// Validate checks the field values on UrlMeta 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 *UrlMeta) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on UrlMeta 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 UrlMetaMultiError, or nil if none found.
func (m *UrlMeta) ValidateAll() error {
return m.validate(true)
}
func (m *UrlMeta) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for Digest
// no validation rules for Tag
@ -119,9 +166,28 @@ func (m *UrlMeta) Validate() error {
// no validation rules for Header
if len(errors) > 0 {
return UrlMetaMultiError(errors)
}
return nil
}
// UrlMetaMultiError is an error wrapping multiple validation errors returned
// by UrlMeta.ValidateAll() if the designated constraints aren't met.
type UrlMetaMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m UrlMetaMultiError) 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 UrlMetaMultiError) AllErrors() []error { return m }
// UrlMetaValidationError is the validation error returned by UrlMeta.Validate
// if the designated constraints aren't met.
type UrlMetaValidationError struct {
@ -177,21 +243,55 @@ var _ interface {
} = UrlMetaValidationError{}
// Validate checks the field values on HostLoad 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 *HostLoad) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on HostLoad 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 HostLoadMultiError, or nil
// if none found.
func (m *HostLoad) ValidateAll() error {
return m.validate(true)
}
func (m *HostLoad) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for CpuRatio
// no validation rules for MemRatio
// no validation rules for DiskRatio
if len(errors) > 0 {
return HostLoadMultiError(errors)
}
return nil
}
// HostLoadMultiError is an error wrapping multiple validation errors returned
// by HostLoad.ValidateAll() if the designated constraints aren't met.
type HostLoadMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m HostLoadMultiError) 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 HostLoadMultiError) AllErrors() []error { return m }
// HostLoadValidationError is the validation error returned by
// HostLoad.Validate if the designated constraints aren't met.
type HostLoadValidationError struct {
@ -247,13 +347,27 @@ var _ interface {
} = HostLoadValidationError{}
// Validate checks the field values on PieceTaskRequest with the rules defined
// in the proto definition for this message. If any rules are violated, an
// error is returned.
// 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.
func (m *PieceTaskRequest) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on PieceTaskRequest 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
// PieceTaskRequestMultiError, or nil if none found.
func (m *PieceTaskRequest) ValidateAll() error {
return m.validate(true)
}
func (m *PieceTaskRequest) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for TaskId
// no validation rules for SrcPid
@ -264,9 +378,29 @@ func (m *PieceTaskRequest) Validate() error {
// no validation rules for Limit
if len(errors) > 0 {
return PieceTaskRequestMultiError(errors)
}
return nil
}
// PieceTaskRequestMultiError is an error wrapping multiple validation errors
// returned by PieceTaskRequest.ValidateAll() if the designated constraints
// aren't met.
type PieceTaskRequestMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m PieceTaskRequestMultiError) 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 PieceTaskRequestMultiError) AllErrors() []error { return m }
// PieceTaskRequestValidationError is the validation error returned by
// PieceTaskRequest.Validate if the designated constraints aren't met.
type PieceTaskRequestValidationError struct {
@ -322,12 +456,27 @@ var _ interface {
} = PieceTaskRequestValidationError{}
// Validate checks the field values on PieceInfo 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 *PieceInfo) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on PieceInfo 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 PieceInfoMultiError, or nil
// if none found.
func (m *PieceInfo) ValidateAll() error {
return m.validate(true)
}
func (m *PieceInfo) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for PieceNum
// no validation rules for RangeStart
@ -340,9 +489,28 @@ func (m *PieceInfo) Validate() error {
// no validation rules for PieceStyle
if len(errors) > 0 {
return PieceInfoMultiError(errors)
}
return nil
}
// PieceInfoMultiError is an error wrapping multiple validation errors returned
// by PieceInfo.ValidateAll() if the designated constraints aren't met.
type PieceInfoMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m PieceInfoMultiError) 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 PieceInfoMultiError) AllErrors() []error { return m }
// PieceInfoValidationError is the validation error returned by
// PieceInfo.Validate if the designated constraints aren't met.
type PieceInfoValidationError struct {
@ -398,13 +566,27 @@ var _ interface {
} = PieceInfoValidationError{}
// Validate checks the field values on PiecePacket 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 *PiecePacket) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on PiecePacket 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 PiecePacketMultiError, or
// nil if none found.
func (m *PiecePacket) ValidateAll() error {
return m.validate(true)
}
func (m *PiecePacket) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for TaskId
// no validation rules for DstPid
@ -414,7 +596,26 @@ func (m *PiecePacket) Validate() error {
for idx, item := range m.GetPieceInfos() {
_, _ = idx, item
if v, ok := interface{}(item).(interface{ Validate() error }); ok {
if all {
switch v := interface{}(item).(type) {
case interface{ ValidateAll() error }:
if err := v.ValidateAll(); err != nil {
errors = append(errors, PiecePacketValidationError{
field: fmt.Sprintf("PieceInfos[%v]", idx),
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, PiecePacketValidationError{
field: fmt.Sprintf("PieceInfos[%v]", idx),
reason: "embedded message failed validation",
cause: err,
})
}
}
} else if v, ok := interface{}(item).(interface{ Validate() error }); ok {
if err := v.Validate(); err != nil {
return PiecePacketValidationError{
field: fmt.Sprintf("PieceInfos[%v]", idx),
@ -432,9 +633,28 @@ func (m *PiecePacket) Validate() error {
// no validation rules for PieceMd5Sign
if len(errors) > 0 {
return PiecePacketMultiError(errors)
}
return nil
}
// PiecePacketMultiError is an error wrapping multiple validation errors
// returned by PiecePacket.ValidateAll() if the designated constraints aren't met.
type PiecePacketMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m PiecePacketMultiError) 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 PiecePacketMultiError) AllErrors() []error { return m }
// PiecePacketValidationError is the validation error returned by
// PiecePacket.Validate if the designated constraints aren't met.
type PiecePacketValidationError struct {

View File

@ -15,14 +15,15 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.26.0
// protoc v3.15.8
// protoc-gen-go v1.25.0
// protoc v3.17.3
// source: pkg/rpc/cdnsystem/cdnsystem.proto
package cdnsystem
import (
base "d7y.io/dragonfly/v2/pkg/rpc/base"
proto "github.com/golang/protobuf/proto"
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
reflect "reflect"
@ -36,6 +37,10 @@ const (
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
)
// This is a compile-time assertion that a sufficiently up-to-date version
// of the legacy proto package is being used.
const _ = proto.ProtoPackageIsVersion4
type SeedRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache

View File

@ -34,18 +34,51 @@ 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
}
var errors []error
// no validation rules for TaskId
// no validation rules for Url
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, 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 +88,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 +165,51 @@ 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
}
var errors []error
// no validation rules for PeerId
// no validation rules for HostUuid
if v, ok := interface{}(m.GetPieceInfo()).(interface{ Validate() error }); ok {
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 +225,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

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

@ -18,11 +18,12 @@ package client
import (
"context"
"io"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/pkg/rpc"
"d7y.io/dragonfly/v2/pkg/rpc/cdnsystem"
"github.com/pkg/errors"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
@ -50,9 +51,9 @@ func newPieceSeedStream(ctx context.Context, sc *cdnClient, hashKey string, sr *
sr: sr,
opts: opts,
RetryMeta: rpc.RetryMeta{
MaxAttempts: 5,
InitBackoff: 0.5,
MaxBackOff: 4.0,
MaxAttempts: 3,
InitBackoff: 0.2,
MaxBackOff: 2.0,
},
}
@ -74,7 +75,10 @@ func (pss *PieceSeedStream) initStream() error {
return client.ObtainSeeds(pss.ctx, pss.sr, pss.opts...)
}, pss.InitBackoff, pss.MaxBackOff, pss.MaxAttempts, nil)
if err != nil {
logger.WithTaskID(pss.hashKey).Infof("initStream: invoke cdn node %s ObtainSeeds failed: %v", target, err)
if errors.Cause(err) == dferrors.ErrNoCandidateNode {
return errors.Wrapf(err, "get grpc server instance failed")
}
logger.WithTaskID(pss.hashKey).Errorf("initStream: invoke cdn node %s ObtainSeeds failed: %v", target, err)
return pss.replaceClient(pss.hashKey, err)
}
pss.stream = stream.(cdnsystem.Seeder_ObtainSeedsClient)
@ -84,16 +88,14 @@ func (pss *PieceSeedStream) initStream() error {
func (pss *PieceSeedStream) Recv() (ps *cdnsystem.PieceSeed, err error) {
pss.sc.UpdateAccessNodeMapByHashKey(pss.hashKey)
if ps, err = pss.stream.Recv(); err != nil && err != io.EOF {
ps, err = pss.retryRecv(err)
}
return
return pss.stream.Recv()
}
func (pss *PieceSeedStream) retryRecv(cause error) (*cdnsystem.PieceSeed, error) {
if status.Code(cause) == codes.DeadlineExceeded || status.Code(cause) == codes.Canceled {
return nil, cause
}
if err := pss.replaceStream(cause); err != nil {
return nil, err
}

View File

@ -180,14 +180,15 @@ func (conn *Connection) UpdateAccessNodeMapByHashKey(key string) {
node, ok := conn.key2NodeMap.Load(key)
if ok {
conn.accessNodeMap.Store(node, time.Now())
logger.With("conn", conn.name).Debugf("successfully update server node %s access time for hashKey %s", node, key)
_, ok := conn.node2ClientMap.Load(node)
if !ok {
logger.With("conn", conn.name).Warnf("successfully update server node %s access time for hashKey %s,"+
if ok {
logger.GrpcLogger.With("conn", conn.name).Debugf("successfully update server node %s access time for hashKey %s", node, key)
} else {
logger.GrpcLogger.With("conn", conn.name).Warnf("successfully update server node %s access time for hashKey %s,"+
"but cannot found client conn in node2ClientMap", node, key)
}
} else {
logger.With("conn", conn.name).Errorf("update access node map failed, hash key (%s) not found in key2NodeMap", key)
logger.GrpcLogger.With("conn", conn.name).Errorf("update access node map failed, hash key (%s) not found in key2NodeMap", key)
}
}
@ -201,7 +202,7 @@ func (conn *Connection) AddServerNodes(addrs []dfnet.NetAddr) error {
for _, addr := range addrs {
serverNode := addr.GetEndpoint()
conn.hashRing = conn.hashRing.AddNode(serverNode)
logger.With("conn", conn.name).Debugf("success add %s to server node list", addr)
logger.GrpcLogger.With("conn", conn.name).Debugf("success add %s to server node list", addr)
}
return nil
}
@ -222,7 +223,7 @@ func (conn *Connection) findCandidateClientConn(key string, exclusiveNodes sets.
ringNodes, ok := conn.hashRing.GetNodes(key, conn.hashRing.Size())
if !ok {
logger.Warnf("cannot obtain expected %d server nodes", conn.hashRing.Size())
logger.GrpcLogger.Warnf("cannot obtain expected %d server nodes", conn.hashRing.Size())
}
if len(ringNodes) == 0 {
return nil, dferrors.ErrNoCandidateNode
@ -233,29 +234,30 @@ func (conn *Connection) findCandidateClientConn(key string, exclusiveNodes sets.
candidateNodes = append(candidateNodes, ringNode)
}
}
logger.With("conn", conn.name).Infof("candidate result for hash key %s: all server node list: %v, exclusiveNodes node list: %v, candidate node list: %v",
logger.GrpcLogger.With("conn", conn.name).Infof("candidate result for hash key %s: all server node list: %v, exclusiveNodes node list: %v, "+
"candidate node list: %v",
key, ringNodes, exclusiveNodes.List(), candidateNodes)
for _, candidateNode := range candidateNodes {
// Check whether there is a corresponding mapping client in the node2ClientMap
// TODO 下面部分可以直接调用loadOrCreate方法但是日志没有这么调用打印全
if client, ok := conn.node2ClientMap.Load(candidateNode); ok {
logger.With("conn", conn.name).Debugf("hit cache candidateNode %s for hash key %s", candidateNode, key)
logger.GrpcLogger.With("conn", conn.name).Debugf("hit cache candidateNode %s for hash key %s", candidateNode, key)
return &candidateClient{
node: candidateNode,
Ref: client,
}, nil
}
logger.With("conn", conn.name).Debugf("attempt to connect candidateNode %s for hash key %s", candidateNode, key)
logger.GrpcLogger.With("conn", conn.name).Debugf("attempt to connect candidateNode %s for hash key %s", candidateNode, key)
clientConn, err := conn.createClient(candidateNode, append(defaultClientOpts, conn.dialOpts...)...)
if err == nil {
logger.With("conn", conn.name).Infof("success connect to candidateNode %s for hash key %s", candidateNode, key)
logger.GrpcLogger.With("conn", conn.name).Infof("success connect to candidateNode %s for hash key %s", candidateNode, key)
return &candidateClient{
node: candidateNode,
Ref: clientConn,
}, nil
}
logger.With("conn", conn.name).Infof("failed to connect candidateNode %s for hash key %s: %v", candidateNode, key, err)
logger.GrpcLogger.With("conn", conn.name).Infof("failed to connect candidateNode %s for hash key %s: %v", candidateNode, key, err)
}
return nil, dferrors.ErrNoCandidateNode
}
@ -285,14 +287,14 @@ func (conn *Connection) GetServerNode(hashKey string) (string, bool) {
}
func (conn *Connection) GetClientConnByTarget(node string) (*grpc.ClientConn, error) {
logger.With("conn", conn.name).Debugf("start to get client conn by target %s", node)
logger.GrpcLogger.With("conn", conn.name).Debugf("start to get client conn by target %s", node)
conn.rwMutex.RLock()
defer conn.rwMutex.RUnlock()
clientConn, err := conn.loadOrCreateClientConnByNode(node)
if err != nil {
return nil, errors.Wrapf(err, "get client conn by conn %s", node)
}
logger.With("conn", conn.name).Debugf("successfully get %s client conn", node)
logger.GrpcLogger.With("conn", conn.name).Debugf("successfully get %s client conn", node)
return clientConn, nil
}
@ -305,14 +307,14 @@ func (conn *Connection) loadOrCreateClientConnByNode(node string) (clientConn *g
conn.accessNodeMap.Store(node, time.Now())
client, ok := conn.node2ClientMap.Load(node)
if ok {
logger.With("conn", conn.name).Debugf("hit cache clientConn associated with node %s", node)
logger.GrpcLogger.With("conn", conn.name).Debugf("hit cache clientConn associated with node %s", node)
return client.(*grpc.ClientConn), nil
}
logger.With("conn", conn.name).Debugf("failed to load clientConn associated with node %s, attempt to create it", node)
logger.GrpcLogger.With("conn", conn.name).Debugf("failed to load clientConn associated with node %s, attempt to create it", node)
clientConn, err = conn.createClient(node, append(defaultClientOpts, conn.dialOpts...)...)
if err == nil {
logger.With("conn", conn.name).Infof("success connect to node %s", node)
logger.GrpcLogger.With("conn", conn.name).Infof("success connect to node %s", node)
// bind
conn.node2ClientMap.Store(node, clientConn)
return clientConn, nil
@ -324,8 +326,8 @@ func (conn *Connection) loadOrCreateClientConnByNode(node string) (clientConn *g
// GetClientConn get conn or bind hashKey to candidate node, don't do the migrate action
// stick whether hash key need already associated with specify node
func (conn *Connection) GetClientConn(hashKey string, stick bool) (*grpc.ClientConn, error) {
logger.With("conn", conn.name).Debugf("start to get client conn hashKey %s, stick %t", hashKey, stick)
defer logger.With("conn", conn.name).Debugf("get client conn done, hashKey %s, stick %t end", hashKey, stick)
logger.GrpcLogger.With("conn", conn.name).Debugf("start to get client conn hashKey %s, stick %t", hashKey, stick)
defer logger.GrpcLogger.With("conn", conn.name).Debugf("get client conn done, hashKey %s, stick %t end", hashKey, stick)
conn.rwMutex.RLock()
node, ok := conn.key2NodeMap.Load(hashKey)
if stick && !ok {
@ -343,7 +345,7 @@ func (conn *Connection) GetClientConn(hashKey string, stick bool) (*grpc.ClientC
}
return clientConn, nil
}
logger.With("conn", conn.name).Infof("no server node associated with hash key %s was found, start find candidate server", hashKey)
logger.GrpcLogger.With("conn", conn.name).Infof("no server node associated with hash key %s was found, start find candidate server", hashKey)
conn.rwMutex.RUnlock()
// if absence
conn.rwMutex.Lock()
@ -361,14 +363,14 @@ func (conn *Connection) GetClientConn(hashKey string, stick bool) (*grpc.ClientC
// TryMigrate migrate key to another hash node other than exclusiveNodes
// preNode node before the migration
func (conn *Connection) TryMigrate(key string, cause error, exclusiveNodes []string) (preNode string, err error) {
logger.With("conn", conn.name).Infof("start try migrate server node for key %s, cause err: %v", key, cause)
logger.GrpcLogger.With("conn", conn.name).Infof("start try migrate server node for key %s, cause err: %v", key, cause)
if status.Code(cause) == codes.DeadlineExceeded || status.Code(cause) == codes.Canceled {
logger.With("conn", conn.name).Infof("migrate server node for key %s failed, cause err: %v", key, cause)
logger.GrpcLogger.With("conn", conn.name).Infof("migrate server node for key %s failed, cause err: %v", key, cause)
return "", cause
}
// TODO recover findCandidateClientConn error
if e, ok := cause.(*dferrors.DfError); ok {
if e.Code != dfcodes.ResourceLacked && e.Code != dfcodes.UnknownError {
if e.Code != dfcodes.ResourceLacked {
return "", cause
}
}
@ -378,8 +380,6 @@ func (conn *Connection) TryMigrate(key string, cause error, exclusiveNodes []str
currentNode = node.(string)
preNode = currentNode
exclusiveNodes = append(exclusiveNodes, preNode)
} else {
logger.With("conn", conn.name).Warnf("failed to find server node for hash key %s", key)
}
conn.rwMutex.RUnlock()
conn.rwMutex.Lock()
@ -388,7 +388,7 @@ func (conn *Connection) TryMigrate(key string, cause error, exclusiveNodes []str
if err != nil {
return "", errors.Wrapf(err, "find candidate client conn for hash key %s", key)
}
logger.With("conn", conn.name).Infof("successfully migrate hash key %s from server node %s to %s", key, currentNode, client.node)
logger.GrpcLogger.With("conn", conn.name).Infof("successfully migrate hash key %s from server node %s to %s", key, currentNode, client.node)
conn.key2NodeMap.Store(key, client.node)
conn.node2ClientMap.Store(client.node, client.Ref)
conn.accessNodeMap.Store(client.node, time.Now())

View File

@ -25,7 +25,6 @@ import (
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/pkg/rpc/base"
@ -186,10 +185,8 @@ type RetryMeta struct {
func ExecuteWithRetry(f func() (interface{}, error), initBackoff float64, maxBackoff float64, maxAttempts int, cause error) (interface{}, error) {
var res interface{}
for i := 0; i < maxAttempts; i++ {
if e, ok := cause.(*dferrors.DfError); ok {
if e.Code != dfcodes.UnknownError {
return res, cause
}
if _, ok := cause.(*dferrors.DfError); ok {
return res, cause
}
if status.Code(cause) == codes.DeadlineExceeded || status.Code(cause) == codes.Canceled {
return res, cause

View File

@ -18,11 +18,12 @@ package client
import (
"context"
"io"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/pkg/rpc"
"d7y.io/dragonfly/v2/pkg/rpc/dfdaemon"
"github.com/pkg/errors"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
@ -49,9 +50,9 @@ func newDownResultStream(ctx context.Context, dc *daemonClient, hashKey string,
opts: opts,
RetryMeta: rpc.RetryMeta{
MaxAttempts: 5,
MaxBackOff: 5.0,
InitBackoff: 1.0,
MaxAttempts: 3,
MaxBackOff: 2.0,
InitBackoff: 0.2,
},
}
@ -73,6 +74,9 @@ func (drs *DownResultStream) initStream() error {
return client.Download(drs.ctx, drs.req, drs.opts...)
}, drs.InitBackoff, drs.MaxBackOff, drs.MaxAttempts, nil)
if err != nil {
if errors.Cause(err) == dferrors.ErrNoCandidateNode {
return errors.Wrapf(err, "get grpc server instance failed")
}
logger.WithTaskID(drs.hashKey).Infof("initStream: invoke daemon node %s Download failed: %v", target, err)
return drs.replaceClient(err)
}
@ -92,11 +96,7 @@ func (drs *DownResultStream) Recv() (dr *dfdaemon.DownResult, err error) {
}
}()
drs.dc.UpdateAccessNodeMapByHashKey(drs.hashKey)
if dr, err = drs.stream.Recv(); err != nil && err != io.EOF {
dr, err = drs.retryRecv(err)
}
return
return drs.stream.Recv()
}
func (drs *DownResultStream) retryRecv(cause error) (*dfdaemon.DownResult, error) {

View File

@ -15,14 +15,15 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.26.0
// protoc v3.15.8
// protoc-gen-go v1.25.0
// protoc v3.17.3
// source: pkg/rpc/dfdaemon/dfdaemon.proto
package dfdaemon
import (
base "d7y.io/dragonfly/v2/pkg/rpc/base"
proto "github.com/golang/protobuf/proto"
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
emptypb "google.golang.org/protobuf/types/known/emptypb"
@ -37,6 +38,10 @@ const (
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
)
// This is a compile-time assertion that a sufficiently up-to-date version
// of the legacy proto package is being used.
const _ = proto.ProtoPackageIsVersion4
type DownRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache

View File

@ -34,13 +34,27 @@ var (
)
// 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
}
var errors []error
// no validation rules for Uuid
// no validation rules for Url
@ -53,7 +67,26 @@ func (m *DownRequest) Validate() error {
// 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",
@ -71,9 +104,28 @@ func (m *DownRequest) Validate() error {
// no validation rules for Gid
if len(errors) > 0 {
return DownRequestMultiError(errors)
}
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 {
@ -129,12 +181,27 @@ var _ interface {
} = DownRequestValidationError{}
// 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
}
var errors []error
// no validation rules for TaskId
// no validation rules for PeerId
@ -143,9 +210,28 @@ func (m *DownResult) Validate() error {
// 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

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

@ -15,14 +15,15 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.26.0
// protoc v3.15.8
// protoc-gen-go v1.25.0
// protoc v3.17.3
// source: pkg/rpc/manager/manager.proto
package manager
import (
_ "github.com/envoyproxy/protoc-gen-validate/validate"
proto "github.com/golang/protobuf/proto"
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
emptypb "google.golang.org/protobuf/types/known/emptypb"
@ -37,6 +38,10 @@ const (
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
)
// This is a compile-time assertion that a sufficiently up-to-date version
// of the legacy proto package is being used.
const _ = proto.ProtoPackageIsVersion4
type SourceType int32
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

@ -148,7 +148,9 @@ func (sc *schedulerClient) retryRegisterPeerTask(ctx context.Context, hashKey st
func (sc *schedulerClient) ReportPieceResult(ctx context.Context, taskID string, ptr *scheduler.PeerTaskRequest, opts ...grpc.CallOption) (PeerPacketStream, error) {
pps, err := newPeerPacketStream(ctx, sc, taskID, ptr, opts)
if err != nil {
return pps, err
}
logger.With("peerId", ptr.PeerId, "errMsg", err).Infof("start to report piece result for taskID: %s", taskID)
// trigger scheduling

View File

@ -18,9 +18,10 @@ package client
import (
"context"
"io"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"github.com/pkg/errors"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
@ -60,9 +61,9 @@ func newPeerPacketStream(ctx context.Context, sc *schedulerClient, hashKey strin
ptr: ptr,
opts: opts,
retryMeta: rpc.RetryMeta{
MaxAttempts: 5,
InitBackoff: 0.5,
MaxBackOff: 4.0,
MaxAttempts: 3,
InitBackoff: 0.2,
MaxBackOff: 2.0,
},
}
@ -77,14 +78,13 @@ func (pps *peerPacketStream) Send(pr *scheduler.PieceResult) (err error) {
pps.sc.UpdateAccessNodeMapByHashKey(pps.hashKey)
err = pps.stream.Send(pr)
if pr.PieceNum == common.EndOfPiece {
if pr.PieceInfo.PieceNum == common.EndOfPiece {
pps.closeSend()
return
}
if err != nil {
pps.closeSend()
err = pps.retrySend(pr, err)
}
return
@ -96,10 +96,7 @@ func (pps *peerPacketStream) closeSend() error {
func (pps *peerPacketStream) Recv() (pp *scheduler.PeerPacket, err error) {
pps.sc.UpdateAccessNodeMapByHashKey(pps.hashKey)
if pp, err = pps.stream.Recv(); err != nil && err != io.EOF {
pp, err = pps.retryRecv(err)
}
return
return pps.stream.Recv()
}
func (pps *peerPacketStream) retrySend(pr *scheduler.PieceResult, cause error) error {
@ -108,7 +105,7 @@ func (pps *peerPacketStream) retrySend(pr *scheduler.PieceResult, cause error) e
}
if err := pps.replaceStream(cause); err != nil {
return err
return cause
}
return pps.Send(pr)
@ -159,6 +156,9 @@ func (pps *peerPacketStream) initStream() error {
return client.ReportPieceResult(pps.ctx, pps.opts...)
}, pps.retryMeta.InitBackoff, pps.retryMeta.MaxBackOff, pps.retryMeta.MaxAttempts, nil)
if err != nil {
if errors.Cause(err) == dferrors.ErrNoCandidateNode {
return errors.Wrapf(err, "get grpc server instance failed")
}
logger.WithTaskID(pps.hashKey).Infof("initStream: invoke scheduler node %s ReportPieceResult failed: %v", target, err)
return pps.replaceClient(err)
}

View File

@ -17,14 +17,22 @@
package scheduler
import (
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/base/common"
)
func NewZeroPieceResult(taskID, peerID string) *PieceResult {
return &PieceResult{
TaskId: taskID,
SrcPid: peerID,
PieceNum: common.ZeroOfPiece,
TaskId: taskID,
SrcPid: peerID,
PieceInfo: &base.PieceInfo{
PieceNum: common.ZeroOfPiece,
RangeStart: 0,
RangeSize: 0,
PieceMd5: "",
PieceOffset: 0,
PieceStyle: 0,
},
}
}
@ -32,7 +40,14 @@ func NewEndPieceResult(taskID, peerID string, finishedCount int32) *PieceResult
return &PieceResult{
TaskId: taskID,
SrcPid: peerID,
PieceNum: common.EndOfPiece,
FinishedCount: finishedCount,
PieceInfo: &base.PieceInfo{
PieceNum: common.EndOfPiece,
RangeStart: 0,
RangeSize: 0,
PieceMd5: "",
PieceOffset: 0,
PieceStyle: 0,
},
}
}

View File

@ -15,14 +15,15 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.26.0
// protoc v3.15.8
// protoc-gen-go v1.25.0
// protoc v3.17.3
// source: pkg/rpc/scheduler/scheduler.proto
package scheduler
import (
base "d7y.io/dragonfly/v2/pkg/rpc/base"
proto "github.com/golang/protobuf/proto"
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
emptypb "google.golang.org/protobuf/types/known/emptypb"
@ -37,6 +38,10 @@ const (
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
)
// This is a compile-time assertion that a sufficiently up-to-date version
// of the legacy proto package is being used.
const _ = proto.ProtoPackageIsVersion4
type PeerTaskRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -429,8 +434,8 @@ type PieceResult struct {
SrcPid string `protobuf:"bytes,2,opt,name=src_pid,json=srcPid,proto3" json:"src_pid,omitempty"`
// dest peer id
DstPid string `protobuf:"bytes,3,opt,name=dst_pid,json=dstPid,proto3" json:"dst_pid,omitempty"`
// piece number
PieceNum int32 `protobuf:"varint,4,opt,name=piece_num,json=pieceNum,proto3" json:"piece_num,omitempty"`
// piece info
PieceInfo *base.PieceInfo `protobuf:"bytes,4,opt,name=piece_info,json=pieceInfo,proto3" json:"piece_info,omitempty"`
// begin time for the piece downloading
BeginTime uint64 `protobuf:"varint,5,opt,name=begin_time,json=beginTime,proto3" json:"begin_time,omitempty"`
// end time for the piece downloading
@ -498,11 +503,11 @@ func (x *PieceResult) GetDstPid() string {
return ""
}
func (x *PieceResult) GetPieceNum() int32 {
func (x *PieceResult) GetPieceInfo() *base.PieceInfo {
if x != nil {
return x.PieceNum
return x.PieceInfo
}
return 0
return nil
}
func (x *PieceResult) GetBeginTime() uint64 {
@ -556,7 +561,6 @@ type PeerPacket struct {
// source peer id
SrcPid string `protobuf:"bytes,3,opt,name=src_pid,json=srcPid,proto3" json:"src_pid,omitempty"`
// concurrent downloading count from main peer
// TODO what effect
ParallelCount int32 `protobuf:"varint,4,opt,name=parallel_count,json=parallelCount,proto3" json:"parallel_count,omitempty"`
MainPeer *PeerPacket_DestPeer `protobuf:"bytes,5,opt,name=main_peer,json=mainPeer,proto3" json:"main_peer,omitempty"`
StealPeers []*PeerPacket_DestPeer `protobuf:"bytes,6,rep,name=steal_peers,json=stealPeers,proto3" json:"steal_peers,omitempty"`
@ -658,7 +662,8 @@ type PeerResult struct {
// 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"`
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"`
}
func (x *PeerResult) Reset() {
@ -770,6 +775,13 @@ func (x *PeerResult) GetCode() base.Code {
return base.Code_X_UNSPECIFIED
}
func (x *PeerResult) GetTotalPieceCount() int32 {
if x != nil {
return x.TotalPieceCount
}
return 0
}
type PeerTarget struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -951,92 +963,96 @@ var file_pkg_rpc_scheduler_scheduler_proto_rawDesc = []byte{
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, 0xbd, 0x02, 0x0a, 0x0b, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75,
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, 0x1b, 0x0a,
0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05,
0x52, 0x08, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4e, 0x75, 0x6d, 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, 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, 0xb1, 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, 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, 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, 0x22, 0x3e, 0x0a, 0x0a,
0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61,
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,
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, 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,
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,
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,
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,
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,
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 (
@ -1076,25 +1092,26 @@ var file_pkg_rpc_scheduler_scheduler_proto_depIdxs = []int32{
11, // 3: scheduler.RegisterResult.size_scope:type_name -> base.SizeScope
2, // 4: scheduler.RegisterResult.single_piece:type_name -> scheduler.SinglePiece
12, // 5: scheduler.SinglePiece.piece_info:type_name -> base.PieceInfo
13, // 6: scheduler.PieceResult.code:type_name -> base.Code
10, // 7: scheduler.PieceResult.host_load:type_name -> base.HostLoad
8, // 8: scheduler.PeerPacket.main_peer:type_name -> scheduler.PeerPacket.DestPeer
8, // 9: scheduler.PeerPacket.steal_peers:type_name -> scheduler.PeerPacket.DestPeer
13, // 10: scheduler.PeerPacket.code:type_name -> base.Code
13, // 11: scheduler.PeerResult.code:type_name -> base.Code
0, // 12: scheduler.Scheduler.RegisterPeerTask:input_type -> scheduler.PeerTaskRequest
4, // 13: scheduler.Scheduler.ReportPieceResult:input_type -> scheduler.PieceResult
6, // 14: scheduler.Scheduler.ReportPeerResult:input_type -> scheduler.PeerResult
7, // 15: scheduler.Scheduler.LeaveTask:input_type -> scheduler.PeerTarget
1, // 16: scheduler.Scheduler.RegisterPeerTask:output_type -> scheduler.RegisterResult
5, // 17: scheduler.Scheduler.ReportPieceResult:output_type -> scheduler.PeerPacket
14, // 18: scheduler.Scheduler.ReportPeerResult:output_type -> google.protobuf.Empty
14, // 19: scheduler.Scheduler.LeaveTask:output_type -> google.protobuf.Empty
16, // [16:20] is the sub-list for method output_type
12, // [12:16] is the sub-list for method input_type
12, // [12:12] is the sub-list for extension type_name
12, // [12:12] is the sub-list for extension extendee
0, // [0:12] is the sub-list for field type_name
12, // 6: scheduler.PieceResult.piece_info:type_name -> base.PieceInfo
13, // 7: scheduler.PieceResult.code:type_name -> base.Code
10, // 8: scheduler.PieceResult.host_load:type_name -> base.HostLoad
8, // 9: scheduler.PeerPacket.main_peer:type_name -> scheduler.PeerPacket.DestPeer
8, // 10: scheduler.PeerPacket.steal_peers:type_name -> scheduler.PeerPacket.DestPeer
13, // 11: scheduler.PeerPacket.code:type_name -> base.Code
13, // 12: scheduler.PeerResult.code:type_name -> base.Code
0, // 13: scheduler.Scheduler.RegisterPeerTask:input_type -> scheduler.PeerTaskRequest
4, // 14: scheduler.Scheduler.ReportPieceResult:input_type -> scheduler.PieceResult
6, // 15: scheduler.Scheduler.ReportPeerResult:input_type -> scheduler.PeerResult
7, // 16: scheduler.Scheduler.LeaveTask:input_type -> scheduler.PeerTarget
1, // 17: scheduler.Scheduler.RegisterPeerTask:output_type -> scheduler.RegisterResult
5, // 18: scheduler.Scheduler.ReportPieceResult:output_type -> scheduler.PeerPacket
14, // 19: scheduler.Scheduler.ReportPeerResult:output_type -> google.protobuf.Empty
14, // 20: scheduler.Scheduler.LeaveTask:output_type -> google.protobuf.Empty
17, // [17:21] is the sub-list for method output_type
13, // [13:17] is the sub-list for method input_type
13, // [13:13] is the sub-list for extension type_name
13, // [13:13] is the sub-list for extension extendee
0, // [0:13] is the sub-list for field type_name
}
func init() { file_pkg_rpc_scheduler_scheduler_proto_init() }

View File

@ -44,16 +44,49 @@ var (
)
// Validate checks the field values on PeerTaskRequest with the rules defined
// in the proto definition for this message. If any rules are violated, an
// error is returned.
// 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.
func (m *PeerTaskRequest) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on PeerTaskRequest 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
// PeerTaskRequestMultiError, or nil if none found.
func (m *PeerTaskRequest) ValidateAll() error {
return m.validate(true)
}
func (m *PeerTaskRequest) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for Url
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, PeerTaskRequestValidationError{
field: "UrlMeta",
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, PeerTaskRequestValidationError{
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 PeerTaskRequestValidationError{
field: "UrlMeta",
@ -65,7 +98,26 @@ func (m *PeerTaskRequest) Validate() error {
// no validation rules for PeerId
if v, ok := interface{}(m.GetPeerHost()).(interface{ Validate() error }); ok {
if all {
switch v := interface{}(m.GetPeerHost()).(type) {
case interface{ ValidateAll() error }:
if err := v.ValidateAll(); err != nil {
errors = append(errors, PeerTaskRequestValidationError{
field: "PeerHost",
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, PeerTaskRequestValidationError{
field: "PeerHost",
reason: "embedded message failed validation",
cause: err,
})
}
}
} else if v, ok := interface{}(m.GetPeerHost()).(interface{ Validate() error }); ok {
if err := v.Validate(); err != nil {
return PeerTaskRequestValidationError{
field: "PeerHost",
@ -75,7 +127,26 @@ func (m *PeerTaskRequest) Validate() error {
}
}
if v, ok := interface{}(m.GetHostLoad()).(interface{ Validate() error }); ok {
if all {
switch v := interface{}(m.GetHostLoad()).(type) {
case interface{ ValidateAll() error }:
if err := v.ValidateAll(); err != nil {
errors = append(errors, PeerTaskRequestValidationError{
field: "HostLoad",
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, PeerTaskRequestValidationError{
field: "HostLoad",
reason: "embedded message failed validation",
cause: err,
})
}
}
} else if v, ok := interface{}(m.GetHostLoad()).(interface{ Validate() error }); ok {
if err := v.Validate(); err != nil {
return PeerTaskRequestValidationError{
field: "HostLoad",
@ -87,9 +158,29 @@ func (m *PeerTaskRequest) Validate() error {
// no validation rules for IsMigrating
if len(errors) > 0 {
return PeerTaskRequestMultiError(errors)
}
return nil
}
// PeerTaskRequestMultiError is an error wrapping multiple validation errors
// returned by PeerTaskRequest.ValidateAll() if the designated constraints
// aren't met.
type PeerTaskRequestMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m PeerTaskRequestMultiError) 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 PeerTaskRequestMultiError) AllErrors() []error { return m }
// PeerTaskRequestValidationError is the validation error returned by
// PeerTaskRequest.Validate if the designated constraints aren't met.
type PeerTaskRequestValidationError struct {
@ -145,13 +236,27 @@ var _ interface {
} = PeerTaskRequestValidationError{}
// Validate checks the field values on RegisterResult 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 *RegisterResult) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on RegisterResult 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 RegisterResultMultiError,
// or nil if none found.
func (m *RegisterResult) ValidateAll() error {
return m.validate(true)
}
func (m *RegisterResult) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for TaskId
// no validation rules for SizeScope
@ -160,7 +265,26 @@ func (m *RegisterResult) Validate() error {
case *RegisterResult_SinglePiece:
if v, ok := interface{}(m.GetSinglePiece()).(interface{ Validate() error }); ok {
if all {
switch v := interface{}(m.GetSinglePiece()).(type) {
case interface{ ValidateAll() error }:
if err := v.ValidateAll(); err != nil {
errors = append(errors, RegisterResultValidationError{
field: "SinglePiece",
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, RegisterResultValidationError{
field: "SinglePiece",
reason: "embedded message failed validation",
cause: err,
})
}
}
} else if v, ok := interface{}(m.GetSinglePiece()).(interface{ Validate() error }); ok {
if err := v.Validate(); err != nil {
return RegisterResultValidationError{
field: "SinglePiece",
@ -175,9 +299,29 @@ func (m *RegisterResult) Validate() error {
}
if len(errors) > 0 {
return RegisterResultMultiError(errors)
}
return nil
}
// RegisterResultMultiError is an error wrapping multiple validation errors
// returned by RegisterResult.ValidateAll() if the designated constraints
// aren't met.
type RegisterResultMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m RegisterResultMultiError) 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 RegisterResultMultiError) AllErrors() []error { return m }
// RegisterResultValidationError is the validation error returned by
// RegisterResult.Validate if the designated constraints aren't met.
type RegisterResultValidationError struct {
@ -233,18 +377,51 @@ var _ interface {
} = RegisterResultValidationError{}
// Validate checks the field values on SinglePiece 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 *SinglePiece) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on SinglePiece 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 SinglePieceMultiError, or
// nil if none found.
func (m *SinglePiece) ValidateAll() error {
return m.validate(true)
}
func (m *SinglePiece) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for DstPid
// no validation rules for DstAddr
if v, ok := interface{}(m.GetPieceInfo()).(interface{ Validate() error }); ok {
if all {
switch v := interface{}(m.GetPieceInfo()).(type) {
case interface{ ValidateAll() error }:
if err := v.ValidateAll(); err != nil {
errors = append(errors, SinglePieceValidationError{
field: "PieceInfo",
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, SinglePieceValidationError{
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 SinglePieceValidationError{
field: "PieceInfo",
@ -254,9 +431,28 @@ func (m *SinglePiece) Validate() error {
}
}
if len(errors) > 0 {
return SinglePieceMultiError(errors)
}
return nil
}
// SinglePieceMultiError is an error wrapping multiple validation errors
// returned by SinglePiece.ValidateAll() if the designated constraints aren't met.
type SinglePieceMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m SinglePieceMultiError) 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 SinglePieceMultiError) AllErrors() []error { return m }
// SinglePieceValidationError is the validation error returned by
// SinglePiece.Validate if the designated constraints aren't met.
type SinglePieceValidationError struct {
@ -312,12 +508,27 @@ var _ interface {
} = SinglePieceValidationError{}
// Validate checks the field values on PeerHost 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 *PeerHost) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on PeerHost 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 PeerHostMultiError, or nil
// if none found.
func (m *PeerHost) ValidateAll() error {
return m.validate(true)
}
func (m *PeerHost) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for Uuid
// no validation rules for Ip
@ -336,9 +547,28 @@ func (m *PeerHost) Validate() error {
// no validation rules for NetTopology
if len(errors) > 0 {
return PeerHostMultiError(errors)
}
return nil
}
// PeerHostMultiError is an error wrapping multiple validation errors returned
// by PeerHost.ValidateAll() if the designated constraints aren't met.
type PeerHostMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m PeerHostMultiError) 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 PeerHostMultiError) AllErrors() []error { return m }
// PeerHostValidationError is the validation error returned by
// PeerHost.Validate if the designated constraints aren't met.
type PeerHostValidationError struct {
@ -394,20 +624,61 @@ var _ interface {
} = PeerHostValidationError{}
// Validate checks the field values on PieceResult 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 *PieceResult) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on PieceResult 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 PieceResultMultiError, or
// nil if none found.
func (m *PieceResult) ValidateAll() error {
return m.validate(true)
}
func (m *PieceResult) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for TaskId
// no validation rules for SrcPid
// no validation rules for DstPid
// no validation rules for PieceNum
if all {
switch v := interface{}(m.GetPieceInfo()).(type) {
case interface{ ValidateAll() error }:
if err := v.ValidateAll(); err != nil {
errors = append(errors, PieceResultValidationError{
field: "PieceInfo",
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, PieceResultValidationError{
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 PieceResultValidationError{
field: "PieceInfo",
reason: "embedded message failed validation",
cause: err,
}
}
}
// no validation rules for BeginTime
@ -417,7 +688,26 @@ func (m *PieceResult) Validate() error {
// no validation rules for Code
if v, ok := interface{}(m.GetHostLoad()).(interface{ Validate() error }); ok {
if all {
switch v := interface{}(m.GetHostLoad()).(type) {
case interface{ ValidateAll() error }:
if err := v.ValidateAll(); err != nil {
errors = append(errors, PieceResultValidationError{
field: "HostLoad",
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, PieceResultValidationError{
field: "HostLoad",
reason: "embedded message failed validation",
cause: err,
})
}
}
} else if v, ok := interface{}(m.GetHostLoad()).(interface{ Validate() error }); ok {
if err := v.Validate(); err != nil {
return PieceResultValidationError{
field: "HostLoad",
@ -429,9 +719,28 @@ func (m *PieceResult) Validate() error {
// no validation rules for FinishedCount
if len(errors) > 0 {
return PieceResultMultiError(errors)
}
return nil
}
// PieceResultMultiError is an error wrapping multiple validation errors
// returned by PieceResult.ValidateAll() if the designated constraints aren't met.
type PieceResultMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m PieceResultMultiError) 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 PieceResultMultiError) AllErrors() []error { return m }
// PieceResultValidationError is the validation error returned by
// PieceResult.Validate if the designated constraints aren't met.
type PieceResultValidationError struct {
@ -487,19 +796,53 @@ var _ interface {
} = PieceResultValidationError{}
// Validate checks the field values on PeerPacket 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 *PeerPacket) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on PeerPacket 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 PeerPacketMultiError, or
// nil if none found.
func (m *PeerPacket) ValidateAll() error {
return m.validate(true)
}
func (m *PeerPacket) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for TaskId
// no validation rules for SrcPid
// no validation rules for ParallelCount
if v, ok := interface{}(m.GetMainPeer()).(interface{ Validate() error }); ok {
if all {
switch v := interface{}(m.GetMainPeer()).(type) {
case interface{ ValidateAll() error }:
if err := v.ValidateAll(); err != nil {
errors = append(errors, PeerPacketValidationError{
field: "MainPeer",
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, PeerPacketValidationError{
field: "MainPeer",
reason: "embedded message failed validation",
cause: err,
})
}
}
} else if v, ok := interface{}(m.GetMainPeer()).(interface{ Validate() error }); ok {
if err := v.Validate(); err != nil {
return PeerPacketValidationError{
field: "MainPeer",
@ -512,7 +855,26 @@ func (m *PeerPacket) Validate() error {
for idx, item := range m.GetStealPeers() {
_, _ = idx, item
if v, ok := interface{}(item).(interface{ Validate() error }); ok {
if all {
switch v := interface{}(item).(type) {
case interface{ ValidateAll() error }:
if err := v.ValidateAll(); err != nil {
errors = append(errors, PeerPacketValidationError{
field: fmt.Sprintf("StealPeers[%v]", idx),
reason: "embedded message failed validation",
cause: err,
})
}
case interface{ Validate() error }:
if err := v.Validate(); err != nil {
errors = append(errors, PeerPacketValidationError{
field: fmt.Sprintf("StealPeers[%v]", idx),
reason: "embedded message failed validation",
cause: err,
})
}
}
} else if v, ok := interface{}(item).(interface{ Validate() error }); ok {
if err := v.Validate(); err != nil {
return PeerPacketValidationError{
field: fmt.Sprintf("StealPeers[%v]", idx),
@ -526,9 +888,28 @@ func (m *PeerPacket) Validate() error {
// no validation rules for Code
if len(errors) > 0 {
return PeerPacketMultiError(errors)
}
return nil
}
// PeerPacketMultiError is an error wrapping multiple validation errors
// returned by PeerPacket.ValidateAll() if the designated constraints aren't met.
type PeerPacketMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m PeerPacketMultiError) 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 PeerPacketMultiError) AllErrors() []error { return m }
// PeerPacketValidationError is the validation error returned by
// PeerPacket.Validate if the designated constraints aren't met.
type PeerPacketValidationError struct {
@ -584,12 +965,27 @@ var _ interface {
} = PeerPacketValidationError{}
// Validate checks the field values on PeerResult 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 *PeerResult) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on PeerResult 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 PeerResultMultiError, or
// nil if none found.
func (m *PeerResult) ValidateAll() error {
return m.validate(true)
}
func (m *PeerResult) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for TaskId
// no validation rules for PeerId
@ -612,9 +1008,30 @@ func (m *PeerResult) Validate() error {
// no validation rules for Code
// no validation rules for TotalPieceCount
if len(errors) > 0 {
return PeerResultMultiError(errors)
}
return nil
}
// PeerResultMultiError is an error wrapping multiple validation errors
// returned by PeerResult.ValidateAll() if the designated constraints aren't met.
type PeerResultMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m PeerResultMultiError) 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 PeerResultMultiError) AllErrors() []error { return m }
// PeerResultValidationError is the validation error returned by
// PeerResult.Validate if the designated constraints aren't met.
type PeerResultValidationError struct {
@ -670,19 +1087,53 @@ var _ interface {
} = PeerResultValidationError{}
// Validate checks the field values on PeerTarget 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 *PeerTarget) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on PeerTarget 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 PeerTargetMultiError, or
// nil if none found.
func (m *PeerTarget) ValidateAll() error {
return m.validate(true)
}
func (m *PeerTarget) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for TaskId
// no validation rules for PeerId
if len(errors) > 0 {
return PeerTargetMultiError(errors)
}
return nil
}
// PeerTargetMultiError is an error wrapping multiple validation errors
// returned by PeerTarget.ValidateAll() if the designated constraints aren't met.
type PeerTargetMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m PeerTargetMultiError) 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 PeerTargetMultiError) AllErrors() []error { return m }
// PeerTargetValidationError is the validation error returned by
// PeerTarget.Validate if the designated constraints aren't met.
type PeerTargetValidationError struct {
@ -739,21 +1190,55 @@ var _ interface {
// Validate checks the field values on PeerPacket_DestPeer with the rules
// defined in the proto definition for this message. If any rules are
// violated, an error is returned.
// violated, the first error encountered is returned, or nil if there are no violations.
func (m *PeerPacket_DestPeer) Validate() error {
return m.validate(false)
}
// ValidateAll checks the field values on PeerPacket_DestPeer 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
// PeerPacket_DestPeerMultiError, or nil if none found.
func (m *PeerPacket_DestPeer) ValidateAll() error {
return m.validate(true)
}
func (m *PeerPacket_DestPeer) validate(all bool) error {
if m == nil {
return nil
}
var errors []error
// no validation rules for Ip
// no validation rules for RpcPort
// no validation rules for PeerId
if len(errors) > 0 {
return PeerPacket_DestPeerMultiError(errors)
}
return nil
}
// PeerPacket_DestPeerMultiError is an error wrapping multiple validation
// errors returned by PeerPacket_DestPeer.ValidateAll() if the designated
// constraints aren't met.
type PeerPacket_DestPeerMultiError []error
// Error returns a concatenation of all the error messages it wraps.
func (m PeerPacket_DestPeerMultiError) 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 PeerPacket_DestPeerMultiError) AllErrors() []error { return m }
// PeerPacket_DestPeerValidationError is the validation error returned by
// PeerPacket_DestPeer.Validate if the designated constraints aren't met.
type PeerPacket_DestPeerValidationError struct {

View File

@ -90,8 +90,8 @@ message PieceResult{
string src_pid = 2;
// dest peer id
string dst_pid = 3;
// piece number
int32 piece_num = 4;
// piece info
base.PieceInfo piece_info = 4;
// begin time for the piece downloading
uint64 begin_time = 5;
// end time for the piece downloading
@ -122,7 +122,6 @@ message PeerPacket{
string src_pid = 3;
// concurrent downloading count from main peer
// TODO what effect
int32 parallel_count = 4;
DestPeer main_peer = 5;
repeated DestPeer steal_peers = 6;
@ -147,6 +146,7 @@ message PeerResult{
bool success = 10;
// result code
base.Code code = 11;
int32 total_piece_count = 12;
}
message PeerTarget{

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

@ -44,15 +44,15 @@ type proxy struct {
scheduler.UnimplementedSchedulerServer
}
// see scheduler.SchedulerServer
// SchedulerServer scheduler.SchedulerServer
type SchedulerServer interface {
// RegisterPeerTask
// RegisterPeerTask register a peer to scheduler
RegisterPeerTask(context.Context, *scheduler.PeerTaskRequest) (*scheduler.RegisterResult, error)
// ReportPieceResult
// ReportPieceResult report piece result to scheduler
ReportPieceResult(scheduler.Scheduler_ReportPieceResultServer) error
// ReportPeerResult
// ReportPeerResult report peer download result to scheduler
ReportPeerResult(context.Context, *scheduler.PeerResult) error
// LeaveTask
// LeaveTask leave peer from scheduler
LeaveTask(context.Context, *scheduler.PeerTarget) error
}

View File

@ -35,12 +35,12 @@ type Config struct {
Manager *ManagerConfig `yaml:"manager" mapstructure:"manager"`
Host *HostConfig `yaml:"host" mapstructure:"host"`
Job *JobConfig `yaml:"job" mapstructure:"job"`
DisableCDN bool `yaml:"disableCDN" mapstructure:"disableCDN"`
}
func New() *Config {
return &Config{
Scheduler: &SchedulerConfig{
DisableCDN: false,
ABTest: false,
AScheduler: "",
BScheduler: "",
@ -53,8 +53,8 @@ func New() *Config {
ClientLoad: 10,
OpenMonitor: true,
GC: &GCConfig{
PeerGCInterval: 5 * time.Minute,
TaskGCInterval: 5 * time.Minute,
PeerGCInterval: 1 * time.Minute,
TaskGCInterval: 1 * time.Minute,
PeerTTL: 10 * time.Minute,
PeerTTI: 3 * time.Minute,
TaskTTL: 10 * time.Minute,
@ -109,6 +109,7 @@ func New() *Config {
BackendDB: 2,
},
},
DisableCDN: false,
}
}
@ -178,12 +179,11 @@ type DynConfig struct {
}
type SchedulerConfig struct {
DisableCDN bool `yaml:"disableCDN" mapstructure:"disableCDN"`
ABTest bool `yaml:"abtest" mapstructure:"abtest"`
AScheduler string `yaml:"ascheduler" mapstructure:"ascheduler"`
BScheduler string `yaml:"bscheduler" mapstructure:"bscheduler"`
WorkerNum int `yaml:"workerNum" mapstructure:"workerNum"`
BackSourceCount int `yaml:"backSourceCount" mapstructure:"backSourceCount"`
BackSourceCount int32 `yaml:"backSourceCount" mapstructure:"backSourceCount"`
// AccessWindow should less than CDN task expireTime
AccessWindow time.Duration `yaml:"accessWindow" mapstructure:"accessWindow"`
CandidateParentCount int `yaml:"candidateParentCount" mapstructure:"candidateParentCount"`
@ -202,6 +202,7 @@ type ServerConfig struct {
type GCConfig struct {
PeerGCInterval time.Duration `yaml:"peerGCInterval" mapstructure:"peerGCInterval"`
// PeerTTL is advised to set the time to be smaller than the expire time of a task in the CDN
PeerTTL time.Duration `yaml:"peerTTL" mapstructure:"peerTTL"`
PeerTTI time.Duration `yaml:"peerTTI" mapstructure:"peerTTI"`
TaskGCInterval time.Duration `yaml:"taskGCInterval" mapstructure:"taskGCInterval"`

View File

@ -35,6 +35,10 @@ const (
AttributePeerID = attribute.Key("d7y.peer.id")
AttributeCDNSeedRequest = attribute.Key("d7y.cdn.seed.request")
AttributeNeedSeedCDN = attribute.Key("d7y.need.seed.cdn")
AttributeTaskStatus = attribute.Key("d7y.task.status")
AttributeLastTriggerTime = attribute.Key("d7y.task.last.trigger.time")
AttributeClientBackSource = attribute.Key("d7y.need.client.back-source")
AttributeTriggerCDNError = attribute.Key("d7y.trigger.cdn.error")
)
const (
@ -46,11 +50,13 @@ const (
)
const (
EventScheduleParentFail = "fail-schedule-parent"
EventPeerNotFound = "peer-not-found"
EventHostNotFound = "host-not-found"
EventCreateCDNPeer = "create-cdn-peer"
EventPieceReceived = "receive-piece"
EventPeerDownloaded = "downloaded"
EventDownloadTinyFile = "download-tiny-file"
EventSmallTaskSelectParentFail = "small-task-select-parent-fail"
EventPeerNotFound = "peer-not-found"
EventHostNotFound = "host-not-found"
EventCreateCDNPeer = "create-cdn-peer"
EventPieceReceived = "receive-piece"
EventPeerDownloaded = "downloaded"
EventDownloadTinyFile = "download-tiny-file"
EventStartReportPieceResult = "start-report-piece-result"
EventCDNFailBackClientSource = "cdn-fail-back-client-source"
)

View File

@ -69,7 +69,7 @@ func (eval *baseEvaluator) NeedAdjustParent(peer *supervisor.Peer) bool {
func (eval *baseEvaluator) IsBadNode(peer *supervisor.Peer) bool {
if peer.IsBad() {
logger.Debugf("peer %s is bad because status is %s", peer.PeerID, peer.GetStatus())
logger.Debugf("peer %s is bad because it's status is %s", peer.PeerID, peer.GetStatus())
return true
}
costHistory := peer.GetCostHistory()

View File

@ -21,11 +21,12 @@ import (
"time"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/pkg/rpc/base"
schedulerRPC "d7y.io/dragonfly/v2/pkg/rpc/scheduler"
"d7y.io/dragonfly/v2/pkg/structure/sortedlist"
"d7y.io/dragonfly/v2/pkg/synclock"
"d7y.io/dragonfly/v2/scheduler/config"
"d7y.io/dragonfly/v2/scheduler/core/scheduler"
"d7y.io/dragonfly/v2/scheduler/supervisor"
"go.opentelemetry.io/otel/trace"
@ -76,20 +77,37 @@ var _ event = startReportPieceResultEvent{}
func (e startReportPieceResultEvent) apply(s *state) {
span := trace.SpanFromContext(e.ctx)
span.AddEvent(config.EventStartReportPieceResult)
if e.peer.GetParent() != nil {
span.AddEvent("no parent")
logger.WithTaskAndPeerID(e.peer.Task.TaskID,
e.peer.PeerID).Warnf("startReportPieceResultEvent: no need schedule parent because peer already had parent %s", e.peer.GetParent().PeerID)
return
}
if e.peer.Task.IsBackSourcePeer(e.peer.PeerID) {
logger.WithTaskAndPeerID(e.peer.Task.TaskID,
e.peer.PeerID).Info("startReportPieceResultEvent: no need schedule parent because peer is back source peer")
s.waitScheduleParentPeerQueue.Done(e.peer)
return
}
parent, candidates, hasParent := s.sched.ScheduleParent(e.peer)
span.AddEvent("parent")
// No parent node is currently available
if !hasParent {
logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("startReportPieceResultEvent: there is no available parentreschedule it in one second")
if e.peer.Task.NeedClientBackSource() {
span.SetAttributes(config.AttributeClientBackSource.Bool(true))
if e.peer.CloseChannel(dferrors.Newf(dfcodes.SchedNeedBackSource, "peer %s need back source", e.peer.PeerID)) == nil {
e.peer.Task.IncreaseBackSourcePeer(e.peer.PeerID)
s.waitScheduleParentPeerQueue.Done(e.peer)
}
return
}
logger.WithTaskAndPeerID(e.peer.Task.TaskID,
e.peer.PeerID).Warnf("startReportPieceResultEvent: no parent node is currently availablereschedule it later")
s.waitScheduleParentPeerQueue.AddAfter(e.peer, time.Second)
return
}
e.peer.SendSchedulePacket(constructSuccessPeerPacket(e.peer, parent, candidates))
if err := e.peer.SendSchedulePacket(constructSuccessPeerPacket(e.peer, parent, candidates)); err != nil {
logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", e.peer.PeerID, err)
}
}
func (e startReportPieceResultEvent) hashKey() string {
@ -106,28 +124,45 @@ var _ event = peerDownloadPieceSuccessEvent{}
func (e peerDownloadPieceSuccessEvent) apply(s *state) {
span := trace.SpanFromContext(e.ctx)
span.AddEvent("piece success")
e.peer.AddPieceInfo(e.pr.FinishedCount, int(e.pr.EndTime-e.pr.BeginTime))
oldParent := e.peer.GetParent()
span.AddEvent(config.EventPieceReceived, trace.WithAttributes(config.AttributePieceReceived.String(e.pr.String())))
e.peer.UpdateProgress(e.pr.FinishedCount, int(e.pr.EndTime-e.pr.BeginTime))
if e.peer.Task.IsBackSourcePeer(e.peer.PeerID) {
e.peer.Task.AddPiece(e.pr.PieceInfo)
if !e.peer.Task.CanSchedule() {
logger.WithTaskAndPeerID(e.peer.Task.TaskID,
e.peer.PeerID).Warnf("peerDownloadPieceSuccessEvent: update task status seeding")
e.peer.Task.SetStatus(supervisor.TaskStatusSeeding)
}
return
}
var candidates []*supervisor.Peer
parentPeer, ok := s.peerManager.Get(e.pr.DstPid)
if !ok || parentPeer.IsLeave() {
if ok {
oldParent := e.peer.GetParent()
if e.pr.DstPid != e.peer.PeerID && (oldParent == nil || oldParent.PeerID != e.pr.DstPid) {
e.peer.ReplaceParent(parentPeer)
}
} else if parentPeer.IsLeave() {
logger.WithTaskAndPeerID(e.peer.Task.TaskID,
e.peer.PeerID).Warnf("peerDownloadPieceSuccessEvent: need reschedule parent for peer because it's parent is leave")
e.peer.ReplaceParent(nil)
var hasParent bool
parentPeer, candidates, hasParent = s.sched.ScheduleParent(e.peer)
if !hasParent {
logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("peerDownloadPieceSuccessEvent: there is no available parent, reschedule it in one second")
logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("peerDownloadPieceSuccessEvent: no parent node is currently available, " +
"reschedule it later")
s.waitScheduleParentPeerQueue.AddAfter(e.peer, time.Second)
return
}
}
parentPeer.Touch()
if oldParent != nil {
candidates = append(candidates, oldParent)
if parentPeer.PeerID == e.pr.DstPid {
return
}
// TODO if parentPeer is equal with oldParent, need schedule again ?
e.peer.SendSchedulePacket(constructSuccessPeerPacket(e.peer, parentPeer, candidates))
return
if err := e.peer.SendSchedulePacket(constructSuccessPeerPacket(e.peer, parentPeer, candidates)); err != nil {
logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", e.peer.PeerID, err)
}
}
func (e peerDownloadPieceSuccessEvent) hashKey() string {
@ -135,6 +170,7 @@ func (e peerDownloadPieceSuccessEvent) hashKey() string {
}
type peerDownloadPieceFailEvent struct {
ctx context.Context
peer *supervisor.Peer
pr *schedulerRPC.PieceResult
}
@ -142,7 +178,14 @@ type peerDownloadPieceFailEvent struct {
var _ event = peerDownloadPieceFailEvent{}
func (e peerDownloadPieceFailEvent) apply(s *state) {
span := trace.SpanFromContext(e.ctx)
span.AddEvent(config.EventPieceReceived, trace.WithAttributes(config.AttributePieceReceived.String(e.pr.String())))
if e.peer.Task.IsBackSourcePeer(e.peer.PeerID) {
return
}
switch e.pr.Code {
case dfcodes.ClientWaitPieceReady:
return
case dfcodes.PeerTaskNotFound, dfcodes.ClientPieceRequestFail, dfcodes.ClientPieceDownloadFail:
// TODO PeerTaskNotFound remove dest peer task, ClientPieceDownloadFail add blank list
reScheduleParent(e.peer, s)
@ -152,16 +195,17 @@ func (e peerDownloadPieceFailEvent) apply(s *state) {
// TODO
synclock.Lock(task.TaskID, false)
defer synclock.UnLock(task.TaskID, false)
task.SetStatus(supervisor.TaskStatusRunning)
if cdnPeer, err := s.cdnManager.StartSeedTask(context.Background(), task); err != nil {
logger.Errorf("start seed task fail: %v", err)
task.SetStatus(supervisor.TaskStatusFailed)
span.AddEvent(config.EventCDNFailBackClientSource, trace.WithAttributes(config.AttributeTriggerCDNError.String(err.Error())))
handleSeedTaskFail(task)
} else {
logger.Debugf("===== successfully obtain seeds from cdn, task: %+v =====", e.peer.Task)
children := s.sched.ScheduleChildren(cdnPeer)
for _, child := range children {
child.SendSchedulePacket(constructSuccessPeerPacket(child, cdnPeer, nil))
if err := child.SendSchedulePacket(constructSuccessPeerPacket(child, cdnPeer, nil)); err != nil {
logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", child.PeerID, err)
}
}
}
}(e.peer.Task)
@ -197,12 +241,16 @@ var _ event = peerDownloadSuccessEvent{}
func (e peerDownloadSuccessEvent) apply(s *state) {
e.peer.SetStatus(supervisor.PeerStatusSuccess)
if e.peer.Task.IsBackSourcePeer(e.peer.PeerID) {
e.peer.Task.UpdateTaskSuccessResult(e.peerResult.TotalPieceCount, e.peerResult.ContentLength)
}
removePeerFromCurrentTree(e.peer, s)
children := s.sched.ScheduleChildren(e.peer)
for _, child := range children {
child.SendSchedulePacket(constructSuccessPeerPacket(child, e.peer, nil))
if err := child.SendSchedulePacket(constructSuccessPeerPacket(child, e.peer, nil)); err != nil {
logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", child.PeerID, err)
}
}
e.peer.UnBindSendChannel()
}
func (e peerDownloadSuccessEvent) hashKey() string {
@ -218,16 +266,23 @@ var _ event = peerDownloadFailEvent{}
func (e peerDownloadFailEvent) apply(s *state) {
e.peer.SetStatus(supervisor.PeerStatusFail)
if e.peer.Task.IsBackSourcePeer(e.peer.PeerID) && !e.peer.Task.IsSuccess() {
e.peer.Task.SetStatus(supervisor.TaskStatusFail)
handleSeedTaskFail(e.peer.Task)
return
}
removePeerFromCurrentTree(e.peer, s)
e.peer.GetChildren().Range(func(key, value interface{}) bool {
child := (value).(*supervisor.Peer)
parent, candidates, hasParent := s.sched.ScheduleParent(child)
if !hasParent {
logger.WithTaskAndPeerID(child.Task.TaskID, child.PeerID).Warnf("peerDownloadFailEvent: there is no available parent, reschedule it in one second")
logger.WithTaskAndPeerID(child.Task.TaskID, child.PeerID).Warnf("peerDownloadFailEvent: there is no available parent, reschedule it later")
s.waitScheduleParentPeerQueue.AddAfter(e.peer, time.Second)
return true
}
child.SendSchedulePacket(constructSuccessPeerPacket(child, parent, candidates))
if err := child.SendSchedulePacket(constructSuccessPeerPacket(child, parent, candidates)); err != nil {
logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", child.PeerID, err)
}
return true
})
}
@ -250,11 +305,13 @@ func (e peerLeaveEvent) apply(s *state) {
child := value.(*supervisor.Peer)
parent, candidates, hasParent := s.sched.ScheduleParent(child)
if !hasParent {
logger.WithTaskAndPeerID(child.Task.TaskID, child.PeerID).Warnf("handlePeerLeave: there is no available parentreschedule it in one second")
logger.WithTaskAndPeerID(child.Task.TaskID, child.PeerID).Warnf("handlePeerLeave: there is no available parentreschedule it later")
s.waitScheduleParentPeerQueue.AddAfter(child, time.Second)
return true
}
child.SendSchedulePacket(constructSuccessPeerPacket(child, parent, candidates))
if err := child.SendSchedulePacket(constructSuccessPeerPacket(child, parent, candidates)); err != nil {
logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", child.PeerID, err)
}
return true
})
s.peerManager.Delete(e.peer.PeerID)
@ -264,6 +321,7 @@ func (e peerLeaveEvent) hashKey() string {
return e.peer.Task.TaskID
}
// constructSuccessPeerPacket construct success peer schedule packet
func constructSuccessPeerPacket(peer *supervisor.Peer, parent *supervisor.Peer, candidates []*supervisor.Peer) *schedulerRPC.PeerPacket {
mainPeer := &schedulerRPC.PeerPacket_DestPeer{
Ip: parent.Host.IP,
@ -290,30 +348,42 @@ func constructSuccessPeerPacket(peer *supervisor.Peer, parent *supervisor.Peer,
return peerPacket
}
func constructFailPeerPacket(peer *supervisor.Peer, errCode base.Code) *schedulerRPC.PeerPacket {
return &schedulerRPC.PeerPacket{
TaskId: peer.Task.TaskID,
SrcPid: peer.PeerID,
Code: errCode,
}
}
func reScheduleParent(peer *supervisor.Peer, s *state) {
parent, candidates, hasParent := s.sched.ScheduleParent(peer)
if !hasParent {
logger.Errorf("handleReplaceParent: failed to schedule parent to peer %s, reschedule it in one second", peer.PeerID)
if peer.Task.NeedClientBackSource() {
if peer.CloseChannel(dferrors.Newf(dfcodes.SchedNeedBackSource, "peer %s need back source", peer.PeerID)) == nil {
peer.Task.IncreaseBackSourcePeer(peer.PeerID)
}
return
}
logger.Errorf("reScheduleParent: failed to schedule parent to peer %s, reschedule it later", peer.PeerID)
//peer.PacketChan <- constructFailPeerPacket(peer, dfcodes.SchedWithoutParentPeer)
s.waitScheduleParentPeerQueue.AddAfter(peer, time.Second)
return
}
peer.SendSchedulePacket(constructSuccessPeerPacket(peer, parent, candidates))
// TODO if parentPeer is equal with oldParent, need schedule again ?
if err := peer.SendSchedulePacket(constructSuccessPeerPacket(peer, parent, candidates)); err != nil {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", peer.PeerID, err)
}
}
func handleSeedTaskFail(task *supervisor.Task) {
if task.IsFail() {
if task.NeedClientBackSource() {
task.ListPeers().Range(func(data sortedlist.Item) bool {
peer := data.(*supervisor.Peer)
peer.SendSchedulePacket(constructFailPeerPacket(peer, dfcodes.SchedNeedBackSource))
if task.NeedClientBackSource() {
if peer.CloseChannel(dferrors.Newf(dfcodes.SchedNeedBackSource, "peer %s need back source", peer.PeerID)) == nil {
task.IncreaseBackSourcePeer(peer.PeerID)
}
return true
}
return false
})
} else {
task.ListPeers().Range(func(data sortedlist.Item) bool {
peer := data.(*supervisor.Peer)
peer.CloseChannel(dferrors.New(dfcodes.SchedTaskStatusError, "schedule task status failed"))
return true
})
}
@ -326,7 +396,9 @@ func removePeerFromCurrentTree(peer *supervisor.Peer, s *state) {
if parent != nil {
children := s.sched.ScheduleChildren(parent)
for _, child := range children {
child.SendSchedulePacket(constructSuccessPeerPacket(child, peer, nil))
if err := child.SendSchedulePacket(constructSuccessPeerPacket(child, peer, nil)); err != nil {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", child.PeerID, err)
}
}
}
}

View File

@ -70,19 +70,20 @@ type Scheduler struct {
}
func (s *Scheduler) ScheduleChildren(peer *supervisor.Peer) (children []*supervisor.Peer) {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debug("start schedule children flow")
if s.evaluator.IsBadNode(peer) {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debug("stop schedule children flow because peer is bad node")
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debug("terminate schedule children flow because peer is bad node")
return
}
freeUpload := peer.Host.GetFreeUploadLoad()
candidateChildren := s.selectCandidateChildren(peer, freeUpload*2)
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("select num %d candidate children %v", len(candidateChildren), candidateChildren)
evalResult := make(map[float64]*supervisor.Peer)
if len(candidateChildren) == 0 {
return nil
}
evalResult := make(map[float64][]*supervisor.Peer)
var evalScore []float64
for _, child := range candidateChildren {
score := s.evaluator.Evaluate(peer, child)
evalResult[score] = child
evalResult[score] = append(evalResult[score], child)
evalScore = append(evalScore, score)
}
sort.Float64s(evalScore)
@ -90,22 +91,26 @@ func (s *Scheduler) ScheduleChildren(peer *supervisor.Peer) (children []*supervi
if freeUpload <= 0 {
break
}
child := evalResult[evalScore[len(evalScore)-i-1]]
if child.GetParent() == peer {
continue
peers := evalResult[evalScore[len(evalScore)-i-1]]
for _, child := range peers {
if freeUpload <= 0 {
break
}
if child.GetParent() == peer {
continue
}
children = append(children, child)
freeUpload--
}
children = append(children, child)
freeUpload--
}
for _, child := range children {
child.ReplaceParent(peer)
}
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("final schedule children list %v", children)
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("schedule children result: %v", children)
return
}
func (s *Scheduler) ScheduleParent(peer *supervisor.Peer) (*supervisor.Peer, []*supervisor.Peer, bool) {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debug("start schedule parent flow")
//if !s.evaluator.NeedAdjustParent(peer) {
// logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("stop schedule parent flow because peer is not need adjust parent", peer.PeerID)
// if peer.GetParent() == nil {
@ -114,24 +119,21 @@ func (s *Scheduler) ScheduleParent(peer *supervisor.Peer) (*supervisor.Peer, []*
// return peer.GetParent(), []*types.Peer{peer.GetParent()}, true
//}
candidateParents := s.selectCandidateParents(peer, s.cfg.CandidateParentCount)
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("select num %d candidates parents, current task tree node count %d ",
len(candidateParents), peer.Task.ListPeers().Size())
if len(candidateParents) == 0 {
return nil, nil, false
}
evalResult := make(map[float64]*supervisor.Peer)
evalResult := make(map[float64][]*supervisor.Peer)
var evalScore []float64
for _, candidate := range candidateParents {
score := s.evaluator.Evaluate(candidate, peer)
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("evaluate score candidate %s is %f", candidate.PeerID, score)
evalResult[score] = candidate
for _, parent := range candidateParents {
score := s.evaluator.Evaluate(parent, peer)
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("evaluate score candidate %s is %f", parent.PeerID, score)
evalResult[score] = append(evalResult[score], parent)
evalScore = append(evalScore, score)
}
sort.Float64s(evalScore)
var parents = make([]*supervisor.Peer, 0, len(candidateParents))
for i := range evalScore {
parent := evalResult[evalScore[len(evalScore)-i-1]]
parents = append(parents, parent)
parents = append(parents, evalResult[evalScore[len(evalScore)-i-1]]...)
}
if parents[0] != peer.GetParent() {
peer.ReplaceParent(parents[0])
@ -140,108 +142,132 @@ func (s *Scheduler) ScheduleParent(peer *supervisor.Peer) (*supervisor.Peer, []*
return parents[0], parents[1:], true
}
func (s *Scheduler) selectCandidateChildren(peer *supervisor.Peer, limit int) (list []*supervisor.Peer) {
return s.peerManager.Pick(peer.Task, limit, func(candidateNode *supervisor.Peer) bool {
func (s *Scheduler) selectCandidateChildren(peer *supervisor.Peer, limit int) (candidateChildren []*supervisor.Peer) {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debug("start schedule children flow")
defer logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("finish schedule parent flow, select num %d candidate children, "+
"current task tree node count %d, back source peers: %s", len(candidateChildren), peer.Task.ListPeers().Size(), peer.Task.GetBackSourcePeers())
candidateChildren = peer.Task.Pick(limit, func(candidateNode *supervisor.Peer) bool {
if candidateNode == nil {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer is not selected because it is nil")
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer is not selected because it is nil******")
return false
}
// TODO IsWaiting
if candidateNode.IsDone() {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it has done",
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it has done******",
candidateNode.PeerID)
return false
}
if candidateNode.IsLeave() {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it has left",
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it has left******",
candidateNode.PeerID)
return false
}
if candidateNode.IsWaiting() {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it's status is Waiting******",
candidateNode.PeerID)
return false
}
if candidateNode == peer {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it and peer are the same",
logger.WithTaskAndPeerID(peer.Task.TaskID,
peer.PeerID).Debugf("******candidate child peer %s is not selected because it and peer are the same******",
candidateNode.PeerID)
return false
}
if candidateNode.IsAncestorOf(peer) {
logger.WithTaskAndPeerID(peer.Task.TaskID,
peer.PeerID).Debugf("******candidate child peer %s is not selected because peer's ancestor is candidate peer", candidateNode.PeerID)
peer.PeerID).Debugf("******candidate child peer %s is not selected because peer's ancestor is candidate peer******", candidateNode.PeerID)
return false
}
if candidateNode.GetFinishedNum() > peer.GetFinishedNum() {
if candidateNode.GetFinishedNum() >= peer.GetFinishedNum() {
logger.WithTaskAndPeerID(peer.Task.TaskID,
peer.PeerID).Debugf("******candidate child peer %s is not selected because it finished number of download is more than peer's",
peer.PeerID).Debugf("******candidate child peer %s is not selected because it finished number of download is equal to or greater than peer's"+
"******",
candidateNode.PeerID)
return false
}
if candidateNode.Host != nil && candidateNode.Host.CDN {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it is a cdn host",
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it is a cdn host******",
candidateNode.PeerID)
return false
}
if !candidateNode.IsConnected() {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it is not connected******",
candidateNode.PeerID)
return false
}
if candidateNode.GetParent() == nil {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is selected because it has not parent",
logger.WithTaskAndPeerID(peer.Task.TaskID,
peer.PeerID).Debugf("******[selected]candidate child peer %s is selected because it has not parent[selected]******",
candidateNode.PeerID)
return true
}
if candidateNode.GetParent() != nil && s.evaluator.IsBadNode(candidateNode.GetParent()) {
logger.WithTaskAndPeerID(peer.Task.TaskID,
peer.PeerID).Debugf("******candidate child peer %s is selected because it has parent and parent status is not health", candidateNode.PeerID)
peer.PeerID).Debugf("******[selected]candidate child peer %s is selected because parent's status is not health[selected]******",
candidateNode.PeerID)
return true
}
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is selected", candidateNode.PeerID)
return false
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******[default]candidate child peer %s is selected[default]******", candidateNode.PeerID)
return true
})
return
}
func (s *Scheduler) selectCandidateParents(peer *supervisor.Peer, limit int) (list []*supervisor.Peer) {
func (s *Scheduler) selectCandidateParents(peer *supervisor.Peer, limit int) (candidateParents []*supervisor.Peer) {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debug("start schedule parent flow")
defer logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("finish schedule parent flow, select num %d candidates parents,"+
"current task tree node count %d, back source peers: %s", len(candidateParents), peer.Task.ListPeers().Size(), peer.Task.GetBackSourcePeers())
if !peer.Task.CanSchedule() {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++peer %s can not be scheduled because task status", peer.PeerID)
logger.WithTaskAndPeerID(peer.Task.TaskID,
peer.PeerID).Debugf("++++++peer can not be scheduled because task cannot be scheduled at this timewaiting task status become seeding. "+
"it current status is %s++++++", peer.Task.GetStatus())
return nil
}
return s.peerManager.PickReverse(peer.Task, limit, func(candidateNode *supervisor.Peer) bool {
candidateParents = peer.Task.PickReverse(limit, func(candidateNode *supervisor.Peer) bool {
if candidateNode == nil {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer is not selected because it is nil")
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer is not selected because it is nil++++++")
return false
}
if s.evaluator.IsBadNode(candidateNode) {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it is badNode",
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it is badNode++++++",
candidateNode.PeerID)
return false
}
if candidateNode.IsLeave() {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it has already left",
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it has already left++++++",
candidateNode.PeerID)
return false
}
if candidateNode == peer {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it and peer are the same",
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it and peer are the same++++++",
candidateNode.PeerID)
return false
}
if candidateNode.IsDescendantOf(peer) {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it's ancestor is peer",
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it's ancestor is peer++++++",
candidateNode.PeerID)
return false
}
if candidateNode.Host.GetFreeUploadLoad() <= 0 {
logger.WithTaskAndPeerID(peer.Task.TaskID,
peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it's free upload load equal to less than zero",
peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it's free upload load equal to less than zero++++++",
candidateNode.PeerID)
return false
}
if candidateNode.IsWaiting() {
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it's status is waiting",
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it's status is waiting++++++",
candidateNode.PeerID)
return false
}
if candidateNode.GetFinishedNum() < peer.GetFinishedNum() {
if candidateNode.GetFinishedNum() <= peer.GetFinishedNum() {
logger.WithTaskAndPeerID(peer.Task.TaskID,
peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it finished number of download is smaller than peer's",
peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it finished number of download is equal to or smaller than peer"+
"'s++++++",
candidateNode.PeerID)
return false
}
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is selected", candidateNode.PeerID)
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++[default]candidate parent peer %s is selected[default]", candidateNode.PeerID)
return true
})
return
}

View File

@ -42,6 +42,25 @@ import (
"k8s.io/client-go/util/workqueue"
)
type Options struct {
openTel bool
disableCDN bool
}
type Option func(options *Options)
func WithOpenTel(openTel bool) Option {
return func(options *Options) {
options.openTel = openTel
}
}
func WithDisableCDN(disableCDN bool) Option {
return func(options *Options) {
options.disableCDN = disableCDN
}
}
type SchedulerService struct {
// cdn mgr
cdnManager supervisor.CDNMgr
@ -62,22 +81,14 @@ type SchedulerService struct {
wg sync.WaitGroup
}
func NewSchedulerService(cfg *config.SchedulerConfig, dynConfig config.DynconfigInterface, openTel bool) (*SchedulerService, error) {
func NewSchedulerService(cfg *config.SchedulerConfig, dynConfig config.DynconfigInterface, options ...Option) (*SchedulerService, error) {
ops := &Options{}
for _, op := range options {
op(ops)
}
hostManager := host.NewManager()
peerManager := peer.NewManager(cfg.GC, hostManager)
var opts []grpc.DialOption
if openTel {
opts = append(opts, grpc.WithChainUnaryInterceptor(otelgrpc.UnaryClientInterceptor()), grpc.WithChainStreamInterceptor(otelgrpc.StreamClientInterceptor()))
}
cdnClient, err := cdn.NewRefreshableCDNClient(dynConfig, opts)
if err != nil {
return nil, errors.Wrap(err, "new refreshable cdn client")
}
cdnManager, err := cdn.NewManager(cdnClient, peerManager, hostManager)
if err != nil {
return nil, errors.Wrap(err, "new cdn manager")
}
taskManager := task.NewManager(cfg.GC, peerManager)
sched, err := scheduler.Get(cfg.Scheduler).Build(cfg, &scheduler.BuildOptions{
PeerManager: peerManager,
@ -89,8 +100,7 @@ func NewSchedulerService(cfg *config.SchedulerConfig, dynConfig config.Dynconfig
work := newEventLoopGroup(cfg.WorkerNum)
downloadMonitor := newMonitor(cfg.OpenMonitor, peerManager)
done := make(chan struct{})
return &SchedulerService{
cdnManager: cdnManager,
s := &SchedulerService{
taskManager: taskManager,
hostManager: hostManager,
peerManager: peerManager,
@ -99,7 +109,23 @@ func NewSchedulerService(cfg *config.SchedulerConfig, dynConfig config.Dynconfig
sched: sched,
config: cfg,
done: done,
}, nil
}
if !ops.disableCDN {
var opts []grpc.DialOption
if ops.openTel {
opts = append(opts, grpc.WithChainUnaryInterceptor(otelgrpc.UnaryClientInterceptor()), grpc.WithChainStreamInterceptor(otelgrpc.StreamClientInterceptor()))
}
cdnClient, err := cdn.NewRefreshableCDNClient(dynConfig, opts)
if err != nil {
return nil, errors.Wrap(err, "new refreshable cdn client")
}
cdnManager, err := cdn.NewManager(cdnClient, peerManager, hostManager)
if err != nil {
return nil, errors.Wrap(err, "new cdn manager")
}
s.cdnManager = cdnManager
}
return s, nil
}
func (s *SchedulerService) Serve() {
@ -138,6 +164,12 @@ func (s *SchedulerService) runReScheduleParentLoop(wsdq workqueue.DelayingInterf
"isLeave %t", peer.GetStatus(), peer.IsLeave())
continue
}
if peer.GetParent() != nil {
logger.WithTaskAndPeerID(peer.Task.TaskID,
peer.PeerID).Debugf("runReScheduleLoop: peer has left from waitScheduleParentPeerQueue because peer has parent %s",
peer.GetParent().PeerID)
continue
}
s.worker.send(reScheduleParentEvent{peer})
}
}
@ -155,6 +187,7 @@ func (s *SchedulerService) Stop() {
if s.worker != nil {
s.worker.stop()
}
s.wg.Wait()
})
}
@ -165,9 +198,8 @@ func (s *SchedulerService) GenerateTaskID(url string, meta *base.UrlMeta, peerID
return idgen.TaskID(url, meta)
}
func (s *SchedulerService) ScheduleParent(peer *supervisor.Peer) (parent *supervisor.Peer, err error) {
func (s *SchedulerService) SelectParent(peer *supervisor.Peer) (parent *supervisor.Peer, err error) {
parent, _, hasParent := s.sched.ScheduleParent(peer)
//logger.Debugf("schedule parent result: parent %v, candidates:%v", parent, candidates)
if !hasParent || parent == nil {
return nil, errors.Errorf("no parent peer available for peer %v", peer.PeerID)
}
@ -178,28 +210,27 @@ func (s *SchedulerService) GetPeerTask(peerTaskID string) (peerTask *supervisor.
return s.peerManager.Get(peerTaskID)
}
func (s *SchedulerService) RegisterPeerTask(req *schedulerRPC.PeerTaskRequest, task *supervisor.Task) (*supervisor.Peer, error) {
var (
peer *supervisor.Peer
ok bool
peerHost *supervisor.PeerHost
)
func (s *SchedulerService) RegisterPeerTask(req *schedulerRPC.PeerTaskRequest, task *supervisor.Task) *supervisor.Peer {
// get or create host
reqPeerHost := req.PeerHost
if peerHost, ok = s.hostManager.Get(reqPeerHost.Uuid); !ok {
peerHost, ok := s.hostManager.Get(reqPeerHost.Uuid)
if !ok {
peerHost = supervisor.NewClientPeerHost(reqPeerHost.Uuid, reqPeerHost.Ip, reqPeerHost.HostName, reqPeerHost.RpcPort, reqPeerHost.DownPort,
reqPeerHost.SecurityDomain, reqPeerHost.Location, reqPeerHost.Idc, reqPeerHost.NetTopology, s.config.ClientLoad)
s.hostManager.Add(peerHost)
}
// get or creat PeerTask
if peer, ok = s.peerManager.Get(req.PeerId); !ok {
peer = supervisor.NewPeer(req.PeerId, task, peerHost)
s.peerManager.Add(peer)
peer, ok := s.peerManager.Get(req.PeerId)
if ok {
logger.Warnf("peer %s has already registered", peer.PeerID)
return peer
}
return peer, nil
peer = supervisor.NewPeer(req.PeerId, task, peerHost)
s.peerManager.Add(peer)
return peer
}
func (s *SchedulerService) GetOrCreateTask(ctx context.Context, task *supervisor.Task) (*supervisor.Task, error) {
func (s *SchedulerService) GetOrCreateTask(ctx context.Context, task *supervisor.Task) *supervisor.Task {
span := trace.SpanFromContext(ctx)
synclock.Lock(task.TaskID, true)
task, ok := s.taskManager.GetOrAdd(task)
@ -207,46 +238,55 @@ func (s *SchedulerService) GetOrCreateTask(ctx context.Context, task *supervisor
if task.GetLastTriggerTime().Add(s.config.AccessWindow).After(time.Now()) || task.IsHealth() {
synclock.UnLock(task.TaskID, true)
span.SetAttributes(config.AttributeNeedSeedCDN.Bool(false))
return task, nil
span.SetAttributes(config.AttributeTaskStatus.String(task.GetStatus().String()))
span.SetAttributes(config.AttributeLastTriggerTime.String(task.GetLastTriggerTime().String()))
return task
}
}
synclock.UnLock(task.TaskID, true)
// do trigger
task.UpdateLastTriggerTime(time.Now())
// register cdn peer task
// notify peer tasks
synclock.Lock(task.TaskID, false)
defer synclock.UnLock(task.TaskID, false)
if task.IsHealth() && task.GetLastTriggerTime().Add(s.config.AccessWindow).After(time.Now()) {
return task, nil
}
if task.IsFrozen() {
task.SetStatus(supervisor.TaskStatusRunning)
if task.IsHealth() {
span.SetAttributes(config.AttributeNeedSeedCDN.Bool(false))
span.SetAttributes(config.AttributeTaskStatus.String(task.GetStatus().String()))
span.SetAttributes(config.AttributeLastTriggerTime.String(task.GetLastTriggerTime().String()))
return task
}
task.SetStatus(supervisor.TaskStatusRunning)
if s.cdnManager == nil {
// client back source
span.SetAttributes(config.AttributeClientBackSource.Bool(true))
task.SetClientBackSourceStatusAndLimit(s.config.BackSourceCount)
return task
}
span.SetAttributes(config.AttributeNeedSeedCDN.Bool(true))
go func() {
if cdnPeer, err := s.cdnManager.StartSeedTask(ctx, task); err != nil {
if errors.Cause(err) != cdn.ErrCDNInvokeFail {
task.SetStatus(supervisor.TaskStatusFailed)
}
logger.Errorf("failed to seed task: %v", err)
// fall back to client back source
logger.Errorf("seed task failed: %v", err)
span.AddEvent(config.EventCDNFailBackClientSource, trace.WithAttributes(config.AttributeTriggerCDNError.String(err.Error())))
task.SetClientBackSourceStatusAndLimit(s.config.BackSourceCount)
if ok = s.worker.send(taskSeedFailEvent{task}); !ok {
logger.Error("failed to send taskSeed fail event, eventLoop is shutdown")
logger.Error("send taskSeed fail event failed, eventLoop is shutdown")
}
} else {
if ok = s.worker.send(peerDownloadSuccessEvent{cdnPeer, nil}); !ok {
logger.Error("failed to send taskSeed fail event, eventLoop is shutdown")
logger.Error("send taskSeed success event failed, eventLoop is shutdown")
}
logger.Debugf("===== successfully obtain seeds from cdn, task: %+v ====", task)
}
}()
return task, nil
return task
}
func (s *SchedulerService) HandlePieceResult(ctx context.Context, peer *supervisor.Peer, pieceResult *schedulerRPC.PieceResult) error {
peer.Touch()
if pieceResult.PieceNum == common.ZeroOfPiece {
if pieceResult.PieceInfo != nil && pieceResult.PieceInfo.PieceNum == common.EndOfPiece {
return nil
} else if pieceResult.PieceInfo != nil && pieceResult.PieceInfo.PieceNum == common.ZeroOfPiece {
s.worker.send(startReportPieceResultEvent{ctx, peer})
return nil
} else if pieceResult.Success {
@ -258,6 +298,7 @@ func (s *SchedulerService) HandlePieceResult(ctx context.Context, peer *supervis
return nil
} else if pieceResult.Code != dfcodes.Success {
s.worker.send(peerDownloadPieceFailEvent{
ctx: ctx,
peer: peer,
pr: pieceResult,
})

View File

@ -31,7 +31,6 @@ type worker interface {
type workerGroup struct {
workerNum int
workerList []*baseWorker
stopCh chan struct{}
}
var _ worker = (*workerGroup)(nil)
@ -40,7 +39,6 @@ func newEventLoopGroup(workerNum int) worker {
return &workerGroup{
workerNum: workerNum,
workerList: make([]*baseWorker, 0, workerNum),
stopCh: make(chan struct{}),
}
}
@ -59,7 +57,6 @@ func (wg *workerGroup) send(e event) bool {
}
func (wg *workerGroup) stop() {
close(wg.stopCh)
for _, worker := range wg.workerList {
worker.stop()
}

View File

@ -20,8 +20,6 @@ import (
"context"
"time"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/internal/idgen"
internaljob "d7y.io/dragonfly/v2/internal/job"
@ -158,11 +156,7 @@ func (t *job) preheat(req string) error {
logger.Debugf("ready to preheat \"%s\", taskID = %s", request.URL, taskID)
task := supervisor.NewTask(taskID, request.URL, meta)
task, err := t.service.GetOrCreateTask(t.ctx, task)
if err != nil {
return dferrors.Newf(dfcodes.SchedCDNSeedFail, "create task failed: %v", err)
}
task = t.service.GetOrCreateTask(t.ctx, task)
return getPreheatResult(task)
}
@ -175,7 +169,7 @@ func getPreheatResult(task *supervisor.Task) error {
select {
case <-ticker.C:
switch task.GetStatus() {
case supervisor.TaskStatusFailed, supervisor.TaskStatusCDNRegisterFail, supervisor.TaskStatusSourceError:
case supervisor.TaskStatusFail:
return errors.Errorf("preheat task fail")
case supervisor.TaskStatusSuccess:
return nil

View File

@ -20,13 +20,11 @@ import (
"context"
"fmt"
"io"
"sync"
"d7y.io/dragonfly/v2/internal/dfcodes"
"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"
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
"d7y.io/dragonfly/v2/pkg/rpc/scheduler/server"
"d7y.io/dragonfly/v2/pkg/util/net/urlutils"
@ -36,9 +34,6 @@ import (
"d7y.io/dragonfly/v2/scheduler/supervisor"
"go.opentelemetry.io/otel"
"go.opentelemetry.io/otel/trace"
"golang.org/x/sync/errgroup"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
var tracer trace.Tracer
@ -76,44 +71,29 @@ func (s *SchedulerServer) RegisterPeerTask(ctx context.Context, request *schedul
}
taskID := s.service.GenerateTaskID(request.Url, request.UrlMeta, request.PeerId)
span.SetAttributes(config.AttributeTaskID.String(taskID))
task := supervisor.NewTask(taskID, request.Url, request.UrlMeta)
task, err = s.service.GetOrCreateTask(ctx, task)
if err != nil {
err = dferrors.Newf(dfcodes.SchedCDNSeedFail, "create task failed: %v", err)
logger.Errorf("get or create task failed: %v", err)
span.RecordError(err)
return
}
task := s.service.GetOrCreateTask(ctx, supervisor.NewTask(taskID, request.Url, request.UrlMeta))
if task.IsFail() {
err = dferrors.Newf(dfcodes.SchedTaskStatusError, "task status is %s", task.GetStatus())
logger.Errorf("task status is unhealthy, task status is: %s", task.GetStatus())
err = dferrors.New(dfcodes.SchedTaskStatusError, "task status is fail")
logger.Error("task %s status is fail", task.TaskID)
span.RecordError(err)
return
}
resp.SizeScope = getTaskSizeScope(task)
span.SetAttributes(config.AttributeTaskSizeScope.String(resp.SizeScope.String()))
resp.TaskId = taskID
switch resp.SizeScope {
case base.SizeScope_TINY:
resp.DirectPiece = &scheduler.RegisterResult_PieceContent{
PieceContent: task.DirectPiece,
}
span.SetAttributes(config.AttributeTaskSizeScope.String(resp.SizeScope.String()))
return
case base.SizeScope_SMALL:
span.SetAttributes(config.AttributeTaskSizeScope.String(resp.SizeScope.String()))
peer, regErr := s.service.RegisterPeerTask(request, task)
if regErr != nil {
err = dferrors.Newf(dfcodes.SchedPeerRegisterFail, "failed to register peer: %v", regErr)
span.RecordError(err)
return
}
parent, schErr := s.service.ScheduleParent(peer)
peer := s.service.RegisterPeerTask(request, task)
parent, schErr := s.service.SelectParent(peer)
if schErr != nil {
span.SetAttributes(config.AttributeTaskSizeScope.String(base.SizeScope_NORMAL.String()))
span.AddEvent(config.EventScheduleParentFail)
span.AddEvent(config.EventSmallTaskSelectParentFail)
resp.SizeScope = base.SizeScope_NORMAL
resp.TaskId = taskID
//err = dferrors.Newf(dfcodes.SchedPeerScheduleFail, "failed to schedule peer %v: %v", peer.PeerID, schErr)
return
}
firstPiece := task.GetPiece(0)
@ -135,12 +115,7 @@ func (s *SchedulerServer) RegisterPeerTask(ctx context.Context, request *schedul
span.SetAttributes(config.AttributeSinglePiece.String(singlePiece.String()))
return
default:
span.SetAttributes(config.AttributeTaskSizeScope.String(resp.SizeScope.String()))
_, regErr := s.service.RegisterPeerTask(request, task)
if regErr != nil {
err = dferrors.Newf(dfcodes.SchedPeerRegisterFail, "failed to register peer: %v", regErr)
span.RecordError(regErr)
}
s.service.RegisterPeerTask(request, task)
return
}
}
@ -149,90 +124,52 @@ func (s *SchedulerServer) ReportPieceResult(stream scheduler.Scheduler_ReportPie
var span trace.Span
ctx, span := tracer.Start(stream.Context(), config.SpanReportPieceResult, trace.WithSpanKind(trace.SpanKindServer))
defer span.End()
peerPacketChan := make(chan *scheduler.PeerPacket, 1)
var peer *supervisor.Peer
initialized := false
ctx, cancel := context.WithCancel(ctx)
g, ctx := errgroup.WithContext(ctx)
var once sync.Once
g.Go(func() error {
defer func() {
cancel()
once.Do(peer.UnBindSendChannel)
}()
for {
select {
case <-ctx.Done():
return nil
default:
pieceResult, err := stream.Recv()
if err == io.EOF {
span.AddEvent("report piece process exited because client has terminated sending the request")
return nil
}
if err != nil {
if status.Code(err) == codes.Canceled {
span.AddEvent("report piece process exited because an error exception was received")
if peer != nil {
logger.Info("peer %s canceled", peer.PeerID)
return nil
}
}
return dferrors.Newf(dfcodes.SchedPeerPieceResultReportFail, "peer piece result report error: %v", err)
}
logger.Debugf("report piece result %v of peer %s", pieceResult, pieceResult.SrcPid)
var ok bool
peer, ok = s.service.GetPeerTask(pieceResult.SrcPid)
if !ok {
return dferrors.Newf(dfcodes.SchedPeerNotFound, "peer %s not found", pieceResult.SrcPid)
}
if !initialized {
peer.BindSendChannel(peerPacketChan)
peer.SetStatus(supervisor.PeerStatusRunning)
initialized = true
span.SetAttributes(config.AttributePeerID.String(peer.PeerID))
span.AddEvent("init")
}
if pieceResult.PieceNum == common.EndOfPiece {
return nil
}
if err := s.service.HandlePieceResult(ctx, peer, pieceResult); err != nil {
logger.Errorf("handle piece result %v fail: %v", pieceResult, err)
}
}
pieceResult, err := stream.Recv()
if err != nil {
if err == io.EOF {
return nil
}
})
err = dferrors.Newf(dfcodes.SchedPeerPieceResultReportFail, "receive an error from peer stream: %v", err)
span.RecordError(err)
return err
}
logger.Debugf("peer %s start report piece result", pieceResult.SrcPid)
peer, ok := s.service.GetPeerTask(pieceResult.SrcPid)
if !ok {
err = dferrors.Newf(dfcodes.SchedPeerNotFound, "peer %s not found", pieceResult.SrcPid)
span.RecordError(err)
return err
}
if peer.Task.IsFail() {
err = dferrors.Newf(dfcodes.SchedTaskStatusError, "peer's task status is fail, task status %s", peer.Task.GetStatus())
span.RecordError(err)
return err
}
if err := s.service.HandlePieceResult(ctx, peer, pieceResult); err != nil {
logger.Errorf("peer %s handle piece result %v fail: %v", peer.PeerID, pieceResult, err)
g.Go(func() error {
defer func() {
cancel()
once.Do(peer.UnBindSendChannel)
}()
for {
select {
case <-ctx.Done():
return nil
case pp, ok := <-peerPacketChan:
if !ok {
span.AddEvent("exit report piece process due to send channel has closed")
return nil
}
span.AddEvent("schedule event", trace.WithAttributes(config.AttributeSchedulePacket.String(pp.String())))
err := stream.Send(pp)
if err != nil {
logger.Errorf("send peer %s schedule packet %v failed: %v", pp.SrcPid, pp, err)
return dferrors.Newf(dfcodes.SchedPeerPieceResultReportFail, "peer piece result report error: %v", err)
}
}
conn := peer.BindNewConn(stream)
logger.Infof("peer %s is connected", peer.PeerID)
defer logger.Infof("peer %s is disconnect", peer.PeerID)
for {
select {
case <-conn.Done():
return conn.Err()
case piece := <-conn.Receiver():
if piece == nil {
logger.Infof("peer %s channel has been closed", peer.PeerID)
continue
}
if err := s.service.HandlePieceResult(ctx, peer, piece); err != nil {
logger.Errorf("peer %s handle piece result %v fail: %v", peer.PeerID, piece, err)
}
}
})
err := g.Wait()
logger.Debugf("report piece result: %v", err)
return err
}
}
func (s *SchedulerServer) ReportPeerResult(ctx context.Context, result *scheduler.PeerResult) (err error) {
logger.Debugf("report peer result %+v", result)
logger.Debugf("report peer result %v", result)
var span trace.Span
ctx, span = tracer.Start(ctx, config.SpanReportPeerResult, trace.WithSpanKind(trace.SpanKindServer))
defer span.End()
@ -244,17 +181,18 @@ func (s *SchedulerServer) ReportPeerResult(ctx context.Context, result *schedule
logger.Warnf("report peer result: peer %s is not exists", result.PeerId)
err = dferrors.Newf(dfcodes.SchedPeerNotFound, "peer %s not found", result.PeerId)
span.RecordError(err)
return err
}
return s.service.HandlePeerResult(ctx, peer, result)
}
func (s *SchedulerServer) LeaveTask(ctx context.Context, target *scheduler.PeerTarget) (err error) {
logger.Debugf("leave task %v", target)
var span trace.Span
ctx, span = tracer.Start(ctx, config.SpanPeerLeave, trace.WithSpanKind(trace.SpanKindServer))
defer span.End()
span.SetAttributes(config.AttributeLeavePeerID.String(target.PeerId))
span.SetAttributes(config.AttributeLeaveTaskID.String(target.TaskId))
logger.Debugf("leave task %+v", target)
peer, ok := s.service.GetPeerTask(target.PeerId)
if !ok {
logger.Warnf("leave task: peer %s is not exists", target.PeerId)

View File

@ -89,7 +89,7 @@ func New(cfg *config.Config) (*Server, error) {
if cfg.Options.Telemetry.Jaeger != "" {
openTel = true
}
schedulerService, err := core.NewSchedulerService(cfg.Scheduler, dynConfig, openTel)
schedulerService, err := core.NewSchedulerService(cfg.Scheduler, dynConfig, core.WithDisableCDN(cfg.DisableCDN), core.WithOpenTel(openTel))
if err != nil {
return nil, err
}

View File

@ -71,6 +71,8 @@ func NewManager(cdnClient RefreshableCDNClient, peerManager supervisor.PeerMgr,
}
func (cm *manager) StartSeedTask(ctx context.Context, task *supervisor.Task) (*supervisor.Peer, error) {
logger.Infof("start seed task %s", task.TaskID)
defer logger.Infof("finish seed task %s", task.TaskID)
var seedSpan trace.Span
ctx, seedSpan = tracer.Start(ctx, config.SpanTriggerCDN)
defer seedSpan.End()
@ -161,15 +163,8 @@ func (cm *manager) receivePiece(ctx context.Context, task *supervisor.Task, stre
span.SetAttributes(config.AttributeContentLength.Int64(task.ContentLength))
return cdnPeer, nil
}
cdnPeer.AddPieceInfo(piece.PieceInfo.PieceNum+1, 0)
task.AddPiece(&supervisor.PieceInfo{
PieceNum: piece.PieceInfo.PieceNum,
RangeStart: piece.PieceInfo.RangeStart,
RangeSize: piece.PieceInfo.RangeSize,
PieceMd5: piece.PieceInfo.PieceMd5,
PieceOffset: piece.PieceInfo.PieceOffset,
PieceStyle: piece.PieceInfo.PieceStyle,
})
cdnPeer.UpdateProgress(piece.PieceInfo.PieceNum+1, 0)
task.AddPiece(piece.PieceInfo)
}
}
}

View File

@ -17,10 +17,12 @@
package supervisor
import (
"io"
"sync"
"time"
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
"github.com/pkg/errors"
"go.uber.org/atomic"
)
@ -32,12 +34,12 @@ func (status PeerStatus) String() string {
return "Waiting"
case PeerStatusRunning:
return "Running"
case PeerStatusSuccess:
return "Success"
case PeerStatusFail:
return "Fail"
case PeerStatusZombie:
return "Zombie"
case PeerStatusFail:
return "Fail"
case PeerStatusSuccess:
return "Success"
default:
return "unknown"
}
@ -46,6 +48,7 @@ func (status PeerStatus) String() string {
const (
PeerStatusWaiting PeerStatus = iota
PeerStatusRunning
// TODO add Seeding status
PeerStatusZombie
PeerStatusFail
PeerStatusSuccess
@ -59,10 +62,7 @@ type Peer struct {
Task *Task
// Host specifies
Host *PeerHost
// bindPacketChan
bindPacketChan bool
// PacketChan send schedulerPacket to peer client
packetChan chan *scheduler.PeerPacket
conn *Channel
// createTime
CreateTime time.Time
// finishedNum specifies downloaded finished piece number
@ -155,7 +155,7 @@ func (peer *Peer) GetCost() int {
return totalCost / len(peer.costHistory)
}
func (peer *Peer) AddPieceInfo(finishedCount int32, cost int) {
func (peer *Peer) UpdateProgress(finishedCount int32, cost int) {
peer.lock.Lock()
if finishedCount > peer.finishedNum.Load() {
peer.finishedNum.Store(finishedCount)
@ -254,38 +254,6 @@ func (peer *Peer) GetChildren() *sync.Map {
return &peer.children
}
func (peer *Peer) BindSendChannel(packetChan chan *scheduler.PeerPacket) {
peer.lock.Lock()
defer peer.lock.Unlock()
peer.bindPacketChan = true
peer.packetChan = packetChan
}
func (peer *Peer) UnBindSendChannel() {
peer.lock.Lock()
defer peer.lock.Unlock()
if peer.bindPacketChan {
if peer.packetChan != nil {
close(peer.packetChan)
}
peer.bindPacketChan = false
}
}
func (peer *Peer) IsBindSendChannel() bool {
peer.lock.RLock()
defer peer.lock.RUnlock()
return peer.bindPacketChan
}
func (peer *Peer) SendSchedulePacket(packet *scheduler.PeerPacket) {
peer.lock.Lock()
defer peer.lock.Unlock()
if peer.bindPacketChan {
peer.packetChan <- packet
}
}
func (peer *Peer) SetStatus(status PeerStatus) {
peer.lock.Lock()
defer peer.lock.Unlock()
@ -333,3 +301,163 @@ func (peer *Peer) IsBad() bool {
func (peer *Peer) IsFail() bool {
return peer.status == PeerStatusFail
}
func (peer *Peer) BindNewConn(stream scheduler.Scheduler_ReportPieceResultServer) *Channel {
peer.lock.Lock()
defer peer.lock.Unlock()
if peer.status == PeerStatusWaiting {
peer.status = PeerStatusRunning
}
peer.conn = newChannel(stream)
return peer.conn
}
func (peer *Peer) IsConnected() bool {
peer.lock.RLock()
defer peer.lock.RUnlock()
if peer.conn == nil {
return false
}
return !peer.conn.IsClosed()
}
func (peer *Peer) SendSchedulePacket(packet *scheduler.PeerPacket) error {
peer.lock.Lock()
defer peer.lock.Unlock()
if peer.conn != nil {
return peer.conn.Send(packet)
}
return errors.New("client peer is not connected")
}
func (peer *Peer) CloseChannel(err error) error {
peer.lock.Lock()
defer peer.lock.Unlock()
if peer.conn != nil {
peer.conn.CloseWithError(err)
return nil
}
return errors.New("client peer is not connected")
}
type Channel struct {
startOnce sync.Once
sender chan *scheduler.PeerPacket
receiver chan *scheduler.PieceResult
stream scheduler.Scheduler_ReportPieceResultServer
closed *atomic.Bool
done chan struct{}
wg sync.WaitGroup
err error
}
func newChannel(stream scheduler.Scheduler_ReportPieceResultServer) *Channel {
c := &Channel{
sender: make(chan *scheduler.PeerPacket),
receiver: make(chan *scheduler.PieceResult),
stream: stream,
closed: atomic.NewBool(false),
done: make(chan struct{}),
}
c.start()
return c
}
func (c *Channel) start() {
c.startOnce.Do(func() {
c.wg.Add(2)
go c.receiveLoop()
go c.sendLoop()
})
}
func (c *Channel) Send(packet *scheduler.PeerPacket) error {
select {
case <-c.done:
return errors.New("conn has closed")
case c.sender <- packet:
return nil
default:
return errors.New("send channel is blocking")
}
}
func (c *Channel) Receiver() <-chan *scheduler.PieceResult {
return c.receiver
}
func (c *Channel) Close() {
if !c.closed.CAS(false, true) {
return
}
go func() {
close(c.done)
c.wg.Wait()
}()
}
func (c *Channel) CloseWithError(err error) {
c.err = err
c.Close()
}
func (c *Channel) Err() error {
err := c.err
return err
}
func (c *Channel) Done() <-chan struct{} {
if c.done == nil {
c.done = make(chan struct{})
}
d := c.done
return d
}
func (c *Channel) IsClosed() bool {
return c.closed.Load()
}
func (c *Channel) receiveLoop() {
defer func() {
close(c.receiver)
c.wg.Done()
c.Close()
}()
for {
select {
case <-c.done:
return
default:
pieceResult, err := c.stream.Recv()
if err == io.EOF {
return
}
if err != nil {
c.err = err
return
}
c.receiver <- pieceResult
}
}
}
func (c *Channel) sendLoop() {
defer func() {
c.wg.Done()
c.Close()
}()
for {
select {
case <-c.done:
return
case packet := <-c.sender:
if err := c.stream.Send(packet); err != nil {
c.err = err
return
}
}
}
}

View File

@ -21,7 +21,6 @@ import (
"time"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/pkg/structure/sortedlist"
"d7y.io/dragonfly/v2/scheduler/config"
"d7y.io/dragonfly/v2/scheduler/supervisor"
)
@ -88,69 +87,36 @@ func (m *manager) Delete(peerID string) {
if ok {
peer.Host.DeletePeer(peerID)
peer.Task.DeletePeer(peer)
peer.UnBindSendChannel()
peer.ReplaceParent(nil)
m.peerMap.Delete(peerID)
}
return
}
func (m *manager) Pick(task *supervisor.Task, limit int, pickFn func(peer *supervisor.Peer) bool) (pickedPeers []*supervisor.Peer) {
return m.pick(task, limit, false, pickFn)
}
func (m *manager) PickReverse(task *supervisor.Task, limit int, pickFn func(peer *supervisor.Peer) bool) (pickedPeers []*supervisor.Peer) {
return m.pick(task, limit, true, pickFn)
}
func (m *manager) pick(task *supervisor.Task, limit int, reverse bool, pickFn func(peer *supervisor.Peer) bool) (pickedPeers []*supervisor.Peer) {
if pickFn == nil {
return
}
if !reverse {
task.ListPeers().Range(func(data sortedlist.Item) bool {
if len(pickedPeers) >= limit {
return false
}
peer := data.(*supervisor.Peer)
if pickFn(peer) {
pickedPeers = append(pickedPeers, peer)
}
return true
})
return
}
task.ListPeers().RangeReverse(func(data sortedlist.Item) bool {
if len(pickedPeers) >= limit {
return false
}
peer := data.(*supervisor.Peer)
if pickFn(peer) {
pickedPeers = append(pickedPeers, peer)
}
return true
})
return
}
func (m *manager) cleanupPeers() {
for range m.cleanupExpiredPeerTicker.C {
m.peerMap.Range(func(key, value interface{}) bool {
peerID := key.(string)
peer := value.(*supervisor.Peer)
elapse := time.Since(peer.GetLastAccessTime())
if elapse > m.peerTTI && !peer.IsDone() {
if !peer.IsBindSendChannel() {
if elapse > m.peerTTI && !peer.IsDone() && !peer.Host.CDN {
if !peer.IsConnected() {
peer.MarkLeave()
}
logger.Debugf("peer %s has been more than %s since last access, set status to zombie", peer.PeerID, m.peerTTI)
peer.SetStatus(supervisor.PeerStatusZombie)
}
if peer.IsLeave() || peer.IsFail() || elapse > m.peerTTL {
logger.Debugf("delete peer %s because %s have passed since last access", peer.PeerID)
m.Delete(key.(string))
if elapse > m.peerTTL {
logger.Debugf("delete peer %s because %s have passed since last access", peer.PeerID)
}
m.Delete(peerID)
if peer.Host.GetPeerTaskNum() == 0 {
m.hostManager.Delete(peer.Host.UUID)
}
if peer.Task.ListPeers().Size() == 0 {
peer.Task.SetStatus(supervisor.TaskStatusWaiting)
}
}
return true
})

View File

@ -30,8 +30,4 @@ type PeerMgr interface {
ListPeersByTask(taskID string) []*Peer
ListPeers() *sync.Map
Pick(task *Task, limit int, pickFn func(pt *Peer) bool) []*Peer
PickReverse(task *Task, limit int, pickFn func(peer *Peer) bool) []*Peer
}

View File

@ -22,6 +22,7 @@ import (
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/structure/sortedlist"
"go.uber.org/atomic"
)
type TaskStatus uint8
@ -32,16 +33,14 @@ func (status TaskStatus) String() string {
return "Waiting"
case TaskStatusRunning:
return "Running"
case TaskStatusZombie:
return "Zombie"
case TaskStatusSeeding:
return "Seeding"
case TaskStatusSuccess:
return "Success"
case TaskStatusCDNRegisterFail:
return "cdnRegisterFail"
case TaskStatusFailed:
return "fail"
case TaskStatusSourceError:
return "sourceError"
case TaskStatusZombie:
return "Zombie"
case TaskStatusFail:
return "Fail"
default:
return "unknown"
}
@ -50,39 +49,41 @@ func (status TaskStatus) String() string {
const (
TaskStatusWaiting TaskStatus = iota
TaskStatusRunning
TaskStatusZombie
TaskStatusSeeding
TaskStatusSuccess
TaskStatusCDNRegisterFail
TaskStatusFailed
TaskStatusSourceError
TaskStatusZombie
TaskStatusFail
)
type Task struct {
TaskID string
URL string
URLMeta *base.UrlMeta
DirectPiece []byte
CreateTime time.Time
lastAccessTime time.Time
lastTriggerTime time.Time
lock sync.RWMutex
pieceList map[int32]*PieceInfo
PieceTotal int32
ContentLength int64
status TaskStatus
peers *sortedlist.SortedList
lock sync.RWMutex
TaskID string
URL string
URLMeta *base.UrlMeta
DirectPiece []byte
CreateTime time.Time
lastAccessTime time.Time
lastTriggerTime time.Time
pieceList map[int32]*base.PieceInfo
PieceTotal int32
ContentLength int64
status TaskStatus
peers *sortedlist.SortedList
backSourceLimit atomic.Int32
needClientBackSource atomic.Bool
backSourcePeers []string
// TODO add cdnPeers
}
func NewTask(taskID, url string, meta *base.UrlMeta) *Task {
return &Task{
TaskID: taskID,
URL: url,
URLMeta: meta,
pieceList: make(map[int32]*PieceInfo),
peers: sortedlist.NewSortedList(),
status: TaskStatusWaiting,
TaskID: taskID,
URL: url,
URLMeta: meta,
CreateTime: time.Now(),
pieceList: make(map[int32]*base.PieceInfo),
peers: sortedlist.NewSortedList(),
status: TaskStatusWaiting,
}
}
@ -114,13 +115,31 @@ func (task *Task) GetStatus() TaskStatus {
return task.status
}
func (task *Task) GetPiece(pieceNum int32) *PieceInfo {
func (task *Task) SetClientBackSourceStatusAndLimit(backSourceLimit int32) {
task.lock.Lock()
defer task.lock.Unlock()
task.backSourcePeers = make([]string, 0, backSourceLimit)
task.needClientBackSource.Store(true)
task.backSourceLimit.Store(backSourceLimit)
}
func (task *Task) NeedClientBackSource() bool {
return task.needClientBackSource.Load()
}
func (task *Task) GetPiece(pieceNum int32) *base.PieceInfo {
task.lock.RLock()
defer task.lock.RUnlock()
return task.pieceList[pieceNum]
}
func (task *Task) AddPiece(p *PieceInfo) {
func (task *Task) AddPiece(p *base.PieceInfo) {
task.lock.RLock()
if _, ok := task.pieceList[p.PieceNum]; ok {
task.lock.RUnlock()
return
}
task.lock.RUnlock()
task.lock.Lock()
defer task.lock.Unlock()
task.pieceList[p.PieceNum] = p
@ -130,22 +149,32 @@ func (task *Task) GetLastTriggerTime() time.Time {
return task.lastTriggerTime
}
func (task *Task) UpdateLastTriggerTime(lastTriggerTime time.Time) {
task.lastTriggerTime = lastTriggerTime
}
func (task *Task) Touch() {
task.lock.Lock()
defer task.lock.Unlock()
task.lastAccessTime = time.Now()
}
func (task *Task) UpdateLastTriggerTime(lastTriggerTime time.Time) {
task.lastTriggerTime = lastTriggerTime
}
func (task *Task) GetLastAccessTime() time.Time {
task.lock.RLock()
defer task.lock.RUnlock()
return task.lastAccessTime
}
func (task *Task) UpdateTaskSuccessResult(pieceTotal int32, contentLength int64) {
task.lock.Lock()
defer task.lock.Unlock()
if task.status != TaskStatusSuccess {
task.status = TaskStatusSuccess
task.PieceTotal = pieceTotal
task.ContentLength = contentLength
}
}
func (task *Task) Lock() {
task.lock.Lock()
}
@ -164,25 +193,16 @@ func (task *Task) RUnlock() {
const TinyFileSize = 128
type PieceInfo struct {
PieceNum int32
RangeStart uint64
RangeSize int32
PieceMd5 string
PieceOffset uint64
PieceStyle base.PieceStyle
}
// IsSuccess determines that whether cdn status is success.
func (task *Task) IsSuccess() bool {
return task.status == TaskStatusSuccess
}
// IsFrozen determines that whether cdn status is frozen
func (task *Task) IsFrozen() bool {
return task.status == TaskStatusFailed || task.status == TaskStatusWaiting ||
task.status == TaskStatusSourceError || task.status == TaskStatusCDNRegisterFail
}
//func (task *Task) IsFrozen() bool {
// return task.status == TaskStatusWaiting || task.status == TaskStatusZombie || task.status == TaskStatusFailed ||
// task.status == TaskStatusSourceError || task.status == TaskStatusCDNRegisterFail
//}
// CanSchedule determines whether task can be scheduled
// only task status is seeding or success can be scheduled
@ -197,10 +217,75 @@ func (task *Task) IsWaiting() bool {
// IsHealth determines whether task is health
func (task *Task) IsHealth() bool {
return task.status == TaskStatusRunning || task.status == TaskStatusSuccess || task.status == TaskStatusSeeding
return task.status == TaskStatusRunning || task.status == TaskStatusSeeding || task.status == TaskStatusSuccess
}
// IsFail determines whether task is fail
func (task *Task) IsFail() bool {
return task.status == TaskStatusFailed || task.status == TaskStatusSourceError || task.status == TaskStatusCDNRegisterFail
return task.status == TaskStatusFail
}
func (task *Task) IncreaseBackSourcePeer(peerID string) {
task.lock.Lock()
defer task.lock.Unlock()
task.backSourcePeers = append(task.backSourcePeers, peerID)
if task.backSourceLimit.Dec() <= 0 {
task.needClientBackSource.Store(false)
}
}
func (task *Task) GetBackSourcePeers() []string {
task.lock.RLock()
defer task.lock.RUnlock()
backSourcePeers := task.backSourcePeers
return backSourcePeers
}
func (task *Task) IsBackSourcePeer(peerID string) bool {
task.lock.RLock()
defer task.lock.RUnlock()
for i := range task.backSourcePeers {
if task.backSourcePeers[i] == peerID {
return true
}
}
return false
}
func (task *Task) Pick(limit int, pickFn func(peer *Peer) bool) (pickedPeers []*Peer) {
return task.pick(limit, false, pickFn)
}
func (task *Task) PickReverse(limit int, pickFn func(peer *Peer) bool) (pickedPeers []*Peer) {
return task.pick(limit, true, pickFn)
}
func (task *Task) pick(limit int, reverse bool, pickFn func(peer *Peer) bool) (pickedPeers []*Peer) {
if pickFn == nil {
return
}
if !reverse {
task.ListPeers().Range(func(data sortedlist.Item) bool {
if len(pickedPeers) >= limit {
return false
}
peer := data.(*Peer)
if pickFn(peer) {
pickedPeers = append(pickedPeers, peer)
}
return true
})
return
}
task.ListPeers().RangeReverse(func(data sortedlist.Item) bool {
if len(pickedPeers) >= limit {
return false
}
peer := data.(*Peer)
if pickFn(peer) {
pickedPeers = append(pickedPeers, peer)
}
return true
})
return
}

View File

@ -72,19 +72,22 @@ func (m *manager) GetOrAdd(task *supervisor.Task) (actual *supervisor.Task, load
func (m *manager) cleanupTasks() {
for range m.cleanupExpiredTaskTicker.C {
m.taskMap.Range(func(key, value interface{}) bool {
taskID := key.(string)
task := value.(*supervisor.Task)
elapse := time.Since(task.GetLastAccessTime())
if elapse > m.taskTTI && task.IsSuccess() {
task.SetStatus(supervisor.TaskStatusZombie)
}
if task.ListPeers().Size() == 0 {
task.SetStatus(supervisor.TaskStatusWaiting)
}
if elapse > m.taskTTL {
taskID := key.(string)
// TODO lock
m.Delete(taskID)
peers := m.peerManager.ListPeersByTask(taskID)
for _, peer := range peers {
m.peerManager.Delete(peer.PeerID)
}
m.Delete(taskID)
}
return true
})