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.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 {
|
if err := checkPieceTasksRequestParams(req); err != nil {
|
||||||
err = dferrors.Newf(dfcodes.BadRequest, "failed to validate seed request for task(%s): %v", req.TaskId, err)
|
err = dferrors.Newf(dfcodes.BadRequest, "failed to validate seed request for task(%s): %v", req.TaskId, err)
|
||||||
span.RecordError(err)
|
span.RecordError(err)
|
||||||
|
|
|
||||||
|
|
@ -68,6 +68,7 @@ type peerTask struct {
|
||||||
backSourceFunc func()
|
backSourceFunc func()
|
||||||
reportPieceResultFunc func(result *pieceTaskResult) error
|
reportPieceResultFunc func(result *pieceTaskResult) error
|
||||||
setContentLengthFunc func(i int64) error
|
setContentLengthFunc func(i int64) error
|
||||||
|
setTotalPiecesFunc func(i int32)
|
||||||
|
|
||||||
request *scheduler.PeerTaskRequest
|
request *scheduler.PeerTaskRequest
|
||||||
|
|
||||||
|
|
@ -172,6 +173,10 @@ func (pt *peerTask) GetTotalPieces() int32 {
|
||||||
return pt.totalPiece
|
return pt.totalPiece
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (pt *peerTask) SetTotalPieces(i int32) {
|
||||||
|
pt.setTotalPiecesFunc(i)
|
||||||
|
}
|
||||||
|
|
||||||
func (pt *peerTask) Context() context.Context {
|
func (pt *peerTask) Context() context.Context {
|
||||||
return pt.ctx
|
return pt.ctx
|
||||||
}
|
}
|
||||||
|
|
@ -369,6 +374,7 @@ func (pt *peerTask) pullPiecesFromPeers(cleanUnfinishedFunc func()) {
|
||||||
}()
|
}()
|
||||||
|
|
||||||
if !pt.waitFirstPeerPacket() {
|
if !pt.waitFirstPeerPacket() {
|
||||||
|
// TODO 如果是客户端直接回源,这里不应该在输出错误日志
|
||||||
pt.Errorf("wait first peer packet error")
|
pt.Errorf("wait first peer packet error")
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
@ -650,7 +656,7 @@ func (pt *peerTask) downloadPieceWorker(id int32, pti Task, requests chan *Downl
|
||||||
TaskId: pt.GetTaskID(),
|
TaskId: pt.GetTaskID(),
|
||||||
SrcPid: pt.GetPeerID(),
|
SrcPid: pt.GetPeerID(),
|
||||||
DstPid: request.DstPid,
|
DstPid: request.DstPid,
|
||||||
PieceNum: request.piece.PieceNum,
|
PieceInfo: request.piece,
|
||||||
Success: false,
|
Success: false,
|
||||||
Code: dfcodes.ClientRequestLimitFail,
|
Code: dfcodes.ClientRequestLimitFail,
|
||||||
HostLoad: nil,
|
HostLoad: nil,
|
||||||
|
|
@ -761,6 +767,7 @@ retry:
|
||||||
TaskId: pt.taskID,
|
TaskId: pt.taskID,
|
||||||
SrcPid: pt.peerID,
|
SrcPid: pt.peerID,
|
||||||
DstPid: peer.PeerId,
|
DstPid: peer.PeerId,
|
||||||
|
PieceInfo: &base.PieceInfo{},
|
||||||
Success: false,
|
Success: false,
|
||||||
Code: code,
|
Code: code,
|
||||||
HostLoad: nil,
|
HostLoad: nil,
|
||||||
|
|
@ -805,6 +812,7 @@ func (pt *peerTask) getPieceTasks(span trace.Span, curPeerPacket *scheduler.Peer
|
||||||
TaskId: pt.taskID,
|
TaskId: pt.taskID,
|
||||||
SrcPid: pt.peerID,
|
SrcPid: pt.peerID,
|
||||||
DstPid: peer.PeerId,
|
DstPid: peer.PeerId,
|
||||||
|
PieceInfo: &base.PieceInfo{},
|
||||||
Success: false,
|
Success: false,
|
||||||
Code: dfcodes.ClientWaitPieceReady,
|
Code: dfcodes.ClientWaitPieceReady,
|
||||||
HostLoad: nil,
|
HostLoad: nil,
|
||||||
|
|
|
||||||
|
|
@ -208,6 +208,7 @@ func newFilePeerTask(ctx context.Context,
|
||||||
// bind func that base peer task did not implement
|
// bind func that base peer task did not implement
|
||||||
pt.backSourceFunc = pt.backSource
|
pt.backSourceFunc = pt.backSource
|
||||||
pt.setContentLengthFunc = pt.SetContentLength
|
pt.setContentLengthFunc = pt.SetContentLength
|
||||||
|
pt.setTotalPiecesFunc = pt.SetTotalPieces
|
||||||
pt.reportPieceResultFunc = pt.ReportPieceResult
|
pt.reportPieceResultFunc = pt.ReportPieceResult
|
||||||
return ctx, pt, nil, nil
|
return ctx, pt, nil, nil
|
||||||
}
|
}
|
||||||
|
|
@ -237,19 +238,19 @@ func (pt *filePeerTask) ReportPieceResult(result *pieceTaskResult) error {
|
||||||
if !result.pieceResult.Success {
|
if !result.pieceResult.Success {
|
||||||
result.pieceResult.FinishedCount = pt.readyPieces.Settled()
|
result.pieceResult.FinishedCount = pt.readyPieces.Settled()
|
||||||
_ = pt.peerPacketStream.Send(result.pieceResult)
|
_ = 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)
|
pt.Errorf("%d download failed, retry later", result.piece.PieceNum)
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
pt.lock.Lock()
|
pt.lock.Lock()
|
||||||
if pt.readyPieces.IsSet(result.pieceResult.PieceNum) {
|
if pt.readyPieces.IsSet(result.pieceResult.PieceInfo.PieceNum) {
|
||||||
pt.lock.Unlock()
|
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
|
return nil
|
||||||
}
|
}
|
||||||
// mark piece processed
|
// mark piece processed
|
||||||
pt.readyPieces.Set(result.pieceResult.PieceNum)
|
pt.readyPieces.Set(result.pieceResult.PieceInfo.PieceNum)
|
||||||
pt.completedLength.Add(int64(result.piece.RangeSize))
|
pt.completedLength.Add(int64(result.piece.RangeSize))
|
||||||
pt.lock.Unlock()
|
pt.lock.Unlock()
|
||||||
|
|
||||||
|
|
@ -420,6 +421,7 @@ func (pt *filePeerTask) cleanUnfinished() {
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// TODO SetContentLength 需要和pt.finish解绑,以便在下载进度处可以看到文件长度
|
||||||
func (pt *filePeerTask) SetContentLength(i int64) error {
|
func (pt *filePeerTask) SetContentLength(i int64) error {
|
||||||
pt.contentLength.Store(i)
|
pt.contentLength.Store(i)
|
||||||
if !pt.isCompleted() {
|
if !pt.isCompleted() {
|
||||||
|
|
@ -429,6 +431,10 @@ func (pt *filePeerTask) SetContentLength(i int64) error {
|
||||||
return pt.finish()
|
return pt.finish()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (pt *filePeerTask) SetTotalPieces(i int32) {
|
||||||
|
pt.totalPiece = i
|
||||||
|
}
|
||||||
|
|
||||||
func (pt *filePeerTask) backSource() {
|
func (pt *filePeerTask) backSource() {
|
||||||
defer pt.cleanUnfinished()
|
defer pt.cleanUnfinished()
|
||||||
if pt.disableBackSource {
|
if pt.disableBackSource {
|
||||||
|
|
@ -436,6 +442,13 @@ func (pt *filePeerTask) backSource() {
|
||||||
pt.failedReason = reasonBackSourceDisabled
|
pt.failedReason = reasonBackSourceDisabled
|
||||||
return
|
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)
|
err := pt.pieceManager.DownloadSource(pt.ctx, pt, pt.request)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
pt.Errorf("download from source error: %s", err)
|
pt.Errorf("download from source error: %s", err)
|
||||||
|
|
@ -444,5 +457,4 @@ func (pt *filePeerTask) backSource() {
|
||||||
}
|
}
|
||||||
pt.Infof("download from source ok")
|
pt.Infof("download from source ok")
|
||||||
_ = pt.finish()
|
_ = pt.finish()
|
||||||
return
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -17,7 +17,6 @@
|
||||||
package peer
|
package peer
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"d7y.io/dragonfly/v2/client/daemon/storage"
|
"d7y.io/dragonfly/v2/client/daemon/storage"
|
||||||
|
|
@ -92,7 +91,7 @@ func (p *filePeerTaskCallback) Done(pt Task) error {
|
||||||
return e
|
return e
|
||||||
}
|
}
|
||||||
p.ptm.PeerTaskDone(p.req.PeerId)
|
p.ptm.PeerTaskDone(p.req.PeerId)
|
||||||
err := p.pt.schedulerClient.ReportPeerResult(context.Background(), &scheduler.PeerResult{
|
err := p.pt.schedulerClient.ReportPeerResult(p.pt.ctx, &scheduler.PeerResult{
|
||||||
TaskId: pt.GetTaskID(),
|
TaskId: pt.GetTaskID(),
|
||||||
PeerId: pt.GetPeerID(),
|
PeerId: pt.GetPeerID(),
|
||||||
SrcIp: p.ptm.host.Ip,
|
SrcIp: p.ptm.host.Ip,
|
||||||
|
|
@ -101,6 +100,7 @@ func (p *filePeerTaskCallback) Done(pt Task) error {
|
||||||
Url: p.req.Url,
|
Url: p.req.Url,
|
||||||
ContentLength: pt.GetContentLength(),
|
ContentLength: pt.GetContentLength(),
|
||||||
Traffic: pt.GetTraffic(),
|
Traffic: pt.GetTraffic(),
|
||||||
|
TotalPieceCount: pt.GetTotalPieces(),
|
||||||
Cost: uint32(cost),
|
Cost: uint32(cost),
|
||||||
Success: true,
|
Success: true,
|
||||||
Code: dfcodes.Success,
|
Code: dfcodes.Success,
|
||||||
|
|
@ -117,7 +117,7 @@ func (p *filePeerTaskCallback) Fail(pt Task, code base.Code, reason string) erro
|
||||||
p.ptm.PeerTaskDone(p.req.PeerId)
|
p.ptm.PeerTaskDone(p.req.PeerId)
|
||||||
var end = time.Now()
|
var end = time.Now()
|
||||||
pt.Log().Errorf("file peer task failed, code: %d, reason: %s", code, reason)
|
pt.Log().Errorf("file peer task failed, code: %d, reason: %s", code, reason)
|
||||||
err := p.pt.schedulerClient.ReportPeerResult(context.Background(), &scheduler.PeerResult{
|
err := p.pt.schedulerClient.ReportPeerResult(p.pt.ctx, &scheduler.PeerResult{
|
||||||
TaskId: pt.GetTaskID(),
|
TaskId: pt.GetTaskID(),
|
||||||
PeerId: pt.GetPeerID(),
|
PeerId: pt.GetPeerID(),
|
||||||
SrcIp: p.ptm.host.Ip,
|
SrcIp: p.ptm.host.Ip,
|
||||||
|
|
@ -126,6 +126,7 @@ func (p *filePeerTaskCallback) Fail(pt Task, code base.Code, reason string) erro
|
||||||
Url: p.req.Url,
|
Url: p.req.Url,
|
||||||
ContentLength: pt.GetContentLength(),
|
ContentLength: pt.GetContentLength(),
|
||||||
Traffic: pt.GetTraffic(),
|
Traffic: pt.GetTraffic(),
|
||||||
|
TotalPieceCount: p.pt.totalPiece,
|
||||||
Cost: uint32(end.Sub(p.start).Milliseconds()),
|
Cost: uint32(end.Sub(p.start).Milliseconds()),
|
||||||
Success: false,
|
Success: false,
|
||||||
Code: code,
|
Code: code,
|
||||||
|
|
|
||||||
|
|
@ -66,6 +66,7 @@ type Task interface {
|
||||||
GetPeerID() string
|
GetPeerID() string
|
||||||
GetTaskID() string
|
GetTaskID() string
|
||||||
GetTotalPieces() int32
|
GetTotalPieces() int32
|
||||||
|
SetTotalPieces(int32)
|
||||||
GetContentLength() int64
|
GetContentLength() int64
|
||||||
// SetContentLength will be called after download completed, when download from source without content length
|
// SetContentLength will be called after download completed, when download from source without content length
|
||||||
SetContentLength(int64) error
|
SetContentLength(int64) error
|
||||||
|
|
|
||||||
|
|
@ -1,5 +1,5 @@
|
||||||
// Code generated by MockGen. DO NOT EDIT.
|
// 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 is a generated GoMock package.
|
||||||
package peer
|
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)
|
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.
|
// MockTaskCallback is a mock of TaskCallback interface.
|
||||||
type MockTaskCallback struct {
|
type MockTaskCallback struct {
|
||||||
ctrl *gomock.Controller
|
ctrl *gomock.Controller
|
||||||
|
|
|
||||||
|
|
@ -184,6 +184,7 @@ func newStreamPeerTask(ctx context.Context,
|
||||||
// bind func that base peer task did not implement
|
// bind func that base peer task did not implement
|
||||||
pt.backSourceFunc = pt.backSource
|
pt.backSourceFunc = pt.backSource
|
||||||
pt.setContentLengthFunc = pt.SetContentLength
|
pt.setContentLengthFunc = pt.SetContentLength
|
||||||
|
pt.setTotalPiecesFunc = pt.SetTotalPieces
|
||||||
pt.reportPieceResultFunc = pt.ReportPieceResult
|
pt.reportPieceResultFunc = pt.ReportPieceResult
|
||||||
return ctx, pt, nil, nil
|
return ctx, pt, nil, nil
|
||||||
}
|
}
|
||||||
|
|
@ -193,18 +194,18 @@ func (s *streamPeerTask) ReportPieceResult(result *pieceTaskResult) error {
|
||||||
// retry failed piece
|
// retry failed piece
|
||||||
if !result.pieceResult.Success {
|
if !result.pieceResult.Success {
|
||||||
_ = s.peerPacketStream.Send(result.pieceResult)
|
_ = s.peerPacketStream.Send(result.pieceResult)
|
||||||
s.failedPieceCh <- result.pieceResult.PieceNum
|
s.failedPieceCh <- result.pieceResult.PieceInfo.PieceNum
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
s.lock.Lock()
|
s.lock.Lock()
|
||||||
if s.readyPieces.IsSet(result.pieceResult.PieceNum) {
|
if s.readyPieces.IsSet(result.pieceResult.PieceInfo.PieceNum) {
|
||||||
s.lock.Unlock()
|
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
|
return nil
|
||||||
}
|
}
|
||||||
// mark piece processed
|
// mark piece processed
|
||||||
s.readyPieces.Set(result.pieceResult.PieceNum)
|
s.readyPieces.Set(result.pieceResult.PieceInfo.PieceNum)
|
||||||
s.completedLength.Add(int64(result.piece.RangeSize))
|
s.completedLength.Add(int64(result.piece.RangeSize))
|
||||||
s.lock.Unlock()
|
s.lock.Unlock()
|
||||||
|
|
||||||
|
|
@ -416,6 +417,10 @@ func (s *streamPeerTask) SetContentLength(i int64) error {
|
||||||
return s.finish()
|
return s.finish()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (s *streamPeerTask) SetTotalPieces(i int32) {
|
||||||
|
s.totalPiece = i
|
||||||
|
}
|
||||||
|
|
||||||
func (s *streamPeerTask) writeTo(w io.Writer, pieceNum int32) (int64, error) {
|
func (s *streamPeerTask) writeTo(w io.Writer, pieceNum int32) (int64, error) {
|
||||||
pr, pc, err := s.pieceManager.ReadPiece(s.ctx, &storage.ReadPieceRequest{
|
pr, pc, err := s.pieceManager.ReadPiece(s.ctx, &storage.ReadPieceRequest{
|
||||||
PeerTaskMetaData: storage.PeerTaskMetaData{
|
PeerTaskMetaData: storage.PeerTaskMetaData{
|
||||||
|
|
@ -440,6 +445,12 @@ func (s *streamPeerTask) writeTo(w io.Writer, pieceNum int32) (int64, error) {
|
||||||
func (s *streamPeerTask) backSource() {
|
func (s *streamPeerTask) backSource() {
|
||||||
s.contentLength.Store(-1)
|
s.contentLength.Store(-1)
|
||||||
_ = s.callback.Init(s)
|
_ = 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)
|
err := s.pieceManager.DownloadSource(s.ctx, s, s.request)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
s.Errorf("download from source error: %s", err)
|
s.Errorf("download from source error: %s", err)
|
||||||
|
|
|
||||||
|
|
@ -17,7 +17,6 @@
|
||||||
package peer
|
package peer
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"d7y.io/dragonfly/v2/client/daemon/storage"
|
"d7y.io/dragonfly/v2/client/daemon/storage"
|
||||||
|
|
@ -90,7 +89,7 @@ func (p *streamPeerTaskCallback) Done(pt Task) error {
|
||||||
return e
|
return e
|
||||||
}
|
}
|
||||||
p.ptm.PeerTaskDone(p.req.PeerId)
|
p.ptm.PeerTaskDone(p.req.PeerId)
|
||||||
err := p.pt.schedulerClient.ReportPeerResult(context.Background(), &scheduler.PeerResult{
|
err := p.pt.schedulerClient.ReportPeerResult(p.pt.ctx, &scheduler.PeerResult{
|
||||||
TaskId: pt.GetTaskID(),
|
TaskId: pt.GetTaskID(),
|
||||||
PeerId: pt.GetPeerID(),
|
PeerId: pt.GetPeerID(),
|
||||||
SrcIp: p.ptm.host.Ip,
|
SrcIp: p.ptm.host.Ip,
|
||||||
|
|
@ -99,6 +98,7 @@ func (p *streamPeerTaskCallback) Done(pt Task) error {
|
||||||
Url: p.req.Url,
|
Url: p.req.Url,
|
||||||
ContentLength: pt.GetContentLength(),
|
ContentLength: pt.GetContentLength(),
|
||||||
Traffic: pt.GetTraffic(),
|
Traffic: pt.GetTraffic(),
|
||||||
|
TotalPieceCount: p.pt.totalPiece,
|
||||||
Cost: uint32(cost),
|
Cost: uint32(cost),
|
||||||
Success: true,
|
Success: true,
|
||||||
Code: dfcodes.Success,
|
Code: dfcodes.Success,
|
||||||
|
|
@ -115,7 +115,7 @@ func (p *streamPeerTaskCallback) Fail(pt Task, code base.Code, reason string) er
|
||||||
p.ptm.PeerTaskDone(p.req.PeerId)
|
p.ptm.PeerTaskDone(p.req.PeerId)
|
||||||
var end = time.Now()
|
var end = time.Now()
|
||||||
pt.Log().Errorf("stream peer task failed, code: %d, reason: %s", code, reason)
|
pt.Log().Errorf("stream peer task failed, code: %d, reason: %s", code, reason)
|
||||||
err := p.pt.schedulerClient.ReportPeerResult(context.Background(), &scheduler.PeerResult{
|
err := p.pt.schedulerClient.ReportPeerResult(p.pt.ctx, &scheduler.PeerResult{
|
||||||
TaskId: pt.GetTaskID(),
|
TaskId: pt.GetTaskID(),
|
||||||
PeerId: pt.GetPeerID(),
|
PeerId: pt.GetPeerID(),
|
||||||
SrcIp: p.ptm.host.Ip,
|
SrcIp: p.ptm.host.Ip,
|
||||||
|
|
@ -124,6 +124,7 @@ func (p *streamPeerTaskCallback) Fail(pt Task, code base.Code, reason string) er
|
||||||
Url: p.req.Url,
|
Url: p.req.Url,
|
||||||
ContentLength: pt.GetContentLength(),
|
ContentLength: pt.GetContentLength(),
|
||||||
Traffic: pt.GetTraffic(),
|
Traffic: pt.GetTraffic(),
|
||||||
|
TotalPieceCount: p.pt.totalPiece,
|
||||||
Cost: uint32(end.Sub(p.start).Milliseconds()),
|
Cost: uint32(end.Sub(p.start).Milliseconds()),
|
||||||
Success: false,
|
Success: false,
|
||||||
Code: code,
|
Code: code,
|
||||||
|
|
|
||||||
|
|
@ -173,13 +173,14 @@ func (pm *pieceManager) pushSuccessResult(peerTask Task, dstPid string, piece *b
|
||||||
TaskId: peerTask.GetTaskID(),
|
TaskId: peerTask.GetTaskID(),
|
||||||
SrcPid: peerTask.GetPeerID(),
|
SrcPid: peerTask.GetPeerID(),
|
||||||
DstPid: dstPid,
|
DstPid: dstPid,
|
||||||
PieceNum: piece.PieceNum,
|
PieceInfo: piece,
|
||||||
BeginTime: uint64(start),
|
BeginTime: uint64(start),
|
||||||
EndTime: uint64(end),
|
EndTime: uint64(end),
|
||||||
Success: true,
|
Success: true,
|
||||||
Code: dfcodes.Success,
|
Code: dfcodes.Success,
|
||||||
HostLoad: nil, // TODO(jim): update host load
|
HostLoad: nil, // TODO(jim): update host load
|
||||||
FinishedCount: 0, // update by peer task
|
FinishedCount: piece.PieceNum + 1, // update by peer task
|
||||||
|
// TODO range_start, range_size, piece_md5, piece_offset, piece_style
|
||||||
},
|
},
|
||||||
err: nil,
|
err: nil,
|
||||||
})
|
})
|
||||||
|
|
@ -196,7 +197,7 @@ func (pm *pieceManager) pushFailResult(peerTask Task, dstPid string, piece *base
|
||||||
TaskId: peerTask.GetTaskID(),
|
TaskId: peerTask.GetTaskID(),
|
||||||
SrcPid: peerTask.GetPeerID(),
|
SrcPid: peerTask.GetPeerID(),
|
||||||
DstPid: dstPid,
|
DstPid: dstPid,
|
||||||
PieceNum: piece.PieceNum,
|
PieceInfo: piece,
|
||||||
BeginTime: uint64(start),
|
BeginTime: uint64(start),
|
||||||
EndTime: uint64(end),
|
EndTime: uint64(end),
|
||||||
Success: false,
|
Success: false,
|
||||||
|
|
@ -371,6 +372,7 @@ func (pm *pieceManager) DownloadSource(ctx context.Context, pt Task, request *sc
|
||||||
},
|
},
|
||||||
ContentLength: contentLength,
|
ContentLength: contentLength,
|
||||||
})
|
})
|
||||||
|
pt.SetTotalPieces(pieceNum + 1)
|
||||||
return pt.SetContentLength(contentLength)
|
return pt.SetContentLength(contentLength)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -398,6 +400,8 @@ func (pm *pieceManager) DownloadSource(ctx context.Context, pt Task, request *sc
|
||||||
return storage.ErrShortRead
|
return storage.ErrShortRead
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
pt.SetTotalPieces(maxPieceNum)
|
||||||
|
pt.SetContentLength(contentLength)
|
||||||
log.Infof("download from source ok")
|
log.Infof("download from source ok")
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -130,6 +130,9 @@ func TestPieceManager_DownloadSource(t *testing.T) {
|
||||||
func(arg0 int64) error {
|
func(arg0 int64) error {
|
||||||
return nil
|
return nil
|
||||||
})
|
})
|
||||||
|
mockPeerTask.EXPECT().SetTotalPieces(gomock.Any()).AnyTimes().DoAndReturn(
|
||||||
|
func(arg0 int32) {
|
||||||
|
})
|
||||||
mockPeerTask.EXPECT().GetPeerID().AnyTimes().DoAndReturn(
|
mockPeerTask.EXPECT().GetPeerID().AnyTimes().DoAndReturn(
|
||||||
func() string {
|
func() string {
|
||||||
return peerID
|
return peerID
|
||||||
|
|
|
||||||
|
|
@ -37,8 +37,7 @@ var (
|
||||||
var rootCmd = &cobra.Command{
|
var rootCmd = &cobra.Command{
|
||||||
Use: "scheduler",
|
Use: "scheduler",
|
||||||
Short: "the scheduler of dragonfly",
|
Short: "the scheduler of dragonfly",
|
||||||
Long: `scheduler is a long-running process and is mainly responsible
|
Long: `scheduler is a long-running process and is mainly responsible for deciding which peers transmit blocks to each other.`,
|
||||||
for deciding which peers transmit blocks to each other.`,
|
|
||||||
Args: cobra.NoArgs,
|
Args: cobra.NoArgs,
|
||||||
DisableAutoGenTag: true,
|
DisableAutoGenTag: true,
|
||||||
SilenceUsage: true,
|
SilenceUsage: true,
|
||||||
|
|
|
||||||
|
|
@ -25,7 +25,7 @@ const (
|
||||||
ServerUnavailable base.Code = 500 // framework can not find server node
|
ServerUnavailable base.Code = 500 // framework can not find server node
|
||||||
|
|
||||||
// common response error 1000-1999
|
// 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
|
BadRequest base.Code = 1400
|
||||||
PeerTaskNotFound base.Code = 1404
|
PeerTaskNotFound base.Code = 1404
|
||||||
UnknownError base.Code = 1500
|
UnknownError base.Code = 1500
|
||||||
|
|
@ -42,15 +42,11 @@ const (
|
||||||
|
|
||||||
// scheduler response error 5000-5999
|
// scheduler response error 5000-5999
|
||||||
SchedError base.Code = 5000
|
SchedError base.Code = 5000
|
||||||
/** @deprecated */
|
|
||||||
SchedNeedBackSource base.Code = 5001 // client should try to download from source
|
SchedNeedBackSource base.Code = 5001 // client should try to download from source
|
||||||
SchedPeerGone base.Code = 5002 // client should disconnect from scheduler
|
SchedPeerGone base.Code = 5002 // client should disconnect from scheduler
|
||||||
SchedPeerRegisterFail base.Code = 5003
|
SchedPeerNotFound base.Code = 5004 // peer not found in scheduler
|
||||||
SchedPeerScheduleFail base.Code = 5004
|
SchedPeerPieceResultReportFail base.Code = 5005 // report piece
|
||||||
SchedPeerNotFound base.Code = 5005
|
SchedTaskStatusError base.Code = 5006 // task status is fail
|
||||||
SchedPeerPieceResultReportFail base.Code = 5006
|
|
||||||
SchedCDNSeedFail base.Code = 5007
|
|
||||||
SchedTaskStatusError base.Code = 5008
|
|
||||||
|
|
||||||
// cdnsystem response error 6000-6999
|
// cdnsystem response error 6000-6999
|
||||||
CdnError base.Code = 6000
|
CdnError base.Code = 6000
|
||||||
|
|
|
||||||
|
|
@ -30,7 +30,7 @@ var (
|
||||||
ErrEmptyValue = errors.New("empty value")
|
ErrEmptyValue = errors.New("empty value")
|
||||||
ErrConvertFailed = errors.New("convert failed")
|
ErrConvertFailed = errors.New("convert failed")
|
||||||
ErrEndOfStream = errors.New("end of stream")
|
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 {
|
func IsEndOfStream(err error) bool {
|
||||||
|
|
|
||||||
|
|
@ -15,13 +15,14 @@
|
||||||
|
|
||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.26.0
|
// protoc-gen-go v1.25.0
|
||||||
// protoc v3.15.8
|
// protoc v3.17.3
|
||||||
// source: pkg/rpc/base/base.proto
|
// source: pkg/rpc/base/base.proto
|
||||||
|
|
||||||
package base
|
package base
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
proto "github.com/golang/protobuf/proto"
|
||||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||||
reflect "reflect"
|
reflect "reflect"
|
||||||
|
|
@ -35,6 +36,10 @@ const (
|
||||||
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
|
_ = 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
|
type Code int32
|
||||||
|
|
||||||
const (
|
const (
|
||||||
|
|
|
||||||
|
|
@ -34,20 +34,53 @@ var (
|
||||||
)
|
)
|
||||||
|
|
||||||
// Validate checks the field values on GrpcDfError with the rules defined in
|
// 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
|
// the proto definition for this message. If any rules are violated, the first
|
||||||
// is returned.
|
// error encountered is returned, or nil if there are no violations.
|
||||||
func (m *GrpcDfError) Validate() error {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for Code
|
// no validation rules for Code
|
||||||
|
|
||||||
// no validation rules for Message
|
// no validation rules for Message
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return GrpcDfErrorMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// GrpcDfErrorValidationError is the validation error returned by
|
||||||
// GrpcDfError.Validate if the designated constraints aren't met.
|
// GrpcDfError.Validate if the designated constraints aren't met.
|
||||||
type GrpcDfErrorValidationError struct {
|
type GrpcDfErrorValidationError struct {
|
||||||
|
|
@ -103,12 +136,26 @@ var _ interface {
|
||||||
} = GrpcDfErrorValidationError{}
|
} = GrpcDfErrorValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on UrlMeta with the rules defined in the
|
// 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 {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for Digest
|
// no validation rules for Digest
|
||||||
|
|
||||||
// no validation rules for Tag
|
// no validation rules for Tag
|
||||||
|
|
@ -119,9 +166,28 @@ func (m *UrlMeta) Validate() error {
|
||||||
|
|
||||||
// no validation rules for Header
|
// no validation rules for Header
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return UrlMetaMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// UrlMetaValidationError is the validation error returned by UrlMeta.Validate
|
||||||
// if the designated constraints aren't met.
|
// if the designated constraints aren't met.
|
||||||
type UrlMetaValidationError struct {
|
type UrlMetaValidationError struct {
|
||||||
|
|
@ -177,21 +243,55 @@ var _ interface {
|
||||||
} = UrlMetaValidationError{}
|
} = UrlMetaValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on HostLoad with the rules defined in the
|
// Validate checks the field values on HostLoad with the rules defined in the
|
||||||
// proto definition for this message. If any rules are violated, 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 {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for CpuRatio
|
// no validation rules for CpuRatio
|
||||||
|
|
||||||
// no validation rules for MemRatio
|
// no validation rules for MemRatio
|
||||||
|
|
||||||
// no validation rules for DiskRatio
|
// no validation rules for DiskRatio
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return HostLoadMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// HostLoadValidationError is the validation error returned by
|
||||||
// HostLoad.Validate if the designated constraints aren't met.
|
// HostLoad.Validate if the designated constraints aren't met.
|
||||||
type HostLoadValidationError struct {
|
type HostLoadValidationError struct {
|
||||||
|
|
@ -247,13 +347,27 @@ var _ interface {
|
||||||
} = HostLoadValidationError{}
|
} = HostLoadValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on PieceTaskRequest with the rules defined
|
// Validate checks the field values on PieceTaskRequest with the rules defined
|
||||||
// in the proto definition for this message. If any rules are violated, an
|
// in the proto definition for this message. If any rules are violated, the
|
||||||
// error is returned.
|
// first error encountered is returned, or nil if there are no violations.
|
||||||
func (m *PieceTaskRequest) Validate() error {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for TaskId
|
// no validation rules for TaskId
|
||||||
|
|
||||||
// no validation rules for SrcPid
|
// no validation rules for SrcPid
|
||||||
|
|
@ -264,9 +378,29 @@ func (m *PieceTaskRequest) Validate() error {
|
||||||
|
|
||||||
// no validation rules for Limit
|
// no validation rules for Limit
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return PieceTaskRequestMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// PieceTaskRequestValidationError is the validation error returned by
|
||||||
// PieceTaskRequest.Validate if the designated constraints aren't met.
|
// PieceTaskRequest.Validate if the designated constraints aren't met.
|
||||||
type PieceTaskRequestValidationError struct {
|
type PieceTaskRequestValidationError struct {
|
||||||
|
|
@ -322,12 +456,27 @@ var _ interface {
|
||||||
} = PieceTaskRequestValidationError{}
|
} = PieceTaskRequestValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on PieceInfo with the rules defined in the
|
// 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 {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for PieceNum
|
// no validation rules for PieceNum
|
||||||
|
|
||||||
// no validation rules for RangeStart
|
// no validation rules for RangeStart
|
||||||
|
|
@ -340,9 +489,28 @@ func (m *PieceInfo) Validate() error {
|
||||||
|
|
||||||
// no validation rules for PieceStyle
|
// no validation rules for PieceStyle
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return PieceInfoMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// PieceInfoValidationError is the validation error returned by
|
||||||
// PieceInfo.Validate if the designated constraints aren't met.
|
// PieceInfo.Validate if the designated constraints aren't met.
|
||||||
type PieceInfoValidationError struct {
|
type PieceInfoValidationError struct {
|
||||||
|
|
@ -398,13 +566,27 @@ var _ interface {
|
||||||
} = PieceInfoValidationError{}
|
} = PieceInfoValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on PiecePacket with the rules defined in
|
// Validate checks the field values on PiecePacket with the rules defined in
|
||||||
// the proto definition for this message. If any rules are violated, an error
|
// the proto definition for this message. If any rules are violated, the first
|
||||||
// is returned.
|
// error encountered is returned, or nil if there are no violations.
|
||||||
func (m *PiecePacket) Validate() error {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for TaskId
|
// no validation rules for TaskId
|
||||||
|
|
||||||
// no validation rules for DstPid
|
// no validation rules for DstPid
|
||||||
|
|
@ -414,7 +596,26 @@ func (m *PiecePacket) Validate() error {
|
||||||
for idx, item := range m.GetPieceInfos() {
|
for idx, item := range m.GetPieceInfos() {
|
||||||
_, _ = idx, item
|
_, _ = 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 {
|
if err := v.Validate(); err != nil {
|
||||||
return PiecePacketValidationError{
|
return PiecePacketValidationError{
|
||||||
field: fmt.Sprintf("PieceInfos[%v]", idx),
|
field: fmt.Sprintf("PieceInfos[%v]", idx),
|
||||||
|
|
@ -432,9 +633,28 @@ func (m *PiecePacket) Validate() error {
|
||||||
|
|
||||||
// no validation rules for PieceMd5Sign
|
// no validation rules for PieceMd5Sign
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return PiecePacketMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// PiecePacketValidationError is the validation error returned by
|
||||||
// PiecePacket.Validate if the designated constraints aren't met.
|
// PiecePacket.Validate if the designated constraints aren't met.
|
||||||
type PiecePacketValidationError struct {
|
type PiecePacketValidationError struct {
|
||||||
|
|
|
||||||
|
|
@ -15,14 +15,15 @@
|
||||||
|
|
||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.26.0
|
// protoc-gen-go v1.25.0
|
||||||
// protoc v3.15.8
|
// protoc v3.17.3
|
||||||
// source: pkg/rpc/cdnsystem/cdnsystem.proto
|
// source: pkg/rpc/cdnsystem/cdnsystem.proto
|
||||||
|
|
||||||
package cdnsystem
|
package cdnsystem
|
||||||
|
|
||||||
import (
|
import (
|
||||||
base "d7y.io/dragonfly/v2/pkg/rpc/base"
|
base "d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||||
|
proto "github.com/golang/protobuf/proto"
|
||||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||||
reflect "reflect"
|
reflect "reflect"
|
||||||
|
|
@ -36,6 +37,10 @@ const (
|
||||||
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
|
_ = 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 {
|
type SeedRequest struct {
|
||||||
state protoimpl.MessageState
|
state protoimpl.MessageState
|
||||||
sizeCache protoimpl.SizeCache
|
sizeCache protoimpl.SizeCache
|
||||||
|
|
|
||||||
|
|
@ -34,18 +34,51 @@ var (
|
||||||
)
|
)
|
||||||
|
|
||||||
// Validate checks the field values on SeedRequest with the rules defined in
|
// Validate checks the field values on SeedRequest with the rules defined in
|
||||||
// the proto definition for this message. If any rules are violated, an error
|
// the proto definition for this message. If any rules are violated, the first
|
||||||
// is returned.
|
// error encountered is returned, or nil if there are no violations.
|
||||||
func (m *SeedRequest) Validate() error {
|
func (m *SeedRequest) Validate() error {
|
||||||
|
return m.validate(false)
|
||||||
|
}
|
||||||
|
|
||||||
|
// ValidateAll checks the field values on SeedRequest with the rules defined in
|
||||||
|
// the proto definition for this message. If any rules are violated, the
|
||||||
|
// result is a list of violation errors wrapped in SeedRequestMultiError, or
|
||||||
|
// nil if none found.
|
||||||
|
func (m *SeedRequest) ValidateAll() error {
|
||||||
|
return m.validate(true)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *SeedRequest) validate(all bool) error {
|
||||||
if m == nil {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for TaskId
|
// no validation rules for TaskId
|
||||||
|
|
||||||
// no validation rules for Url
|
// 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 {
|
if err := v.Validate(); err != nil {
|
||||||
return SeedRequestValidationError{
|
return SeedRequestValidationError{
|
||||||
field: "UrlMeta",
|
field: "UrlMeta",
|
||||||
|
|
@ -55,9 +88,28 @@ func (m *SeedRequest) Validate() error {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return SeedRequestMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SeedRequestMultiError is an error wrapping multiple validation errors
|
||||||
|
// returned by SeedRequest.ValidateAll() if the designated constraints aren't met.
|
||||||
|
type SeedRequestMultiError []error
|
||||||
|
|
||||||
|
// Error returns a concatenation of all the error messages it wraps.
|
||||||
|
func (m SeedRequestMultiError) Error() string {
|
||||||
|
var msgs []string
|
||||||
|
for _, err := range m {
|
||||||
|
msgs = append(msgs, err.Error())
|
||||||
|
}
|
||||||
|
return strings.Join(msgs, "; ")
|
||||||
|
}
|
||||||
|
|
||||||
|
// AllErrors returns a list of validation violation errors.
|
||||||
|
func (m SeedRequestMultiError) AllErrors() []error { return m }
|
||||||
|
|
||||||
// SeedRequestValidationError is the validation error returned by
|
// SeedRequestValidationError is the validation error returned by
|
||||||
// SeedRequest.Validate if the designated constraints aren't met.
|
// SeedRequest.Validate if the designated constraints aren't met.
|
||||||
type SeedRequestValidationError struct {
|
type SeedRequestValidationError struct {
|
||||||
|
|
@ -113,17 +165,51 @@ var _ interface {
|
||||||
} = SeedRequestValidationError{}
|
} = SeedRequestValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on PieceSeed with the rules defined in the
|
// Validate checks the field values on PieceSeed with the rules defined in the
|
||||||
// proto definition for this message. If any rules are violated, an error is returned.
|
// proto definition for this message. If any rules are violated, the first
|
||||||
|
// error encountered is returned, or nil if there are no violations.
|
||||||
func (m *PieceSeed) Validate() error {
|
func (m *PieceSeed) Validate() error {
|
||||||
|
return m.validate(false)
|
||||||
|
}
|
||||||
|
|
||||||
|
// ValidateAll checks the field values on PieceSeed with the rules defined in
|
||||||
|
// the proto definition for this message. If any rules are violated, the
|
||||||
|
// result is a list of violation errors wrapped in PieceSeedMultiError, or nil
|
||||||
|
// if none found.
|
||||||
|
func (m *PieceSeed) ValidateAll() error {
|
||||||
|
return m.validate(true)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *PieceSeed) validate(all bool) error {
|
||||||
if m == nil {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for PeerId
|
// no validation rules for PeerId
|
||||||
|
|
||||||
// no validation rules for HostUuid
|
// 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 {
|
if err := v.Validate(); err != nil {
|
||||||
return PieceSeedValidationError{
|
return PieceSeedValidationError{
|
||||||
field: "PieceInfo",
|
field: "PieceInfo",
|
||||||
|
|
@ -139,9 +225,28 @@ func (m *PieceSeed) Validate() error {
|
||||||
|
|
||||||
// no validation rules for TotalPieceCount
|
// no validation rules for TotalPieceCount
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return PieceSeedMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// PieceSeedMultiError is an error wrapping multiple validation errors returned
|
||||||
|
// by PieceSeed.ValidateAll() if the designated constraints aren't met.
|
||||||
|
type PieceSeedMultiError []error
|
||||||
|
|
||||||
|
// Error returns a concatenation of all the error messages it wraps.
|
||||||
|
func (m PieceSeedMultiError) Error() string {
|
||||||
|
var msgs []string
|
||||||
|
for _, err := range m {
|
||||||
|
msgs = append(msgs, err.Error())
|
||||||
|
}
|
||||||
|
return strings.Join(msgs, "; ")
|
||||||
|
}
|
||||||
|
|
||||||
|
// AllErrors returns a list of validation violation errors.
|
||||||
|
func (m PieceSeedMultiError) AllErrors() []error { return m }
|
||||||
|
|
||||||
// PieceSeedValidationError is the validation error returned by
|
// PieceSeedValidationError is the validation error returned by
|
||||||
// PieceSeed.Validate if the designated constraints aren't met.
|
// PieceSeed.Validate if the designated constraints aren't met.
|
||||||
type PieceSeedValidationError struct {
|
type PieceSeedValidationError struct {
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,6 @@ import (
|
||||||
|
|
||||||
// This is a compile-time assertion to ensure that this generated file
|
// This is a compile-time assertion to ensure that this generated file
|
||||||
// is compatible with the grpc package it is being compiled against.
|
// is compatible with the grpc package it is being compiled against.
|
||||||
// Requires gRPC-Go v1.32.0 or later.
|
|
||||||
const _ = grpc.SupportPackageIsVersion7
|
const _ = grpc.SupportPackageIsVersion7
|
||||||
|
|
||||||
// SeederClient is the client API for Seeder service.
|
// SeederClient is the client API for Seeder service.
|
||||||
|
|
@ -34,7 +33,7 @@ func NewSeederClient(cc grpc.ClientConnInterface) SeederClient {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *seederClient) ObtainSeeds(ctx context.Context, in *SeedRequest, opts ...grpc.CallOption) (Seeder_ObtainSeedsClient, error) {
|
func (c *seederClient) ObtainSeeds(ctx context.Context, in *SeedRequest, opts ...grpc.CallOption) (Seeder_ObtainSeedsClient, error) {
|
||||||
stream, err := c.cc.NewStream(ctx, &Seeder_ServiceDesc.Streams[0], "/cdnsystem.Seeder/ObtainSeeds", opts...)
|
stream, err := c.cc.NewStream(ctx, &_Seeder_serviceDesc.Streams[0], "/cdnsystem.Seeder/ObtainSeeds", opts...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
@ -105,7 +104,7 @@ type UnsafeSeederServer interface {
|
||||||
}
|
}
|
||||||
|
|
||||||
func RegisterSeederServer(s grpc.ServiceRegistrar, srv SeederServer) {
|
func RegisterSeederServer(s grpc.ServiceRegistrar, srv SeederServer) {
|
||||||
s.RegisterService(&Seeder_ServiceDesc, srv)
|
s.RegisterService(&_Seeder_serviceDesc, srv)
|
||||||
}
|
}
|
||||||
|
|
||||||
func _Seeder_ObtainSeeds_Handler(srv interface{}, stream grpc.ServerStream) error {
|
func _Seeder_ObtainSeeds_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||||
|
|
@ -147,10 +146,7 @@ func _Seeder_GetPieceTasks_Handler(srv interface{}, ctx context.Context, dec fun
|
||||||
return interceptor(ctx, in, info, handler)
|
return interceptor(ctx, in, info, handler)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Seeder_ServiceDesc is the grpc.ServiceDesc for Seeder service.
|
var _Seeder_serviceDesc = grpc.ServiceDesc{
|
||||||
// It's only intended for direct use with grpc.RegisterService,
|
|
||||||
// and not to be introspected or modified (even as a copy)
|
|
||||||
var Seeder_ServiceDesc = grpc.ServiceDesc{
|
|
||||||
ServiceName: "cdnsystem.Seeder",
|
ServiceName: "cdnsystem.Seeder",
|
||||||
HandlerType: (*SeederServer)(nil),
|
HandlerType: (*SeederServer)(nil),
|
||||||
Methods: []grpc.MethodDesc{
|
Methods: []grpc.MethodDesc{
|
||||||
|
|
|
||||||
|
|
@ -18,11 +18,12 @@ package client
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"io"
|
|
||||||
|
|
||||||
|
"d7y.io/dragonfly/v2/internal/dferrors"
|
||||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||||
"d7y.io/dragonfly/v2/pkg/rpc"
|
"d7y.io/dragonfly/v2/pkg/rpc"
|
||||||
"d7y.io/dragonfly/v2/pkg/rpc/cdnsystem"
|
"d7y.io/dragonfly/v2/pkg/rpc/cdnsystem"
|
||||||
|
"github.com/pkg/errors"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/codes"
|
"google.golang.org/grpc/codes"
|
||||||
"google.golang.org/grpc/status"
|
"google.golang.org/grpc/status"
|
||||||
|
|
@ -50,9 +51,9 @@ func newPieceSeedStream(ctx context.Context, sc *cdnClient, hashKey string, sr *
|
||||||
sr: sr,
|
sr: sr,
|
||||||
opts: opts,
|
opts: opts,
|
||||||
RetryMeta: rpc.RetryMeta{
|
RetryMeta: rpc.RetryMeta{
|
||||||
MaxAttempts: 5,
|
MaxAttempts: 3,
|
||||||
InitBackoff: 0.5,
|
InitBackoff: 0.2,
|
||||||
MaxBackOff: 4.0,
|
MaxBackOff: 2.0,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -74,7 +75,10 @@ func (pss *PieceSeedStream) initStream() error {
|
||||||
return client.ObtainSeeds(pss.ctx, pss.sr, pss.opts...)
|
return client.ObtainSeeds(pss.ctx, pss.sr, pss.opts...)
|
||||||
}, pss.InitBackoff, pss.MaxBackOff, pss.MaxAttempts, nil)
|
}, pss.InitBackoff, pss.MaxBackOff, pss.MaxAttempts, nil)
|
||||||
if err != 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)
|
return pss.replaceClient(pss.hashKey, err)
|
||||||
}
|
}
|
||||||
pss.stream = stream.(cdnsystem.Seeder_ObtainSeedsClient)
|
pss.stream = stream.(cdnsystem.Seeder_ObtainSeedsClient)
|
||||||
|
|
@ -84,16 +88,14 @@ func (pss *PieceSeedStream) initStream() error {
|
||||||
|
|
||||||
func (pss *PieceSeedStream) Recv() (ps *cdnsystem.PieceSeed, err error) {
|
func (pss *PieceSeedStream) Recv() (ps *cdnsystem.PieceSeed, err error) {
|
||||||
pss.sc.UpdateAccessNodeMapByHashKey(pss.hashKey)
|
pss.sc.UpdateAccessNodeMapByHashKey(pss.hashKey)
|
||||||
if ps, err = pss.stream.Recv(); err != nil && err != io.EOF {
|
return pss.stream.Recv()
|
||||||
ps, err = pss.retryRecv(err)
|
|
||||||
}
|
|
||||||
return
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pss *PieceSeedStream) retryRecv(cause error) (*cdnsystem.PieceSeed, error) {
|
func (pss *PieceSeedStream) retryRecv(cause error) (*cdnsystem.PieceSeed, error) {
|
||||||
if status.Code(cause) == codes.DeadlineExceeded || status.Code(cause) == codes.Canceled {
|
if status.Code(cause) == codes.DeadlineExceeded || status.Code(cause) == codes.Canceled {
|
||||||
return nil, cause
|
return nil, cause
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := pss.replaceStream(cause); err != nil {
|
if err := pss.replaceStream(cause); err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -180,14 +180,15 @@ func (conn *Connection) UpdateAccessNodeMapByHashKey(key string) {
|
||||||
node, ok := conn.key2NodeMap.Load(key)
|
node, ok := conn.key2NodeMap.Load(key)
|
||||||
if ok {
|
if ok {
|
||||||
conn.accessNodeMap.Store(node, time.Now())
|
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)
|
_, ok := conn.node2ClientMap.Load(node)
|
||||||
if !ok {
|
if ok {
|
||||||
logger.With("conn", conn.name).Warnf("successfully update server node %s access time for hashKey %s,"+
|
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)
|
"but cannot found client conn in node2ClientMap", node, key)
|
||||||
}
|
}
|
||||||
} else {
|
} 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 {
|
for _, addr := range addrs {
|
||||||
serverNode := addr.GetEndpoint()
|
serverNode := addr.GetEndpoint()
|
||||||
conn.hashRing = conn.hashRing.AddNode(serverNode)
|
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
|
return nil
|
||||||
}
|
}
|
||||||
|
|
@ -222,7 +223,7 @@ func (conn *Connection) findCandidateClientConn(key string, exclusiveNodes sets.
|
||||||
|
|
||||||
ringNodes, ok := conn.hashRing.GetNodes(key, conn.hashRing.Size())
|
ringNodes, ok := conn.hashRing.GetNodes(key, conn.hashRing.Size())
|
||||||
if !ok {
|
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 {
|
if len(ringNodes) == 0 {
|
||||||
return nil, dferrors.ErrNoCandidateNode
|
return nil, dferrors.ErrNoCandidateNode
|
||||||
|
|
@ -233,29 +234,30 @@ func (conn *Connection) findCandidateClientConn(key string, exclusiveNodes sets.
|
||||||
candidateNodes = append(candidateNodes, ringNode)
|
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)
|
key, ringNodes, exclusiveNodes.List(), candidateNodes)
|
||||||
for _, candidateNode := range candidateNodes {
|
for _, candidateNode := range candidateNodes {
|
||||||
// Check whether there is a corresponding mapping client in the node2ClientMap
|
// Check whether there is a corresponding mapping client in the node2ClientMap
|
||||||
// TODO 下面部分可以直接调用loadOrCreate方法,但是日志没有这么调用打印全
|
// TODO 下面部分可以直接调用loadOrCreate方法,但是日志没有这么调用打印全
|
||||||
if client, ok := conn.node2ClientMap.Load(candidateNode); ok {
|
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{
|
return &candidateClient{
|
||||||
node: candidateNode,
|
node: candidateNode,
|
||||||
Ref: client,
|
Ref: client,
|
||||||
}, nil
|
}, 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...)...)
|
clientConn, err := conn.createClient(candidateNode, append(defaultClientOpts, conn.dialOpts...)...)
|
||||||
if err == nil {
|
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{
|
return &candidateClient{
|
||||||
node: candidateNode,
|
node: candidateNode,
|
||||||
Ref: clientConn,
|
Ref: clientConn,
|
||||||
}, nil
|
}, 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
|
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) {
|
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()
|
conn.rwMutex.RLock()
|
||||||
defer conn.rwMutex.RUnlock()
|
defer conn.rwMutex.RUnlock()
|
||||||
clientConn, err := conn.loadOrCreateClientConnByNode(node)
|
clientConn, err := conn.loadOrCreateClientConnByNode(node)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, errors.Wrapf(err, "get client conn by conn %s", node)
|
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
|
return clientConn, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -305,14 +307,14 @@ func (conn *Connection) loadOrCreateClientConnByNode(node string) (clientConn *g
|
||||||
conn.accessNodeMap.Store(node, time.Now())
|
conn.accessNodeMap.Store(node, time.Now())
|
||||||
client, ok := conn.node2ClientMap.Load(node)
|
client, ok := conn.node2ClientMap.Load(node)
|
||||||
if ok {
|
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
|
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...)...)
|
clientConn, err = conn.createClient(node, append(defaultClientOpts, conn.dialOpts...)...)
|
||||||
if err == nil {
|
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
|
// bind
|
||||||
conn.node2ClientMap.Store(node, clientConn)
|
conn.node2ClientMap.Store(node, clientConn)
|
||||||
return clientConn, nil
|
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
|
// 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
|
// stick whether hash key need already associated with specify node
|
||||||
func (conn *Connection) GetClientConn(hashKey string, stick bool) (*grpc.ClientConn, error) {
|
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)
|
logger.GrpcLogger.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)
|
defer logger.GrpcLogger.With("conn", conn.name).Debugf("get client conn done, hashKey %s, stick %t end", hashKey, stick)
|
||||||
conn.rwMutex.RLock()
|
conn.rwMutex.RLock()
|
||||||
node, ok := conn.key2NodeMap.Load(hashKey)
|
node, ok := conn.key2NodeMap.Load(hashKey)
|
||||||
if stick && !ok {
|
if stick && !ok {
|
||||||
|
|
@ -343,7 +345,7 @@ func (conn *Connection) GetClientConn(hashKey string, stick bool) (*grpc.ClientC
|
||||||
}
|
}
|
||||||
return clientConn, nil
|
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()
|
conn.rwMutex.RUnlock()
|
||||||
// if absence
|
// if absence
|
||||||
conn.rwMutex.Lock()
|
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
|
// TryMigrate migrate key to another hash node other than exclusiveNodes
|
||||||
// preNode node before the migration
|
// preNode node before the migration
|
||||||
func (conn *Connection) TryMigrate(key string, cause error, exclusiveNodes []string) (preNode string, err error) {
|
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 {
|
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
|
return "", cause
|
||||||
}
|
}
|
||||||
// TODO recover findCandidateClientConn error
|
// TODO recover findCandidateClientConn error
|
||||||
if e, ok := cause.(*dferrors.DfError); ok {
|
if e, ok := cause.(*dferrors.DfError); ok {
|
||||||
if e.Code != dfcodes.ResourceLacked && e.Code != dfcodes.UnknownError {
|
if e.Code != dfcodes.ResourceLacked {
|
||||||
return "", cause
|
return "", cause
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -378,8 +380,6 @@ func (conn *Connection) TryMigrate(key string, cause error, exclusiveNodes []str
|
||||||
currentNode = node.(string)
|
currentNode = node.(string)
|
||||||
preNode = currentNode
|
preNode = currentNode
|
||||||
exclusiveNodes = append(exclusiveNodes, preNode)
|
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.RUnlock()
|
||||||
conn.rwMutex.Lock()
|
conn.rwMutex.Lock()
|
||||||
|
|
@ -388,7 +388,7 @@ func (conn *Connection) TryMigrate(key string, cause error, exclusiveNodes []str
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return "", errors.Wrapf(err, "find candidate client conn for hash key %s", key)
|
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.key2NodeMap.Store(key, client.node)
|
||||||
conn.node2ClientMap.Store(client.node, client.Ref)
|
conn.node2ClientMap.Store(client.node, client.Ref)
|
||||||
conn.accessNodeMap.Store(client.node, time.Now())
|
conn.accessNodeMap.Store(client.node, time.Now())
|
||||||
|
|
|
||||||
|
|
@ -25,7 +25,6 @@ import (
|
||||||
"google.golang.org/grpc/codes"
|
"google.golang.org/grpc/codes"
|
||||||
"google.golang.org/grpc/status"
|
"google.golang.org/grpc/status"
|
||||||
|
|
||||||
"d7y.io/dragonfly/v2/internal/dfcodes"
|
|
||||||
"d7y.io/dragonfly/v2/internal/dferrors"
|
"d7y.io/dragonfly/v2/internal/dferrors"
|
||||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||||
|
|
@ -186,11 +185,9 @@ type RetryMeta struct {
|
||||||
func ExecuteWithRetry(f func() (interface{}, error), initBackoff float64, maxBackoff float64, maxAttempts int, cause error) (interface{}, error) {
|
func ExecuteWithRetry(f func() (interface{}, error), initBackoff float64, maxBackoff float64, maxAttempts int, cause error) (interface{}, error) {
|
||||||
var res interface{}
|
var res interface{}
|
||||||
for i := 0; i < maxAttempts; i++ {
|
for i := 0; i < maxAttempts; i++ {
|
||||||
if e, ok := cause.(*dferrors.DfError); ok {
|
if _, ok := cause.(*dferrors.DfError); ok {
|
||||||
if e.Code != dfcodes.UnknownError {
|
|
||||||
return res, cause
|
return res, cause
|
||||||
}
|
}
|
||||||
}
|
|
||||||
if status.Code(cause) == codes.DeadlineExceeded || status.Code(cause) == codes.Canceled {
|
if status.Code(cause) == codes.DeadlineExceeded || status.Code(cause) == codes.Canceled {
|
||||||
return res, cause
|
return res, cause
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -18,11 +18,12 @@ package client
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"io"
|
|
||||||
|
|
||||||
|
"d7y.io/dragonfly/v2/internal/dferrors"
|
||||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||||
"d7y.io/dragonfly/v2/pkg/rpc"
|
"d7y.io/dragonfly/v2/pkg/rpc"
|
||||||
"d7y.io/dragonfly/v2/pkg/rpc/dfdaemon"
|
"d7y.io/dragonfly/v2/pkg/rpc/dfdaemon"
|
||||||
|
"github.com/pkg/errors"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/codes"
|
"google.golang.org/grpc/codes"
|
||||||
"google.golang.org/grpc/status"
|
"google.golang.org/grpc/status"
|
||||||
|
|
@ -49,9 +50,9 @@ func newDownResultStream(ctx context.Context, dc *daemonClient, hashKey string,
|
||||||
opts: opts,
|
opts: opts,
|
||||||
|
|
||||||
RetryMeta: rpc.RetryMeta{
|
RetryMeta: rpc.RetryMeta{
|
||||||
MaxAttempts: 5,
|
MaxAttempts: 3,
|
||||||
MaxBackOff: 5.0,
|
MaxBackOff: 2.0,
|
||||||
InitBackoff: 1.0,
|
InitBackoff: 0.2,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -73,6 +74,9 @@ func (drs *DownResultStream) initStream() error {
|
||||||
return client.Download(drs.ctx, drs.req, drs.opts...)
|
return client.Download(drs.ctx, drs.req, drs.opts...)
|
||||||
}, drs.InitBackoff, drs.MaxBackOff, drs.MaxAttempts, nil)
|
}, drs.InitBackoff, drs.MaxBackOff, drs.MaxAttempts, nil)
|
||||||
if err != 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)
|
logger.WithTaskID(drs.hashKey).Infof("initStream: invoke daemon node %s Download failed: %v", target, err)
|
||||||
return drs.replaceClient(err)
|
return drs.replaceClient(err)
|
||||||
}
|
}
|
||||||
|
|
@ -92,11 +96,7 @@ func (drs *DownResultStream) Recv() (dr *dfdaemon.DownResult, err error) {
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
drs.dc.UpdateAccessNodeMapByHashKey(drs.hashKey)
|
drs.dc.UpdateAccessNodeMapByHashKey(drs.hashKey)
|
||||||
if dr, err = drs.stream.Recv(); err != nil && err != io.EOF {
|
return drs.stream.Recv()
|
||||||
dr, err = drs.retryRecv(err)
|
|
||||||
}
|
|
||||||
|
|
||||||
return
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (drs *DownResultStream) retryRecv(cause error) (*dfdaemon.DownResult, error) {
|
func (drs *DownResultStream) retryRecv(cause error) (*dfdaemon.DownResult, error) {
|
||||||
|
|
|
||||||
|
|
@ -15,14 +15,15 @@
|
||||||
|
|
||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.26.0
|
// protoc-gen-go v1.25.0
|
||||||
// protoc v3.15.8
|
// protoc v3.17.3
|
||||||
// source: pkg/rpc/dfdaemon/dfdaemon.proto
|
// source: pkg/rpc/dfdaemon/dfdaemon.proto
|
||||||
|
|
||||||
package dfdaemon
|
package dfdaemon
|
||||||
|
|
||||||
import (
|
import (
|
||||||
base "d7y.io/dragonfly/v2/pkg/rpc/base"
|
base "d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||||
|
proto "github.com/golang/protobuf/proto"
|
||||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||||
emptypb "google.golang.org/protobuf/types/known/emptypb"
|
emptypb "google.golang.org/protobuf/types/known/emptypb"
|
||||||
|
|
@ -37,6 +38,10 @@ const (
|
||||||
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
|
_ = 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 {
|
type DownRequest struct {
|
||||||
state protoimpl.MessageState
|
state protoimpl.MessageState
|
||||||
sizeCache protoimpl.SizeCache
|
sizeCache protoimpl.SizeCache
|
||||||
|
|
|
||||||
|
|
@ -34,13 +34,27 @@ var (
|
||||||
)
|
)
|
||||||
|
|
||||||
// Validate checks the field values on DownRequest with the rules defined in
|
// Validate checks the field values on DownRequest with the rules defined in
|
||||||
// the proto definition for this message. If any rules are violated, an error
|
// the proto definition for this message. If any rules are violated, the first
|
||||||
// is returned.
|
// error encountered is returned, or nil if there are no violations.
|
||||||
func (m *DownRequest) Validate() error {
|
func (m *DownRequest) Validate() error {
|
||||||
|
return m.validate(false)
|
||||||
|
}
|
||||||
|
|
||||||
|
// ValidateAll checks the field values on DownRequest with the rules defined in
|
||||||
|
// the proto definition for this message. If any rules are violated, the
|
||||||
|
// result is a list of violation errors wrapped in DownRequestMultiError, or
|
||||||
|
// nil if none found.
|
||||||
|
func (m *DownRequest) ValidateAll() error {
|
||||||
|
return m.validate(true)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *DownRequest) validate(all bool) error {
|
||||||
if m == nil {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for Uuid
|
// no validation rules for Uuid
|
||||||
|
|
||||||
// no validation rules for Url
|
// no validation rules for Url
|
||||||
|
|
@ -53,7 +67,26 @@ func (m *DownRequest) Validate() error {
|
||||||
|
|
||||||
// no validation rules for DisableBackSource
|
// no validation rules for DisableBackSource
|
||||||
|
|
||||||
if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok {
|
if all {
|
||||||
|
switch v := interface{}(m.GetUrlMeta()).(type) {
|
||||||
|
case interface{ ValidateAll() error }:
|
||||||
|
if err := v.ValidateAll(); err != nil {
|
||||||
|
errors = append(errors, DownRequestValidationError{
|
||||||
|
field: "UrlMeta",
|
||||||
|
reason: "embedded message failed validation",
|
||||||
|
cause: err,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
case interface{ Validate() error }:
|
||||||
|
if err := v.Validate(); err != nil {
|
||||||
|
errors = append(errors, DownRequestValidationError{
|
||||||
|
field: "UrlMeta",
|
||||||
|
reason: "embedded message failed validation",
|
||||||
|
cause: err,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok {
|
||||||
if err := v.Validate(); err != nil {
|
if err := v.Validate(); err != nil {
|
||||||
return DownRequestValidationError{
|
return DownRequestValidationError{
|
||||||
field: "UrlMeta",
|
field: "UrlMeta",
|
||||||
|
|
@ -71,9 +104,28 @@ func (m *DownRequest) Validate() error {
|
||||||
|
|
||||||
// no validation rules for Gid
|
// no validation rules for Gid
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return DownRequestMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// DownRequestMultiError is an error wrapping multiple validation errors
|
||||||
|
// returned by DownRequest.ValidateAll() if the designated constraints aren't met.
|
||||||
|
type DownRequestMultiError []error
|
||||||
|
|
||||||
|
// Error returns a concatenation of all the error messages it wraps.
|
||||||
|
func (m DownRequestMultiError) Error() string {
|
||||||
|
var msgs []string
|
||||||
|
for _, err := range m {
|
||||||
|
msgs = append(msgs, err.Error())
|
||||||
|
}
|
||||||
|
return strings.Join(msgs, "; ")
|
||||||
|
}
|
||||||
|
|
||||||
|
// AllErrors returns a list of validation violation errors.
|
||||||
|
func (m DownRequestMultiError) AllErrors() []error { return m }
|
||||||
|
|
||||||
// DownRequestValidationError is the validation error returned by
|
// DownRequestValidationError is the validation error returned by
|
||||||
// DownRequest.Validate if the designated constraints aren't met.
|
// DownRequest.Validate if the designated constraints aren't met.
|
||||||
type DownRequestValidationError struct {
|
type DownRequestValidationError struct {
|
||||||
|
|
@ -129,12 +181,27 @@ var _ interface {
|
||||||
} = DownRequestValidationError{}
|
} = DownRequestValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on DownResult with the rules defined in the
|
// Validate checks the field values on DownResult with the rules defined in the
|
||||||
// proto definition for this message. If any rules are violated, an error is returned.
|
// proto definition for this message. If any rules are violated, the first
|
||||||
|
// error encountered is returned, or nil if there are no violations.
|
||||||
func (m *DownResult) Validate() error {
|
func (m *DownResult) Validate() error {
|
||||||
|
return m.validate(false)
|
||||||
|
}
|
||||||
|
|
||||||
|
// ValidateAll checks the field values on DownResult with the rules defined in
|
||||||
|
// the proto definition for this message. If any rules are violated, the
|
||||||
|
// result is a list of violation errors wrapped in DownResultMultiError, or
|
||||||
|
// nil if none found.
|
||||||
|
func (m *DownResult) ValidateAll() error {
|
||||||
|
return m.validate(true)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *DownResult) validate(all bool) error {
|
||||||
if m == nil {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for TaskId
|
// no validation rules for TaskId
|
||||||
|
|
||||||
// no validation rules for PeerId
|
// no validation rules for PeerId
|
||||||
|
|
@ -143,9 +210,28 @@ func (m *DownResult) Validate() error {
|
||||||
|
|
||||||
// no validation rules for Done
|
// no validation rules for Done
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return DownResultMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// DownResultMultiError is an error wrapping multiple validation errors
|
||||||
|
// returned by DownResult.ValidateAll() if the designated constraints aren't met.
|
||||||
|
type DownResultMultiError []error
|
||||||
|
|
||||||
|
// Error returns a concatenation of all the error messages it wraps.
|
||||||
|
func (m DownResultMultiError) Error() string {
|
||||||
|
var msgs []string
|
||||||
|
for _, err := range m {
|
||||||
|
msgs = append(msgs, err.Error())
|
||||||
|
}
|
||||||
|
return strings.Join(msgs, "; ")
|
||||||
|
}
|
||||||
|
|
||||||
|
// AllErrors returns a list of validation violation errors.
|
||||||
|
func (m DownResultMultiError) AllErrors() []error { return m }
|
||||||
|
|
||||||
// DownResultValidationError is the validation error returned by
|
// DownResultValidationError is the validation error returned by
|
||||||
// DownResult.Validate if the designated constraints aren't met.
|
// DownResult.Validate if the designated constraints aren't met.
|
||||||
type DownResultValidationError struct {
|
type DownResultValidationError struct {
|
||||||
|
|
|
||||||
|
|
@ -13,7 +13,6 @@ import (
|
||||||
|
|
||||||
// This is a compile-time assertion to ensure that this generated file
|
// This is a compile-time assertion to ensure that this generated file
|
||||||
// is compatible with the grpc package it is being compiled against.
|
// is compatible with the grpc package it is being compiled against.
|
||||||
// Requires gRPC-Go v1.32.0 or later.
|
|
||||||
const _ = grpc.SupportPackageIsVersion7
|
const _ = grpc.SupportPackageIsVersion7
|
||||||
|
|
||||||
// DaemonClient is the client API for Daemon service.
|
// DaemonClient is the client API for Daemon service.
|
||||||
|
|
@ -37,7 +36,7 @@ func NewDaemonClient(cc grpc.ClientConnInterface) DaemonClient {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *daemonClient) Download(ctx context.Context, in *DownRequest, opts ...grpc.CallOption) (Daemon_DownloadClient, error) {
|
func (c *daemonClient) Download(ctx context.Context, in *DownRequest, opts ...grpc.CallOption) (Daemon_DownloadClient, error) {
|
||||||
stream, err := c.cc.NewStream(ctx, &Daemon_ServiceDesc.Streams[0], "/dfdaemon.Daemon/Download", opts...)
|
stream, err := c.cc.NewStream(ctx, &_Daemon_serviceDesc.Streams[0], "/dfdaemon.Daemon/Download", opts...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
@ -122,7 +121,7 @@ type UnsafeDaemonServer interface {
|
||||||
}
|
}
|
||||||
|
|
||||||
func RegisterDaemonServer(s grpc.ServiceRegistrar, srv DaemonServer) {
|
func RegisterDaemonServer(s grpc.ServiceRegistrar, srv DaemonServer) {
|
||||||
s.RegisterService(&Daemon_ServiceDesc, srv)
|
s.RegisterService(&_Daemon_serviceDesc, srv)
|
||||||
}
|
}
|
||||||
|
|
||||||
func _Daemon_Download_Handler(srv interface{}, stream grpc.ServerStream) error {
|
func _Daemon_Download_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||||
|
|
@ -182,10 +181,7 @@ func _Daemon_CheckHealth_Handler(srv interface{}, ctx context.Context, dec func(
|
||||||
return interceptor(ctx, in, info, handler)
|
return interceptor(ctx, in, info, handler)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Daemon_ServiceDesc is the grpc.ServiceDesc for Daemon service.
|
var _Daemon_serviceDesc = grpc.ServiceDesc{
|
||||||
// It's only intended for direct use with grpc.RegisterService,
|
|
||||||
// and not to be introspected or modified (even as a copy)
|
|
||||||
var Daemon_ServiceDesc = grpc.ServiceDesc{
|
|
||||||
ServiceName: "dfdaemon.Daemon",
|
ServiceName: "dfdaemon.Daemon",
|
||||||
HandlerType: (*DaemonServer)(nil),
|
HandlerType: (*DaemonServer)(nil),
|
||||||
Methods: []grpc.MethodDesc{
|
Methods: []grpc.MethodDesc{
|
||||||
|
|
|
||||||
|
|
@ -15,14 +15,15 @@
|
||||||
|
|
||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.26.0
|
// protoc-gen-go v1.25.0
|
||||||
// protoc v3.15.8
|
// protoc v3.17.3
|
||||||
// source: pkg/rpc/manager/manager.proto
|
// source: pkg/rpc/manager/manager.proto
|
||||||
|
|
||||||
package manager
|
package manager
|
||||||
|
|
||||||
import (
|
import (
|
||||||
_ "github.com/envoyproxy/protoc-gen-validate/validate"
|
_ "github.com/envoyproxy/protoc-gen-validate/validate"
|
||||||
|
proto "github.com/golang/protobuf/proto"
|
||||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||||
emptypb "google.golang.org/protobuf/types/known/emptypb"
|
emptypb "google.golang.org/protobuf/types/known/emptypb"
|
||||||
|
|
@ -37,6 +38,10 @@ const (
|
||||||
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
|
_ = 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
|
type SourceType int32
|
||||||
|
|
||||||
const (
|
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
|
// This is a compile-time assertion to ensure that this generated file
|
||||||
// is compatible with the grpc package it is being compiled against.
|
// is compatible with the grpc package it is being compiled against.
|
||||||
// Requires gRPC-Go v1.32.0 or later.
|
|
||||||
const _ = grpc.SupportPackageIsVersion7
|
const _ = grpc.SupportPackageIsVersion7
|
||||||
|
|
||||||
// ManagerClient is the client API for Manager service.
|
// ManagerClient is the client API for Manager service.
|
||||||
|
|
@ -87,7 +86,7 @@ func (c *managerClient) ListSchedulers(ctx context.Context, in *ListSchedulersRe
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *managerClient) KeepAlive(ctx context.Context, opts ...grpc.CallOption) (Manager_KeepAliveClient, error) {
|
func (c *managerClient) KeepAlive(ctx context.Context, opts ...grpc.CallOption) (Manager_KeepAliveClient, error) {
|
||||||
stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[0], "/manager.Manager/KeepAlive", opts...)
|
stream, err := c.cc.NewStream(ctx, &_Manager_serviceDesc.Streams[0], "/manager.Manager/KeepAlive", opts...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
@ -171,7 +170,7 @@ type UnsafeManagerServer interface {
|
||||||
}
|
}
|
||||||
|
|
||||||
func RegisterManagerServer(s grpc.ServiceRegistrar, srv ManagerServer) {
|
func RegisterManagerServer(s grpc.ServiceRegistrar, srv ManagerServer) {
|
||||||
s.RegisterService(&Manager_ServiceDesc, srv)
|
s.RegisterService(&_Manager_serviceDesc, srv)
|
||||||
}
|
}
|
||||||
|
|
||||||
func _Manager_GetCDN_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
func _Manager_GetCDN_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||||
|
|
@ -290,10 +289,7 @@ func (x *managerKeepAliveServer) Recv() (*KeepAliveRequest, error) {
|
||||||
return m, nil
|
return m, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Manager_ServiceDesc is the grpc.ServiceDesc for Manager service.
|
var _Manager_serviceDesc = grpc.ServiceDesc{
|
||||||
// It's only intended for direct use with grpc.RegisterService,
|
|
||||||
// and not to be introspected or modified (even as a copy)
|
|
||||||
var Manager_ServiceDesc = grpc.ServiceDesc{
|
|
||||||
ServiceName: "manager.Manager",
|
ServiceName: "manager.Manager",
|
||||||
HandlerType: (*ManagerServer)(nil),
|
HandlerType: (*ManagerServer)(nil),
|
||||||
Methods: []grpc.MethodDesc{
|
Methods: []grpc.MethodDesc{
|
||||||
|
|
|
||||||
|
|
@ -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) {
|
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)
|
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)
|
logger.With("peerId", ptr.PeerId, "errMsg", err).Infof("start to report piece result for taskID: %s", taskID)
|
||||||
|
|
||||||
// trigger scheduling
|
// trigger scheduling
|
||||||
|
|
|
||||||
|
|
@ -18,9 +18,10 @@ package client
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"io"
|
|
||||||
|
|
||||||
|
"d7y.io/dragonfly/v2/internal/dferrors"
|
||||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||||
|
"github.com/pkg/errors"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/codes"
|
"google.golang.org/grpc/codes"
|
||||||
"google.golang.org/grpc/status"
|
"google.golang.org/grpc/status"
|
||||||
|
|
@ -60,9 +61,9 @@ func newPeerPacketStream(ctx context.Context, sc *schedulerClient, hashKey strin
|
||||||
ptr: ptr,
|
ptr: ptr,
|
||||||
opts: opts,
|
opts: opts,
|
||||||
retryMeta: rpc.RetryMeta{
|
retryMeta: rpc.RetryMeta{
|
||||||
MaxAttempts: 5,
|
MaxAttempts: 3,
|
||||||
InitBackoff: 0.5,
|
InitBackoff: 0.2,
|
||||||
MaxBackOff: 4.0,
|
MaxBackOff: 2.0,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -77,14 +78,13 @@ func (pps *peerPacketStream) Send(pr *scheduler.PieceResult) (err error) {
|
||||||
pps.sc.UpdateAccessNodeMapByHashKey(pps.hashKey)
|
pps.sc.UpdateAccessNodeMapByHashKey(pps.hashKey)
|
||||||
err = pps.stream.Send(pr)
|
err = pps.stream.Send(pr)
|
||||||
|
|
||||||
if pr.PieceNum == common.EndOfPiece {
|
if pr.PieceInfo.PieceNum == common.EndOfPiece {
|
||||||
pps.closeSend()
|
pps.closeSend()
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
pps.closeSend()
|
pps.closeSend()
|
||||||
err = pps.retrySend(pr, err)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return
|
return
|
||||||
|
|
@ -96,10 +96,7 @@ func (pps *peerPacketStream) closeSend() error {
|
||||||
|
|
||||||
func (pps *peerPacketStream) Recv() (pp *scheduler.PeerPacket, err error) {
|
func (pps *peerPacketStream) Recv() (pp *scheduler.PeerPacket, err error) {
|
||||||
pps.sc.UpdateAccessNodeMapByHashKey(pps.hashKey)
|
pps.sc.UpdateAccessNodeMapByHashKey(pps.hashKey)
|
||||||
if pp, err = pps.stream.Recv(); err != nil && err != io.EOF {
|
return pps.stream.Recv()
|
||||||
pp, err = pps.retryRecv(err)
|
|
||||||
}
|
|
||||||
return
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pps *peerPacketStream) retrySend(pr *scheduler.PieceResult, cause error) error {
|
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 {
|
if err := pps.replaceStream(cause); err != nil {
|
||||||
return err
|
return cause
|
||||||
}
|
}
|
||||||
|
|
||||||
return pps.Send(pr)
|
return pps.Send(pr)
|
||||||
|
|
@ -159,6 +156,9 @@ func (pps *peerPacketStream) initStream() error {
|
||||||
return client.ReportPieceResult(pps.ctx, pps.opts...)
|
return client.ReportPieceResult(pps.ctx, pps.opts...)
|
||||||
}, pps.retryMeta.InitBackoff, pps.retryMeta.MaxBackOff, pps.retryMeta.MaxAttempts, nil)
|
}, pps.retryMeta.InitBackoff, pps.retryMeta.MaxBackOff, pps.retryMeta.MaxAttempts, nil)
|
||||||
if err != 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)
|
logger.WithTaskID(pps.hashKey).Infof("initStream: invoke scheduler node %s ReportPieceResult failed: %v", target, err)
|
||||||
return pps.replaceClient(err)
|
return pps.replaceClient(err)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -17,6 +17,7 @@
|
||||||
package scheduler
|
package scheduler
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||||
"d7y.io/dragonfly/v2/pkg/rpc/base/common"
|
"d7y.io/dragonfly/v2/pkg/rpc/base/common"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
@ -24,7 +25,14 @@ func NewZeroPieceResult(taskID, peerID string) *PieceResult {
|
||||||
return &PieceResult{
|
return &PieceResult{
|
||||||
TaskId: taskID,
|
TaskId: taskID,
|
||||||
SrcPid: peerID,
|
SrcPid: peerID,
|
||||||
|
PieceInfo: &base.PieceInfo{
|
||||||
PieceNum: common.ZeroOfPiece,
|
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{
|
return &PieceResult{
|
||||||
TaskId: taskID,
|
TaskId: taskID,
|
||||||
SrcPid: peerID,
|
SrcPid: peerID,
|
||||||
PieceNum: common.EndOfPiece,
|
|
||||||
FinishedCount: finishedCount,
|
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.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.26.0
|
// protoc-gen-go v1.25.0
|
||||||
// protoc v3.15.8
|
// protoc v3.17.3
|
||||||
// source: pkg/rpc/scheduler/scheduler.proto
|
// source: pkg/rpc/scheduler/scheduler.proto
|
||||||
|
|
||||||
package scheduler
|
package scheduler
|
||||||
|
|
||||||
import (
|
import (
|
||||||
base "d7y.io/dragonfly/v2/pkg/rpc/base"
|
base "d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||||
|
proto "github.com/golang/protobuf/proto"
|
||||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||||
emptypb "google.golang.org/protobuf/types/known/emptypb"
|
emptypb "google.golang.org/protobuf/types/known/emptypb"
|
||||||
|
|
@ -37,6 +38,10 @@ const (
|
||||||
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
|
_ = 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 {
|
type PeerTaskRequest struct {
|
||||||
state protoimpl.MessageState
|
state protoimpl.MessageState
|
||||||
sizeCache protoimpl.SizeCache
|
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"`
|
SrcPid string `protobuf:"bytes,2,opt,name=src_pid,json=srcPid,proto3" json:"src_pid,omitempty"`
|
||||||
// dest peer id
|
// dest peer id
|
||||||
DstPid string `protobuf:"bytes,3,opt,name=dst_pid,json=dstPid,proto3" json:"dst_pid,omitempty"`
|
DstPid string `protobuf:"bytes,3,opt,name=dst_pid,json=dstPid,proto3" json:"dst_pid,omitempty"`
|
||||||
// piece number
|
// piece info
|
||||||
PieceNum int32 `protobuf:"varint,4,opt,name=piece_num,json=pieceNum,proto3" json:"piece_num,omitempty"`
|
PieceInfo *base.PieceInfo `protobuf:"bytes,4,opt,name=piece_info,json=pieceInfo,proto3" json:"piece_info,omitempty"`
|
||||||
// begin time for the piece downloading
|
// begin time for the piece downloading
|
||||||
BeginTime uint64 `protobuf:"varint,5,opt,name=begin_time,json=beginTime,proto3" json:"begin_time,omitempty"`
|
BeginTime uint64 `protobuf:"varint,5,opt,name=begin_time,json=beginTime,proto3" json:"begin_time,omitempty"`
|
||||||
// end time for the piece downloading
|
// end time for the piece downloading
|
||||||
|
|
@ -498,11 +503,11 @@ func (x *PieceResult) GetDstPid() string {
|
||||||
return ""
|
return ""
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *PieceResult) GetPieceNum() int32 {
|
func (x *PieceResult) GetPieceInfo() *base.PieceInfo {
|
||||||
if x != nil {
|
if x != nil {
|
||||||
return x.PieceNum
|
return x.PieceInfo
|
||||||
}
|
}
|
||||||
return 0
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *PieceResult) GetBeginTime() uint64 {
|
func (x *PieceResult) GetBeginTime() uint64 {
|
||||||
|
|
@ -556,7 +561,6 @@ type PeerPacket struct {
|
||||||
// source peer id
|
// source peer id
|
||||||
SrcPid string `protobuf:"bytes,3,opt,name=src_pid,json=srcPid,proto3" json:"src_pid,omitempty"`
|
SrcPid string `protobuf:"bytes,3,opt,name=src_pid,json=srcPid,proto3" json:"src_pid,omitempty"`
|
||||||
// concurrent downloading count from main peer
|
// 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"`
|
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"`
|
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"`
|
StealPeers []*PeerPacket_DestPeer `protobuf:"bytes,6,rep,name=steal_peers,json=stealPeers,proto3" json:"steal_peers,omitempty"`
|
||||||
|
|
@ -659,6 +663,7 @@ type PeerResult struct {
|
||||||
Success bool `protobuf:"varint,10,opt,name=success,proto3" json:"success,omitempty"`
|
Success bool `protobuf:"varint,10,opt,name=success,proto3" json:"success,omitempty"`
|
||||||
// result code
|
// result code
|
||||||
Code base.Code `protobuf:"varint,11,opt,name=code,proto3,enum=base.Code" json:"code,omitempty"`
|
Code base.Code `protobuf:"varint,11,opt,name=code,proto3,enum=base.Code" json:"code,omitempty"`
|
||||||
|
TotalPieceCount int32 `protobuf:"varint,12,opt,name=total_piece_count,json=totalPieceCount,proto3" json:"total_piece_count,omitempty"`
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *PeerResult) Reset() {
|
func (x *PeerResult) Reset() {
|
||||||
|
|
@ -770,6 +775,13 @@ func (x *PeerResult) GetCode() base.Code {
|
||||||
return base.Code_X_UNSPECIFIED
|
return base.Code_X_UNSPECIFIED
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (x *PeerResult) GetTotalPieceCount() int32 {
|
||||||
|
if x != nil {
|
||||||
|
return x.TotalPieceCount
|
||||||
|
}
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
type PeerTarget struct {
|
type PeerTarget struct {
|
||||||
state protoimpl.MessageState
|
state protoimpl.MessageState
|
||||||
sizeCache protoimpl.SizeCache
|
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,
|
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,
|
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,
|
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,
|
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,
|
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,
|
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,
|
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,
|
0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x2e, 0x0a,
|
||||||
0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05,
|
0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28,
|
||||||
0x52, 0x08, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4e, 0x75, 0x6d, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x65,
|
0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e,
|
||||||
0x67, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09,
|
0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x0a,
|
||||||
0x62, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x65, 0x6e, 0x64,
|
0x0a, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28,
|
||||||
0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x65, 0x6e, 0x64,
|
0x04, 0x52, 0x09, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08,
|
||||||
0x54, 0x69, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18,
|
0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07,
|
||||||
0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1e,
|
0x65, 0x6e, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65,
|
||||||
0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62,
|
0x73, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73,
|
||||||
0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x2b,
|
0x73, 0x12, 0x1e, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32,
|
||||||
0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28,
|
0x0a, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64,
|
||||||
0x0b, 0x32, 0x0e, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x48, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61,
|
0x65, 0x12, 0x2b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09,
|
||||||
0x64, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x66,
|
0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x48, 0x6f, 0x73, 0x74,
|
||||||
0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0a, 0x20,
|
0x4c, 0x6f, 0x61, 0x64, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x25,
|
||||||
0x01, 0x28, 0x05, 0x52, 0x0d, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x43, 0x6f, 0x75,
|
0x0a, 0x0e, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74,
|
||||||
0x6e, 0x74, 0x22, 0xd3, 0x02, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x50, 0x61, 0x63, 0x6b, 0x65,
|
0x18, 0x0a, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64,
|
||||||
0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01,
|
0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xd3, 0x02, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x50, 0x61,
|
||||||
0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x73, 0x72,
|
0x63, 0x6b, 0x65, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18,
|
||||||
0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x72, 0x63,
|
0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a,
|
||||||
0x50, 0x69, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f,
|
0x07, 0x73, 0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06,
|
||||||
0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x70, 0x61, 0x72,
|
0x73, 0x72, 0x63, 0x50, 0x69, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c,
|
||||||
0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x3b, 0x0a, 0x09, 0x6d, 0x61,
|
0x65, 0x6c, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d,
|
||||||
0x69, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e,
|
0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x3b, 0x0a,
|
||||||
0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x50, 0x61,
|
0x09, 0x6d, 0x61, 0x69, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b,
|
||||||
0x63, 0x6b, 0x65, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, 0x08, 0x6d,
|
0x32, 0x1e, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65,
|
||||||
0x61, 0x69, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x0b, 0x73, 0x74, 0x65, 0x61, 0x6c,
|
0x72, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72,
|
||||||
0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x73,
|
0x52, 0x08, 0x6d, 0x61, 0x69, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x0b, 0x73, 0x74,
|
||||||
0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x50, 0x61, 0x63,
|
0x65, 0x61, 0x6c, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32,
|
||||||
0x6b, 0x65, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x74,
|
0x1e, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72,
|
||||||
0x65, 0x61, 0x6c, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x1e, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65,
|
0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52,
|
||||||
0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f,
|
0x0a, 0x73, 0x74, 0x65, 0x61, 0x6c, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x1e, 0x0a, 0x04, 0x63,
|
||||||
0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x1a, 0x4e, 0x0a, 0x08, 0x44, 0x65, 0x73, 0x74,
|
0x6f, 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61, 0x73, 0x65,
|
||||||
0x50, 0x65, 0x65, 0x72, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
|
0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x1a, 0x4e, 0x0a, 0x08, 0x44,
|
||||||
0x52, 0x02, 0x69, 0x70, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x70, 0x63, 0x5f, 0x70, 0x6f, 0x72, 0x74,
|
0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x70, 0x18, 0x01, 0x20,
|
||||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x72, 0x70, 0x63, 0x50, 0x6f, 0x72, 0x74, 0x12,
|
0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x70, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x70, 0x63, 0x5f, 0x70,
|
||||||
0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09,
|
0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x72, 0x70, 0x63, 0x50, 0x6f,
|
||||||
0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x22, 0xb1, 0x02, 0x0a, 0x0a, 0x50, 0x65, 0x65,
|
0x72, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20,
|
||||||
0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f,
|
0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x22, 0xdd, 0x02, 0x0a, 0x0a,
|
||||||
0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64,
|
0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61,
|
||||||
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,
|
|
||||||
0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73,
|
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,
|
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,
|
0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x15, 0x0a, 0x06,
|
||||||
0x09, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x10, 0x52, 0x65,
|
0x73, 0x72, 0x63, 0x5f, 0x69, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x72,
|
||||||
0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x1a,
|
0x63, 0x49, 0x70, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f,
|
||||||
0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54,
|
0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x65,
|
||||||
0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x73, 0x63, 0x68,
|
0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03,
|
||||||
0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52,
|
0x69, 0x64, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x63, 0x12, 0x10,
|
||||||
0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x46, 0x0a, 0x11, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50,
|
0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c,
|
||||||
0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x2e, 0x73, 0x63, 0x68,
|
0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x65, 0x6e, 0x67,
|
||||||
0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75,
|
0x74, 0x68, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e,
|
||||||
0x6c, 0x74, 0x1a, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50,
|
0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x72, 0x61, 0x66, 0x66,
|
||||||
0x65, 0x65, 0x72, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x28, 0x01, 0x30, 0x01, 0x12, 0x41, 0x0a,
|
0x69, 0x63, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x74, 0x72, 0x61, 0x66, 0x66, 0x69,
|
||||||
0x10, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c,
|
0x63, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x73, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0d, 0x52,
|
||||||
0x74, 0x12, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65,
|
0x04, 0x63, 0x6f, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73,
|
||||||
0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
|
0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12,
|
||||||
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79,
|
0x1e, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e,
|
||||||
0x12, 0x3a, 0x0a, 0x09, 0x4c, 0x65, 0x61, 0x76, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x15, 0x2e,
|
0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12,
|
||||||
|
0x2a, 0x0a, 0x11, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x63,
|
||||||
|
0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61,
|
||||||
|
0x6c, 0x50, 0x69, 0x65, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x3e, 0x0a, 0x0a, 0x50,
|
||||||
|
0x65, 0x65, 0x72, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73,
|
||||||
|
0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b,
|
||||||
|
0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20,
|
||||||
|
0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x32, 0x9d, 0x02, 0x0a, 0x09,
|
||||||
|
0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x10, 0x52, 0x65, 0x67,
|
||||||
|
0x69, 0x73, 0x74, 0x65, 0x72, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x1a, 0x2e,
|
||||||
0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61,
|
0x73, 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,
|
0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x73, 0x63, 0x68, 0x65,
|
||||||
0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x27, 0x5a, 0x25,
|
0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65,
|
||||||
0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79,
|
0x73, 0x75, 0x6c, 0x74, 0x12, 0x46, 0x0a, 0x11, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69,
|
||||||
0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x73, 0x63, 0x68, 0x65,
|
0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x2e, 0x73, 0x63, 0x68, 0x65,
|
||||||
0x64, 0x75, 0x6c, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
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 (
|
var (
|
||||||
|
|
@ -1076,25 +1092,26 @@ var file_pkg_rpc_scheduler_scheduler_proto_depIdxs = []int32{
|
||||||
11, // 3: scheduler.RegisterResult.size_scope:type_name -> base.SizeScope
|
11, // 3: scheduler.RegisterResult.size_scope:type_name -> base.SizeScope
|
||||||
2, // 4: scheduler.RegisterResult.single_piece:type_name -> scheduler.SinglePiece
|
2, // 4: scheduler.RegisterResult.single_piece:type_name -> scheduler.SinglePiece
|
||||||
12, // 5: scheduler.SinglePiece.piece_info:type_name -> base.PieceInfo
|
12, // 5: scheduler.SinglePiece.piece_info:type_name -> base.PieceInfo
|
||||||
13, // 6: scheduler.PieceResult.code:type_name -> base.Code
|
12, // 6: scheduler.PieceResult.piece_info:type_name -> base.PieceInfo
|
||||||
10, // 7: scheduler.PieceResult.host_load:type_name -> base.HostLoad
|
13, // 7: scheduler.PieceResult.code:type_name -> base.Code
|
||||||
8, // 8: scheduler.PeerPacket.main_peer:type_name -> scheduler.PeerPacket.DestPeer
|
10, // 8: scheduler.PieceResult.host_load:type_name -> base.HostLoad
|
||||||
8, // 9: scheduler.PeerPacket.steal_peers:type_name -> scheduler.PeerPacket.DestPeer
|
8, // 9: scheduler.PeerPacket.main_peer:type_name -> scheduler.PeerPacket.DestPeer
|
||||||
13, // 10: scheduler.PeerPacket.code:type_name -> base.Code
|
8, // 10: scheduler.PeerPacket.steal_peers:type_name -> scheduler.PeerPacket.DestPeer
|
||||||
13, // 11: scheduler.PeerResult.code:type_name -> base.Code
|
13, // 11: scheduler.PeerPacket.code:type_name -> base.Code
|
||||||
0, // 12: scheduler.Scheduler.RegisterPeerTask:input_type -> scheduler.PeerTaskRequest
|
13, // 12: scheduler.PeerResult.code:type_name -> base.Code
|
||||||
4, // 13: scheduler.Scheduler.ReportPieceResult:input_type -> scheduler.PieceResult
|
0, // 13: scheduler.Scheduler.RegisterPeerTask:input_type -> scheduler.PeerTaskRequest
|
||||||
6, // 14: scheduler.Scheduler.ReportPeerResult:input_type -> scheduler.PeerResult
|
4, // 14: scheduler.Scheduler.ReportPieceResult:input_type -> scheduler.PieceResult
|
||||||
7, // 15: scheduler.Scheduler.LeaveTask:input_type -> scheduler.PeerTarget
|
6, // 15: scheduler.Scheduler.ReportPeerResult:input_type -> scheduler.PeerResult
|
||||||
1, // 16: scheduler.Scheduler.RegisterPeerTask:output_type -> scheduler.RegisterResult
|
7, // 16: scheduler.Scheduler.LeaveTask:input_type -> scheduler.PeerTarget
|
||||||
5, // 17: scheduler.Scheduler.ReportPieceResult:output_type -> scheduler.PeerPacket
|
1, // 17: scheduler.Scheduler.RegisterPeerTask:output_type -> scheduler.RegisterResult
|
||||||
14, // 18: scheduler.Scheduler.ReportPeerResult:output_type -> google.protobuf.Empty
|
5, // 18: scheduler.Scheduler.ReportPieceResult:output_type -> scheduler.PeerPacket
|
||||||
14, // 19: scheduler.Scheduler.LeaveTask:output_type -> google.protobuf.Empty
|
14, // 19: scheduler.Scheduler.ReportPeerResult:output_type -> google.protobuf.Empty
|
||||||
16, // [16:20] is the sub-list for method output_type
|
14, // 20: scheduler.Scheduler.LeaveTask:output_type -> google.protobuf.Empty
|
||||||
12, // [12:16] is the sub-list for method input_type
|
17, // [17:21] is the sub-list for method output_type
|
||||||
12, // [12:12] is the sub-list for extension type_name
|
13, // [13:17] is the sub-list for method input_type
|
||||||
12, // [12:12] is the sub-list for extension extendee
|
13, // [13:13] is the sub-list for extension type_name
|
||||||
0, // [0:12] is the sub-list for field 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() }
|
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
|
// Validate checks the field values on PeerTaskRequest with the rules defined
|
||||||
// in the proto definition for this message. If any rules are violated, an
|
// in the proto definition for this message. If any rules are violated, the
|
||||||
// error is returned.
|
// first error encountered is returned, or nil if there are no violations.
|
||||||
func (m *PeerTaskRequest) Validate() error {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for Url
|
// 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 {
|
if err := v.Validate(); err != nil {
|
||||||
return PeerTaskRequestValidationError{
|
return PeerTaskRequestValidationError{
|
||||||
field: "UrlMeta",
|
field: "UrlMeta",
|
||||||
|
|
@ -65,7 +98,26 @@ func (m *PeerTaskRequest) Validate() error {
|
||||||
|
|
||||||
// no validation rules for PeerId
|
// 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 {
|
if err := v.Validate(); err != nil {
|
||||||
return PeerTaskRequestValidationError{
|
return PeerTaskRequestValidationError{
|
||||||
field: "PeerHost",
|
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 {
|
if err := v.Validate(); err != nil {
|
||||||
return PeerTaskRequestValidationError{
|
return PeerTaskRequestValidationError{
|
||||||
field: "HostLoad",
|
field: "HostLoad",
|
||||||
|
|
@ -87,9 +158,29 @@ func (m *PeerTaskRequest) Validate() error {
|
||||||
|
|
||||||
// no validation rules for IsMigrating
|
// no validation rules for IsMigrating
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return PeerTaskRequestMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// PeerTaskRequestValidationError is the validation error returned by
|
||||||
// PeerTaskRequest.Validate if the designated constraints aren't met.
|
// PeerTaskRequest.Validate if the designated constraints aren't met.
|
||||||
type PeerTaskRequestValidationError struct {
|
type PeerTaskRequestValidationError struct {
|
||||||
|
|
@ -145,13 +236,27 @@ var _ interface {
|
||||||
} = PeerTaskRequestValidationError{}
|
} = PeerTaskRequestValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on RegisterResult with the rules defined in
|
// 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
|
// the proto definition for this message. If any rules are violated, the first
|
||||||
// is returned.
|
// error encountered is returned, or nil if there are no violations.
|
||||||
func (m *RegisterResult) Validate() error {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for TaskId
|
// no validation rules for TaskId
|
||||||
|
|
||||||
// no validation rules for SizeScope
|
// no validation rules for SizeScope
|
||||||
|
|
@ -160,7 +265,26 @@ func (m *RegisterResult) Validate() error {
|
||||||
|
|
||||||
case *RegisterResult_SinglePiece:
|
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 {
|
if err := v.Validate(); err != nil {
|
||||||
return RegisterResultValidationError{
|
return RegisterResultValidationError{
|
||||||
field: "SinglePiece",
|
field: "SinglePiece",
|
||||||
|
|
@ -175,9 +299,29 @@ func (m *RegisterResult) Validate() error {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return RegisterResultMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// RegisterResultValidationError is the validation error returned by
|
||||||
// RegisterResult.Validate if the designated constraints aren't met.
|
// RegisterResult.Validate if the designated constraints aren't met.
|
||||||
type RegisterResultValidationError struct {
|
type RegisterResultValidationError struct {
|
||||||
|
|
@ -233,18 +377,51 @@ var _ interface {
|
||||||
} = RegisterResultValidationError{}
|
} = RegisterResultValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on SinglePiece with the rules defined in
|
// 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
|
// the proto definition for this message. If any rules are violated, the first
|
||||||
// is returned.
|
// error encountered is returned, or nil if there are no violations.
|
||||||
func (m *SinglePiece) Validate() error {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for DstPid
|
// no validation rules for DstPid
|
||||||
|
|
||||||
// no validation rules for DstAddr
|
// 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 {
|
if err := v.Validate(); err != nil {
|
||||||
return SinglePieceValidationError{
|
return SinglePieceValidationError{
|
||||||
field: "PieceInfo",
|
field: "PieceInfo",
|
||||||
|
|
@ -254,9 +431,28 @@ func (m *SinglePiece) Validate() error {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return SinglePieceMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// SinglePieceValidationError is the validation error returned by
|
||||||
// SinglePiece.Validate if the designated constraints aren't met.
|
// SinglePiece.Validate if the designated constraints aren't met.
|
||||||
type SinglePieceValidationError struct {
|
type SinglePieceValidationError struct {
|
||||||
|
|
@ -312,12 +508,27 @@ var _ interface {
|
||||||
} = SinglePieceValidationError{}
|
} = SinglePieceValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on PeerHost with the rules defined in the
|
// 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 {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for Uuid
|
// no validation rules for Uuid
|
||||||
|
|
||||||
// no validation rules for Ip
|
// no validation rules for Ip
|
||||||
|
|
@ -336,9 +547,28 @@ func (m *PeerHost) Validate() error {
|
||||||
|
|
||||||
// no validation rules for NetTopology
|
// no validation rules for NetTopology
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return PeerHostMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// PeerHostValidationError is the validation error returned by
|
||||||
// PeerHost.Validate if the designated constraints aren't met.
|
// PeerHost.Validate if the designated constraints aren't met.
|
||||||
type PeerHostValidationError struct {
|
type PeerHostValidationError struct {
|
||||||
|
|
@ -394,20 +624,61 @@ var _ interface {
|
||||||
} = PeerHostValidationError{}
|
} = PeerHostValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on PieceResult with the rules defined in
|
// 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
|
// the proto definition for this message. If any rules are violated, the first
|
||||||
// is returned.
|
// error encountered is returned, or nil if there are no violations.
|
||||||
func (m *PieceResult) Validate() error {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for TaskId
|
// no validation rules for TaskId
|
||||||
|
|
||||||
// no validation rules for SrcPid
|
// no validation rules for SrcPid
|
||||||
|
|
||||||
// no validation rules for DstPid
|
// 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
|
// no validation rules for BeginTime
|
||||||
|
|
||||||
|
|
@ -417,7 +688,26 @@ func (m *PieceResult) Validate() error {
|
||||||
|
|
||||||
// no validation rules for Code
|
// 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 {
|
if err := v.Validate(); err != nil {
|
||||||
return PieceResultValidationError{
|
return PieceResultValidationError{
|
||||||
field: "HostLoad",
|
field: "HostLoad",
|
||||||
|
|
@ -429,9 +719,28 @@ func (m *PieceResult) Validate() error {
|
||||||
|
|
||||||
// no validation rules for FinishedCount
|
// no validation rules for FinishedCount
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return PieceResultMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// PieceResultValidationError is the validation error returned by
|
||||||
// PieceResult.Validate if the designated constraints aren't met.
|
// PieceResult.Validate if the designated constraints aren't met.
|
||||||
type PieceResultValidationError struct {
|
type PieceResultValidationError struct {
|
||||||
|
|
@ -487,19 +796,53 @@ var _ interface {
|
||||||
} = PieceResultValidationError{}
|
} = PieceResultValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on PeerPacket with the rules defined in the
|
// 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 {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for TaskId
|
// no validation rules for TaskId
|
||||||
|
|
||||||
// no validation rules for SrcPid
|
// no validation rules for SrcPid
|
||||||
|
|
||||||
// no validation rules for ParallelCount
|
// 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 {
|
if err := v.Validate(); err != nil {
|
||||||
return PeerPacketValidationError{
|
return PeerPacketValidationError{
|
||||||
field: "MainPeer",
|
field: "MainPeer",
|
||||||
|
|
@ -512,7 +855,26 @@ func (m *PeerPacket) Validate() error {
|
||||||
for idx, item := range m.GetStealPeers() {
|
for idx, item := range m.GetStealPeers() {
|
||||||
_, _ = idx, item
|
_, _ = 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 {
|
if err := v.Validate(); err != nil {
|
||||||
return PeerPacketValidationError{
|
return PeerPacketValidationError{
|
||||||
field: fmt.Sprintf("StealPeers[%v]", idx),
|
field: fmt.Sprintf("StealPeers[%v]", idx),
|
||||||
|
|
@ -526,9 +888,28 @@ func (m *PeerPacket) Validate() error {
|
||||||
|
|
||||||
// no validation rules for Code
|
// no validation rules for Code
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return PeerPacketMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// PeerPacketValidationError is the validation error returned by
|
||||||
// PeerPacket.Validate if the designated constraints aren't met.
|
// PeerPacket.Validate if the designated constraints aren't met.
|
||||||
type PeerPacketValidationError struct {
|
type PeerPacketValidationError struct {
|
||||||
|
|
@ -584,12 +965,27 @@ var _ interface {
|
||||||
} = PeerPacketValidationError{}
|
} = PeerPacketValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on PeerResult with the rules defined in the
|
// 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 {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for TaskId
|
// no validation rules for TaskId
|
||||||
|
|
||||||
// no validation rules for PeerId
|
// no validation rules for PeerId
|
||||||
|
|
@ -612,9 +1008,30 @@ func (m *PeerResult) Validate() error {
|
||||||
|
|
||||||
// no validation rules for Code
|
// no validation rules for Code
|
||||||
|
|
||||||
|
// no validation rules for TotalPieceCount
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return PeerResultMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// PeerResultValidationError is the validation error returned by
|
||||||
// PeerResult.Validate if the designated constraints aren't met.
|
// PeerResult.Validate if the designated constraints aren't met.
|
||||||
type PeerResultValidationError struct {
|
type PeerResultValidationError struct {
|
||||||
|
|
@ -670,19 +1087,53 @@ var _ interface {
|
||||||
} = PeerResultValidationError{}
|
} = PeerResultValidationError{}
|
||||||
|
|
||||||
// Validate checks the field values on PeerTarget with the rules defined in the
|
// 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 {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for TaskId
|
// no validation rules for TaskId
|
||||||
|
|
||||||
// no validation rules for PeerId
|
// no validation rules for PeerId
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return PeerTargetMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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
|
// PeerTargetValidationError is the validation error returned by
|
||||||
// PeerTarget.Validate if the designated constraints aren't met.
|
// PeerTarget.Validate if the designated constraints aren't met.
|
||||||
type PeerTargetValidationError struct {
|
type PeerTargetValidationError struct {
|
||||||
|
|
@ -739,21 +1190,55 @@ var _ interface {
|
||||||
|
|
||||||
// Validate checks the field values on PeerPacket_DestPeer with the rules
|
// Validate checks the field values on PeerPacket_DestPeer with the rules
|
||||||
// defined in the proto definition for this message. If any rules are
|
// 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 {
|
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 {
|
if m == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var errors []error
|
||||||
|
|
||||||
// no validation rules for Ip
|
// no validation rules for Ip
|
||||||
|
|
||||||
// no validation rules for RpcPort
|
// no validation rules for RpcPort
|
||||||
|
|
||||||
// no validation rules for PeerId
|
// no validation rules for PeerId
|
||||||
|
|
||||||
|
if len(errors) > 0 {
|
||||||
|
return PeerPacket_DestPeerMultiError(errors)
|
||||||
|
}
|
||||||
return nil
|
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_DestPeerValidationError is the validation error returned by
|
||||||
// PeerPacket_DestPeer.Validate if the designated constraints aren't met.
|
// PeerPacket_DestPeer.Validate if the designated constraints aren't met.
|
||||||
type PeerPacket_DestPeerValidationError struct {
|
type PeerPacket_DestPeerValidationError struct {
|
||||||
|
|
|
||||||
|
|
@ -90,8 +90,8 @@ message PieceResult{
|
||||||
string src_pid = 2;
|
string src_pid = 2;
|
||||||
// dest peer id
|
// dest peer id
|
||||||
string dst_pid = 3;
|
string dst_pid = 3;
|
||||||
// piece number
|
// piece info
|
||||||
int32 piece_num = 4;
|
base.PieceInfo piece_info = 4;
|
||||||
// begin time for the piece downloading
|
// begin time for the piece downloading
|
||||||
uint64 begin_time = 5;
|
uint64 begin_time = 5;
|
||||||
// end time for the piece downloading
|
// end time for the piece downloading
|
||||||
|
|
@ -122,7 +122,6 @@ message PeerPacket{
|
||||||
string src_pid = 3;
|
string src_pid = 3;
|
||||||
|
|
||||||
// concurrent downloading count from main peer
|
// concurrent downloading count from main peer
|
||||||
// TODO what effect
|
|
||||||
int32 parallel_count = 4;
|
int32 parallel_count = 4;
|
||||||
DestPeer main_peer = 5;
|
DestPeer main_peer = 5;
|
||||||
repeated DestPeer steal_peers = 6;
|
repeated DestPeer steal_peers = 6;
|
||||||
|
|
@ -147,6 +146,7 @@ message PeerResult{
|
||||||
bool success = 10;
|
bool success = 10;
|
||||||
// result code
|
// result code
|
||||||
base.Code code = 11;
|
base.Code code = 11;
|
||||||
|
int32 total_piece_count = 12;
|
||||||
}
|
}
|
||||||
|
|
||||||
message PeerTarget{
|
message PeerTarget{
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,6 @@ import (
|
||||||
|
|
||||||
// This is a compile-time assertion to ensure that this generated file
|
// This is a compile-time assertion to ensure that this generated file
|
||||||
// is compatible with the grpc package it is being compiled against.
|
// is compatible with the grpc package it is being compiled against.
|
||||||
// Requires gRPC-Go v1.32.0 or later.
|
|
||||||
const _ = grpc.SupportPackageIsVersion7
|
const _ = grpc.SupportPackageIsVersion7
|
||||||
|
|
||||||
// SchedulerClient is the client API for Scheduler service.
|
// SchedulerClient is the client API for Scheduler service.
|
||||||
|
|
@ -49,7 +48,7 @@ func (c *schedulerClient) RegisterPeerTask(ctx context.Context, in *PeerTaskRequ
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *schedulerClient) ReportPieceResult(ctx context.Context, opts ...grpc.CallOption) (Scheduler_ReportPieceResultClient, error) {
|
func (c *schedulerClient) ReportPieceResult(ctx context.Context, opts ...grpc.CallOption) (Scheduler_ReportPieceResultClient, error) {
|
||||||
stream, err := c.cc.NewStream(ctx, &Scheduler_ServiceDesc.Streams[0], "/scheduler.Scheduler/ReportPieceResult", opts...)
|
stream, err := c.cc.NewStream(ctx, &_Scheduler_serviceDesc.Streams[0], "/scheduler.Scheduler/ReportPieceResult", opts...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
@ -140,7 +139,7 @@ type UnsafeSchedulerServer interface {
|
||||||
}
|
}
|
||||||
|
|
||||||
func RegisterSchedulerServer(s grpc.ServiceRegistrar, srv SchedulerServer) {
|
func RegisterSchedulerServer(s grpc.ServiceRegistrar, srv SchedulerServer) {
|
||||||
s.RegisterService(&Scheduler_ServiceDesc, srv)
|
s.RegisterService(&_Scheduler_serviceDesc, srv)
|
||||||
}
|
}
|
||||||
|
|
||||||
func _Scheduler_RegisterPeerTask_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
func _Scheduler_RegisterPeerTask_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||||
|
|
@ -223,10 +222,7 @@ func _Scheduler_LeaveTask_Handler(srv interface{}, ctx context.Context, dec func
|
||||||
return interceptor(ctx, in, info, handler)
|
return interceptor(ctx, in, info, handler)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Scheduler_ServiceDesc is the grpc.ServiceDesc for Scheduler service.
|
var _Scheduler_serviceDesc = grpc.ServiceDesc{
|
||||||
// It's only intended for direct use with grpc.RegisterService,
|
|
||||||
// and not to be introspected or modified (even as a copy)
|
|
||||||
var Scheduler_ServiceDesc = grpc.ServiceDesc{
|
|
||||||
ServiceName: "scheduler.Scheduler",
|
ServiceName: "scheduler.Scheduler",
|
||||||
HandlerType: (*SchedulerServer)(nil),
|
HandlerType: (*SchedulerServer)(nil),
|
||||||
Methods: []grpc.MethodDesc{
|
Methods: []grpc.MethodDesc{
|
||||||
|
|
|
||||||
|
|
@ -44,15 +44,15 @@ type proxy struct {
|
||||||
scheduler.UnimplementedSchedulerServer
|
scheduler.UnimplementedSchedulerServer
|
||||||
}
|
}
|
||||||
|
|
||||||
// see scheduler.SchedulerServer
|
// SchedulerServer scheduler.SchedulerServer
|
||||||
type SchedulerServer interface {
|
type SchedulerServer interface {
|
||||||
// RegisterPeerTask
|
// RegisterPeerTask register a peer to scheduler
|
||||||
RegisterPeerTask(context.Context, *scheduler.PeerTaskRequest) (*scheduler.RegisterResult, error)
|
RegisterPeerTask(context.Context, *scheduler.PeerTaskRequest) (*scheduler.RegisterResult, error)
|
||||||
// ReportPieceResult
|
// ReportPieceResult report piece result to scheduler
|
||||||
ReportPieceResult(scheduler.Scheduler_ReportPieceResultServer) error
|
ReportPieceResult(scheduler.Scheduler_ReportPieceResultServer) error
|
||||||
// ReportPeerResult
|
// ReportPeerResult report peer download result to scheduler
|
||||||
ReportPeerResult(context.Context, *scheduler.PeerResult) error
|
ReportPeerResult(context.Context, *scheduler.PeerResult) error
|
||||||
// LeaveTask
|
// LeaveTask leave peer from scheduler
|
||||||
LeaveTask(context.Context, *scheduler.PeerTarget) error
|
LeaveTask(context.Context, *scheduler.PeerTarget) error
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -35,12 +35,12 @@ type Config struct {
|
||||||
Manager *ManagerConfig `yaml:"manager" mapstructure:"manager"`
|
Manager *ManagerConfig `yaml:"manager" mapstructure:"manager"`
|
||||||
Host *HostConfig `yaml:"host" mapstructure:"host"`
|
Host *HostConfig `yaml:"host" mapstructure:"host"`
|
||||||
Job *JobConfig `yaml:"job" mapstructure:"job"`
|
Job *JobConfig `yaml:"job" mapstructure:"job"`
|
||||||
|
DisableCDN bool `yaml:"disableCDN" mapstructure:"disableCDN"`
|
||||||
}
|
}
|
||||||
|
|
||||||
func New() *Config {
|
func New() *Config {
|
||||||
return &Config{
|
return &Config{
|
||||||
Scheduler: &SchedulerConfig{
|
Scheduler: &SchedulerConfig{
|
||||||
DisableCDN: false,
|
|
||||||
ABTest: false,
|
ABTest: false,
|
||||||
AScheduler: "",
|
AScheduler: "",
|
||||||
BScheduler: "",
|
BScheduler: "",
|
||||||
|
|
@ -53,8 +53,8 @@ func New() *Config {
|
||||||
ClientLoad: 10,
|
ClientLoad: 10,
|
||||||
OpenMonitor: true,
|
OpenMonitor: true,
|
||||||
GC: &GCConfig{
|
GC: &GCConfig{
|
||||||
PeerGCInterval: 5 * time.Minute,
|
PeerGCInterval: 1 * time.Minute,
|
||||||
TaskGCInterval: 5 * time.Minute,
|
TaskGCInterval: 1 * time.Minute,
|
||||||
PeerTTL: 10 * time.Minute,
|
PeerTTL: 10 * time.Minute,
|
||||||
PeerTTI: 3 * time.Minute,
|
PeerTTI: 3 * time.Minute,
|
||||||
TaskTTL: 10 * time.Minute,
|
TaskTTL: 10 * time.Minute,
|
||||||
|
|
@ -109,6 +109,7 @@ func New() *Config {
|
||||||
BackendDB: 2,
|
BackendDB: 2,
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
|
DisableCDN: false,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -178,12 +179,11 @@ type DynConfig struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
type SchedulerConfig struct {
|
type SchedulerConfig struct {
|
||||||
DisableCDN bool `yaml:"disableCDN" mapstructure:"disableCDN"`
|
|
||||||
ABTest bool `yaml:"abtest" mapstructure:"abtest"`
|
ABTest bool `yaml:"abtest" mapstructure:"abtest"`
|
||||||
AScheduler string `yaml:"ascheduler" mapstructure:"ascheduler"`
|
AScheduler string `yaml:"ascheduler" mapstructure:"ascheduler"`
|
||||||
BScheduler string `yaml:"bscheduler" mapstructure:"bscheduler"`
|
BScheduler string `yaml:"bscheduler" mapstructure:"bscheduler"`
|
||||||
WorkerNum int `yaml:"workerNum" mapstructure:"workerNum"`
|
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 should less than CDN task expireTime
|
||||||
AccessWindow time.Duration `yaml:"accessWindow" mapstructure:"accessWindow"`
|
AccessWindow time.Duration `yaml:"accessWindow" mapstructure:"accessWindow"`
|
||||||
CandidateParentCount int `yaml:"candidateParentCount" mapstructure:"candidateParentCount"`
|
CandidateParentCount int `yaml:"candidateParentCount" mapstructure:"candidateParentCount"`
|
||||||
|
|
@ -202,6 +202,7 @@ type ServerConfig struct {
|
||||||
|
|
||||||
type GCConfig struct {
|
type GCConfig struct {
|
||||||
PeerGCInterval time.Duration `yaml:"peerGCInterval" mapstructure:"peerGCInterval"`
|
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"`
|
PeerTTL time.Duration `yaml:"peerTTL" mapstructure:"peerTTL"`
|
||||||
PeerTTI time.Duration `yaml:"peerTTI" mapstructure:"peerTTI"`
|
PeerTTI time.Duration `yaml:"peerTTI" mapstructure:"peerTTI"`
|
||||||
TaskGCInterval time.Duration `yaml:"taskGCInterval" mapstructure:"taskGCInterval"`
|
TaskGCInterval time.Duration `yaml:"taskGCInterval" mapstructure:"taskGCInterval"`
|
||||||
|
|
|
||||||
|
|
@ -35,6 +35,10 @@ const (
|
||||||
AttributePeerID = attribute.Key("d7y.peer.id")
|
AttributePeerID = attribute.Key("d7y.peer.id")
|
||||||
AttributeCDNSeedRequest = attribute.Key("d7y.cdn.seed.request")
|
AttributeCDNSeedRequest = attribute.Key("d7y.cdn.seed.request")
|
||||||
AttributeNeedSeedCDN = attribute.Key("d7y.need.seed.cdn")
|
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 (
|
const (
|
||||||
|
|
@ -46,11 +50,13 @@ const (
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
EventScheduleParentFail = "fail-schedule-parent"
|
EventSmallTaskSelectParentFail = "small-task-select-parent-fail"
|
||||||
EventPeerNotFound = "peer-not-found"
|
EventPeerNotFound = "peer-not-found"
|
||||||
EventHostNotFound = "host-not-found"
|
EventHostNotFound = "host-not-found"
|
||||||
EventCreateCDNPeer = "create-cdn-peer"
|
EventCreateCDNPeer = "create-cdn-peer"
|
||||||
EventPieceReceived = "receive-piece"
|
EventPieceReceived = "receive-piece"
|
||||||
EventPeerDownloaded = "downloaded"
|
EventPeerDownloaded = "downloaded"
|
||||||
EventDownloadTinyFile = "download-tiny-file"
|
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 {
|
func (eval *baseEvaluator) IsBadNode(peer *supervisor.Peer) bool {
|
||||||
if peer.IsBad() {
|
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
|
return true
|
||||||
}
|
}
|
||||||
costHistory := peer.GetCostHistory()
|
costHistory := peer.GetCostHistory()
|
||||||
|
|
|
||||||
|
|
@ -21,11 +21,12 @@ import (
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"d7y.io/dragonfly/v2/internal/dfcodes"
|
"d7y.io/dragonfly/v2/internal/dfcodes"
|
||||||
|
"d7y.io/dragonfly/v2/internal/dferrors"
|
||||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
|
||||||
schedulerRPC "d7y.io/dragonfly/v2/pkg/rpc/scheduler"
|
schedulerRPC "d7y.io/dragonfly/v2/pkg/rpc/scheduler"
|
||||||
"d7y.io/dragonfly/v2/pkg/structure/sortedlist"
|
"d7y.io/dragonfly/v2/pkg/structure/sortedlist"
|
||||||
"d7y.io/dragonfly/v2/pkg/synclock"
|
"d7y.io/dragonfly/v2/pkg/synclock"
|
||||||
|
"d7y.io/dragonfly/v2/scheduler/config"
|
||||||
"d7y.io/dragonfly/v2/scheduler/core/scheduler"
|
"d7y.io/dragonfly/v2/scheduler/core/scheduler"
|
||||||
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
||||||
"go.opentelemetry.io/otel/trace"
|
"go.opentelemetry.io/otel/trace"
|
||||||
|
|
@ -76,20 +77,37 @@ var _ event = startReportPieceResultEvent{}
|
||||||
|
|
||||||
func (e startReportPieceResultEvent) apply(s *state) {
|
func (e startReportPieceResultEvent) apply(s *state) {
|
||||||
span := trace.SpanFromContext(e.ctx)
|
span := trace.SpanFromContext(e.ctx)
|
||||||
|
span.AddEvent(config.EventStartReportPieceResult)
|
||||||
if e.peer.GetParent() != nil {
|
if e.peer.GetParent() != nil {
|
||||||
span.AddEvent("no parent")
|
|
||||||
logger.WithTaskAndPeerID(e.peer.Task.TaskID,
|
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)
|
e.peer.PeerID).Warnf("startReportPieceResultEvent: no need schedule parent because peer already had parent %s", e.peer.GetParent().PeerID)
|
||||||
return
|
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)
|
parent, candidates, hasParent := s.sched.ScheduleParent(e.peer)
|
||||||
span.AddEvent("parent")
|
// No parent node is currently available
|
||||||
if !hasParent {
|
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)
|
s.waitScheduleParentPeerQueue.AddAfter(e.peer, time.Second)
|
||||||
return
|
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 {
|
func (e startReportPieceResultEvent) hashKey() string {
|
||||||
|
|
@ -106,28 +124,45 @@ var _ event = peerDownloadPieceSuccessEvent{}
|
||||||
|
|
||||||
func (e peerDownloadPieceSuccessEvent) apply(s *state) {
|
func (e peerDownloadPieceSuccessEvent) apply(s *state) {
|
||||||
span := trace.SpanFromContext(e.ctx)
|
span := trace.SpanFromContext(e.ctx)
|
||||||
span.AddEvent("piece success")
|
span.AddEvent(config.EventPieceReceived, trace.WithAttributes(config.AttributePieceReceived.String(e.pr.String())))
|
||||||
e.peer.AddPieceInfo(e.pr.FinishedCount, int(e.pr.EndTime-e.pr.BeginTime))
|
e.peer.UpdateProgress(e.pr.FinishedCount, int(e.pr.EndTime-e.pr.BeginTime))
|
||||||
oldParent := e.peer.GetParent()
|
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
|
var candidates []*supervisor.Peer
|
||||||
parentPeer, ok := s.peerManager.Get(e.pr.DstPid)
|
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)
|
e.peer.ReplaceParent(nil)
|
||||||
var hasParent bool
|
var hasParent bool
|
||||||
parentPeer, candidates, hasParent = s.sched.ScheduleParent(e.peer)
|
parentPeer, candidates, hasParent = s.sched.ScheduleParent(e.peer)
|
||||||
if !hasParent {
|
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)
|
s.waitScheduleParentPeerQueue.AddAfter(e.peer, time.Second)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
parentPeer.Touch()
|
parentPeer.Touch()
|
||||||
if oldParent != nil {
|
if parentPeer.PeerID == e.pr.DstPid {
|
||||||
candidates = append(candidates, oldParent)
|
return
|
||||||
}
|
}
|
||||||
// TODO if parentPeer is equal with oldParent, need schedule again ?
|
// TODO if parentPeer is equal with oldParent, need schedule again ?
|
||||||
e.peer.SendSchedulePacket(constructSuccessPeerPacket(e.peer, parentPeer, candidates))
|
if err := e.peer.SendSchedulePacket(constructSuccessPeerPacket(e.peer, parentPeer, candidates)); err != nil {
|
||||||
return
|
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 {
|
func (e peerDownloadPieceSuccessEvent) hashKey() string {
|
||||||
|
|
@ -135,6 +170,7 @@ func (e peerDownloadPieceSuccessEvent) hashKey() string {
|
||||||
}
|
}
|
||||||
|
|
||||||
type peerDownloadPieceFailEvent struct {
|
type peerDownloadPieceFailEvent struct {
|
||||||
|
ctx context.Context
|
||||||
peer *supervisor.Peer
|
peer *supervisor.Peer
|
||||||
pr *schedulerRPC.PieceResult
|
pr *schedulerRPC.PieceResult
|
||||||
}
|
}
|
||||||
|
|
@ -142,7 +178,14 @@ type peerDownloadPieceFailEvent struct {
|
||||||
var _ event = peerDownloadPieceFailEvent{}
|
var _ event = peerDownloadPieceFailEvent{}
|
||||||
|
|
||||||
func (e peerDownloadPieceFailEvent) apply(s *state) {
|
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 {
|
switch e.pr.Code {
|
||||||
|
case dfcodes.ClientWaitPieceReady:
|
||||||
|
return
|
||||||
case dfcodes.PeerTaskNotFound, dfcodes.ClientPieceRequestFail, dfcodes.ClientPieceDownloadFail:
|
case dfcodes.PeerTaskNotFound, dfcodes.ClientPieceRequestFail, dfcodes.ClientPieceDownloadFail:
|
||||||
// TODO PeerTaskNotFound remove dest peer task, ClientPieceDownloadFail add blank list
|
// TODO PeerTaskNotFound remove dest peer task, ClientPieceDownloadFail add blank list
|
||||||
reScheduleParent(e.peer, s)
|
reScheduleParent(e.peer, s)
|
||||||
|
|
@ -152,16 +195,17 @@ func (e peerDownloadPieceFailEvent) apply(s *state) {
|
||||||
// TODO
|
// TODO
|
||||||
synclock.Lock(task.TaskID, false)
|
synclock.Lock(task.TaskID, false)
|
||||||
defer synclock.UnLock(task.TaskID, false)
|
defer synclock.UnLock(task.TaskID, false)
|
||||||
task.SetStatus(supervisor.TaskStatusRunning)
|
|
||||||
if cdnPeer, err := s.cdnManager.StartSeedTask(context.Background(), task); err != nil {
|
if cdnPeer, err := s.cdnManager.StartSeedTask(context.Background(), task); err != nil {
|
||||||
logger.Errorf("start seed task fail: %v", err)
|
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)
|
handleSeedTaskFail(task)
|
||||||
} else {
|
} else {
|
||||||
logger.Debugf("===== successfully obtain seeds from cdn, task: %+v =====", e.peer.Task)
|
logger.Debugf("===== successfully obtain seeds from cdn, task: %+v =====", e.peer.Task)
|
||||||
children := s.sched.ScheduleChildren(cdnPeer)
|
children := s.sched.ScheduleChildren(cdnPeer)
|
||||||
for _, child := range children {
|
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)
|
}(e.peer.Task)
|
||||||
|
|
@ -197,12 +241,16 @@ var _ event = peerDownloadSuccessEvent{}
|
||||||
|
|
||||||
func (e peerDownloadSuccessEvent) apply(s *state) {
|
func (e peerDownloadSuccessEvent) apply(s *state) {
|
||||||
e.peer.SetStatus(supervisor.PeerStatusSuccess)
|
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)
|
removePeerFromCurrentTree(e.peer, s)
|
||||||
children := s.sched.ScheduleChildren(e.peer)
|
children := s.sched.ScheduleChildren(e.peer)
|
||||||
for _, child := range children {
|
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 {
|
func (e peerDownloadSuccessEvent) hashKey() string {
|
||||||
|
|
@ -218,16 +266,23 @@ var _ event = peerDownloadFailEvent{}
|
||||||
|
|
||||||
func (e peerDownloadFailEvent) apply(s *state) {
|
func (e peerDownloadFailEvent) apply(s *state) {
|
||||||
e.peer.SetStatus(supervisor.PeerStatusFail)
|
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)
|
removePeerFromCurrentTree(e.peer, s)
|
||||||
e.peer.GetChildren().Range(func(key, value interface{}) bool {
|
e.peer.GetChildren().Range(func(key, value interface{}) bool {
|
||||||
child := (value).(*supervisor.Peer)
|
child := (value).(*supervisor.Peer)
|
||||||
parent, candidates, hasParent := s.sched.ScheduleParent(child)
|
parent, candidates, hasParent := s.sched.ScheduleParent(child)
|
||||||
if !hasParent {
|
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)
|
s.waitScheduleParentPeerQueue.AddAfter(e.peer, time.Second)
|
||||||
return true
|
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
|
return true
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
@ -250,11 +305,13 @@ func (e peerLeaveEvent) apply(s *state) {
|
||||||
child := value.(*supervisor.Peer)
|
child := value.(*supervisor.Peer)
|
||||||
parent, candidates, hasParent := s.sched.ScheduleParent(child)
|
parent, candidates, hasParent := s.sched.ScheduleParent(child)
|
||||||
if !hasParent {
|
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)
|
s.waitScheduleParentPeerQueue.AddAfter(child, time.Second)
|
||||||
return true
|
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
|
return true
|
||||||
})
|
})
|
||||||
s.peerManager.Delete(e.peer.PeerID)
|
s.peerManager.Delete(e.peer.PeerID)
|
||||||
|
|
@ -264,6 +321,7 @@ func (e peerLeaveEvent) hashKey() string {
|
||||||
return e.peer.Task.TaskID
|
return e.peer.Task.TaskID
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// constructSuccessPeerPacket construct success peer schedule packet
|
||||||
func constructSuccessPeerPacket(peer *supervisor.Peer, parent *supervisor.Peer, candidates []*supervisor.Peer) *schedulerRPC.PeerPacket {
|
func constructSuccessPeerPacket(peer *supervisor.Peer, parent *supervisor.Peer, candidates []*supervisor.Peer) *schedulerRPC.PeerPacket {
|
||||||
mainPeer := &schedulerRPC.PeerPacket_DestPeer{
|
mainPeer := &schedulerRPC.PeerPacket_DestPeer{
|
||||||
Ip: parent.Host.IP,
|
Ip: parent.Host.IP,
|
||||||
|
|
@ -290,30 +348,42 @@ func constructSuccessPeerPacket(peer *supervisor.Peer, parent *supervisor.Peer,
|
||||||
return peerPacket
|
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) {
|
func reScheduleParent(peer *supervisor.Peer, s *state) {
|
||||||
parent, candidates, hasParent := s.sched.ScheduleParent(peer)
|
parent, candidates, hasParent := s.sched.ScheduleParent(peer)
|
||||||
if !hasParent {
|
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)
|
//peer.PacketChan <- constructFailPeerPacket(peer, dfcodes.SchedWithoutParentPeer)
|
||||||
s.waitScheduleParentPeerQueue.AddAfter(peer, time.Second)
|
s.waitScheduleParentPeerQueue.AddAfter(peer, time.Second)
|
||||||
return
|
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) {
|
func handleSeedTaskFail(task *supervisor.Task) {
|
||||||
if task.IsFail() {
|
if task.NeedClientBackSource() {
|
||||||
task.ListPeers().Range(func(data sortedlist.Item) bool {
|
task.ListPeers().Range(func(data sortedlist.Item) bool {
|
||||||
peer := data.(*supervisor.Peer)
|
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
|
return true
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
@ -326,7 +396,9 @@ func removePeerFromCurrentTree(peer *supervisor.Peer, s *state) {
|
||||||
if parent != nil {
|
if parent != nil {
|
||||||
children := s.sched.ScheduleChildren(parent)
|
children := s.sched.ScheduleChildren(parent)
|
||||||
for _, child := range children {
|
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) {
|
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) {
|
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
|
return
|
||||||
}
|
}
|
||||||
freeUpload := peer.Host.GetFreeUploadLoad()
|
freeUpload := peer.Host.GetFreeUploadLoad()
|
||||||
candidateChildren := s.selectCandidateChildren(peer, freeUpload*2)
|
candidateChildren := s.selectCandidateChildren(peer, freeUpload*2)
|
||||||
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("select num %d candidate children %v", len(candidateChildren), candidateChildren)
|
if len(candidateChildren) == 0 {
|
||||||
evalResult := make(map[float64]*supervisor.Peer)
|
return nil
|
||||||
|
}
|
||||||
|
evalResult := make(map[float64][]*supervisor.Peer)
|
||||||
var evalScore []float64
|
var evalScore []float64
|
||||||
for _, child := range candidateChildren {
|
for _, child := range candidateChildren {
|
||||||
score := s.evaluator.Evaluate(peer, child)
|
score := s.evaluator.Evaluate(peer, child)
|
||||||
evalResult[score] = child
|
evalResult[score] = append(evalResult[score], child)
|
||||||
evalScore = append(evalScore, score)
|
evalScore = append(evalScore, score)
|
||||||
}
|
}
|
||||||
sort.Float64s(evalScore)
|
sort.Float64s(evalScore)
|
||||||
|
|
@ -90,22 +91,26 @@ func (s *Scheduler) ScheduleChildren(peer *supervisor.Peer) (children []*supervi
|
||||||
if freeUpload <= 0 {
|
if freeUpload <= 0 {
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
child := evalResult[evalScore[len(evalScore)-i-1]]
|
peers := evalResult[evalScore[len(evalScore)-i-1]]
|
||||||
|
for _, child := range peers {
|
||||||
|
if freeUpload <= 0 {
|
||||||
|
break
|
||||||
|
}
|
||||||
if child.GetParent() == peer {
|
if child.GetParent() == peer {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
children = append(children, child)
|
children = append(children, child)
|
||||||
freeUpload--
|
freeUpload--
|
||||||
}
|
}
|
||||||
|
}
|
||||||
for _, child := range children {
|
for _, child := range children {
|
||||||
child.ReplaceParent(peer)
|
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
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Scheduler) ScheduleParent(peer *supervisor.Peer) (*supervisor.Peer, []*supervisor.Peer, bool) {
|
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) {
|
//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)
|
// 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 {
|
// 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
|
// return peer.GetParent(), []*types.Peer{peer.GetParent()}, true
|
||||||
//}
|
//}
|
||||||
candidateParents := s.selectCandidateParents(peer, s.cfg.CandidateParentCount)
|
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 {
|
if len(candidateParents) == 0 {
|
||||||
return nil, nil, false
|
return nil, nil, false
|
||||||
}
|
}
|
||||||
evalResult := make(map[float64]*supervisor.Peer)
|
evalResult := make(map[float64][]*supervisor.Peer)
|
||||||
var evalScore []float64
|
var evalScore []float64
|
||||||
for _, candidate := range candidateParents {
|
for _, parent := range candidateParents {
|
||||||
score := s.evaluator.Evaluate(candidate, peer)
|
score := s.evaluator.Evaluate(parent, peer)
|
||||||
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("evaluate score candidate %s is %f", candidate.PeerID, score)
|
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("evaluate score candidate %s is %f", parent.PeerID, score)
|
||||||
evalResult[score] = candidate
|
evalResult[score] = append(evalResult[score], parent)
|
||||||
evalScore = append(evalScore, score)
|
evalScore = append(evalScore, score)
|
||||||
}
|
}
|
||||||
sort.Float64s(evalScore)
|
sort.Float64s(evalScore)
|
||||||
var parents = make([]*supervisor.Peer, 0, len(candidateParents))
|
var parents = make([]*supervisor.Peer, 0, len(candidateParents))
|
||||||
for i := range evalScore {
|
for i := range evalScore {
|
||||||
parent := evalResult[evalScore[len(evalScore)-i-1]]
|
parents = append(parents, evalResult[evalScore[len(evalScore)-i-1]]...)
|
||||||
parents = append(parents, parent)
|
|
||||||
}
|
}
|
||||||
if parents[0] != peer.GetParent() {
|
if parents[0] != peer.GetParent() {
|
||||||
peer.ReplaceParent(parents[0])
|
peer.ReplaceParent(parents[0])
|
||||||
|
|
@ -140,108 +142,132 @@ func (s *Scheduler) ScheduleParent(peer *supervisor.Peer) (*supervisor.Peer, []*
|
||||||
return parents[0], parents[1:], true
|
return parents[0], parents[1:], true
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Scheduler) selectCandidateChildren(peer *supervisor.Peer, limit int) (list []*supervisor.Peer) {
|
func (s *Scheduler) selectCandidateChildren(peer *supervisor.Peer, limit int) (candidateChildren []*supervisor.Peer) {
|
||||||
return s.peerManager.Pick(peer.Task, limit, func(candidateNode *supervisor.Peer) bool {
|
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 {
|
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
|
return false
|
||||||
}
|
}
|
||||||
// TODO IsWaiting
|
|
||||||
if candidateNode.IsDone() {
|
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)
|
candidateNode.PeerID)
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if candidateNode.IsLeave() {
|
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)
|
candidateNode.PeerID)
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if candidateNode == peer {
|
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)
|
candidateNode.PeerID)
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if candidateNode.IsAncestorOf(peer) {
|
if candidateNode.IsAncestorOf(peer) {
|
||||||
logger.WithTaskAndPeerID(peer.Task.TaskID,
|
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
|
return false
|
||||||
}
|
}
|
||||||
if candidateNode.GetFinishedNum() > peer.GetFinishedNum() {
|
if candidateNode.GetFinishedNum() >= peer.GetFinishedNum() {
|
||||||
logger.WithTaskAndPeerID(peer.Task.TaskID,
|
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)
|
candidateNode.PeerID)
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if candidateNode.Host != nil && candidateNode.Host.CDN {
|
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)
|
candidateNode.PeerID)
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if candidateNode.GetParent() == nil {
|
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)
|
candidateNode.PeerID)
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
if candidateNode.GetParent() != nil && s.evaluator.IsBadNode(candidateNode.GetParent()) {
|
if candidateNode.GetParent() != nil && s.evaluator.IsBadNode(candidateNode.GetParent()) {
|
||||||
logger.WithTaskAndPeerID(peer.Task.TaskID,
|
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
|
return true
|
||||||
}
|
}
|
||||||
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is selected", candidateNode.PeerID)
|
logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******[default]candidate child peer %s is selected[default]******", candidateNode.PeerID)
|
||||||
return false
|
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() {
|
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 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 {
|
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
|
return false
|
||||||
}
|
}
|
||||||
if s.evaluator.IsBadNode(candidateNode) {
|
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)
|
candidateNode.PeerID)
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if candidateNode.IsLeave() {
|
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)
|
candidateNode.PeerID)
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if candidateNode == peer {
|
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)
|
candidateNode.PeerID)
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if candidateNode.IsDescendantOf(peer) {
|
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)
|
candidateNode.PeerID)
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if candidateNode.Host.GetFreeUploadLoad() <= 0 {
|
if candidateNode.Host.GetFreeUploadLoad() <= 0 {
|
||||||
logger.WithTaskAndPeerID(peer.Task.TaskID,
|
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)
|
candidateNode.PeerID)
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if candidateNode.IsWaiting() {
|
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)
|
candidateNode.PeerID)
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if candidateNode.GetFinishedNum() < peer.GetFinishedNum() {
|
if candidateNode.GetFinishedNum() <= peer.GetFinishedNum() {
|
||||||
logger.WithTaskAndPeerID(peer.Task.TaskID,
|
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)
|
candidateNode.PeerID)
|
||||||
return false
|
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 true
|
||||||
})
|
})
|
||||||
|
return
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -42,6 +42,25 @@ import (
|
||||||
"k8s.io/client-go/util/workqueue"
|
"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 {
|
type SchedulerService struct {
|
||||||
// cdn mgr
|
// cdn mgr
|
||||||
cdnManager supervisor.CDNMgr
|
cdnManager supervisor.CDNMgr
|
||||||
|
|
@ -62,22 +81,14 @@ type SchedulerService struct {
|
||||||
wg sync.WaitGroup
|
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()
|
hostManager := host.NewManager()
|
||||||
peerManager := peer.NewManager(cfg.GC, hostManager)
|
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)
|
taskManager := task.NewManager(cfg.GC, peerManager)
|
||||||
sched, err := scheduler.Get(cfg.Scheduler).Build(cfg, &scheduler.BuildOptions{
|
sched, err := scheduler.Get(cfg.Scheduler).Build(cfg, &scheduler.BuildOptions{
|
||||||
PeerManager: peerManager,
|
PeerManager: peerManager,
|
||||||
|
|
@ -89,8 +100,7 @@ func NewSchedulerService(cfg *config.SchedulerConfig, dynConfig config.Dynconfig
|
||||||
work := newEventLoopGroup(cfg.WorkerNum)
|
work := newEventLoopGroup(cfg.WorkerNum)
|
||||||
downloadMonitor := newMonitor(cfg.OpenMonitor, peerManager)
|
downloadMonitor := newMonitor(cfg.OpenMonitor, peerManager)
|
||||||
done := make(chan struct{})
|
done := make(chan struct{})
|
||||||
return &SchedulerService{
|
s := &SchedulerService{
|
||||||
cdnManager: cdnManager,
|
|
||||||
taskManager: taskManager,
|
taskManager: taskManager,
|
||||||
hostManager: hostManager,
|
hostManager: hostManager,
|
||||||
peerManager: peerManager,
|
peerManager: peerManager,
|
||||||
|
|
@ -99,7 +109,23 @@ func NewSchedulerService(cfg *config.SchedulerConfig, dynConfig config.Dynconfig
|
||||||
sched: sched,
|
sched: sched,
|
||||||
config: cfg,
|
config: cfg,
|
||||||
done: done,
|
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() {
|
func (s *SchedulerService) Serve() {
|
||||||
|
|
@ -138,6 +164,12 @@ func (s *SchedulerService) runReScheduleParentLoop(wsdq workqueue.DelayingInterf
|
||||||
"isLeave %t", peer.GetStatus(), peer.IsLeave())
|
"isLeave %t", peer.GetStatus(), peer.IsLeave())
|
||||||
continue
|
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})
|
s.worker.send(reScheduleParentEvent{peer})
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -155,6 +187,7 @@ func (s *SchedulerService) Stop() {
|
||||||
if s.worker != nil {
|
if s.worker != nil {
|
||||||
s.worker.stop()
|
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)
|
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)
|
parent, _, hasParent := s.sched.ScheduleParent(peer)
|
||||||
//logger.Debugf("schedule parent result: parent %v, candidates:%v", parent, candidates)
|
|
||||||
if !hasParent || parent == nil {
|
if !hasParent || parent == nil {
|
||||||
return nil, errors.Errorf("no parent peer available for peer %v", peer.PeerID)
|
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)
|
return s.peerManager.Get(peerTaskID)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *SchedulerService) RegisterPeerTask(req *schedulerRPC.PeerTaskRequest, task *supervisor.Task) (*supervisor.Peer, error) {
|
func (s *SchedulerService) RegisterPeerTask(req *schedulerRPC.PeerTaskRequest, task *supervisor.Task) *supervisor.Peer {
|
||||||
var (
|
// get or create host
|
||||||
peer *supervisor.Peer
|
|
||||||
ok bool
|
|
||||||
peerHost *supervisor.PeerHost
|
|
||||||
)
|
|
||||||
reqPeerHost := req.PeerHost
|
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,
|
peerHost = supervisor.NewClientPeerHost(reqPeerHost.Uuid, reqPeerHost.Ip, reqPeerHost.HostName, reqPeerHost.RpcPort, reqPeerHost.DownPort,
|
||||||
reqPeerHost.SecurityDomain, reqPeerHost.Location, reqPeerHost.Idc, reqPeerHost.NetTopology, s.config.ClientLoad)
|
reqPeerHost.SecurityDomain, reqPeerHost.Location, reqPeerHost.Idc, reqPeerHost.NetTopology, s.config.ClientLoad)
|
||||||
s.hostManager.Add(peerHost)
|
s.hostManager.Add(peerHost)
|
||||||
}
|
}
|
||||||
|
|
||||||
// get or creat PeerTask
|
// get or creat PeerTask
|
||||||
if peer, ok = s.peerManager.Get(req.PeerId); !ok {
|
peer, ok := s.peerManager.Get(req.PeerId)
|
||||||
|
if ok {
|
||||||
|
logger.Warnf("peer %s has already registered", peer.PeerID)
|
||||||
|
return peer
|
||||||
|
}
|
||||||
peer = supervisor.NewPeer(req.PeerId, task, peerHost)
|
peer = supervisor.NewPeer(req.PeerId, task, peerHost)
|
||||||
s.peerManager.Add(peer)
|
s.peerManager.Add(peer)
|
||||||
}
|
return peer
|
||||||
return peer, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
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)
|
span := trace.SpanFromContext(ctx)
|
||||||
synclock.Lock(task.TaskID, true)
|
synclock.Lock(task.TaskID, true)
|
||||||
task, ok := s.taskManager.GetOrAdd(task)
|
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() {
|
if task.GetLastTriggerTime().Add(s.config.AccessWindow).After(time.Now()) || task.IsHealth() {
|
||||||
synclock.UnLock(task.TaskID, true)
|
synclock.UnLock(task.TaskID, true)
|
||||||
span.SetAttributes(config.AttributeNeedSeedCDN.Bool(false))
|
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)
|
synclock.UnLock(task.TaskID, true)
|
||||||
// do trigger
|
// do trigger
|
||||||
task.UpdateLastTriggerTime(time.Now())
|
task.UpdateLastTriggerTime(time.Now())
|
||||||
// register cdn peer task
|
|
||||||
// notify peer tasks
|
|
||||||
synclock.Lock(task.TaskID, false)
|
synclock.Lock(task.TaskID, false)
|
||||||
defer synclock.UnLock(task.TaskID, false)
|
defer synclock.UnLock(task.TaskID, false)
|
||||||
if task.IsHealth() && task.GetLastTriggerTime().Add(s.config.AccessWindow).After(time.Now()) {
|
if task.IsHealth() {
|
||||||
return task, nil
|
|
||||||
}
|
|
||||||
if task.IsFrozen() {
|
|
||||||
task.SetStatus(supervisor.TaskStatusRunning)
|
|
||||||
span.SetAttributes(config.AttributeNeedSeedCDN.Bool(false))
|
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))
|
span.SetAttributes(config.AttributeNeedSeedCDN.Bool(true))
|
||||||
go func() {
|
go func() {
|
||||||
if cdnPeer, err := s.cdnManager.StartSeedTask(ctx, task); err != nil {
|
if cdnPeer, err := s.cdnManager.StartSeedTask(ctx, task); err != nil {
|
||||||
if errors.Cause(err) != cdn.ErrCDNInvokeFail {
|
// fall back to client back source
|
||||||
task.SetStatus(supervisor.TaskStatusFailed)
|
logger.Errorf("seed task failed: %v", err)
|
||||||
}
|
span.AddEvent(config.EventCDNFailBackClientSource, trace.WithAttributes(config.AttributeTriggerCDNError.String(err.Error())))
|
||||||
logger.Errorf("failed to seed task: %v", err)
|
task.SetClientBackSourceStatusAndLimit(s.config.BackSourceCount)
|
||||||
if ok = s.worker.send(taskSeedFailEvent{task}); !ok {
|
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 {
|
} else {
|
||||||
if ok = s.worker.send(peerDownloadSuccessEvent{cdnPeer, nil}); !ok {
|
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)
|
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 {
|
func (s *SchedulerService) HandlePieceResult(ctx context.Context, peer *supervisor.Peer, pieceResult *schedulerRPC.PieceResult) error {
|
||||||
peer.Touch()
|
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})
|
s.worker.send(startReportPieceResultEvent{ctx, peer})
|
||||||
return nil
|
return nil
|
||||||
} else if pieceResult.Success {
|
} else if pieceResult.Success {
|
||||||
|
|
@ -258,6 +298,7 @@ func (s *SchedulerService) HandlePieceResult(ctx context.Context, peer *supervis
|
||||||
return nil
|
return nil
|
||||||
} else if pieceResult.Code != dfcodes.Success {
|
} else if pieceResult.Code != dfcodes.Success {
|
||||||
s.worker.send(peerDownloadPieceFailEvent{
|
s.worker.send(peerDownloadPieceFailEvent{
|
||||||
|
ctx: ctx,
|
||||||
peer: peer,
|
peer: peer,
|
||||||
pr: pieceResult,
|
pr: pieceResult,
|
||||||
})
|
})
|
||||||
|
|
|
||||||
|
|
@ -31,7 +31,6 @@ type worker interface {
|
||||||
type workerGroup struct {
|
type workerGroup struct {
|
||||||
workerNum int
|
workerNum int
|
||||||
workerList []*baseWorker
|
workerList []*baseWorker
|
||||||
stopCh chan struct{}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
var _ worker = (*workerGroup)(nil)
|
var _ worker = (*workerGroup)(nil)
|
||||||
|
|
@ -40,7 +39,6 @@ func newEventLoopGroup(workerNum int) worker {
|
||||||
return &workerGroup{
|
return &workerGroup{
|
||||||
workerNum: workerNum,
|
workerNum: workerNum,
|
||||||
workerList: make([]*baseWorker, 0, 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() {
|
func (wg *workerGroup) stop() {
|
||||||
close(wg.stopCh)
|
|
||||||
for _, worker := range wg.workerList {
|
for _, worker := range wg.workerList {
|
||||||
worker.stop()
|
worker.stop()
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -20,8 +20,6 @@ import (
|
||||||
"context"
|
"context"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"d7y.io/dragonfly/v2/internal/dfcodes"
|
|
||||||
"d7y.io/dragonfly/v2/internal/dferrors"
|
|
||||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||||
"d7y.io/dragonfly/v2/internal/idgen"
|
"d7y.io/dragonfly/v2/internal/idgen"
|
||||||
internaljob "d7y.io/dragonfly/v2/internal/job"
|
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)
|
logger.Debugf("ready to preheat \"%s\", taskID = %s", request.URL, taskID)
|
||||||
|
|
||||||
task := supervisor.NewTask(taskID, request.URL, meta)
|
task := supervisor.NewTask(taskID, request.URL, meta)
|
||||||
task, err := t.service.GetOrCreateTask(t.ctx, task)
|
task = t.service.GetOrCreateTask(t.ctx, task)
|
||||||
if err != nil {
|
|
||||||
return dferrors.Newf(dfcodes.SchedCDNSeedFail, "create task failed: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
return getPreheatResult(task)
|
return getPreheatResult(task)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -175,7 +169,7 @@ func getPreheatResult(task *supervisor.Task) error {
|
||||||
select {
|
select {
|
||||||
case <-ticker.C:
|
case <-ticker.C:
|
||||||
switch task.GetStatus() {
|
switch task.GetStatus() {
|
||||||
case supervisor.TaskStatusFailed, supervisor.TaskStatusCDNRegisterFail, supervisor.TaskStatusSourceError:
|
case supervisor.TaskStatusFail:
|
||||||
return errors.Errorf("preheat task fail")
|
return errors.Errorf("preheat task fail")
|
||||||
case supervisor.TaskStatusSuccess:
|
case supervisor.TaskStatusSuccess:
|
||||||
return nil
|
return nil
|
||||||
|
|
|
||||||
|
|
@ -20,13 +20,11 @@ import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
"io"
|
"io"
|
||||||
"sync"
|
|
||||||
|
|
||||||
"d7y.io/dragonfly/v2/internal/dfcodes"
|
"d7y.io/dragonfly/v2/internal/dfcodes"
|
||||||
"d7y.io/dragonfly/v2/internal/dferrors"
|
"d7y.io/dragonfly/v2/internal/dferrors"
|
||||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||||
"d7y.io/dragonfly/v2/pkg/rpc/base/common"
|
|
||||||
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
|
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
|
||||||
"d7y.io/dragonfly/v2/pkg/rpc/scheduler/server"
|
"d7y.io/dragonfly/v2/pkg/rpc/scheduler/server"
|
||||||
"d7y.io/dragonfly/v2/pkg/util/net/urlutils"
|
"d7y.io/dragonfly/v2/pkg/util/net/urlutils"
|
||||||
|
|
@ -36,9 +34,6 @@ import (
|
||||||
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
||||||
"go.opentelemetry.io/otel"
|
"go.opentelemetry.io/otel"
|
||||||
"go.opentelemetry.io/otel/trace"
|
"go.opentelemetry.io/otel/trace"
|
||||||
"golang.org/x/sync/errgroup"
|
|
||||||
"google.golang.org/grpc/codes"
|
|
||||||
"google.golang.org/grpc/status"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
var tracer trace.Tracer
|
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)
|
taskID := s.service.GenerateTaskID(request.Url, request.UrlMeta, request.PeerId)
|
||||||
span.SetAttributes(config.AttributeTaskID.String(taskID))
|
span.SetAttributes(config.AttributeTaskID.String(taskID))
|
||||||
task := supervisor.NewTask(taskID, request.Url, request.UrlMeta)
|
task := s.service.GetOrCreateTask(ctx, 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
|
|
||||||
}
|
|
||||||
if task.IsFail() {
|
if task.IsFail() {
|
||||||
err = dferrors.Newf(dfcodes.SchedTaskStatusError, "task status is %s", task.GetStatus())
|
err = dferrors.New(dfcodes.SchedTaskStatusError, "task status is fail")
|
||||||
logger.Errorf("task status is unhealthy, task status is: %s", task.GetStatus())
|
logger.Error("task %s status is fail", task.TaskID)
|
||||||
span.RecordError(err)
|
span.RecordError(err)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
resp.SizeScope = getTaskSizeScope(task)
|
resp.SizeScope = getTaskSizeScope(task)
|
||||||
|
span.SetAttributes(config.AttributeTaskSizeScope.String(resp.SizeScope.String()))
|
||||||
resp.TaskId = taskID
|
resp.TaskId = taskID
|
||||||
switch resp.SizeScope {
|
switch resp.SizeScope {
|
||||||
case base.SizeScope_TINY:
|
case base.SizeScope_TINY:
|
||||||
resp.DirectPiece = &scheduler.RegisterResult_PieceContent{
|
resp.DirectPiece = &scheduler.RegisterResult_PieceContent{
|
||||||
PieceContent: task.DirectPiece,
|
PieceContent: task.DirectPiece,
|
||||||
}
|
}
|
||||||
span.SetAttributes(config.AttributeTaskSizeScope.String(resp.SizeScope.String()))
|
|
||||||
return
|
return
|
||||||
case base.SizeScope_SMALL:
|
case base.SizeScope_SMALL:
|
||||||
span.SetAttributes(config.AttributeTaskSizeScope.String(resp.SizeScope.String()))
|
peer := s.service.RegisterPeerTask(request, task)
|
||||||
peer, regErr := s.service.RegisterPeerTask(request, task)
|
parent, schErr := s.service.SelectParent(peer)
|
||||||
if regErr != nil {
|
|
||||||
err = dferrors.Newf(dfcodes.SchedPeerRegisterFail, "failed to register peer: %v", regErr)
|
|
||||||
span.RecordError(err)
|
|
||||||
return
|
|
||||||
}
|
|
||||||
parent, schErr := s.service.ScheduleParent(peer)
|
|
||||||
if schErr != nil {
|
if schErr != nil {
|
||||||
span.SetAttributes(config.AttributeTaskSizeScope.String(base.SizeScope_NORMAL.String()))
|
span.AddEvent(config.EventSmallTaskSelectParentFail)
|
||||||
span.AddEvent(config.EventScheduleParentFail)
|
|
||||||
resp.SizeScope = base.SizeScope_NORMAL
|
resp.SizeScope = base.SizeScope_NORMAL
|
||||||
resp.TaskId = taskID
|
resp.TaskId = taskID
|
||||||
//err = dferrors.Newf(dfcodes.SchedPeerScheduleFail, "failed to schedule peer %v: %v", peer.PeerID, schErr)
|
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
firstPiece := task.GetPiece(0)
|
firstPiece := task.GetPiece(0)
|
||||||
|
|
@ -135,12 +115,7 @@ func (s *SchedulerServer) RegisterPeerTask(ctx context.Context, request *schedul
|
||||||
span.SetAttributes(config.AttributeSinglePiece.String(singlePiece.String()))
|
span.SetAttributes(config.AttributeSinglePiece.String(singlePiece.String()))
|
||||||
return
|
return
|
||||||
default:
|
default:
|
||||||
span.SetAttributes(config.AttributeTaskSizeScope.String(resp.SizeScope.String()))
|
s.service.RegisterPeerTask(request, task)
|
||||||
_, regErr := s.service.RegisterPeerTask(request, task)
|
|
||||||
if regErr != nil {
|
|
||||||
err = dferrors.Newf(dfcodes.SchedPeerRegisterFail, "failed to register peer: %v", regErr)
|
|
||||||
span.RecordError(regErr)
|
|
||||||
}
|
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -149,90 +124,52 @@ func (s *SchedulerServer) ReportPieceResult(stream scheduler.Scheduler_ReportPie
|
||||||
var span trace.Span
|
var span trace.Span
|
||||||
ctx, span := tracer.Start(stream.Context(), config.SpanReportPieceResult, trace.WithSpanKind(trace.SpanKindServer))
|
ctx, span := tracer.Start(stream.Context(), config.SpanReportPieceResult, trace.WithSpanKind(trace.SpanKindServer))
|
||||||
defer span.End()
|
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()
|
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 err != nil {
|
||||||
if status.Code(err) == codes.Canceled {
|
if err == io.EOF {
|
||||||
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 nil
|
||||||
}
|
}
|
||||||
|
err = dferrors.Newf(dfcodes.SchedPeerPieceResultReportFail, "receive an error from peer stream: %v", err)
|
||||||
|
span.RecordError(err)
|
||||||
|
return err
|
||||||
}
|
}
|
||||||
return dferrors.Newf(dfcodes.SchedPeerPieceResultReportFail, "peer piece result report error: %v", err)
|
logger.Debugf("peer %s start report piece result", pieceResult.SrcPid)
|
||||||
}
|
peer, ok := s.service.GetPeerTask(pieceResult.SrcPid)
|
||||||
logger.Debugf("report piece result %v of peer %s", pieceResult, pieceResult.SrcPid)
|
|
||||||
var ok bool
|
|
||||||
peer, ok = s.service.GetPeerTask(pieceResult.SrcPid)
|
|
||||||
if !ok {
|
if !ok {
|
||||||
return dferrors.Newf(dfcodes.SchedPeerNotFound, "peer %s not found", pieceResult.SrcPid)
|
err = dferrors.Newf(dfcodes.SchedPeerNotFound, "peer %s not found", pieceResult.SrcPid)
|
||||||
|
span.RecordError(err)
|
||||||
|
return err
|
||||||
}
|
}
|
||||||
if !initialized {
|
if peer.Task.IsFail() {
|
||||||
peer.BindSendChannel(peerPacketChan)
|
err = dferrors.Newf(dfcodes.SchedTaskStatusError, "peer's task status is fail, task status %s", peer.Task.GetStatus())
|
||||||
peer.SetStatus(supervisor.PeerStatusRunning)
|
span.RecordError(err)
|
||||||
initialized = true
|
return err
|
||||||
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 {
|
if err := s.service.HandlePieceResult(ctx, peer, pieceResult); err != nil {
|
||||||
logger.Errorf("handle piece result %v fail: %v", pieceResult, err)
|
logger.Errorf("peer %s handle piece result %v fail: %v", peer.PeerID, pieceResult, err)
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
})
|
|
||||||
|
|
||||||
g.Go(func() error {
|
}
|
||||||
defer func() {
|
conn := peer.BindNewConn(stream)
|
||||||
cancel()
|
logger.Infof("peer %s is connected", peer.PeerID)
|
||||||
once.Do(peer.UnBindSendChannel)
|
defer logger.Infof("peer %s is disconnect", peer.PeerID)
|
||||||
}()
|
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case <-ctx.Done():
|
case <-conn.Done():
|
||||||
return nil
|
return conn.Err()
|
||||||
case pp, ok := <-peerPacketChan:
|
case piece := <-conn.Receiver():
|
||||||
if !ok {
|
if piece == nil {
|
||||||
span.AddEvent("exit report piece process due to send channel has closed")
|
logger.Infof("peer %s channel has been closed", peer.PeerID)
|
||||||
return nil
|
continue
|
||||||
}
|
}
|
||||||
span.AddEvent("schedule event", trace.WithAttributes(config.AttributeSchedulePacket.String(pp.String())))
|
if err := s.service.HandlePieceResult(ctx, peer, piece); err != nil {
|
||||||
err := stream.Send(pp)
|
logger.Errorf("peer %s handle piece result %v fail: %v", peer.PeerID, piece, err)
|
||||||
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)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
})
|
|
||||||
err := g.Wait()
|
|
||||||
logger.Debugf("report piece result: %v", err)
|
|
||||||
return err
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *SchedulerServer) ReportPeerResult(ctx context.Context, result *scheduler.PeerResult) (err error) {
|
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
|
var span trace.Span
|
||||||
ctx, span = tracer.Start(ctx, config.SpanReportPeerResult, trace.WithSpanKind(trace.SpanKindServer))
|
ctx, span = tracer.Start(ctx, config.SpanReportPeerResult, trace.WithSpanKind(trace.SpanKindServer))
|
||||||
defer span.End()
|
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)
|
logger.Warnf("report peer result: peer %s is not exists", result.PeerId)
|
||||||
err = dferrors.Newf(dfcodes.SchedPeerNotFound, "peer %s not found", result.PeerId)
|
err = dferrors.Newf(dfcodes.SchedPeerNotFound, "peer %s not found", result.PeerId)
|
||||||
span.RecordError(err)
|
span.RecordError(err)
|
||||||
|
return err
|
||||||
}
|
}
|
||||||
return s.service.HandlePeerResult(ctx, peer, result)
|
return s.service.HandlePeerResult(ctx, peer, result)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *SchedulerServer) LeaveTask(ctx context.Context, target *scheduler.PeerTarget) (err error) {
|
func (s *SchedulerServer) LeaveTask(ctx context.Context, target *scheduler.PeerTarget) (err error) {
|
||||||
|
logger.Debugf("leave task %v", target)
|
||||||
var span trace.Span
|
var span trace.Span
|
||||||
ctx, span = tracer.Start(ctx, config.SpanPeerLeave, trace.WithSpanKind(trace.SpanKindServer))
|
ctx, span = tracer.Start(ctx, config.SpanPeerLeave, trace.WithSpanKind(trace.SpanKindServer))
|
||||||
defer span.End()
|
defer span.End()
|
||||||
span.SetAttributes(config.AttributeLeavePeerID.String(target.PeerId))
|
span.SetAttributes(config.AttributeLeavePeerID.String(target.PeerId))
|
||||||
span.SetAttributes(config.AttributeLeaveTaskID.String(target.TaskId))
|
span.SetAttributes(config.AttributeLeaveTaskID.String(target.TaskId))
|
||||||
logger.Debugf("leave task %+v", target)
|
|
||||||
peer, ok := s.service.GetPeerTask(target.PeerId)
|
peer, ok := s.service.GetPeerTask(target.PeerId)
|
||||||
if !ok {
|
if !ok {
|
||||||
logger.Warnf("leave task: peer %s is not exists", target.PeerId)
|
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 != "" {
|
if cfg.Options.Telemetry.Jaeger != "" {
|
||||||
openTel = true
|
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 {
|
if err != nil {
|
||||||
return nil, err
|
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) {
|
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
|
var seedSpan trace.Span
|
||||||
ctx, seedSpan = tracer.Start(ctx, config.SpanTriggerCDN)
|
ctx, seedSpan = tracer.Start(ctx, config.SpanTriggerCDN)
|
||||||
defer seedSpan.End()
|
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))
|
span.SetAttributes(config.AttributeContentLength.Int64(task.ContentLength))
|
||||||
return cdnPeer, nil
|
return cdnPeer, nil
|
||||||
}
|
}
|
||||||
cdnPeer.AddPieceInfo(piece.PieceInfo.PieceNum+1, 0)
|
cdnPeer.UpdateProgress(piece.PieceInfo.PieceNum+1, 0)
|
||||||
task.AddPiece(&supervisor.PieceInfo{
|
task.AddPiece(piece.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,
|
|
||||||
})
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -17,10 +17,12 @@
|
||||||
package supervisor
|
package supervisor
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"io"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
|
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
|
||||||
|
"github.com/pkg/errors"
|
||||||
"go.uber.org/atomic"
|
"go.uber.org/atomic"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
@ -32,12 +34,12 @@ func (status PeerStatus) String() string {
|
||||||
return "Waiting"
|
return "Waiting"
|
||||||
case PeerStatusRunning:
|
case PeerStatusRunning:
|
||||||
return "Running"
|
return "Running"
|
||||||
case PeerStatusSuccess:
|
|
||||||
return "Success"
|
|
||||||
case PeerStatusFail:
|
|
||||||
return "Fail"
|
|
||||||
case PeerStatusZombie:
|
case PeerStatusZombie:
|
||||||
return "Zombie"
|
return "Zombie"
|
||||||
|
case PeerStatusFail:
|
||||||
|
return "Fail"
|
||||||
|
case PeerStatusSuccess:
|
||||||
|
return "Success"
|
||||||
default:
|
default:
|
||||||
return "unknown"
|
return "unknown"
|
||||||
}
|
}
|
||||||
|
|
@ -46,6 +48,7 @@ func (status PeerStatus) String() string {
|
||||||
const (
|
const (
|
||||||
PeerStatusWaiting PeerStatus = iota
|
PeerStatusWaiting PeerStatus = iota
|
||||||
PeerStatusRunning
|
PeerStatusRunning
|
||||||
|
// TODO add Seeding status
|
||||||
PeerStatusZombie
|
PeerStatusZombie
|
||||||
PeerStatusFail
|
PeerStatusFail
|
||||||
PeerStatusSuccess
|
PeerStatusSuccess
|
||||||
|
|
@ -59,10 +62,7 @@ type Peer struct {
|
||||||
Task *Task
|
Task *Task
|
||||||
// Host specifies
|
// Host specifies
|
||||||
Host *PeerHost
|
Host *PeerHost
|
||||||
// bindPacketChan
|
conn *Channel
|
||||||
bindPacketChan bool
|
|
||||||
// PacketChan send schedulerPacket to peer client
|
|
||||||
packetChan chan *scheduler.PeerPacket
|
|
||||||
// createTime
|
// createTime
|
||||||
CreateTime time.Time
|
CreateTime time.Time
|
||||||
// finishedNum specifies downloaded finished piece number
|
// finishedNum specifies downloaded finished piece number
|
||||||
|
|
@ -155,7 +155,7 @@ func (peer *Peer) GetCost() int {
|
||||||
return totalCost / len(peer.costHistory)
|
return totalCost / len(peer.costHistory)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (peer *Peer) AddPieceInfo(finishedCount int32, cost int) {
|
func (peer *Peer) UpdateProgress(finishedCount int32, cost int) {
|
||||||
peer.lock.Lock()
|
peer.lock.Lock()
|
||||||
if finishedCount > peer.finishedNum.Load() {
|
if finishedCount > peer.finishedNum.Load() {
|
||||||
peer.finishedNum.Store(finishedCount)
|
peer.finishedNum.Store(finishedCount)
|
||||||
|
|
@ -254,38 +254,6 @@ func (peer *Peer) GetChildren() *sync.Map {
|
||||||
return &peer.children
|
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) {
|
func (peer *Peer) SetStatus(status PeerStatus) {
|
||||||
peer.lock.Lock()
|
peer.lock.Lock()
|
||||||
defer peer.lock.Unlock()
|
defer peer.lock.Unlock()
|
||||||
|
|
@ -333,3 +301,163 @@ func (peer *Peer) IsBad() bool {
|
||||||
func (peer *Peer) IsFail() bool {
|
func (peer *Peer) IsFail() bool {
|
||||||
return peer.status == PeerStatusFail
|
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"
|
"time"
|
||||||
|
|
||||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
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/config"
|
||||||
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
||||||
)
|
)
|
||||||
|
|
@ -88,69 +87,36 @@ func (m *manager) Delete(peerID string) {
|
||||||
if ok {
|
if ok {
|
||||||
peer.Host.DeletePeer(peerID)
|
peer.Host.DeletePeer(peerID)
|
||||||
peer.Task.DeletePeer(peer)
|
peer.Task.DeletePeer(peer)
|
||||||
peer.UnBindSendChannel()
|
|
||||||
peer.ReplaceParent(nil)
|
peer.ReplaceParent(nil)
|
||||||
m.peerMap.Delete(peerID)
|
m.peerMap.Delete(peerID)
|
||||||
}
|
}
|
||||||
return
|
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() {
|
func (m *manager) cleanupPeers() {
|
||||||
for range m.cleanupExpiredPeerTicker.C {
|
for range m.cleanupExpiredPeerTicker.C {
|
||||||
m.peerMap.Range(func(key, value interface{}) bool {
|
m.peerMap.Range(func(key, value interface{}) bool {
|
||||||
|
peerID := key.(string)
|
||||||
peer := value.(*supervisor.Peer)
|
peer := value.(*supervisor.Peer)
|
||||||
elapse := time.Since(peer.GetLastAccessTime())
|
elapse := time.Since(peer.GetLastAccessTime())
|
||||||
if elapse > m.peerTTI && !peer.IsDone() {
|
if elapse > m.peerTTI && !peer.IsDone() && !peer.Host.CDN {
|
||||||
if !peer.IsBindSendChannel() {
|
if !peer.IsConnected() {
|
||||||
peer.MarkLeave()
|
peer.MarkLeave()
|
||||||
}
|
}
|
||||||
logger.Debugf("peer %s has been more than %s since last access, set status to zombie", peer.PeerID, m.peerTTI)
|
logger.Debugf("peer %s has been more than %s since last access, set status to zombie", peer.PeerID, m.peerTTI)
|
||||||
peer.SetStatus(supervisor.PeerStatusZombie)
|
peer.SetStatus(supervisor.PeerStatusZombie)
|
||||||
}
|
}
|
||||||
if peer.IsLeave() || peer.IsFail() || elapse > m.peerTTL {
|
if peer.IsLeave() || peer.IsFail() || elapse > m.peerTTL {
|
||||||
|
if elapse > m.peerTTL {
|
||||||
logger.Debugf("delete peer %s because %s have passed since last access", peer.PeerID)
|
logger.Debugf("delete peer %s because %s have passed since last access", peer.PeerID)
|
||||||
m.Delete(key.(string))
|
}
|
||||||
|
m.Delete(peerID)
|
||||||
if peer.Host.GetPeerTaskNum() == 0 {
|
if peer.Host.GetPeerTaskNum() == 0 {
|
||||||
m.hostManager.Delete(peer.Host.UUID)
|
m.hostManager.Delete(peer.Host.UUID)
|
||||||
}
|
}
|
||||||
|
if peer.Task.ListPeers().Size() == 0 {
|
||||||
|
peer.Task.SetStatus(supervisor.TaskStatusWaiting)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return true
|
return true
|
||||||
})
|
})
|
||||||
|
|
|
||||||
|
|
@ -30,8 +30,4 @@ type PeerMgr interface {
|
||||||
ListPeersByTask(taskID string) []*Peer
|
ListPeersByTask(taskID string) []*Peer
|
||||||
|
|
||||||
ListPeers() *sync.Map
|
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/rpc/base"
|
||||||
"d7y.io/dragonfly/v2/pkg/structure/sortedlist"
|
"d7y.io/dragonfly/v2/pkg/structure/sortedlist"
|
||||||
|
"go.uber.org/atomic"
|
||||||
)
|
)
|
||||||
|
|
||||||
type TaskStatus uint8
|
type TaskStatus uint8
|
||||||
|
|
@ -32,16 +33,14 @@ func (status TaskStatus) String() string {
|
||||||
return "Waiting"
|
return "Waiting"
|
||||||
case TaskStatusRunning:
|
case TaskStatusRunning:
|
||||||
return "Running"
|
return "Running"
|
||||||
case TaskStatusZombie:
|
case TaskStatusSeeding:
|
||||||
return "Zombie"
|
return "Seeding"
|
||||||
case TaskStatusSuccess:
|
case TaskStatusSuccess:
|
||||||
return "Success"
|
return "Success"
|
||||||
case TaskStatusCDNRegisterFail:
|
case TaskStatusZombie:
|
||||||
return "cdnRegisterFail"
|
return "Zombie"
|
||||||
case TaskStatusFailed:
|
case TaskStatusFail:
|
||||||
return "fail"
|
return "Fail"
|
||||||
case TaskStatusSourceError:
|
|
||||||
return "sourceError"
|
|
||||||
default:
|
default:
|
||||||
return "unknown"
|
return "unknown"
|
||||||
}
|
}
|
||||||
|
|
@ -50,15 +49,14 @@ func (status TaskStatus) String() string {
|
||||||
const (
|
const (
|
||||||
TaskStatusWaiting TaskStatus = iota
|
TaskStatusWaiting TaskStatus = iota
|
||||||
TaskStatusRunning
|
TaskStatusRunning
|
||||||
TaskStatusZombie
|
|
||||||
TaskStatusSeeding
|
TaskStatusSeeding
|
||||||
TaskStatusSuccess
|
TaskStatusSuccess
|
||||||
TaskStatusCDNRegisterFail
|
TaskStatusZombie
|
||||||
TaskStatusFailed
|
TaskStatusFail
|
||||||
TaskStatusSourceError
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type Task struct {
|
type Task struct {
|
||||||
|
lock sync.RWMutex
|
||||||
TaskID string
|
TaskID string
|
||||||
URL string
|
URL string
|
||||||
URLMeta *base.UrlMeta
|
URLMeta *base.UrlMeta
|
||||||
|
|
@ -66,12 +64,14 @@ type Task struct {
|
||||||
CreateTime time.Time
|
CreateTime time.Time
|
||||||
lastAccessTime time.Time
|
lastAccessTime time.Time
|
||||||
lastTriggerTime time.Time
|
lastTriggerTime time.Time
|
||||||
lock sync.RWMutex
|
pieceList map[int32]*base.PieceInfo
|
||||||
pieceList map[int32]*PieceInfo
|
|
||||||
PieceTotal int32
|
PieceTotal int32
|
||||||
ContentLength int64
|
ContentLength int64
|
||||||
status TaskStatus
|
status TaskStatus
|
||||||
peers *sortedlist.SortedList
|
peers *sortedlist.SortedList
|
||||||
|
backSourceLimit atomic.Int32
|
||||||
|
needClientBackSource atomic.Bool
|
||||||
|
backSourcePeers []string
|
||||||
// TODO add cdnPeers
|
// TODO add cdnPeers
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -80,7 +80,8 @@ func NewTask(taskID, url string, meta *base.UrlMeta) *Task {
|
||||||
TaskID: taskID,
|
TaskID: taskID,
|
||||||
URL: url,
|
URL: url,
|
||||||
URLMeta: meta,
|
URLMeta: meta,
|
||||||
pieceList: make(map[int32]*PieceInfo),
|
CreateTime: time.Now(),
|
||||||
|
pieceList: make(map[int32]*base.PieceInfo),
|
||||||
peers: sortedlist.NewSortedList(),
|
peers: sortedlist.NewSortedList(),
|
||||||
status: TaskStatusWaiting,
|
status: TaskStatusWaiting,
|
||||||
}
|
}
|
||||||
|
|
@ -114,13 +115,31 @@ func (task *Task) GetStatus() TaskStatus {
|
||||||
return task.status
|
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()
|
task.lock.RLock()
|
||||||
defer task.lock.RUnlock()
|
defer task.lock.RUnlock()
|
||||||
return task.pieceList[pieceNum]
|
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()
|
task.lock.Lock()
|
||||||
defer task.lock.Unlock()
|
defer task.lock.Unlock()
|
||||||
task.pieceList[p.PieceNum] = p
|
task.pieceList[p.PieceNum] = p
|
||||||
|
|
@ -130,22 +149,32 @@ func (task *Task) GetLastTriggerTime() time.Time {
|
||||||
return task.lastTriggerTime
|
return task.lastTriggerTime
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (task *Task) UpdateLastTriggerTime(lastTriggerTime time.Time) {
|
||||||
|
task.lastTriggerTime = lastTriggerTime
|
||||||
|
}
|
||||||
|
|
||||||
func (task *Task) Touch() {
|
func (task *Task) Touch() {
|
||||||
task.lock.Lock()
|
task.lock.Lock()
|
||||||
defer task.lock.Unlock()
|
defer task.lock.Unlock()
|
||||||
task.lastAccessTime = time.Now()
|
task.lastAccessTime = time.Now()
|
||||||
}
|
}
|
||||||
|
|
||||||
func (task *Task) UpdateLastTriggerTime(lastTriggerTime time.Time) {
|
|
||||||
task.lastTriggerTime = lastTriggerTime
|
|
||||||
}
|
|
||||||
|
|
||||||
func (task *Task) GetLastAccessTime() time.Time {
|
func (task *Task) GetLastAccessTime() time.Time {
|
||||||
task.lock.RLock()
|
task.lock.RLock()
|
||||||
defer task.lock.RUnlock()
|
defer task.lock.RUnlock()
|
||||||
return task.lastAccessTime
|
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() {
|
func (task *Task) Lock() {
|
||||||
task.lock.Lock()
|
task.lock.Lock()
|
||||||
}
|
}
|
||||||
|
|
@ -164,25 +193,16 @@ func (task *Task) RUnlock() {
|
||||||
|
|
||||||
const TinyFileSize = 128
|
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.
|
// IsSuccess determines that whether cdn status is success.
|
||||||
func (task *Task) IsSuccess() bool {
|
func (task *Task) IsSuccess() bool {
|
||||||
return task.status == TaskStatusSuccess
|
return task.status == TaskStatusSuccess
|
||||||
}
|
}
|
||||||
|
|
||||||
// IsFrozen determines that whether cdn status is frozen
|
// IsFrozen determines that whether cdn status is frozen
|
||||||
func (task *Task) IsFrozen() bool {
|
//func (task *Task) IsFrozen() bool {
|
||||||
return task.status == TaskStatusFailed || task.status == TaskStatusWaiting ||
|
// return task.status == TaskStatusWaiting || task.status == TaskStatusZombie || task.status == TaskStatusFailed ||
|
||||||
task.status == TaskStatusSourceError || task.status == TaskStatusCDNRegisterFail
|
// task.status == TaskStatusSourceError || task.status == TaskStatusCDNRegisterFail
|
||||||
}
|
//}
|
||||||
|
|
||||||
// CanSchedule determines whether task can be scheduled
|
// CanSchedule determines whether task can be scheduled
|
||||||
// only task status is seeding or success 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
|
// IsHealth determines whether task is health
|
||||||
func (task *Task) IsHealth() bool {
|
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
|
// IsFail determines whether task is fail
|
||||||
func (task *Task) IsFail() bool {
|
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() {
|
func (m *manager) cleanupTasks() {
|
||||||
for range m.cleanupExpiredTaskTicker.C {
|
for range m.cleanupExpiredTaskTicker.C {
|
||||||
m.taskMap.Range(func(key, value interface{}) bool {
|
m.taskMap.Range(func(key, value interface{}) bool {
|
||||||
|
taskID := key.(string)
|
||||||
task := value.(*supervisor.Task)
|
task := value.(*supervisor.Task)
|
||||||
elapse := time.Since(task.GetLastAccessTime())
|
elapse := time.Since(task.GetLastAccessTime())
|
||||||
if elapse > m.taskTTI && task.IsSuccess() {
|
if elapse > m.taskTTI && task.IsSuccess() {
|
||||||
task.SetStatus(supervisor.TaskStatusZombie)
|
task.SetStatus(supervisor.TaskStatusZombie)
|
||||||
}
|
}
|
||||||
|
if task.ListPeers().Size() == 0 {
|
||||||
|
task.SetStatus(supervisor.TaskStatusWaiting)
|
||||||
|
}
|
||||||
if elapse > m.taskTTL {
|
if elapse > m.taskTTL {
|
||||||
taskID := key.(string)
|
|
||||||
// TODO lock
|
// TODO lock
|
||||||
m.Delete(taskID)
|
|
||||||
peers := m.peerManager.ListPeersByTask(taskID)
|
peers := m.peerManager.ListPeersByTask(taskID)
|
||||||
for _, peer := range peers {
|
for _, peer := range peers {
|
||||||
m.peerManager.Delete(peer.PeerID)
|
m.peerManager.Delete(peer.PeerID)
|
||||||
}
|
}
|
||||||
|
m.Delete(taskID)
|
||||||
}
|
}
|
||||||
return true
|
return true
|
||||||
})
|
})
|
||||||
|
|
|
||||||
Loading…
Reference in New Issue