move RPC code definition to proto file (#829)

* move rpc code to proto
* replace Cdn to CDN

Signed-off-by: sunwp <244372610@qq.com>
This commit is contained in:
sunwp 2021-11-29 11:13:35 +08:00 committed by Gaius
parent 6bad029d81
commit d79a2a6877
No known key found for this signature in database
GPG Key ID: 8B4E5D1290FA2FFB
25 changed files with 499 additions and 200 deletions

View File

@ -30,7 +30,6 @@ import (
cdnerrors "d7y.io/dragonfly/v2/cdn/errors"
"d7y.io/dragonfly/v2/cdn/supervisor"
"d7y.io/dragonfly/v2/cdn/types"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/internal/idgen"
@ -112,7 +111,7 @@ func (css *server) ObtainSeeds(ctx context.Context, req *cdnsystem.SeedRequest,
logger.Infof("obtain seeds request: %+v", req)
defer func() {
if r := recover(); r != nil {
err = dferrors.Newf(dfcodes.UnknownError, "obtain task(%s) seeds encounter an panic: %v", req.TaskId, r)
err = dferrors.Newf(base.Code_UnknownError, "obtain task(%s) seeds encounter an panic: %v", req.TaskId, r)
span.RecordError(err)
logger.WithTaskID(req.TaskId).Errorf("%v", err)
}
@ -120,7 +119,7 @@ func (css *server) ObtainSeeds(ctx context.Context, req *cdnsystem.SeedRequest,
}()
registerRequest, err := constructRegisterRequest(req)
if err != nil {
err = dferrors.Newf(dfcodes.BadRequest, "bad seed request for task(%s): %v", req.TaskId, err)
err = dferrors.Newf(base.Code_BadRequest, "bad seed request for task(%s): %v", req.TaskId, err)
span.RecordError(err)
return err
}
@ -128,11 +127,11 @@ func (css *server) ObtainSeeds(ctx context.Context, req *cdnsystem.SeedRequest,
pieceChan, err := css.taskMgr.Register(ctx, registerRequest)
if err != nil {
if cdnerrors.IsResourcesLacked(err) {
err = dferrors.Newf(dfcodes.ResourceLacked, "resources lacked for task(%s): %v", req.TaskId, err)
err = dferrors.Newf(base.Code_ResourceLacked, "resources lacked for task(%s): %v", req.TaskId, err)
span.RecordError(err)
return err
}
err = dferrors.Newf(dfcodes.CdnTaskRegistryFail, "failed to register seed task(%s): %v", req.TaskId, err)
err = dferrors.Newf(base.Code_CDNTaskRegistryFail, "failed to register seed task(%s): %v", req.TaskId, err)
span.RecordError(err)
return err
}
@ -154,12 +153,12 @@ func (css *server) ObtainSeeds(ctx context.Context, req *cdnsystem.SeedRequest,
}
task, err := css.taskMgr.Get(req.TaskId)
if err != nil {
err = dferrors.Newf(dfcodes.CdnError, "failed to get task(%s): %v", req.TaskId, err)
err = dferrors.Newf(base.Code_CDNError, "failed to get task(%s): %v", req.TaskId, err)
span.RecordError(err)
return err
}
if !task.IsSuccess() {
err = dferrors.Newf(dfcodes.CdnTaskDownloadFail, "task(%s) status error , status: %s", req.TaskId, task.CdnStatus)
err = dferrors.Newf(base.Code_CDNTaskDownloadFail, "task(%s) status error , status: %s", req.TaskId, task.CdnStatus)
span.RecordError(err)
return err
}
@ -181,36 +180,36 @@ func (css *server) GetPieceTasks(ctx context.Context, req *base.PieceTaskRequest
span.SetAttributes(config.AttributeTaskID.String(req.TaskId))
defer func() {
if r := recover(); r != nil {
err = dferrors.Newf(dfcodes.UnknownError, "get task(%s) piece tasks encounter an panic: %v", req.TaskId, r)
err = dferrors.Newf(base.Code_UnknownError, "get task(%s) piece tasks encounter an panic: %v", req.TaskId, r)
span.RecordError(err)
logger.WithTaskID(req.TaskId).Errorf("%v", err)
}
}()
logger.Infof("get piece tasks: %+v", req)
if err := checkPieceTasksRequestParams(req); err != nil {
err = dferrors.Newf(dfcodes.BadRequest, "failed to validate seed request for task(%s): %v", req.TaskId, err)
err = dferrors.Newf(base.Code_BadRequest, "failed to validate seed request for task(%s): %v", req.TaskId, err)
span.RecordError(err)
return nil, err
}
task, err := css.taskMgr.Get(req.TaskId)
if err != nil {
if cdnerrors.IsDataNotFound(err) {
err = dferrors.Newf(dfcodes.CdnTaskNotFound, "failed to get task(%s) from cdn: %v", req.TaskId, err)
err = dferrors.Newf(base.Code_CDNTaskNotFound, "failed to get task(%s) from cdn: %v", req.TaskId, err)
span.RecordError(err)
return nil, err
}
err = dferrors.Newf(dfcodes.CdnError, "failed to get task(%s) from cdn: %v", req.TaskId, err)
err = dferrors.Newf(base.Code_CDNError, "failed to get task(%s) from cdn: %v", req.TaskId, err)
span.RecordError(err)
return nil, err
}
if task.IsError() {
err = dferrors.Newf(dfcodes.CdnTaskDownloadFail, "fail to download task(%s), cdnStatus: %s", task.TaskID, task.CdnStatus)
err = dferrors.Newf(base.Code_CDNTaskDownloadFail, "fail to download task(%s), cdnStatus: %s", task.TaskID, task.CdnStatus)
span.RecordError(err)
return nil, err
}
pieces, err := css.taskMgr.GetPieces(ctx, req.TaskId)
if err != nil {
err = dferrors.Newf(dfcodes.CdnError, "failed to get pieces of task(%s) from cdn: %v", task.TaskID, err)
err = dferrors.Newf(base.Code_CDNError, "failed to get pieces of task(%s) from cdn: %v", task.TaskID, err)
span.RecordError(err)
return nil, err
}

View File

@ -32,7 +32,6 @@ import (
"google.golang.org/grpc/status"
"d7y.io/dragonfly/v2/client/config"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/pkg/retry"
@ -245,9 +244,9 @@ loop:
if pt.success {
return
}
pt.failedCode = dfcodes.UnknownError
pt.failedCode = base.Code_UnknownError
if de, ok := err.(*dferrors.DfError); ok {
if de.Code == dfcodes.SchedNeedBackSource {
if de.Code == base.Code_SchedNeedBackSource {
pt.needBackSource = true
close(pt.peerPacketReady)
return
@ -266,7 +265,7 @@ loop:
}
logger.Debugf("receive peerPacket %v for peer %s", peerPacket, pt.peerID)
if peerPacket.Code != dfcodes.Success {
if peerPacket.Code != base.Code_Success {
pt.Errorf("receive peer packet with error: %d", peerPacket.Code)
if pt.isExitPeerPacketCode(peerPacket) {
pt.cancel()
@ -316,21 +315,21 @@ loop:
func (pt *peerTask) isExitPeerPacketCode(pp *scheduler.PeerPacket) bool {
switch pp.Code {
case dfcodes.ResourceLacked, dfcodes.BadRequest, dfcodes.PeerTaskNotFound, dfcodes.UnknownError, dfcodes.RequestTimeOut:
case base.Code_ResourceLacked, base.Code_BadRequest, base.Code_PeerTaskNotFound, base.Code_UnknownError, base.Code_RequestTimeOut:
// 1xxx
pt.failedCode = pp.Code
pt.failedReason = fmt.Sprintf("receive exit peer packet with code %d", pp.Code)
return true
case dfcodes.SchedError:
case base.Code_SchedError:
// 5xxx
pt.failedCode = pp.Code
pt.failedReason = fmt.Sprintf("receive exit peer packet with code %d", pp.Code)
return true
case dfcodes.SchedPeerGone:
case base.Code_SchedPeerGone:
pt.failedReason = reasonPeerGoneFromScheduler
pt.failedCode = dfcodes.SchedPeerGone
pt.failedCode = base.Code_SchedPeerGone
return true
case dfcodes.CdnError, dfcodes.CdnTaskRegistryFail, dfcodes.CdnTaskDownloadFail:
case base.Code_CDNError, base.Code_CDNTaskRegistryFail, base.Code_CDNTaskDownloadFail:
// 6xxx
pt.failedCode = pp.Code
pt.failedReason = fmt.Sprintf("receive exit peer packet with code %d", pp.Code)
@ -351,7 +350,7 @@ func (pt *peerTask) pullSinglePiece(cleanUnfinishedFunc func()) {
pt.SetPieceMd5Sign(pt.singlePiece.PieceInfo.PieceMd5)
if err := pt.callback.Init(pt); err != nil {
pt.failedReason = err.Error()
pt.failedCode = dfcodes.ClientError
pt.failedCode = base.Code_ClientError
cleanUnfinishedFunc()
span.RecordError(err)
span.SetAttributes(config.AttributePieceSuccess.Bool(false))
@ -414,7 +413,7 @@ loop:
if !pt.success {
if pt.failedCode == failedCodeNotSet {
pt.failedReason = reasonContextCanceled
pt.failedCode = dfcodes.ClientContextCanceled
pt.failedCode = base.Code_ClientContextCanceled
if err := pt.callback.Fail(pt, pt.failedCode, pt.ctx.Err().Error()); err != nil {
pt.Errorf("peer task callback failed %s", err)
}
@ -501,7 +500,7 @@ func (pt *peerTask) init(piecePacket *base.PiecePacket, pieceBufferSize int32) (
if err := pt.callback.Init(pt); err != nil {
pt.span.RecordError(err)
pt.failedReason = err.Error()
pt.failedCode = dfcodes.ClientError
pt.failedCode = base.Code_ClientError
return nil, false
}
pc := pt.peerPacket.Load().(*scheduler.PeerPacket).ParallelCount
@ -529,8 +528,8 @@ func (pt *peerTask) waitFirstPeerPacket() (done bool, backSource bool) {
time.Now().Sub(pt.callback.GetStartTime()).Microseconds(), pt.peerPacket.Load().(*scheduler.PeerPacket).MainPeer)
return true, false
}
// when scheduler says dfcodes.SchedNeedBackSource, receivePeerPacket will close pt.peerPacketReady
pt.Infof("start download from source due to dfcodes.SchedNeedBackSource")
// when scheduler says base.Code_SchedNeedBackSource, receivePeerPacket will close pt.peerPacketReady
pt.Infof("start download from source due to base.Code_SchedNeedBackSource")
pt.span.AddEvent("back source due to scheduler says need back source")
pt.needBackSource = true
pt.backSource()
@ -538,7 +537,7 @@ func (pt *peerTask) waitFirstPeerPacket() (done bool, backSource bool) {
case <-time.After(pt.schedulerOption.ScheduleTimeout.Duration):
if pt.schedulerOption.DisableAutoBackSource {
pt.failedReason = reasonScheduleTimeout
pt.failedCode = dfcodes.ClientScheduleTimeout
pt.failedCode = base.Code_ClientScheduleTimeout
err := fmt.Errorf("%s, auto back source disabled", pt.failedReason)
pt.span.RecordError(err)
pt.Errorf(err.Error())
@ -564,7 +563,7 @@ func (pt *peerTask) waitAvailablePeerPacket() (int32, bool) {
if !pt.success {
if pt.failedCode == failedCodeNotSet {
pt.failedReason = reasonContextCanceled
pt.failedCode = dfcodes.ClientContextCanceled
pt.failedCode = base.Code_ClientContextCanceled
}
}
case _, ok := <-pt.peerPacketReady:
@ -574,8 +573,8 @@ func (pt *peerTask) waitAvailablePeerPacket() (int32, bool) {
// research from piece 0
return pt.getNextPieceNum(0), true
}
// when scheduler says dfcodes.SchedNeedBackSource, receivePeerPacket will close pt.peerPacketReady
pt.Infof("start download from source due to dfcodes.SchedNeedBackSource")
// when scheduler says base.Code_SchedNeedBackSource, receivePeerPacket will close pt.peerPacketReady
pt.Infof("start download from source due to base.Code_SchedNeedBackSource")
pt.span.AddEvent("back source due to scheduler says need back source ")
pt.needBackSource = true
// TODO optimize back source when already downloaded some pieces
@ -583,7 +582,7 @@ func (pt *peerTask) waitAvailablePeerPacket() (int32, bool) {
case <-time.After(pt.schedulerOption.ScheduleTimeout.Duration):
if pt.schedulerOption.DisableAutoBackSource {
pt.failedReason = reasonReScheduleTimeout
pt.failedCode = dfcodes.ClientScheduleTimeout
pt.failedCode = base.Code_ClientScheduleTimeout
err := fmt.Errorf("%s, auto back source disabled", pt.failedReason)
pt.span.RecordError(err)
pt.Errorf(err.Error())
@ -620,7 +619,7 @@ func (pt *peerTask) dispatchPieceRequest(pieceRequestCh chan *DownloadPieceReque
if !pt.success {
if pt.failedCode == failedCodeNotSet {
pt.failedReason = reasonContextCanceled
pt.failedCode = dfcodes.ClientContextCanceled
pt.failedCode = base.Code_ClientContextCanceled
}
}
}
@ -673,7 +672,7 @@ func (pt *peerTask) downloadPieceWorker(id int32, pti Task, requests chan *Downl
DstPid: request.DstPid,
PieceInfo: request.piece,
Success: false,
Code: dfcodes.ClientRequestLimitFail,
Code: base.Code_ClientRequestLimitFail,
HostLoad: nil,
FinishedCount: 0, // update by peer task
},
@ -683,7 +682,7 @@ func (pt *peerTask) downloadPieceWorker(id int32, pti Task, requests chan *Downl
}
pt.failedReason = err.Error()
pt.failedCode = dfcodes.ClientRequestLimitFail
pt.failedCode = base.Code_ClientRequestLimitFail
pt.cancel()
span.SetAttributes(config.AttributePieceSuccess.Bool(false))
span.End()
@ -749,7 +748,7 @@ func (pt *peerTask) preparePieceTasksByPeer(curPeerPacket *scheduler.PeerPacket,
span.SetAttributes(config.AttributeGetPieceLimit.Int(int(request.Limit)))
defer span.End()
// when cdn returns dfcodes.CdnTaskNotFound, report it to scheduler and wait cdn download it.
// when cdn returns base.Code_CDNTaskNotFound, report it to scheduler and wait cdn download it.
retry:
pt.Debugf("try get piece task from peer %s, piece num: %d, limit: %d\"", peer.PeerId, request.StartNum, request.Limit)
p, err := pt.getPieceTasks(span, curPeerPacket, peer, request)
@ -774,7 +773,7 @@ retry:
return nil, err
}
}
code := dfcodes.ClientPieceRequestFail
code := base.Code_ClientPieceRequestFail
// not grpc error
if de, ok := err.(*dferrors.DfError); ok && uint32(de.Code) > uint32(codes.Unauthenticated) {
pt.Debugf("get piece task from peer %s with df error, code: %d", peer.PeerId, de.Code)
@ -796,8 +795,8 @@ retry:
pt.Errorf("send piece result error: %s, code to send: %d", err, code)
}
if code == dfcodes.CdnTaskNotFound && curPeerPacket == pt.peerPacket.Load().(*scheduler.PeerPacket) {
span.AddEvent("retry for CdnTaskNotFound")
if code == base.Code_CDNTaskNotFound && curPeerPacket == pt.peerPacket.Load().(*scheduler.PeerPacket) {
span.AddEvent("retry for CDNTaskNotFound")
goto retry
}
return nil, err
@ -832,13 +831,13 @@ func (pt *peerTask) getPieceTasks(span trace.Span, curPeerPacket *scheduler.Peer
DstPid: peer.PeerId,
PieceInfo: &base.PieceInfo{},
Success: false,
Code: dfcodes.ClientWaitPieceReady,
Code: base.Code_ClientWaitPieceReady,
HostLoad: nil,
FinishedCount: pt.readyPieces.Settled(),
})
if er != nil {
span.RecordError(er)
pt.Errorf("send piece result with dfcodes.ClientWaitPieceReady error: %s", er)
pt.Errorf("send piece result with base.Code_ClientWaitPieceReady error: %s", er)
}
// fast way to exit retry
lastPeerPacket := pt.peerPacket.Load().(*scheduler.PeerPacket)

View File

@ -21,9 +21,9 @@ import (
"google.golang.org/grpc"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
"d7y.io/dragonfly/v2/pkg/basic/dfnet"
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
schedulerclient "d7y.io/dragonfly/v2/pkg/rpc/scheduler/client"
)
@ -58,7 +58,7 @@ type dummyPeerPacketStream struct {
}
func (d *dummyPeerPacketStream) Recv() (pp *scheduler.PeerPacket, err error) {
return nil, dferrors.New(dfcodes.SchedNeedBackSource, "")
return nil, dferrors.New(base.Code_SchedNeedBackSource, "")
}
func (d *dummyPeerPacketStream) Send(pr *scheduler.PieceResult) (err error) {

View File

@ -27,7 +27,6 @@ import (
"golang.org/x/time/rate"
"d7y.io/dragonfly/v2/client/config"
"d7y.io/dragonfly/v2/internal/dfcodes"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/internal/idgen"
"d7y.io/dragonfly/v2/pkg/rpc/base"
@ -193,7 +192,7 @@ func newFilePeerTask(ctx context.Context,
requestedPieces: NewBitmap(),
failedPieceCh: make(chan int32, config.DefaultPieceChanSize),
failedReason: failedReasonNotSet,
failedCode: dfcodes.UnknownError,
failedCode: base.Code_UnknownError,
contentLength: atomic.NewInt64(-1),
pieceParallelCount: atomic.NewInt32(0),
totalPiece: -1,
@ -315,7 +314,7 @@ func (pt *filePeerTask) finish() error {
var (
success = true
code = dfcodes.Success
code = base.Code_Success
message = "Success"
progressDone bool
)
@ -325,7 +324,7 @@ func (pt *filePeerTask) finish() error {
pt.Errorf("peer task done callback failed: %s", err)
pt.span.RecordError(err)
success = false
code = dfcodes.ClientError
code = base.Code_ClientError
message = err.Error()
}

View File

@ -24,7 +24,6 @@ import (
"d7y.io/dragonfly/v2/client/config"
"d7y.io/dragonfly/v2/client/daemon/storage"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
)
@ -112,7 +111,7 @@ func (p *filePeerTaskCallback) Done(pt Task) error {
TotalPieceCount: pt.GetTotalPieces(),
Cost: uint32(cost),
Success: true,
Code: dfcodes.Success,
Code: base.Code_Success,
})
if err != nil {
peerResultSpan.RecordError(err)

View File

@ -42,7 +42,6 @@ import (
"d7y.io/dragonfly/v2/client/daemon/test"
mock_daemon "d7y.io/dragonfly/v2/client/daemon/test/mock/daemon"
mock_scheduler "d7y.io/dragonfly/v2/client/daemon/test/mock/scheduler"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/pkg/basic/dfnet"
"d7y.io/dragonfly/v2/pkg/rpc"
"d7y.io/dragonfly/v2/pkg/rpc/base"
@ -124,7 +123,7 @@ func setupPeerTaskManagerComponents(ctrl *gomock.Controller, opt componentsOptio
delayCount++
}
return &scheduler.PeerPacket{
Code: dfcodes.Success,
Code: base.Code_Success,
TaskId: opt.taskID,
SrcPid: "127.0.0.1",
ParallelCount: opt.pieceParallelCount,

View File

@ -28,9 +28,9 @@ import (
"d7y.io/dragonfly/v2/client/config"
"d7y.io/dragonfly/v2/client/daemon/storage"
"d7y.io/dragonfly/v2/internal/dfcodes"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/internal/idgen"
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
)
@ -83,7 +83,7 @@ func (ptm *peerTaskManager) tryReuseFilePeerTask(ctx context.Context,
pg := &FilePeerTaskProgress{
State: &ProgressState{
Success: true,
Code: dfcodes.Success,
Code: base.Code_Success,
Msg: "Success",
},
TaskID: taskID,

View File

@ -30,7 +30,6 @@ import (
"d7y.io/dragonfly/v2/client/config"
"d7y.io/dragonfly/v2/client/daemon/storage"
"d7y.io/dragonfly/v2/internal/dfcodes"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/internal/idgen"
"d7y.io/dragonfly/v2/pkg/rpc/base"
@ -169,7 +168,7 @@ func newStreamPeerTask(ctx context.Context,
requestedPieces: NewBitmap(),
failedPieceCh: make(chan int32, config.DefaultPieceChanSize),
failedReason: failedReasonNotSet,
failedCode: dfcodes.UnknownError,
failedCode: base.Code_UnknownError,
contentLength: atomic.NewInt64(-1),
pieceParallelCount: atomic.NewInt32(0),
totalPiece: -1,

View File

@ -40,7 +40,6 @@ import (
"d7y.io/dragonfly/v2/client/daemon/test"
mock_daemon "d7y.io/dragonfly/v2/client/daemon/test/mock/daemon"
mock_scheduler "d7y.io/dragonfly/v2/client/daemon/test/mock/scheduler"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
"d7y.io/dragonfly/v2/pkg/basic/dfnet"
"d7y.io/dragonfly/v2/pkg/rpc"
@ -129,11 +128,11 @@ func setupBackSourcePartialComponents(ctrl *gomock.Controller, testBytes []byte,
if schedPeerPacket {
// send back source after piece 0 is done
wg.Wait()
return nil, dferrors.New(dfcodes.SchedNeedBackSource, "")
return nil, dferrors.New(base.Code_SchedNeedBackSource, "")
}
schedPeerPacket = true
return &scheduler.PeerPacket{
Code: dfcodes.Success,
Code: base.Code_Success,
TaskId: opt.taskID,
SrcPid: "127.0.0.1",
ParallelCount: opt.pieceParallelCount,

View File

@ -24,7 +24,6 @@ import (
"d7y.io/dragonfly/v2/client/config"
"d7y.io/dragonfly/v2/client/daemon/storage"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
)
@ -110,7 +109,7 @@ func (p *streamPeerTaskCallback) Done(pt Task) error {
TotalPieceCount: p.pt.totalPiece,
Cost: uint32(cost),
Success: true,
Code: dfcodes.Success,
Code: base.Code_Success,
})
if err != nil {
peerResultSpan.RecordError(err)

View File

@ -28,7 +28,6 @@ import (
"d7y.io/dragonfly/v2/client/clientutil"
"d7y.io/dragonfly/v2/client/config"
"d7y.io/dragonfly/v2/client/daemon/storage"
"d7y.io/dragonfly/v2/internal/dfcodes"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
@ -171,7 +170,7 @@ func (pm *pieceManager) pushSuccessResult(peerTask Task, dstPid string, piece *b
BeginTime: uint64(start),
EndTime: uint64(end),
Success: true,
Code: dfcodes.Success,
Code: base.Code_Success,
HostLoad: nil, // TODO(jim): update host load
FinishedCount: piece.PieceNum + 1, // update by peer task
// TODO range_start, range_size, piece_md5, piece_offset, piece_style
@ -195,7 +194,7 @@ func (pm *pieceManager) pushFailResult(peerTask Task, dstPid string, piece *base
BeginTime: uint64(start),
EndTime: uint64(end),
Success: false,
Code: dfcodes.ClientPieceDownloadFail,
Code: base.Code_ClientPieceDownloadFail,
HostLoad: nil,
FinishedCount: 0, // update by peer task
},

View File

@ -30,7 +30,6 @@ import (
"d7y.io/dragonfly/v2/client/clientutil"
"d7y.io/dragonfly/v2/client/daemon/peer"
"d7y.io/dragonfly/v2/client/daemon/storage"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/internal/idgen"
@ -88,7 +87,7 @@ func (m *server) GetPieceTasks(ctx context.Context, request *base.PieceTaskReque
m.Keep()
p, err := m.storageManager.GetPieces(ctx, request)
if err != nil {
code := dfcodes.UnknownError
code := base.Code_UnknownError
if err != storage.ErrTaskNotFound {
logger.Errorf("get piece tasks error: %s, task id: %s, src peer: %s, dst peer: %s, piece num: %d, limit: %d",
err, request.TaskId, request.SrcPid, request.DstPid, request.StartNum, request.Limit)
@ -96,7 +95,7 @@ func (m *server) GetPieceTasks(ctx context.Context, request *base.PieceTaskReque
}
// dst peer is not running
if !m.peerTaskManager.IsPeerTaskRunning(request.DstPid) {
code = dfcodes.PeerTaskNotFound
code = base.Code_PeerTaskNotFound
logger.Errorf("get piece tasks error: peer task not found, task id: %s, src peer: %s, dst peer: %s, piece num: %d, limit: %d",
request.TaskId, request.SrcPid, request.DstPid, request.StartNum, request.Limit)
return nil, dferrors.New(code, err.Error())
@ -151,7 +150,7 @@ func (m *server) Download(ctx context.Context,
peerTaskProgress, tiny, err := m.peerTaskManager.StartFilePeerTask(ctx, peerTask)
if err != nil {
return dferrors.New(dfcodes.UnknownError, fmt.Sprintf("%s", err))
return dferrors.New(base.Code_UnknownError, fmt.Sprintf("%s", err))
}
if tiny != nil {
results <- &dfdaemongrpc.DownResult{
@ -176,7 +175,7 @@ func (m *server) Download(ctx context.Context,
if !ok {
err = errors.New("progress closed unexpected")
log.Errorf(err.Error())
return dferrors.New(dfcodes.UnknownError, err.Error())
return dferrors.New(base.Code_UnknownError, err.Error())
}
if !p.State.Success {
log.Errorf("task %s/%s failed: %d/%s", p.PeerID, p.TaskID, p.State.Code, p.State.Msg)

View File

@ -1,65 +0,0 @@
/*
* Copyright 2020 The Dragonfly Authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package dfcodes
import "d7y.io/dragonfly/v2/pkg/rpc/base"
// rpc code
const (
// success code 200-299
Success base.Code = 200
ServerUnavailable base.Code = 500 // framework can not find server node
// common response error 1000-1999
ResourceLacked base.Code = 1000 // client can be migrated to another scheduler/CDN
BadRequest base.Code = 1400
PeerTaskNotFound base.Code = 1404
UnknownError base.Code = 1500
RequestTimeOut base.Code = 1504
// client response error 4000-4999
ClientError base.Code = 4000
ClientPieceRequestFail base.Code = 4001 // get piece task from other peer error
ClientScheduleTimeout base.Code = 4002 // wait scheduler response timeout
ClientContextCanceled base.Code = 4003
ClientWaitPieceReady base.Code = 4004 // when target peer downloads from source slowly, should wait
ClientPieceDownloadFail base.Code = 4005
ClientRequestLimitFail base.Code = 4006
// scheduler response error 5000-5999
SchedError base.Code = 5000
SchedNeedBackSource base.Code = 5001 // client should try to download from source
SchedPeerGone base.Code = 5002 // client should disconnect from scheduler
SchedPeerNotFound base.Code = 5004 // peer not found in scheduler
SchedPeerPieceResultReportFail base.Code = 5005 // report piece
SchedTaskStatusError base.Code = 5006 // task status is fail
// cdnsystem response error 6000-6999
CdnError base.Code = 6000
CdnTaskRegistryFail base.Code = 6001
CdnTaskDownloadFail base.Code = 6002
CdnTaskNotFound base.Code = 6404
// manager response error 7000-7999
ManagerError base.Code = 7000
InvalidResourceType base.Code = 7001
ManagerHostError base.Code = 7002
ManagerStoreError base.Code = 7003
ManagerConfigError base.Code = 7004
ManagerStoreNotFound base.Code = 7005
SchedulerNodesNotFound base.Code = 7006
)

View File

@ -27,8 +27,8 @@ import (
"golang.org/x/crypto/bcrypt"
"gorm.io/gorm"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
"d7y.io/dragonfly/v2/pkg/rpc/base"
)
type ErrorResponse struct {
@ -57,7 +57,7 @@ func Error() gin.HandlerFunc {
// RPC error handler
if err, ok := errors.Cause(err.Err).(*dferrors.DfError); ok {
switch err.Code {
case dfcodes.InvalidResourceType:
case base.Code_InvalidResourceType:
c.JSON(http.StatusBadRequest, ErrorResponse{
Message: http.StatusText(http.StatusBadRequest),
})

View File

@ -39,15 +39,98 @@ type Code int32
const (
Code_X_UNSPECIFIED Code = 0
// success code 200-299
Code_Success Code = 200
// framework can not find server node
Code_ServerUnavailable Code = 500
// common response error 1000-1999
// client can be migrated to another scheduler/CDN
Code_ResourceLacked Code = 1000
Code_BadRequest Code = 1400
Code_PeerTaskNotFound Code = 1404
Code_UnknownError Code = 1500
Code_RequestTimeOut Code = 1504
// client response error 4000-4999
Code_ClientError Code = 4000
Code_ClientPieceRequestFail Code = 4001 // get piece task from other peer error
Code_ClientScheduleTimeout Code = 4002 // wait scheduler response timeout
Code_ClientContextCanceled Code = 4003
Code_ClientWaitPieceReady Code = 4004 // when target peer downloads from source slowly, should wait
Code_ClientPieceDownloadFail Code = 4005
Code_ClientRequestLimitFail Code = 4006
// scheduler response error 5000-5999
Code_SchedError Code = 5000
Code_SchedNeedBackSource Code = 5001 // client should try to download from source
Code_SchedPeerGone Code = 5002 // client should disconnect from scheduler
Code_SchedPeerNotFound Code = 5004 // peer not found in scheduler
Code_SchedPeerPieceResultReportFail Code = 5005 // report piece
Code_SchedTaskStatusError Code = 5006 // task status is fail
// cdnsystem response error 6000-6999
Code_CDNError Code = 6000
Code_CDNTaskRegistryFail Code = 6001
Code_CDNTaskDownloadFail Code = 6002
Code_CDNTaskNotFound Code = 6404
// manager response error 7000-7999
Code_InvalidResourceType Code = 7001
)
// Enum value maps for Code.
var (
Code_name = map[int32]string{
0: "X_UNSPECIFIED",
200: "Success",
500: "ServerUnavailable",
1000: "ResourceLacked",
1400: "BadRequest",
1404: "PeerTaskNotFound",
1500: "UnknownError",
1504: "RequestTimeOut",
4000: "ClientError",
4001: "ClientPieceRequestFail",
4002: "ClientScheduleTimeout",
4003: "ClientContextCanceled",
4004: "ClientWaitPieceReady",
4005: "ClientPieceDownloadFail",
4006: "ClientRequestLimitFail",
5000: "SchedError",
5001: "SchedNeedBackSource",
5002: "SchedPeerGone",
5004: "SchedPeerNotFound",
5005: "SchedPeerPieceResultReportFail",
5006: "SchedTaskStatusError",
6000: "CDNError",
6001: "CDNTaskRegistryFail",
6002: "CDNTaskDownloadFail",
6404: "CDNTaskNotFound",
7001: "InvalidResourceType",
}
Code_value = map[string]int32{
"X_UNSPECIFIED": 0,
"Success": 200,
"ServerUnavailable": 500,
"ResourceLacked": 1000,
"BadRequest": 1400,
"PeerTaskNotFound": 1404,
"UnknownError": 1500,
"RequestTimeOut": 1504,
"ClientError": 4000,
"ClientPieceRequestFail": 4001,
"ClientScheduleTimeout": 4002,
"ClientContextCanceled": 4003,
"ClientWaitPieceReady": 4004,
"ClientPieceDownloadFail": 4005,
"ClientRequestLimitFail": 4006,
"SchedError": 5000,
"SchedNeedBackSource": 5001,
"SchedPeerGone": 5002,
"SchedPeerNotFound": 5004,
"SchedPeerPieceResultReportFail": 5005,
"SchedTaskStatusError": 5006,
"CDNError": 6000,
"CDNTaskRegistryFail": 6001,
"CDNTaskDownloadFail": 6002,
"CDNTaskNotFound": 6404,
"InvalidResourceType": 7001,
}
)
@ -712,16 +795,53 @@ var file_pkg_rpc_base_base_proto_rawDesc = []byte{
0x74, 0x68, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e,
0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x70, 0x69, 0x65, 0x63, 0x65,
0x5f, 0x6d, 0x64, 0x35, 0x5f, 0x73, 0x69, 0x67, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52,
0x0c, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4d, 0x64, 0x35, 0x53, 0x69, 0x67, 0x6e, 0x2a, 0x19, 0x0a,
0x04, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x58, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45,
0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x2a, 0x17, 0x0a, 0x0a, 0x50, 0x69, 0x65, 0x63,
0x65, 0x53, 0x74, 0x79, 0x6c, 0x65, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x4c, 0x41, 0x49, 0x4e, 0x10,
0x00, 0x2a, 0x2c, 0x0a, 0x09, 0x53, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x0a,
0x0a, 0x06, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x53, 0x4d,
0x41, 0x4c, 0x4c, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x54, 0x49, 0x4e, 0x59, 0x10, 0x02, 0x42,
0x22, 0x5a, 0x20, 0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e,
0x66, 0x6c, 0x79, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x62,
0x61, 0x73, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x0c, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4d, 0x64, 0x35, 0x53, 0x69, 0x67, 0x6e, 0x2a, 0xeb, 0x04,
0x0a, 0x04, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x58, 0x5f, 0x55, 0x4e, 0x53, 0x50,
0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x07, 0x53, 0x75, 0x63,
0x63, 0x65, 0x73, 0x73, 0x10, 0xc8, 0x01, 0x12, 0x16, 0x0a, 0x11, 0x53, 0x65, 0x72, 0x76, 0x65,
0x72, 0x55, 0x6e, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x10, 0xf4, 0x03, 0x12,
0x13, 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4c, 0x61, 0x63, 0x6b, 0x65,
0x64, 0x10, 0xe8, 0x07, 0x12, 0x0f, 0x0a, 0x0a, 0x42, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x10, 0xf8, 0x0a, 0x12, 0x15, 0x0a, 0x10, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73,
0x6b, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0xfc, 0x0a, 0x12, 0x11, 0x0a, 0x0c,
0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xdc, 0x0b, 0x12,
0x13, 0x0a, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x4f, 0x75,
0x74, 0x10, 0xe0, 0x0b, 0x12, 0x10, 0x0a, 0x0b, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x45, 0x72,
0x72, 0x6f, 0x72, 0x10, 0xa0, 0x1f, 0x12, 0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74,
0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x46, 0x61, 0x69, 0x6c,
0x10, 0xa1, 0x1f, 0x12, 0x1a, 0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x68,
0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x10, 0xa2, 0x1f, 0x12,
0x1a, 0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74,
0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x10, 0xa3, 0x1f, 0x12, 0x19, 0x0a, 0x14, 0x43,
0x6c, 0x69, 0x65, 0x6e, 0x74, 0x57, 0x61, 0x69, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65,
0x61, 0x64, 0x79, 0x10, 0xa4, 0x1f, 0x12, 0x1c, 0x0a, 0x17, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74,
0x50, 0x69, 0x65, 0x63, 0x65, 0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x46, 0x61, 0x69,
0x6c, 0x10, 0xa5, 0x1f, 0x12, 0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa6,
0x1f, 0x12, 0x0f, 0x0a, 0x0a, 0x53, 0x63, 0x68, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10,
0x88, 0x27, 0x12, 0x18, 0x0a, 0x13, 0x53, 0x63, 0x68, 0x65, 0x64, 0x4e, 0x65, 0x65, 0x64, 0x42,
0x61, 0x63, 0x6b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x10, 0x89, 0x27, 0x12, 0x12, 0x0a, 0x0d,
0x53, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72, 0x47, 0x6f, 0x6e, 0x65, 0x10, 0x8a, 0x27,
0x12, 0x16, 0x0a, 0x11, 0x53, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x6f, 0x74,
0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x8c, 0x27, 0x12, 0x23, 0x0a, 0x1e, 0x53, 0x63, 0x68, 0x65,
0x64, 0x50, 0x65, 0x65, 0x72, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74,
0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0x8d, 0x27, 0x12, 0x19, 0x0a,
0x14, 0x53, 0x63, 0x68, 0x65, 0x64, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73,
0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x8e, 0x27, 0x12, 0x0d, 0x0a, 0x08, 0x43, 0x44, 0x4e, 0x45,
0x72, 0x72, 0x6f, 0x72, 0x10, 0xf0, 0x2e, 0x12, 0x18, 0x0a, 0x13, 0x43, 0x44, 0x4e, 0x54, 0x61,
0x73, 0x6b, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x79, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xf1,
0x2e, 0x12, 0x18, 0x0a, 0x13, 0x43, 0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b, 0x44, 0x6f, 0x77, 0x6e,
0x6c, 0x6f, 0x61, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xf2, 0x2e, 0x12, 0x14, 0x0a, 0x0f, 0x43,
0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x84,
0x32, 0x12, 0x18, 0x0a, 0x13, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x52, 0x65, 0x73, 0x6f,
0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x10, 0xd9, 0x36, 0x2a, 0x17, 0x0a, 0x0a, 0x50,
0x69, 0x65, 0x63, 0x65, 0x53, 0x74, 0x79, 0x6c, 0x65, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x4c, 0x41,
0x49, 0x4e, 0x10, 0x00, 0x2a, 0x2c, 0x0a, 0x09, 0x53, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70,
0x65, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a,
0x05, 0x53, 0x4d, 0x41, 0x4c, 0x4c, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x54, 0x49, 0x4e, 0x59,
0x10, 0x02, 0x42, 0x22, 0x5a, 0x20, 0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61,
0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70,
0x63, 0x2f, 0x62, 0x61, 0x73, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (

View File

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

View File

@ -22,6 +22,44 @@ option go_package = "d7y.io/dragonfly/v2/pkg/rpc/base";
enum Code{
X_UNSPECIFIED = 0;
// success code 200-299
Success = 200;
// framework can not find server node
ServerUnavailable = 500;
// common response error 1000-1999
// client can be migrated to another scheduler/CDN
ResourceLacked = 1000;
BadRequest = 1400;
PeerTaskNotFound = 1404;
UnknownError = 1500;
RequestTimeOut = 1504;
// client response error 4000-4999
ClientError = 4000;
ClientPieceRequestFail = 4001; // get piece task from other peer error
ClientScheduleTimeout = 4002; // wait scheduler response timeout
ClientContextCanceled = 4003;
ClientWaitPieceReady = 4004; // when target peer downloads from source slowly, should wait
ClientPieceDownloadFail = 4005;
ClientRequestLimitFail = 4006;
// scheduler response error 5000-5999
SchedError = 5000;
SchedNeedBackSource = 5001; // client should try to download from source
SchedPeerGone = 5002; // client should disconnect from scheduler
SchedPeerNotFound = 5004; // peer not found in scheduler
SchedPeerPieceResultReportFail = 5005; // report piece
SchedTaskStatusError = 5006; // task status is fail
// cdnsystem response error 6000-6999
CDNError = 6000;
CDNTaskRegistryFail = 6001;
CDNTaskDownloadFail = 6002;
CDNTaskNotFound = 6404;
// manager response error 7000-7999
InvalidResourceType = 7001;
}
enum PieceStyle{

View File

@ -22,7 +22,6 @@ import (
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/pkg/rpc/base"
)
@ -53,11 +52,11 @@ func NewResWithErr(ptr interface{}, err error) interface{} {
var code base.Code
switch st.Code() {
case codes.DeadlineExceeded:
code = dfcodes.RequestTimeOut
code = base.Code_RequestTimeOut
case codes.OK:
code = dfcodes.Success
code = base.Code_Success
default:
code = dfcodes.UnknownError
code = base.Code_UnknownError
}
return NewResWithCodeAndMsg(ptr, code, st.Message())
}

View File

@ -30,10 +30,10 @@ import (
"google.golang.org/grpc/status"
"k8s.io/apimachinery/pkg/util/sets"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/pkg/basic/dfnet"
"d7y.io/dragonfly/v2/pkg/rpc/base"
)
const (
@ -372,7 +372,7 @@ func (conn *Connection) TryMigrate(key string, cause error, exclusiveNodes []str
}
// TODO recover findCandidateClientConn error
if e, ok := cause.(*dferrors.DfError); ok {
if e.Code != dfcodes.ResourceLacked {
if e.Code != base.Code_ResourceLacked {
return "", cause
}
}

View File

@ -25,7 +25,6 @@ import (
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"d7y.io/dragonfly/v2/internal/dfcodes"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/internal/idgen"
"d7y.io/dragonfly/v2/pkg/basic/dfnet"
@ -199,7 +198,7 @@ func (sc *schedulerClient) retryReportPeerResult(ctx context.Context, pr *schedu
var client scheduler.SchedulerClient
client, schedulerNode, err = sc.getSchedulerClient(pr.TaskId, true)
if err != nil {
code = dfcodes.ServerUnavailable
code = base.Code_ServerUnavailable
return nil, err
}
return client.ReportPeerResult(ctx, pr, opts...)

View File

@ -24,10 +24,10 @@ import (
"k8s.io/apimachinery/pkg/util/sets"
"k8s.io/client-go/util/workqueue"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/pkg/container/list"
"d7y.io/dragonfly/v2/pkg/rpc/base"
schedulerRPC "d7y.io/dragonfly/v2/pkg/rpc/scheduler"
"d7y.io/dragonfly/v2/scheduler/config"
"d7y.io/dragonfly/v2/scheduler/core/scheduler"
@ -72,7 +72,7 @@ func (e reScheduleParentEvent) apply(s *state) {
rsPeer.times = rsPeer.times + 1
peer := rsPeer.peer
if peer.Task.IsFail() {
if err := peer.CloseChannelWithError(dferrors.New(dfcodes.SchedTaskStatusError, "schedule task status failed")); err != nil {
if err := peer.CloseChannelWithError(dferrors.New(base.Code_SchedTaskStatusError, "schedule task status failed")); err != nil {
logger.WithTaskAndPeerID(peer.Task.ID, peer.ID).Warnf("close peer channel failed: %v", err)
}
return
@ -88,7 +88,7 @@ func (e reScheduleParentEvent) apply(s *state) {
parent, candidates, hasParent := s.sched.ScheduleParent(peer, blankParents)
if !hasParent {
if peer.Task.CanBackToSource() && !peer.Task.ContainsBackToSourcePeer(peer.ID) {
if peer.CloseChannelWithError(dferrors.Newf(dfcodes.SchedNeedBackSource, "peer %s need back source", peer.ID)) == nil {
if peer.CloseChannelWithError(dferrors.Newf(base.Code_SchedNeedBackSource, "peer %s need back source", peer.ID)) == nil {
peer.Task.AddBackToSourcePeer(peer.ID)
}
return
@ -134,7 +134,7 @@ func (e startReportPieceResultEvent) apply(s *state) {
if !hasParent {
if e.peer.Task.CanBackToSource() && !e.peer.Task.ContainsBackToSourcePeer(e.peer.ID) {
span.SetAttributes(config.AttributeClientBackSource.Bool(true))
if e.peer.CloseChannelWithError(dferrors.Newf(dfcodes.SchedNeedBackSource, "peer %s need back source", e.peer.ID)) == nil {
if e.peer.CloseChannelWithError(dferrors.Newf(base.Code_SchedNeedBackSource, "peer %s need back source", e.peer.ID)) == nil {
e.peer.Task.AddBackToSourcePeer(e.peer.ID)
}
logger.WithTaskAndPeerID(e.peer.Task.ID,
@ -226,11 +226,11 @@ func (e peerDownloadPieceFailEvent) apply(s *state) {
return
}
switch e.pr.Code {
case dfcodes.ClientWaitPieceReady:
case base.Code_ClientWaitPieceReady:
return
case dfcodes.PeerTaskNotFound:
case base.Code_PeerTaskNotFound:
s.peerManager.Delete(e.pr.DstPid)
case dfcodes.CdnTaskNotFound, dfcodes.CdnError, dfcodes.CdnTaskDownloadFail:
case base.Code_CDNTaskNotFound, base.Code_CDNError, base.Code_CDNTaskDownloadFail:
s.peerManager.Delete(e.pr.DstPid)
go func() {
if _, err := s.cdn.StartSeedTask(e.ctx, e.peer.Task); err != nil {
@ -370,7 +370,7 @@ func constructSuccessPeerPacket(peer *supervisor.Peer, parent *supervisor.Peer,
ParallelCount: 1,
MainPeer: mainPeer,
StealPeers: stealPeers,
Code: dfcodes.Success,
Code: base.Code_Success,
}
logger.Debugf("send peerPacket %+v to peer %s", peerPacket, peer.ID)
return peerPacket
@ -386,7 +386,7 @@ func handleCDNSeedTaskFail(task *supervisor.Task) {
if task.CanBackToSource() {
if !task.ContainsBackToSourcePeer(peer.ID) {
if peer.CloseChannelWithError(dferrors.Newf(dfcodes.SchedNeedBackSource, "peer %s need back source because cdn seed task failed", peer.ID)) == nil {
if peer.CloseChannelWithError(dferrors.Newf(base.Code_SchedNeedBackSource, "peer %s need back source because cdn seed task failed", peer.ID)) == nil {
task.AddBackToSourcePeer(peer.ID)
}
}
@ -403,7 +403,7 @@ func handleCDNSeedTaskFail(task *supervisor.Task) {
return true
}
if err := peer.CloseChannelWithError(dferrors.New(dfcodes.SchedTaskStatusError, "schedule task status failed")); err != nil {
if err := peer.CloseChannelWithError(dferrors.New(base.Code_SchedTaskStatusError, "schedule task status failed")); err != nil {
peer.Log().Warnf("close peer conn channel failed: %v", err)
}
return true

View File

@ -28,10 +28,10 @@ import (
"k8s.io/apimachinery/pkg/util/sets"
"k8s.io/client-go/util/workqueue"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/pkg/gc"
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/base/common"
schedulerRPC "d7y.io/dragonfly/v2/pkg/rpc/scheduler"
"d7y.io/dragonfly/v2/pkg/synclock"
@ -171,7 +171,7 @@ func (s *SchedulerService) runReScheduleParentLoop(wsdq workqueue.DelayingInterf
peer := rsPeer.peer
wsdq.Done(v)
if rsPeer.times > maxRescheduleTimes {
if peer.CloseChannelWithError(dferrors.Newf(dfcodes.SchedNeedBackSource, "reschedule parent for peer %s already reaches max reschedule times",
if peer.CloseChannelWithError(dferrors.Newf(base.Code_SchedNeedBackSource, "reschedule parent for peer %s already reaches max reschedule times",
peer.ID)) == nil {
peer.Task.AddBackToSourcePeer(peer.ID)
}
@ -327,7 +327,7 @@ func (s *SchedulerService) HandlePieceResult(ctx context.Context, peer *supervis
pr: pieceResult,
})
return nil
} else if pieceResult.Code != dfcodes.Success {
} else if pieceResult.Code != base.Code_Success {
s.worker.send(peerDownloadPieceFailEvent{
ctx: ctx,
peer: peer,

View File

@ -25,7 +25,6 @@ import (
"go.opentelemetry.io/otel/trace"
"google.golang.org/grpc"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/internal/idgen"
@ -67,7 +66,7 @@ func (s *server) RegisterPeerTask(ctx context.Context, request *scheduler.PeerTa
logger.Debugf("register peer task, req: %+v", request)
resp = new(scheduler.RegisterResult)
if verifyErr := validateParams(request); verifyErr != nil {
err = dferrors.Newf(dfcodes.BadRequest, "bad request param: %v", verifyErr)
err = dferrors.Newf(base.Code_BadRequest, "bad request param: %v", verifyErr)
logger.Errorf("register request: %v", err)
span.RecordError(err)
return
@ -77,7 +76,7 @@ func (s *server) RegisterPeerTask(ctx context.Context, request *scheduler.PeerTa
span.SetAttributes(config.AttributeTaskID.String(taskID))
task := s.service.GetOrCreateTask(ctx, supervisor.NewTask(taskID, request.Url, request.UrlMeta))
if task.IsFail() {
err = dferrors.New(dfcodes.SchedTaskStatusError, "task status is fail")
err = dferrors.New(base.Code_SchedTaskStatusError, "task status is fail")
logger.Errorf("task %s status is fail", task.ID)
span.RecordError(err)
return
@ -133,7 +132,7 @@ func (s *server) ReportPieceResult(stream scheduler.Scheduler_ReportPieceResultS
if err == io.EOF {
return nil
}
err = dferrors.Newf(dfcodes.SchedPeerPieceResultReportFail, "receive an error from peer stream: %v", err)
err = dferrors.Newf(base.Code_SchedPeerPieceResultReportFail, "receive an error from peer stream: %v", err)
span.RecordError(err)
return err
}
@ -141,20 +140,20 @@ func (s *server) ReportPieceResult(stream scheduler.Scheduler_ReportPieceResultS
peer, ok := s.service.GetPeer(pieceResult.SrcPid)
if !ok {
err = dferrors.Newf(dfcodes.SchedPeerNotFound, "peer %s not found", pieceResult.SrcPid)
err = dferrors.Newf(base.Code_SchedPeerNotFound, "peer %s not found", pieceResult.SrcPid)
span.RecordError(err)
return err
}
if peer.Task.IsFail() {
err = dferrors.Newf(dfcodes.SchedTaskStatusError, "peer's task status is fail, task status %s", peer.Task.GetStatus())
err = dferrors.Newf(base.Code_SchedTaskStatusError, "peer's task status is fail, task status %s", peer.Task.GetStatus())
span.RecordError(err)
return err
}
conn, ok := peer.BindNewConn(stream)
if !ok {
err = dferrors.Newf(dfcodes.SchedPeerPieceResultReportFail, "peer can not bind conn")
err = dferrors.Newf(base.Code_SchedPeerPieceResultReportFail, "peer can not bind conn")
span.RecordError(err)
return err
}
@ -194,7 +193,7 @@ func (s *server) ReportPeerResult(ctx context.Context, result *scheduler.PeerRes
peer, ok := s.service.GetPeer(result.PeerId)
if !ok {
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(base.Code_SchedPeerNotFound, "peer %s not found", result.PeerId)
span.RecordError(err)
return err
}

View File

@ -31,11 +31,11 @@ import (
"go.opentelemetry.io/otel/trace"
"google.golang.org/grpc"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/internal/idgen"
"d7y.io/dragonfly/v2/pkg/basic/dfnet"
"d7y.io/dragonfly/v2/pkg/rpc/base"
"d7y.io/dragonfly/v2/pkg/rpc/cdnsystem"
cdnclient "d7y.io/dragonfly/v2/pkg/rpc/cdnsystem/client"
"d7y.io/dragonfly/v2/scheduler/config"
@ -110,9 +110,9 @@ func (c *cdn) StartSeedTask(ctx context.Context, task *Task) (*Peer, error) {
if cdnErr, ok := err.(*dferrors.DfError); ok {
logger.Errorf("failed to obtain cdn seed: %v", cdnErr)
switch cdnErr.Code {
case dfcodes.CdnTaskRegistryFail:
case base.Code_CDNTaskRegistryFail:
return nil, errors.Wrap(ErrCDNRegisterFail, "obtain seeds")
case dfcodes.CdnTaskDownloadFail:
case base.Code_CDNTaskDownloadFail:
return nil, errors.Wrapf(ErrCDNDownloadFail, "obtain seeds")
default:
return nil, errors.Wrapf(ErrCDNUnknown, "obtain seeds")
@ -145,9 +145,9 @@ func (c *cdn) receivePiece(ctx context.Context, task *Task, stream *cdnclient.Pi
logger.Errorf("task %s add piece err %v", task.ID, err)
if recvErr, ok := err.(*dferrors.DfError); ok {
switch recvErr.Code {
case dfcodes.CdnTaskRegistryFail:
case base.Code_CDNTaskRegistryFail:
return nil, errors.Wrapf(ErrCDNRegisterFail, "receive piece")
case dfcodes.CdnTaskDownloadFail:
case base.Code_CDNTaskDownloadFail:
return nil, errors.Wrapf(ErrCDNDownloadFail, "receive piece")
default:
return nil, errors.Wrapf(ErrCDNUnknown, "recive piece")

View File

@ -31,7 +31,6 @@ import (
"github.com/pkg/errors"
"github.com/stretchr/testify/assert"
"d7y.io/dragonfly/v2/internal/dfcodes"
"d7y.io/dragonfly/v2/internal/dferrors"
logger "d7y.io/dragonfly/v2/internal/dflog"
"d7y.io/dragonfly/v2/pkg/rpc/base"
@ -102,13 +101,13 @@ func TestCDN_Initial(t *testing.T) {
expect func(t *testing.T, cdn supervisor.CDN, peer *supervisor.Peer, err error)
}{
{
name: "ObtainSeeds cause CdnTaskRegistryFail",
name: "ObtainSeeds cause CDNTaskRegistryFail",
status: supervisor.TaskStatusWaiting,
mock: func(t *testing.T) (supervisor.CDNDynmaicClient, supervisor.PeerManager, supervisor.HostManager, *gomonkey.Patches) {
ctl := gomock.NewController(t)
defer ctl.Finish()
err := dferrors.New(dfcodes.CdnTaskRegistryFail, "mockError")
err := dferrors.New(base.Code_CDNTaskRegistryFail, "mockError")
mockCDNDynmaicClient := mocks.NewMockCDNDynmaicClient(ctl)
mockPeerManager := mocks.NewMockPeerManager(ctl)
mockHostManager := mocks.NewMockHostManager(ctl)
@ -124,13 +123,13 @@ func TestCDN_Initial(t *testing.T) {
},
},
{
name: "ObtainSeeds cause CdnTaskDownloadFail",
name: "ObtainSeeds cause CDNTaskDownloadFail",
status: supervisor.TaskStatusWaiting,
mock: func(t *testing.T) (supervisor.CDNDynmaicClient, supervisor.PeerManager, supervisor.HostManager, *gomonkey.Patches) {
ctl := gomock.NewController(t)
defer ctl.Finish()
err := dferrors.New(dfcodes.CdnTaskDownloadFail, "mockError")
err := dferrors.New(base.Code_CDNTaskDownloadFail, "mockError")
mockCDNDynmaicClient := mocks.NewMockCDNDynmaicClient(ctl)
mockPeerManager := mocks.NewMockPeerManager(ctl)
mockHostManager := mocks.NewMockHostManager(ctl)
@ -265,7 +264,7 @@ func TestCDN_Initial(t *testing.T) {
},
},
{
name: "receivePiece cause CdnTaskRegistryFail",
name: "receivePiece cause CDNTaskRegistryFail",
status: supervisor.TaskStatusWaiting,
mock: func(t *testing.T) (supervisor.CDNDynmaicClient, supervisor.PeerManager, supervisor.HostManager, *gomonkey.Patches) {
ctl := gomock.NewController(t)
@ -276,7 +275,7 @@ func TestCDN_Initial(t *testing.T) {
mockHostManager := mocks.NewMockHostManager(ctl)
mockCDNDynmaicClient.EXPECT().ObtainSeeds(gomock.Any(), gomock.Any()).Return(mockPieceSeedStream, nil).AnyTimes()
err := dferrors.New(dfcodes.CdnTaskRegistryFail, "mockError")
err := dferrors.New(base.Code_CDNTaskRegistryFail, "mockError")
streamRet := []gomonkey.OutputCell{
{Values: gomonkey.Params{nil, err}},
}
@ -290,7 +289,7 @@ func TestCDN_Initial(t *testing.T) {
},
},
{
name: "receivePiece cause CdnTaskDownloadFail",
name: "receivePiece cause CDNTaskDownloadFail",
status: supervisor.TaskStatusWaiting,
mock: func(t *testing.T) (supervisor.CDNDynmaicClient, supervisor.PeerManager, supervisor.HostManager, *gomonkey.Patches) {
ctl := gomock.NewController(t)
@ -301,7 +300,7 @@ func TestCDN_Initial(t *testing.T) {
mockHostManager := mocks.NewMockHostManager(ctl)
mockCDNDynmaicClient.EXPECT().ObtainSeeds(gomock.Any(), gomock.Any()).Return(mockPieceSeedStream, nil).AnyTimes()
err := dferrors.New(dfcodes.CdnTaskDownloadFail, "mockError")
err := dferrors.New(base.Code_CDNTaskDownloadFail, "mockError")
streamRet := []gomonkey.OutputCell{
{Values: gomonkey.Params{nil, err}},
}