feat: client back source (#579)
* feat: client back source Signed-off-by: santong <244372610@qq.com>
This commit is contained in:
parent
3c917604d9
commit
60bee061d2
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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 (
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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{
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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())
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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{
|
||||
|
|
|
|||
|
|
@ -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
|
|
@ -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{
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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() }
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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{
|
||||
|
|
|
|||
|
|
@ -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{
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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"`
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
)
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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 parent,reschedule 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 available,reschedule 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 parent,reschedule it in one second")
|
||||
logger.WithTaskAndPeerID(child.Task.TaskID, child.PeerID).Warnf("handlePeerLeave: there is no available parent,reschedule 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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 time,waiting 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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
})
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
})
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
})
|
||||
|
|
|
|||
Loading…
Reference in New Issue