From 60bee061d2d8992c28e276056540a84d136d1dea Mon Sep 17 00:00:00 2001 From: sunwp <244372610@qq.com> Date: Mon, 6 Sep 2021 10:28:07 +0800 Subject: [PATCH] feat: client back source (#579) * feat: client back source Signed-off-by: santong <244372610@qq.com> --- cdnsystem/rpcserver/rpcserver.go | 1 + client/daemon/peer/peertask_base.go | 10 +- client/daemon/peer/peertask_file.go | 22 +- client/daemon/peer/peertask_file_callback.go | 51 +- client/daemon/peer/peertask_manager.go | 1 + .../daemon/peer/peertask_manager_mock_test.go | 14 +- client/daemon/peer/peertask_stream.go | 19 +- .../daemon/peer/peertask_stream_callback.go | 51 +- client/daemon/peer/piece_manager.go | 14 +- client/daemon/peer/piece_manager_test.go | 3 + cmd/scheduler/cmd/root.go | 7 +- internal/dfcodes/rpc_code.go | 14 +- internal/dferrors/error.go | 2 +- pkg/rpc/base/base.pb.go | 9 +- pkg/rpc/base/base.pb.validate.go | 240 +++++- pkg/rpc/cdnsystem/cdnsystem.pb.go | 9 +- pkg/rpc/cdnsystem/cdnsystem.pb.validate.go | 115 ++- pkg/rpc/cdnsystem/cdnsystem_grpc.pb.go | 10 +- pkg/rpc/cdnsystem/client/piece_seed_stream.go | 20 +- pkg/rpc/client.go | 50 +- pkg/rpc/client_util.go | 7 +- pkg/rpc/dfdaemon/client/down_result_stream.go | 18 +- pkg/rpc/dfdaemon/dfdaemon.pb.go | 9 +- pkg/rpc/dfdaemon/dfdaemon.pb.validate.go | 94 ++- pkg/rpc/dfdaemon/dfdaemon_grpc.pb.go | 10 +- pkg/rpc/manager/manager.pb.go | 9 +- pkg/rpc/manager/manager.pb.validate.go | 741 ++++++++++++++++-- pkg/rpc/manager/manager_grpc.pb.go | 10 +- pkg/rpc/scheduler/client/client.go | 4 +- .../scheduler/client/peer_packet_stream.go | 22 +- pkg/rpc/scheduler/scheduler.go | 23 +- pkg/rpc/scheduler/scheduler.pb.go | 231 +++--- pkg/rpc/scheduler/scheduler.pb.validate.go | 529 ++++++++++++- pkg/rpc/scheduler/scheduler.proto | 6 +- pkg/rpc/scheduler/scheduler_grpc.pb.go | 10 +- pkg/rpc/scheduler/server/server.go | 10 +- scheduler/config/config.go | 11 +- scheduler/config/constants_otel.go | 20 +- .../core/evaluator/basic/basic_evaluator.go | 2 +- scheduler/core/events.go | 144 +++- .../core/scheduler/basic/basic_scheduler.go | 124 +-- scheduler/core/service.go | 137 ++-- scheduler/core/worker.go | 3 - scheduler/job/job.go | 10 +- scheduler/rpcserver/rpcserver.go | 160 ++-- scheduler/scheduler.go | 2 +- scheduler/supervisor/cdn/manager.go | 13 +- scheduler/supervisor/peer.go | 210 ++++- scheduler/supervisor/peer/manager.go | 54 +- scheduler/supervisor/peer_mgr.go | 4 - scheduler/supervisor/task.go | 189 +++-- scheduler/supervisor/task/manager.go | 7 +- 52 files changed, 2676 insertions(+), 809 deletions(-) diff --git a/cdnsystem/rpcserver/rpcserver.go b/cdnsystem/rpcserver/rpcserver.go index 33c4188d1..83ab895be 100644 --- a/cdnsystem/rpcserver/rpcserver.go +++ b/cdnsystem/rpcserver/rpcserver.go @@ -207,6 +207,7 @@ func (css *CdnSeedServer) GetPieceTasks(ctx context.Context, req *base.PieceTask logger.WithTaskID(req.TaskId).Errorf("failed to get piece tasks, req=%+v: %v", req, err) } }() + logger.Infof("get piece tasks: %+v", req) if err := checkPieceTasksRequestParams(req); err != nil { err = dferrors.Newf(dfcodes.BadRequest, "failed to validate seed request for task(%s): %v", req.TaskId, err) span.RecordError(err) diff --git a/client/daemon/peer/peertask_base.go b/client/daemon/peer/peertask_base.go index 51b8a13f4..64149e847 100644 --- a/client/daemon/peer/peertask_base.go +++ b/client/daemon/peer/peertask_base.go @@ -68,6 +68,7 @@ type peerTask struct { backSourceFunc func() reportPieceResultFunc func(result *pieceTaskResult) error setContentLengthFunc func(i int64) error + setTotalPiecesFunc func(i int32) request *scheduler.PeerTaskRequest @@ -172,6 +173,10 @@ func (pt *peerTask) GetTotalPieces() int32 { return pt.totalPiece } +func (pt *peerTask) SetTotalPieces(i int32) { + pt.setTotalPiecesFunc(i) +} + func (pt *peerTask) Context() context.Context { return pt.ctx } @@ -369,6 +374,7 @@ func (pt *peerTask) pullPiecesFromPeers(cleanUnfinishedFunc func()) { }() if !pt.waitFirstPeerPacket() { + // TODO 如果是客户端直接回源,这里不应该在输出错误日志 pt.Errorf("wait first peer packet error") return } @@ -650,7 +656,7 @@ func (pt *peerTask) downloadPieceWorker(id int32, pti Task, requests chan *Downl TaskId: pt.GetTaskID(), SrcPid: pt.GetPeerID(), DstPid: request.DstPid, - PieceNum: request.piece.PieceNum, + PieceInfo: request.piece, Success: false, Code: dfcodes.ClientRequestLimitFail, HostLoad: nil, @@ -761,6 +767,7 @@ retry: TaskId: pt.taskID, SrcPid: pt.peerID, DstPid: peer.PeerId, + PieceInfo: &base.PieceInfo{}, Success: false, Code: code, HostLoad: nil, @@ -805,6 +812,7 @@ func (pt *peerTask) getPieceTasks(span trace.Span, curPeerPacket *scheduler.Peer TaskId: pt.taskID, SrcPid: pt.peerID, DstPid: peer.PeerId, + PieceInfo: &base.PieceInfo{}, Success: false, Code: dfcodes.ClientWaitPieceReady, HostLoad: nil, diff --git a/client/daemon/peer/peertask_file.go b/client/daemon/peer/peertask_file.go index f0f7ab48c..2ee6bdba7 100644 --- a/client/daemon/peer/peertask_file.go +++ b/client/daemon/peer/peertask_file.go @@ -208,6 +208,7 @@ func newFilePeerTask(ctx context.Context, // bind func that base peer task did not implement pt.backSourceFunc = pt.backSource pt.setContentLengthFunc = pt.SetContentLength + pt.setTotalPiecesFunc = pt.SetTotalPieces pt.reportPieceResultFunc = pt.ReportPieceResult return ctx, pt, nil, nil } @@ -237,19 +238,19 @@ func (pt *filePeerTask) ReportPieceResult(result *pieceTaskResult) error { if !result.pieceResult.Success { result.pieceResult.FinishedCount = pt.readyPieces.Settled() _ = pt.peerPacketStream.Send(result.pieceResult) - pt.failedPieceCh <- result.pieceResult.PieceNum + pt.failedPieceCh <- result.pieceResult.PieceInfo.PieceNum pt.Errorf("%d download failed, retry later", result.piece.PieceNum) return nil } pt.lock.Lock() - if pt.readyPieces.IsSet(result.pieceResult.PieceNum) { + if pt.readyPieces.IsSet(result.pieceResult.PieceInfo.PieceNum) { pt.lock.Unlock() - pt.Warnf("piece %d is already reported, skipped", result.pieceResult.PieceNum) + pt.Warnf("piece %d is already reported, skipped", result.pieceResult.PieceInfo.PieceNum) return nil } // mark piece processed - pt.readyPieces.Set(result.pieceResult.PieceNum) + pt.readyPieces.Set(result.pieceResult.PieceInfo.PieceNum) pt.completedLength.Add(int64(result.piece.RangeSize)) pt.lock.Unlock() @@ -420,6 +421,7 @@ func (pt *filePeerTask) cleanUnfinished() { }) } +// TODO SetContentLength 需要和pt.finish解绑,以便在下载进度处可以看到文件长度 func (pt *filePeerTask) SetContentLength(i int64) error { pt.contentLength.Store(i) if !pt.isCompleted() { @@ -429,6 +431,10 @@ func (pt *filePeerTask) SetContentLength(i int64) error { return pt.finish() } +func (pt *filePeerTask) SetTotalPieces(i int32) { + pt.totalPiece = i +} + func (pt *filePeerTask) backSource() { defer pt.cleanUnfinished() if pt.disableBackSource { @@ -436,6 +442,13 @@ func (pt *filePeerTask) backSource() { pt.failedReason = reasonBackSourceDisabled return } + _ = pt.callback.Init(pt) + if peerPacketStream, err := pt.schedulerClient.ReportPieceResult(pt.ctx, pt.taskID, pt.request); err != nil { + logger.Errorf("step 2: peer %s report piece failed: err", pt.request.PeerId, err) + } else { + pt.peerPacketStream = peerPacketStream + } + logger.Infof("step 2: start report peer %s back source piece result", pt.request.PeerId) err := pt.pieceManager.DownloadSource(pt.ctx, pt, pt.request) if err != nil { pt.Errorf("download from source error: %s", err) @@ -444,5 +457,4 @@ func (pt *filePeerTask) backSource() { } pt.Infof("download from source ok") _ = pt.finish() - return } diff --git a/client/daemon/peer/peertask_file_callback.go b/client/daemon/peer/peertask_file_callback.go index cdd1c5a82..cd07cd9cd 100644 --- a/client/daemon/peer/peertask_file_callback.go +++ b/client/daemon/peer/peertask_file_callback.go @@ -17,7 +17,6 @@ package peer import ( - "context" "time" "d7y.io/dragonfly/v2/client/daemon/storage" @@ -92,18 +91,19 @@ func (p *filePeerTaskCallback) Done(pt Task) error { return e } p.ptm.PeerTaskDone(p.req.PeerId) - err := p.pt.schedulerClient.ReportPeerResult(context.Background(), &scheduler.PeerResult{ - TaskId: pt.GetTaskID(), - PeerId: pt.GetPeerID(), - SrcIp: p.ptm.host.Ip, - SecurityDomain: p.ptm.host.SecurityDomain, - Idc: p.ptm.host.Idc, - Url: p.req.Url, - ContentLength: pt.GetContentLength(), - Traffic: pt.GetTraffic(), - Cost: uint32(cost), - Success: true, - Code: dfcodes.Success, + err := p.pt.schedulerClient.ReportPeerResult(p.pt.ctx, &scheduler.PeerResult{ + TaskId: pt.GetTaskID(), + PeerId: pt.GetPeerID(), + SrcIp: p.ptm.host.Ip, + SecurityDomain: p.ptm.host.SecurityDomain, + Idc: p.ptm.host.Idc, + Url: p.req.Url, + ContentLength: pt.GetContentLength(), + Traffic: pt.GetTraffic(), + TotalPieceCount: pt.GetTotalPieces(), + Cost: uint32(cost), + Success: true, + Code: dfcodes.Success, }) if err != nil { pt.Log().Errorf("step 3: report successful peer result, error: %v", err) @@ -117,18 +117,19 @@ func (p *filePeerTaskCallback) Fail(pt Task, code base.Code, reason string) erro p.ptm.PeerTaskDone(p.req.PeerId) var end = time.Now() pt.Log().Errorf("file peer task failed, code: %d, reason: %s", code, reason) - err := p.pt.schedulerClient.ReportPeerResult(context.Background(), &scheduler.PeerResult{ - TaskId: pt.GetTaskID(), - PeerId: pt.GetPeerID(), - SrcIp: p.ptm.host.Ip, - SecurityDomain: p.ptm.host.SecurityDomain, - Idc: p.ptm.host.Idc, - Url: p.req.Url, - ContentLength: pt.GetContentLength(), - Traffic: pt.GetTraffic(), - Cost: uint32(end.Sub(p.start).Milliseconds()), - Success: false, - Code: code, + err := p.pt.schedulerClient.ReportPeerResult(p.pt.ctx, &scheduler.PeerResult{ + TaskId: pt.GetTaskID(), + PeerId: pt.GetPeerID(), + SrcIp: p.ptm.host.Ip, + SecurityDomain: p.ptm.host.SecurityDomain, + Idc: p.ptm.host.Idc, + Url: p.req.Url, + ContentLength: pt.GetContentLength(), + Traffic: pt.GetTraffic(), + TotalPieceCount: p.pt.totalPiece, + Cost: uint32(end.Sub(p.start).Milliseconds()), + Success: false, + Code: code, }) if err != nil { pt.Log().Errorf("step 3: report fail peer result, error: %v", err) diff --git a/client/daemon/peer/peertask_manager.go b/client/daemon/peer/peertask_manager.go index 452329fa7..f89431cef 100644 --- a/client/daemon/peer/peertask_manager.go +++ b/client/daemon/peer/peertask_manager.go @@ -66,6 +66,7 @@ type Task interface { GetPeerID() string GetTaskID() string GetTotalPieces() int32 + SetTotalPieces(int32) GetContentLength() int64 // SetContentLength will be called after download completed, when download from source without content length SetContentLength(int64) error diff --git a/client/daemon/peer/peertask_manager_mock_test.go b/client/daemon/peer/peertask_manager_mock_test.go index d4ea54326..1ead1e6ba 100644 --- a/client/daemon/peer/peertask_manager_mock_test.go +++ b/client/daemon/peer/peertask_manager_mock_test.go @@ -1,5 +1,5 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: peertask_manager.go +// Source: client/daemon/peer/peertask_manager.go // Package peer is a generated GoMock package. package peer @@ -272,6 +272,18 @@ func (mr *MockTaskMockRecorder) SetContentLength(arg0 interface{}) *gomock.Call return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetContentLength", reflect.TypeOf((*MockTask)(nil).SetContentLength), arg0) } +// SetTotalPieces mocks base method. +func (m *MockTask) SetTotalPieces(arg0 int32) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "SetTotalPieces", arg0) +} + +// SetTotalPieces indicates an expected call of SetTotalPieces. +func (mr *MockTaskMockRecorder) SetTotalPieces(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetTotalPieces", reflect.TypeOf((*MockTask)(nil).SetTotalPieces), arg0) +} + // MockTaskCallback is a mock of TaskCallback interface. type MockTaskCallback struct { ctrl *gomock.Controller diff --git a/client/daemon/peer/peertask_stream.go b/client/daemon/peer/peertask_stream.go index 44cb55b99..e062a589c 100644 --- a/client/daemon/peer/peertask_stream.go +++ b/client/daemon/peer/peertask_stream.go @@ -184,6 +184,7 @@ func newStreamPeerTask(ctx context.Context, // bind func that base peer task did not implement pt.backSourceFunc = pt.backSource pt.setContentLengthFunc = pt.SetContentLength + pt.setTotalPiecesFunc = pt.SetTotalPieces pt.reportPieceResultFunc = pt.ReportPieceResult return ctx, pt, nil, nil } @@ -193,18 +194,18 @@ func (s *streamPeerTask) ReportPieceResult(result *pieceTaskResult) error { // retry failed piece if !result.pieceResult.Success { _ = s.peerPacketStream.Send(result.pieceResult) - s.failedPieceCh <- result.pieceResult.PieceNum + s.failedPieceCh <- result.pieceResult.PieceInfo.PieceNum return nil } s.lock.Lock() - if s.readyPieces.IsSet(result.pieceResult.PieceNum) { + if s.readyPieces.IsSet(result.pieceResult.PieceInfo.PieceNum) { s.lock.Unlock() - s.Warnf("piece %d is already reported, skipped", result.pieceResult.PieceNum) + s.Warnf("piece %d is already reported, skipped", result.pieceResult.PieceInfo.PieceNum) return nil } // mark piece processed - s.readyPieces.Set(result.pieceResult.PieceNum) + s.readyPieces.Set(result.pieceResult.PieceInfo.PieceNum) s.completedLength.Add(int64(result.piece.RangeSize)) s.lock.Unlock() @@ -416,6 +417,10 @@ func (s *streamPeerTask) SetContentLength(i int64) error { return s.finish() } +func (s *streamPeerTask) SetTotalPieces(i int32) { + s.totalPiece = i +} + func (s *streamPeerTask) writeTo(w io.Writer, pieceNum int32) (int64, error) { pr, pc, err := s.pieceManager.ReadPiece(s.ctx, &storage.ReadPieceRequest{ PeerTaskMetaData: storage.PeerTaskMetaData{ @@ -440,6 +445,12 @@ func (s *streamPeerTask) writeTo(w io.Writer, pieceNum int32) (int64, error) { func (s *streamPeerTask) backSource() { s.contentLength.Store(-1) _ = s.callback.Init(s) + if peerPacketStream, err := s.schedulerClient.ReportPieceResult(s.ctx, s.taskID, s.request); err != nil { + logger.Errorf("step 2: peer %s report piece failed: err", s.request.PeerId, err) + } else { + s.peerPacketStream = peerPacketStream + } + logger.Infof("step 2: start report peer %s back source piece result", s.request.PeerId) err := s.pieceManager.DownloadSource(s.ctx, s, s.request) if err != nil { s.Errorf("download from source error: %s", err) diff --git a/client/daemon/peer/peertask_stream_callback.go b/client/daemon/peer/peertask_stream_callback.go index 917de1f3b..c30817060 100644 --- a/client/daemon/peer/peertask_stream_callback.go +++ b/client/daemon/peer/peertask_stream_callback.go @@ -17,7 +17,6 @@ package peer import ( - "context" "time" "d7y.io/dragonfly/v2/client/daemon/storage" @@ -90,18 +89,19 @@ func (p *streamPeerTaskCallback) Done(pt Task) error { return e } p.ptm.PeerTaskDone(p.req.PeerId) - err := p.pt.schedulerClient.ReportPeerResult(context.Background(), &scheduler.PeerResult{ - TaskId: pt.GetTaskID(), - PeerId: pt.GetPeerID(), - SrcIp: p.ptm.host.Ip, - SecurityDomain: p.ptm.host.SecurityDomain, - Idc: p.ptm.host.Idc, - Url: p.req.Url, - ContentLength: pt.GetContentLength(), - Traffic: pt.GetTraffic(), - Cost: uint32(cost), - Success: true, - Code: dfcodes.Success, + err := p.pt.schedulerClient.ReportPeerResult(p.pt.ctx, &scheduler.PeerResult{ + TaskId: pt.GetTaskID(), + PeerId: pt.GetPeerID(), + SrcIp: p.ptm.host.Ip, + SecurityDomain: p.ptm.host.SecurityDomain, + Idc: p.ptm.host.Idc, + Url: p.req.Url, + ContentLength: pt.GetContentLength(), + Traffic: pt.GetTraffic(), + TotalPieceCount: p.pt.totalPiece, + Cost: uint32(cost), + Success: true, + Code: dfcodes.Success, }) if err != nil { pt.Log().Errorf("step 3: report successful peer result, error: %v", err) @@ -115,18 +115,19 @@ func (p *streamPeerTaskCallback) Fail(pt Task, code base.Code, reason string) er p.ptm.PeerTaskDone(p.req.PeerId) var end = time.Now() pt.Log().Errorf("stream peer task failed, code: %d, reason: %s", code, reason) - err := p.pt.schedulerClient.ReportPeerResult(context.Background(), &scheduler.PeerResult{ - TaskId: pt.GetTaskID(), - PeerId: pt.GetPeerID(), - SrcIp: p.ptm.host.Ip, - SecurityDomain: p.ptm.host.SecurityDomain, - Idc: p.ptm.host.Idc, - Url: p.req.Url, - ContentLength: pt.GetContentLength(), - Traffic: pt.GetTraffic(), - Cost: uint32(end.Sub(p.start).Milliseconds()), - Success: false, - Code: code, + err := p.pt.schedulerClient.ReportPeerResult(p.pt.ctx, &scheduler.PeerResult{ + TaskId: pt.GetTaskID(), + PeerId: pt.GetPeerID(), + SrcIp: p.ptm.host.Ip, + SecurityDomain: p.ptm.host.SecurityDomain, + Idc: p.ptm.host.Idc, + Url: p.req.Url, + ContentLength: pt.GetContentLength(), + Traffic: pt.GetTraffic(), + TotalPieceCount: p.pt.totalPiece, + Cost: uint32(end.Sub(p.start).Milliseconds()), + Success: false, + Code: code, }) if err != nil { pt.Log().Errorf("step 3: report fail peer result, error: %v", err) diff --git a/client/daemon/peer/piece_manager.go b/client/daemon/peer/piece_manager.go index 0b8dc6f6e..a5e04bc5f 100644 --- a/client/daemon/peer/piece_manager.go +++ b/client/daemon/peer/piece_manager.go @@ -173,13 +173,14 @@ func (pm *pieceManager) pushSuccessResult(peerTask Task, dstPid string, piece *b TaskId: peerTask.GetTaskID(), SrcPid: peerTask.GetPeerID(), DstPid: dstPid, - PieceNum: piece.PieceNum, + PieceInfo: piece, BeginTime: uint64(start), EndTime: uint64(end), Success: true, Code: dfcodes.Success, - HostLoad: nil, // TODO(jim): update host load - FinishedCount: 0, // update by peer task + HostLoad: nil, // TODO(jim): update host load + FinishedCount: piece.PieceNum + 1, // update by peer task + // TODO range_start, range_size, piece_md5, piece_offset, piece_style }, err: nil, }) @@ -196,7 +197,7 @@ func (pm *pieceManager) pushFailResult(peerTask Task, dstPid string, piece *base TaskId: peerTask.GetTaskID(), SrcPid: peerTask.GetPeerID(), DstPid: dstPid, - PieceNum: piece.PieceNum, + PieceInfo: piece, BeginTime: uint64(start), EndTime: uint64(end), Success: false, @@ -371,10 +372,11 @@ func (pm *pieceManager) DownloadSource(ctx context.Context, pt Task, request *sc }, ContentLength: contentLength, }) + pt.SetTotalPieces(pieceNum + 1) return pt.SetContentLength(contentLength) } } - // unreachable code + //unreachable code //return nil } @@ -398,6 +400,8 @@ func (pm *pieceManager) DownloadSource(ctx context.Context, pt Task, request *sc return storage.ErrShortRead } } + pt.SetTotalPieces(maxPieceNum) + pt.SetContentLength(contentLength) log.Infof("download from source ok") return nil } diff --git a/client/daemon/peer/piece_manager_test.go b/client/daemon/peer/piece_manager_test.go index e5fe8e446..abd9bde60 100644 --- a/client/daemon/peer/piece_manager_test.go +++ b/client/daemon/peer/piece_manager_test.go @@ -130,6 +130,9 @@ func TestPieceManager_DownloadSource(t *testing.T) { func(arg0 int64) error { return nil }) + mockPeerTask.EXPECT().SetTotalPieces(gomock.Any()).AnyTimes().DoAndReturn( + func(arg0 int32) { + }) mockPeerTask.EXPECT().GetPeerID().AnyTimes().DoAndReturn( func() string { return peerID diff --git a/cmd/scheduler/cmd/root.go b/cmd/scheduler/cmd/root.go index 7b44bc04b..12d93dfe2 100644 --- a/cmd/scheduler/cmd/root.go +++ b/cmd/scheduler/cmd/root.go @@ -35,10 +35,9 @@ var ( // rootCmd represents the base command when called without any subcommands var rootCmd = &cobra.Command{ - Use: "scheduler", - Short: "the scheduler of dragonfly", - Long: `scheduler is a long-running process and is mainly responsible -for deciding which peers transmit blocks to each other.`, + Use: "scheduler", + Short: "the scheduler of dragonfly", + Long: `scheduler is a long-running process and is mainly responsible for deciding which peers transmit blocks to each other.`, Args: cobra.NoArgs, DisableAutoGenTag: true, SilenceUsage: true, diff --git a/internal/dfcodes/rpc_code.go b/internal/dfcodes/rpc_code.go index a1b4aeea9..1a3f9882b 100644 --- a/internal/dfcodes/rpc_code.go +++ b/internal/dfcodes/rpc_code.go @@ -25,7 +25,7 @@ const ( ServerUnavailable base.Code = 500 // framework can not find server node // common response error 1000-1999 - ResourceLacked base.Code = 1000 // client can be migrated to another scheduler + ResourceLacked base.Code = 1000 // client can be migrated to another scheduler/CDN BadRequest base.Code = 1400 PeerTaskNotFound base.Code = 1404 UnknownError base.Code = 1500 @@ -41,16 +41,12 @@ const ( ClientRequestLimitFail base.Code = 4006 // scheduler response error 5000-5999 - SchedError base.Code = 5000 - /** @deprecated */ + SchedError base.Code = 5000 SchedNeedBackSource base.Code = 5001 // client should try to download from source SchedPeerGone base.Code = 5002 // client should disconnect from scheduler - SchedPeerRegisterFail base.Code = 5003 - SchedPeerScheduleFail base.Code = 5004 - SchedPeerNotFound base.Code = 5005 - SchedPeerPieceResultReportFail base.Code = 5006 - SchedCDNSeedFail base.Code = 5007 - SchedTaskStatusError base.Code = 5008 + SchedPeerNotFound base.Code = 5004 // peer not found in scheduler + SchedPeerPieceResultReportFail base.Code = 5005 // report piece + SchedTaskStatusError base.Code = 5006 // task status is fail // cdnsystem response error 6000-6999 CdnError base.Code = 6000 diff --git a/internal/dferrors/error.go b/internal/dferrors/error.go index ea418a5c3..612d72656 100644 --- a/internal/dferrors/error.go +++ b/internal/dferrors/error.go @@ -30,7 +30,7 @@ var ( ErrEmptyValue = errors.New("empty value") ErrConvertFailed = errors.New("convert failed") ErrEndOfStream = errors.New("end of stream") - ErrNoCandidateNode = errors.New("candidate server node not found") + ErrNoCandidateNode = errors.New("no candidate server node") ) func IsEndOfStream(err error) bool { diff --git a/pkg/rpc/base/base.pb.go b/pkg/rpc/base/base.pb.go index 1533f9194..74097bcb1 100644 --- a/pkg/rpc/base/base.pb.go +++ b/pkg/rpc/base/base.pb.go @@ -15,13 +15,14 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.26.0 -// protoc v3.15.8 +// protoc-gen-go v1.25.0 +// protoc v3.17.3 // source: pkg/rpc/base/base.proto package base import ( + proto "github.com/golang/protobuf/proto" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" @@ -35,6 +36,10 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +// This is a compile-time assertion that a sufficiently up-to-date version +// of the legacy proto package is being used. +const _ = proto.ProtoPackageIsVersion4 + type Code int32 const ( diff --git a/pkg/rpc/base/base.pb.validate.go b/pkg/rpc/base/base.pb.validate.go index 5c4ac4140..e0ae9d0e9 100644 --- a/pkg/rpc/base/base.pb.validate.go +++ b/pkg/rpc/base/base.pb.validate.go @@ -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 { diff --git a/pkg/rpc/cdnsystem/cdnsystem.pb.go b/pkg/rpc/cdnsystem/cdnsystem.pb.go index 745f520f7..36617b57f 100644 --- a/pkg/rpc/cdnsystem/cdnsystem.pb.go +++ b/pkg/rpc/cdnsystem/cdnsystem.pb.go @@ -15,14 +15,15 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.26.0 -// protoc v3.15.8 +// protoc-gen-go v1.25.0 +// protoc v3.17.3 // source: pkg/rpc/cdnsystem/cdnsystem.proto package cdnsystem import ( base "d7y.io/dragonfly/v2/pkg/rpc/base" + proto "github.com/golang/protobuf/proto" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" @@ -36,6 +37,10 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +// This is a compile-time assertion that a sufficiently up-to-date version +// of the legacy proto package is being used. +const _ = proto.ProtoPackageIsVersion4 + type SeedRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache diff --git a/pkg/rpc/cdnsystem/cdnsystem.pb.validate.go b/pkg/rpc/cdnsystem/cdnsystem.pb.validate.go index 68f7b8a9f..be22dc1ef 100644 --- a/pkg/rpc/cdnsystem/cdnsystem.pb.validate.go +++ b/pkg/rpc/cdnsystem/cdnsystem.pb.validate.go @@ -34,18 +34,51 @@ var ( ) // Validate checks the field values on SeedRequest with the rules defined in -// the proto definition for this message. If any rules are violated, an error -// is returned. +// the proto definition for this message. If any rules are violated, the first +// error encountered is returned, or nil if there are no violations. func (m *SeedRequest) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on SeedRequest with the rules defined in +// the proto definition for this message. If any rules are violated, the +// result is a list of violation errors wrapped in SeedRequestMultiError, or +// nil if none found. +func (m *SeedRequest) ValidateAll() error { + return m.validate(true) +} + +func (m *SeedRequest) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for TaskId // no validation rules for Url - if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetUrlMeta()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, SeedRequestValidationError{ + field: "UrlMeta", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, SeedRequestValidationError{ + field: "UrlMeta", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return SeedRequestValidationError{ field: "UrlMeta", @@ -55,9 +88,28 @@ func (m *SeedRequest) Validate() error { } } + if len(errors) > 0 { + return SeedRequestMultiError(errors) + } return nil } +// SeedRequestMultiError is an error wrapping multiple validation errors +// returned by SeedRequest.ValidateAll() if the designated constraints aren't met. +type SeedRequestMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m SeedRequestMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m SeedRequestMultiError) AllErrors() []error { return m } + // SeedRequestValidationError is the validation error returned by // SeedRequest.Validate if the designated constraints aren't met. type SeedRequestValidationError struct { @@ -113,17 +165,51 @@ var _ interface { } = SeedRequestValidationError{} // Validate checks the field values on PieceSeed with the rules defined in the -// proto definition for this message. If any rules are violated, an error is returned. +// proto definition for this message. If any rules are violated, the first +// error encountered is returned, or nil if there are no violations. func (m *PieceSeed) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on PieceSeed with the rules defined in +// the proto definition for this message. If any rules are violated, the +// result is a list of violation errors wrapped in PieceSeedMultiError, or nil +// if none found. +func (m *PieceSeed) ValidateAll() error { + return m.validate(true) +} + +func (m *PieceSeed) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for PeerId // no validation rules for HostUuid - if v, ok := interface{}(m.GetPieceInfo()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetPieceInfo()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, PieceSeedValidationError{ + field: "PieceInfo", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, PieceSeedValidationError{ + field: "PieceInfo", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetPieceInfo()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return PieceSeedValidationError{ field: "PieceInfo", @@ -139,9 +225,28 @@ func (m *PieceSeed) Validate() error { // no validation rules for TotalPieceCount + if len(errors) > 0 { + return PieceSeedMultiError(errors) + } return nil } +// PieceSeedMultiError is an error wrapping multiple validation errors returned +// by PieceSeed.ValidateAll() if the designated constraints aren't met. +type PieceSeedMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m PieceSeedMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m PieceSeedMultiError) AllErrors() []error { return m } + // PieceSeedValidationError is the validation error returned by // PieceSeed.Validate if the designated constraints aren't met. type PieceSeedValidationError struct { diff --git a/pkg/rpc/cdnsystem/cdnsystem_grpc.pb.go b/pkg/rpc/cdnsystem/cdnsystem_grpc.pb.go index f2994552e..d5b70a7c4 100644 --- a/pkg/rpc/cdnsystem/cdnsystem_grpc.pb.go +++ b/pkg/rpc/cdnsystem/cdnsystem_grpc.pb.go @@ -12,7 +12,6 @@ import ( // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -// Requires gRPC-Go v1.32.0 or later. const _ = grpc.SupportPackageIsVersion7 // SeederClient is the client API for Seeder service. @@ -34,7 +33,7 @@ func NewSeederClient(cc grpc.ClientConnInterface) SeederClient { } func (c *seederClient) ObtainSeeds(ctx context.Context, in *SeedRequest, opts ...grpc.CallOption) (Seeder_ObtainSeedsClient, error) { - stream, err := c.cc.NewStream(ctx, &Seeder_ServiceDesc.Streams[0], "/cdnsystem.Seeder/ObtainSeeds", opts...) + stream, err := c.cc.NewStream(ctx, &_Seeder_serviceDesc.Streams[0], "/cdnsystem.Seeder/ObtainSeeds", opts...) if err != nil { return nil, err } @@ -105,7 +104,7 @@ type UnsafeSeederServer interface { } func RegisterSeederServer(s grpc.ServiceRegistrar, srv SeederServer) { - s.RegisterService(&Seeder_ServiceDesc, srv) + s.RegisterService(&_Seeder_serviceDesc, srv) } func _Seeder_ObtainSeeds_Handler(srv interface{}, stream grpc.ServerStream) error { @@ -147,10 +146,7 @@ func _Seeder_GetPieceTasks_Handler(srv interface{}, ctx context.Context, dec fun return interceptor(ctx, in, info, handler) } -// Seeder_ServiceDesc is the grpc.ServiceDesc for Seeder service. -// It's only intended for direct use with grpc.RegisterService, -// and not to be introspected or modified (even as a copy) -var Seeder_ServiceDesc = grpc.ServiceDesc{ +var _Seeder_serviceDesc = grpc.ServiceDesc{ ServiceName: "cdnsystem.Seeder", HandlerType: (*SeederServer)(nil), Methods: []grpc.MethodDesc{ diff --git a/pkg/rpc/cdnsystem/client/piece_seed_stream.go b/pkg/rpc/cdnsystem/client/piece_seed_stream.go index 781426fa1..da42b0615 100644 --- a/pkg/rpc/cdnsystem/client/piece_seed_stream.go +++ b/pkg/rpc/cdnsystem/client/piece_seed_stream.go @@ -18,11 +18,12 @@ package client import ( "context" - "io" + "d7y.io/dragonfly/v2/internal/dferrors" logger "d7y.io/dragonfly/v2/internal/dflog" "d7y.io/dragonfly/v2/pkg/rpc" "d7y.io/dragonfly/v2/pkg/rpc/cdnsystem" + "github.com/pkg/errors" "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -50,9 +51,9 @@ func newPieceSeedStream(ctx context.Context, sc *cdnClient, hashKey string, sr * sr: sr, opts: opts, RetryMeta: rpc.RetryMeta{ - MaxAttempts: 5, - InitBackoff: 0.5, - MaxBackOff: 4.0, + MaxAttempts: 3, + InitBackoff: 0.2, + MaxBackOff: 2.0, }, } @@ -74,7 +75,10 @@ func (pss *PieceSeedStream) initStream() error { return client.ObtainSeeds(pss.ctx, pss.sr, pss.opts...) }, pss.InitBackoff, pss.MaxBackOff, pss.MaxAttempts, nil) if err != nil { - logger.WithTaskID(pss.hashKey).Infof("initStream: invoke cdn node %s ObtainSeeds failed: %v", target, err) + if errors.Cause(err) == dferrors.ErrNoCandidateNode { + return errors.Wrapf(err, "get grpc server instance failed") + } + logger.WithTaskID(pss.hashKey).Errorf("initStream: invoke cdn node %s ObtainSeeds failed: %v", target, err) return pss.replaceClient(pss.hashKey, err) } pss.stream = stream.(cdnsystem.Seeder_ObtainSeedsClient) @@ -84,16 +88,14 @@ func (pss *PieceSeedStream) initStream() error { func (pss *PieceSeedStream) Recv() (ps *cdnsystem.PieceSeed, err error) { pss.sc.UpdateAccessNodeMapByHashKey(pss.hashKey) - if ps, err = pss.stream.Recv(); err != nil && err != io.EOF { - ps, err = pss.retryRecv(err) - } - return + return pss.stream.Recv() } func (pss *PieceSeedStream) retryRecv(cause error) (*cdnsystem.PieceSeed, error) { if status.Code(cause) == codes.DeadlineExceeded || status.Code(cause) == codes.Canceled { return nil, cause } + if err := pss.replaceStream(cause); err != nil { return nil, err } diff --git a/pkg/rpc/client.go b/pkg/rpc/client.go index bdf2bbbd3..b9968d8d7 100644 --- a/pkg/rpc/client.go +++ b/pkg/rpc/client.go @@ -180,14 +180,15 @@ func (conn *Connection) UpdateAccessNodeMapByHashKey(key string) { node, ok := conn.key2NodeMap.Load(key) if ok { conn.accessNodeMap.Store(node, time.Now()) - logger.With("conn", conn.name).Debugf("successfully update server node %s access time for hashKey %s", node, key) _, ok := conn.node2ClientMap.Load(node) - if !ok { - logger.With("conn", conn.name).Warnf("successfully update server node %s access time for hashKey %s,"+ + if ok { + logger.GrpcLogger.With("conn", conn.name).Debugf("successfully update server node %s access time for hashKey %s", node, key) + } else { + logger.GrpcLogger.With("conn", conn.name).Warnf("successfully update server node %s access time for hashKey %s,"+ "but cannot found client conn in node2ClientMap", node, key) } } else { - logger.With("conn", conn.name).Errorf("update access node map failed, hash key (%s) not found in key2NodeMap", key) + logger.GrpcLogger.With("conn", conn.name).Errorf("update access node map failed, hash key (%s) not found in key2NodeMap", key) } } @@ -201,7 +202,7 @@ func (conn *Connection) AddServerNodes(addrs []dfnet.NetAddr) error { for _, addr := range addrs { serverNode := addr.GetEndpoint() conn.hashRing = conn.hashRing.AddNode(serverNode) - logger.With("conn", conn.name).Debugf("success add %s to server node list", addr) + logger.GrpcLogger.With("conn", conn.name).Debugf("success add %s to server node list", addr) } return nil } @@ -222,7 +223,7 @@ func (conn *Connection) findCandidateClientConn(key string, exclusiveNodes sets. ringNodes, ok := conn.hashRing.GetNodes(key, conn.hashRing.Size()) if !ok { - logger.Warnf("cannot obtain expected %d server nodes", conn.hashRing.Size()) + logger.GrpcLogger.Warnf("cannot obtain expected %d server nodes", conn.hashRing.Size()) } if len(ringNodes) == 0 { return nil, dferrors.ErrNoCandidateNode @@ -233,29 +234,30 @@ func (conn *Connection) findCandidateClientConn(key string, exclusiveNodes sets. candidateNodes = append(candidateNodes, ringNode) } } - logger.With("conn", conn.name).Infof("candidate result for hash key %s: all server node list: %v, exclusiveNodes node list: %v, candidate node list: %v", + logger.GrpcLogger.With("conn", conn.name).Infof("candidate result for hash key %s: all server node list: %v, exclusiveNodes node list: %v, "+ + "candidate node list: %v", key, ringNodes, exclusiveNodes.List(), candidateNodes) for _, candidateNode := range candidateNodes { // Check whether there is a corresponding mapping client in the node2ClientMap // TODO 下面部分可以直接调用loadOrCreate方法,但是日志没有这么调用打印全 if client, ok := conn.node2ClientMap.Load(candidateNode); ok { - logger.With("conn", conn.name).Debugf("hit cache candidateNode %s for hash key %s", candidateNode, key) + logger.GrpcLogger.With("conn", conn.name).Debugf("hit cache candidateNode %s for hash key %s", candidateNode, key) return &candidateClient{ node: candidateNode, Ref: client, }, nil } - logger.With("conn", conn.name).Debugf("attempt to connect candidateNode %s for hash key %s", candidateNode, key) + logger.GrpcLogger.With("conn", conn.name).Debugf("attempt to connect candidateNode %s for hash key %s", candidateNode, key) clientConn, err := conn.createClient(candidateNode, append(defaultClientOpts, conn.dialOpts...)...) if err == nil { - logger.With("conn", conn.name).Infof("success connect to candidateNode %s for hash key %s", candidateNode, key) + logger.GrpcLogger.With("conn", conn.name).Infof("success connect to candidateNode %s for hash key %s", candidateNode, key) return &candidateClient{ node: candidateNode, Ref: clientConn, }, nil } - logger.With("conn", conn.name).Infof("failed to connect candidateNode %s for hash key %s: %v", candidateNode, key, err) + logger.GrpcLogger.With("conn", conn.name).Infof("failed to connect candidateNode %s for hash key %s: %v", candidateNode, key, err) } return nil, dferrors.ErrNoCandidateNode } @@ -285,14 +287,14 @@ func (conn *Connection) GetServerNode(hashKey string) (string, bool) { } func (conn *Connection) GetClientConnByTarget(node string) (*grpc.ClientConn, error) { - logger.With("conn", conn.name).Debugf("start to get client conn by target %s", node) + logger.GrpcLogger.With("conn", conn.name).Debugf("start to get client conn by target %s", node) conn.rwMutex.RLock() defer conn.rwMutex.RUnlock() clientConn, err := conn.loadOrCreateClientConnByNode(node) if err != nil { return nil, errors.Wrapf(err, "get client conn by conn %s", node) } - logger.With("conn", conn.name).Debugf("successfully get %s client conn", node) + logger.GrpcLogger.With("conn", conn.name).Debugf("successfully get %s client conn", node) return clientConn, nil } @@ -305,14 +307,14 @@ func (conn *Connection) loadOrCreateClientConnByNode(node string) (clientConn *g conn.accessNodeMap.Store(node, time.Now()) client, ok := conn.node2ClientMap.Load(node) if ok { - logger.With("conn", conn.name).Debugf("hit cache clientConn associated with node %s", node) + logger.GrpcLogger.With("conn", conn.name).Debugf("hit cache clientConn associated with node %s", node) return client.(*grpc.ClientConn), nil } - logger.With("conn", conn.name).Debugf("failed to load clientConn associated with node %s, attempt to create it", node) + logger.GrpcLogger.With("conn", conn.name).Debugf("failed to load clientConn associated with node %s, attempt to create it", node) clientConn, err = conn.createClient(node, append(defaultClientOpts, conn.dialOpts...)...) if err == nil { - logger.With("conn", conn.name).Infof("success connect to node %s", node) + logger.GrpcLogger.With("conn", conn.name).Infof("success connect to node %s", node) // bind conn.node2ClientMap.Store(node, clientConn) return clientConn, nil @@ -324,8 +326,8 @@ func (conn *Connection) loadOrCreateClientConnByNode(node string) (clientConn *g // GetClientConn get conn or bind hashKey to candidate node, don't do the migrate action // stick whether hash key need already associated with specify node func (conn *Connection) GetClientConn(hashKey string, stick bool) (*grpc.ClientConn, error) { - logger.With("conn", conn.name).Debugf("start to get client conn hashKey %s, stick %t", hashKey, stick) - defer logger.With("conn", conn.name).Debugf("get client conn done, hashKey %s, stick %t end", hashKey, stick) + logger.GrpcLogger.With("conn", conn.name).Debugf("start to get client conn hashKey %s, stick %t", hashKey, stick) + defer logger.GrpcLogger.With("conn", conn.name).Debugf("get client conn done, hashKey %s, stick %t end", hashKey, stick) conn.rwMutex.RLock() node, ok := conn.key2NodeMap.Load(hashKey) if stick && !ok { @@ -343,7 +345,7 @@ func (conn *Connection) GetClientConn(hashKey string, stick bool) (*grpc.ClientC } return clientConn, nil } - logger.With("conn", conn.name).Infof("no server node associated with hash key %s was found, start find candidate server", hashKey) + logger.GrpcLogger.With("conn", conn.name).Infof("no server node associated with hash key %s was found, start find candidate server", hashKey) conn.rwMutex.RUnlock() // if absence conn.rwMutex.Lock() @@ -361,14 +363,14 @@ func (conn *Connection) GetClientConn(hashKey string, stick bool) (*grpc.ClientC // TryMigrate migrate key to another hash node other than exclusiveNodes // preNode node before the migration func (conn *Connection) TryMigrate(key string, cause error, exclusiveNodes []string) (preNode string, err error) { - logger.With("conn", conn.name).Infof("start try migrate server node for key %s, cause err: %v", key, cause) + logger.GrpcLogger.With("conn", conn.name).Infof("start try migrate server node for key %s, cause err: %v", key, cause) if status.Code(cause) == codes.DeadlineExceeded || status.Code(cause) == codes.Canceled { - logger.With("conn", conn.name).Infof("migrate server node for key %s failed, cause err: %v", key, cause) + logger.GrpcLogger.With("conn", conn.name).Infof("migrate server node for key %s failed, cause err: %v", key, cause) return "", cause } // TODO recover findCandidateClientConn error if e, ok := cause.(*dferrors.DfError); ok { - if e.Code != dfcodes.ResourceLacked && e.Code != dfcodes.UnknownError { + if e.Code != dfcodes.ResourceLacked { return "", cause } } @@ -378,8 +380,6 @@ func (conn *Connection) TryMigrate(key string, cause error, exclusiveNodes []str currentNode = node.(string) preNode = currentNode exclusiveNodes = append(exclusiveNodes, preNode) - } else { - logger.With("conn", conn.name).Warnf("failed to find server node for hash key %s", key) } conn.rwMutex.RUnlock() conn.rwMutex.Lock() @@ -388,7 +388,7 @@ func (conn *Connection) TryMigrate(key string, cause error, exclusiveNodes []str if err != nil { return "", errors.Wrapf(err, "find candidate client conn for hash key %s", key) } - logger.With("conn", conn.name).Infof("successfully migrate hash key %s from server node %s to %s", key, currentNode, client.node) + logger.GrpcLogger.With("conn", conn.name).Infof("successfully migrate hash key %s from server node %s to %s", key, currentNode, client.node) conn.key2NodeMap.Store(key, client.node) conn.node2ClientMap.Store(client.node, client.Ref) conn.accessNodeMap.Store(client.node, time.Now()) diff --git a/pkg/rpc/client_util.go b/pkg/rpc/client_util.go index 2d3a47a1e..dc5f09450 100644 --- a/pkg/rpc/client_util.go +++ b/pkg/rpc/client_util.go @@ -25,7 +25,6 @@ import ( "google.golang.org/grpc/codes" "google.golang.org/grpc/status" - "d7y.io/dragonfly/v2/internal/dfcodes" "d7y.io/dragonfly/v2/internal/dferrors" logger "d7y.io/dragonfly/v2/internal/dflog" "d7y.io/dragonfly/v2/pkg/rpc/base" @@ -186,10 +185,8 @@ type RetryMeta struct { func ExecuteWithRetry(f func() (interface{}, error), initBackoff float64, maxBackoff float64, maxAttempts int, cause error) (interface{}, error) { var res interface{} for i := 0; i < maxAttempts; i++ { - if e, ok := cause.(*dferrors.DfError); ok { - if e.Code != dfcodes.UnknownError { - return res, cause - } + if _, ok := cause.(*dferrors.DfError); ok { + return res, cause } if status.Code(cause) == codes.DeadlineExceeded || status.Code(cause) == codes.Canceled { return res, cause diff --git a/pkg/rpc/dfdaemon/client/down_result_stream.go b/pkg/rpc/dfdaemon/client/down_result_stream.go index 91a12fd8f..f6f6e72fc 100644 --- a/pkg/rpc/dfdaemon/client/down_result_stream.go +++ b/pkg/rpc/dfdaemon/client/down_result_stream.go @@ -18,11 +18,12 @@ package client import ( "context" - "io" + "d7y.io/dragonfly/v2/internal/dferrors" logger "d7y.io/dragonfly/v2/internal/dflog" "d7y.io/dragonfly/v2/pkg/rpc" "d7y.io/dragonfly/v2/pkg/rpc/dfdaemon" + "github.com/pkg/errors" "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -49,9 +50,9 @@ func newDownResultStream(ctx context.Context, dc *daemonClient, hashKey string, opts: opts, RetryMeta: rpc.RetryMeta{ - MaxAttempts: 5, - MaxBackOff: 5.0, - InitBackoff: 1.0, + MaxAttempts: 3, + MaxBackOff: 2.0, + InitBackoff: 0.2, }, } @@ -73,6 +74,9 @@ func (drs *DownResultStream) initStream() error { return client.Download(drs.ctx, drs.req, drs.opts...) }, drs.InitBackoff, drs.MaxBackOff, drs.MaxAttempts, nil) if err != nil { + if errors.Cause(err) == dferrors.ErrNoCandidateNode { + return errors.Wrapf(err, "get grpc server instance failed") + } logger.WithTaskID(drs.hashKey).Infof("initStream: invoke daemon node %s Download failed: %v", target, err) return drs.replaceClient(err) } @@ -92,11 +96,7 @@ func (drs *DownResultStream) Recv() (dr *dfdaemon.DownResult, err error) { } }() drs.dc.UpdateAccessNodeMapByHashKey(drs.hashKey) - if dr, err = drs.stream.Recv(); err != nil && err != io.EOF { - dr, err = drs.retryRecv(err) - } - - return + return drs.stream.Recv() } func (drs *DownResultStream) retryRecv(cause error) (*dfdaemon.DownResult, error) { diff --git a/pkg/rpc/dfdaemon/dfdaemon.pb.go b/pkg/rpc/dfdaemon/dfdaemon.pb.go index 2cf15322e..78b0ea7ae 100644 --- a/pkg/rpc/dfdaemon/dfdaemon.pb.go +++ b/pkg/rpc/dfdaemon/dfdaemon.pb.go @@ -15,14 +15,15 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.26.0 -// protoc v3.15.8 +// protoc-gen-go v1.25.0 +// protoc v3.17.3 // source: pkg/rpc/dfdaemon/dfdaemon.proto package dfdaemon import ( base "d7y.io/dragonfly/v2/pkg/rpc/base" + proto "github.com/golang/protobuf/proto" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" emptypb "google.golang.org/protobuf/types/known/emptypb" @@ -37,6 +38,10 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +// This is a compile-time assertion that a sufficiently up-to-date version +// of the legacy proto package is being used. +const _ = proto.ProtoPackageIsVersion4 + type DownRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache diff --git a/pkg/rpc/dfdaemon/dfdaemon.pb.validate.go b/pkg/rpc/dfdaemon/dfdaemon.pb.validate.go index 82ec8c63a..d716a2d67 100644 --- a/pkg/rpc/dfdaemon/dfdaemon.pb.validate.go +++ b/pkg/rpc/dfdaemon/dfdaemon.pb.validate.go @@ -34,13 +34,27 @@ var ( ) // Validate checks the field values on DownRequest with the rules defined in -// the proto definition for this message. If any rules are violated, an error -// is returned. +// the proto definition for this message. If any rules are violated, the first +// error encountered is returned, or nil if there are no violations. func (m *DownRequest) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on DownRequest with the rules defined in +// the proto definition for this message. If any rules are violated, the +// result is a list of violation errors wrapped in DownRequestMultiError, or +// nil if none found. +func (m *DownRequest) ValidateAll() error { + return m.validate(true) +} + +func (m *DownRequest) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for Uuid // no validation rules for Url @@ -53,7 +67,26 @@ func (m *DownRequest) Validate() error { // no validation rules for DisableBackSource - if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetUrlMeta()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, DownRequestValidationError{ + field: "UrlMeta", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, DownRequestValidationError{ + field: "UrlMeta", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return DownRequestValidationError{ field: "UrlMeta", @@ -71,9 +104,28 @@ func (m *DownRequest) Validate() error { // no validation rules for Gid + if len(errors) > 0 { + return DownRequestMultiError(errors) + } return nil } +// DownRequestMultiError is an error wrapping multiple validation errors +// returned by DownRequest.ValidateAll() if the designated constraints aren't met. +type DownRequestMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m DownRequestMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m DownRequestMultiError) AllErrors() []error { return m } + // DownRequestValidationError is the validation error returned by // DownRequest.Validate if the designated constraints aren't met. type DownRequestValidationError struct { @@ -129,12 +181,27 @@ var _ interface { } = DownRequestValidationError{} // Validate checks the field values on DownResult with the rules defined in the -// proto definition for this message. If any rules are violated, an error is returned. +// proto definition for this message. If any rules are violated, the first +// error encountered is returned, or nil if there are no violations. func (m *DownResult) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on DownResult with the rules defined in +// the proto definition for this message. If any rules are violated, the +// result is a list of violation errors wrapped in DownResultMultiError, or +// nil if none found. +func (m *DownResult) ValidateAll() error { + return m.validate(true) +} + +func (m *DownResult) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for TaskId // no validation rules for PeerId @@ -143,9 +210,28 @@ func (m *DownResult) Validate() error { // no validation rules for Done + if len(errors) > 0 { + return DownResultMultiError(errors) + } return nil } +// DownResultMultiError is an error wrapping multiple validation errors +// returned by DownResult.ValidateAll() if the designated constraints aren't met. +type DownResultMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m DownResultMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m DownResultMultiError) AllErrors() []error { return m } + // DownResultValidationError is the validation error returned by // DownResult.Validate if the designated constraints aren't met. type DownResultValidationError struct { diff --git a/pkg/rpc/dfdaemon/dfdaemon_grpc.pb.go b/pkg/rpc/dfdaemon/dfdaemon_grpc.pb.go index c5e63251d..851f7c117 100644 --- a/pkg/rpc/dfdaemon/dfdaemon_grpc.pb.go +++ b/pkg/rpc/dfdaemon/dfdaemon_grpc.pb.go @@ -13,7 +13,6 @@ import ( // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -// Requires gRPC-Go v1.32.0 or later. const _ = grpc.SupportPackageIsVersion7 // DaemonClient is the client API for Daemon service. @@ -37,7 +36,7 @@ func NewDaemonClient(cc grpc.ClientConnInterface) DaemonClient { } func (c *daemonClient) Download(ctx context.Context, in *DownRequest, opts ...grpc.CallOption) (Daemon_DownloadClient, error) { - stream, err := c.cc.NewStream(ctx, &Daemon_ServiceDesc.Streams[0], "/dfdaemon.Daemon/Download", opts...) + stream, err := c.cc.NewStream(ctx, &_Daemon_serviceDesc.Streams[0], "/dfdaemon.Daemon/Download", opts...) if err != nil { return nil, err } @@ -122,7 +121,7 @@ type UnsafeDaemonServer interface { } func RegisterDaemonServer(s grpc.ServiceRegistrar, srv DaemonServer) { - s.RegisterService(&Daemon_ServiceDesc, srv) + s.RegisterService(&_Daemon_serviceDesc, srv) } func _Daemon_Download_Handler(srv interface{}, stream grpc.ServerStream) error { @@ -182,10 +181,7 @@ func _Daemon_CheckHealth_Handler(srv interface{}, ctx context.Context, dec func( return interceptor(ctx, in, info, handler) } -// Daemon_ServiceDesc is the grpc.ServiceDesc for Daemon service. -// It's only intended for direct use with grpc.RegisterService, -// and not to be introspected or modified (even as a copy) -var Daemon_ServiceDesc = grpc.ServiceDesc{ +var _Daemon_serviceDesc = grpc.ServiceDesc{ ServiceName: "dfdaemon.Daemon", HandlerType: (*DaemonServer)(nil), Methods: []grpc.MethodDesc{ diff --git a/pkg/rpc/manager/manager.pb.go b/pkg/rpc/manager/manager.pb.go index 4cde099dd..fb78bb6a0 100644 --- a/pkg/rpc/manager/manager.pb.go +++ b/pkg/rpc/manager/manager.pb.go @@ -15,14 +15,15 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.26.0 -// protoc v3.15.8 +// protoc-gen-go v1.25.0 +// protoc v3.17.3 // source: pkg/rpc/manager/manager.proto package manager import ( _ "github.com/envoyproxy/protoc-gen-validate/validate" + proto "github.com/golang/protobuf/proto" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" emptypb "google.golang.org/protobuf/types/known/emptypb" @@ -37,6 +38,10 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +// This is a compile-time assertion that a sufficiently up-to-date version +// of the legacy proto package is being used. +const _ = proto.ProtoPackageIsVersion4 + type SourceType int32 const ( diff --git a/pkg/rpc/manager/manager.pb.validate.go b/pkg/rpc/manager/manager.pb.validate.go index 41a1f3918..d9000e942 100644 --- a/pkg/rpc/manager/manager.pb.validate.go +++ b/pkg/rpc/manager/manager.pb.validate.go @@ -34,12 +34,27 @@ var ( ) // Validate checks the field values on CDNCluster 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 *CDNCluster) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on CDNCluster 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 CDNClusterMultiError, or +// nil if none found. +func (m *CDNCluster) ValidateAll() error { + return m.validate(true) +} + +func (m *CDNCluster) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for Id // no validation rules for Name @@ -48,7 +63,26 @@ func (m *CDNCluster) Validate() error { // no validation rules for Config - if v, ok := interface{}(m.GetSecurityGroup()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetSecurityGroup()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, CDNClusterValidationError{ + field: "SecurityGroup", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, CDNClusterValidationError{ + field: "SecurityGroup", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetSecurityGroup()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return CDNClusterValidationError{ field: "SecurityGroup", @@ -58,9 +92,28 @@ func (m *CDNCluster) Validate() error { } } + if len(errors) > 0 { + return CDNClusterMultiError(errors) + } return nil } +// CDNClusterMultiError is an error wrapping multiple validation errors +// returned by CDNCluster.ValidateAll() if the designated constraints aren't met. +type CDNClusterMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m CDNClusterMultiError) 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 CDNClusterMultiError) AllErrors() []error { return m } + // CDNClusterValidationError is the validation error returned by // CDNCluster.Validate if the designated constraints aren't met. type CDNClusterValidationError struct { @@ -116,13 +169,27 @@ var _ interface { } = CDNClusterValidationError{} // Validate checks the field values on SecurityGroup 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 *SecurityGroup) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on SecurityGroup 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 SecurityGroupMultiError, or +// nil if none found. +func (m *SecurityGroup) ValidateAll() error { + return m.validate(true) +} + +func (m *SecurityGroup) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for Id // no validation rules for Name @@ -133,9 +200,29 @@ func (m *SecurityGroup) Validate() error { // no validation rules for ProxyDomain + if len(errors) > 0 { + return SecurityGroupMultiError(errors) + } return nil } +// SecurityGroupMultiError is an error wrapping multiple validation errors +// returned by SecurityGroup.ValidateAll() if the designated constraints +// aren't met. +type SecurityGroupMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m SecurityGroupMultiError) 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 SecurityGroupMultiError) AllErrors() []error { return m } + // SecurityGroupValidationError is the validation error returned by // SecurityGroup.Validate if the designated constraints aren't met. type SecurityGroupValidationError struct { @@ -191,12 +278,26 @@ var _ interface { } = SecurityGroupValidationError{} // Validate checks the field values on CDN with the rules defined in the proto -// definition for this message. If any rules are violated, an error is returned. +// definition for this message. If any rules are violated, the first error +// encountered is returned, or nil if there are no violations. func (m *CDN) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on CDN 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 CDNMultiError, or nil if none found. +func (m *CDN) ValidateAll() error { + return m.validate(true) +} + +func (m *CDN) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for Id // no validation rules for HostName @@ -215,7 +316,26 @@ func (m *CDN) Validate() error { // no validation rules for CdnClusterId - if v, ok := interface{}(m.GetCdnCluster()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetCdnCluster()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, CDNValidationError{ + field: "CdnCluster", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, CDNValidationError{ + field: "CdnCluster", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetCdnCluster()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return CDNValidationError{ field: "CdnCluster", @@ -225,9 +345,28 @@ func (m *CDN) Validate() error { } } + if len(errors) > 0 { + return CDNMultiError(errors) + } return nil } +// CDNMultiError is an error wrapping multiple validation errors returned by +// CDN.ValidateAll() if the designated constraints aren't met. +type CDNMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m CDNMultiError) 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 CDNMultiError) AllErrors() []error { return m } + // CDNValidationError is the validation error returned by CDN.Validate if the // designated constraints aren't met. type CDNValidationError struct { @@ -283,35 +422,64 @@ var _ interface { } = CDNValidationError{} // Validate checks the field values on GetCDNRequest 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 *GetCDNRequest) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on GetCDNRequest 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 GetCDNRequestMultiError, or +// nil if none found. +func (m *GetCDNRequest) ValidateAll() error { + return m.validate(true) +} + +func (m *GetCDNRequest) validate(all bool) error { if m == nil { return nil } + var errors []error + if _, ok := SourceType_name[int32(m.GetSourceType())]; !ok { - return GetCDNRequestValidationError{ + err := GetCDNRequestValidationError{ field: "SourceType", reason: "value must be one of the defined enum values", } + if !all { + return err + } + errors = append(errors, err) } if err := m._validateHostname(m.GetHostName()); err != nil { - return GetCDNRequestValidationError{ + err = GetCDNRequestValidationError{ field: "HostName", reason: "value must be a valid hostname", cause: err, } + if !all { + return err + } + errors = append(errors, err) } if m.GetCdnClusterId() < 1 { - return GetCDNRequestValidationError{ + err := GetCDNRequestValidationError{ field: "CdnClusterId", reason: "value must be greater than or equal to 1", } + if !all { + return err + } + errors = append(errors, err) } + if len(errors) > 0 { + return GetCDNRequestMultiError(errors) + } return nil } @@ -345,6 +513,23 @@ func (m *GetCDNRequest) _validateHostname(host string) error { return nil } +// GetCDNRequestMultiError is an error wrapping multiple validation errors +// returned by GetCDNRequest.ValidateAll() if the designated constraints +// aren't met. +type GetCDNRequestMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m GetCDNRequestMultiError) 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 GetCDNRequestMultiError) AllErrors() []error { return m } + // GetCDNRequestValidationError is the validation error returned by // GetCDNRequest.Validate if the designated constraints aren't met. type GetCDNRequestValidationError struct { @@ -400,35 +585,61 @@ var _ interface { } = GetCDNRequestValidationError{} // Validate checks the field values on UpdateCDNRequest 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 *UpdateCDNRequest) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on UpdateCDNRequest 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 +// UpdateCDNRequestMultiError, or nil if none found. +func (m *UpdateCDNRequest) ValidateAll() error { + return m.validate(true) +} + +func (m *UpdateCDNRequest) validate(all bool) error { if m == nil { return nil } + var errors []error + if _, ok := SourceType_name[int32(m.GetSourceType())]; !ok { - return UpdateCDNRequestValidationError{ + err := UpdateCDNRequestValidationError{ field: "SourceType", reason: "value must be one of the defined enum values", } + if !all { + return err + } + errors = append(errors, err) } if err := m._validateHostname(m.GetHostName()); err != nil { - return UpdateCDNRequestValidationError{ + err = UpdateCDNRequestValidationError{ field: "HostName", reason: "value must be a valid hostname", cause: err, } + if !all { + return err + } + errors = append(errors, err) } if m.GetIdc() != "" { if l := utf8.RuneCountInString(m.GetIdc()); l < 1 || l > 1024 { - return UpdateCDNRequestValidationError{ + err := UpdateCDNRequestValidationError{ field: "Idc", reason: "value length must be between 1 and 1024 runes, inclusive", } + if !all { + return err + } + errors = append(errors, err) } } @@ -436,42 +647,65 @@ func (m *UpdateCDNRequest) Validate() error { if m.GetLocation() != "" { if utf8.RuneCountInString(m.GetLocation()) > 1024 { - return UpdateCDNRequestValidationError{ + err := UpdateCDNRequestValidationError{ field: "Location", reason: "value length must be at most 1024 runes", } + if !all { + return err + } + errors = append(errors, err) } } if ip := net.ParseIP(m.GetIp()); ip == nil { - return UpdateCDNRequestValidationError{ + err := UpdateCDNRequestValidationError{ field: "Ip", reason: "value must be a valid IP address", } + if !all { + return err + } + errors = append(errors, err) } if val := m.GetPort(); val < 1024 || val >= 65535 { - return UpdateCDNRequestValidationError{ + err := UpdateCDNRequestValidationError{ field: "Port", reason: "value must be inside range [1024, 65535)", } + if !all { + return err + } + errors = append(errors, err) } if val := m.GetDownloadPort(); val < 1024 || val >= 65535 { - return UpdateCDNRequestValidationError{ + err := UpdateCDNRequestValidationError{ field: "DownloadPort", reason: "value must be inside range [1024, 65535)", } + if !all { + return err + } + errors = append(errors, err) } if m.GetCdnClusterId() < 1 { - return UpdateCDNRequestValidationError{ + err := UpdateCDNRequestValidationError{ field: "CdnClusterId", reason: "value must be greater than or equal to 1", } + if !all { + return err + } + errors = append(errors, err) } + if len(errors) > 0 { + return UpdateCDNRequestMultiError(errors) + } return nil } @@ -505,6 +739,23 @@ func (m *UpdateCDNRequest) _validateHostname(host string) error { return nil } +// UpdateCDNRequestMultiError is an error wrapping multiple validation errors +// returned by UpdateCDNRequest.ValidateAll() if the designated constraints +// aren't met. +type UpdateCDNRequestMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m UpdateCDNRequestMultiError) 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 UpdateCDNRequestMultiError) AllErrors() []error { return m } + // UpdateCDNRequestValidationError is the validation error returned by // UpdateCDNRequest.Validate if the designated constraints aren't met. type UpdateCDNRequestValidationError struct { @@ -560,13 +811,27 @@ var _ interface { } = UpdateCDNRequestValidationError{} // Validate checks the field values on SchedulerCluster 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 *SchedulerCluster) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on SchedulerCluster 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 +// SchedulerClusterMultiError, or nil if none found. +func (m *SchedulerCluster) ValidateAll() error { + return m.validate(true) +} + +func (m *SchedulerCluster) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for Id // no validation rules for Name @@ -577,7 +842,26 @@ func (m *SchedulerCluster) Validate() error { // no validation rules for ClientConfig - if v, ok := interface{}(m.GetSecurityGroup()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetSecurityGroup()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, SchedulerClusterValidationError{ + field: "SecurityGroup", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, SchedulerClusterValidationError{ + field: "SecurityGroup", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetSecurityGroup()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return SchedulerClusterValidationError{ field: "SecurityGroup", @@ -587,9 +871,29 @@ func (m *SchedulerCluster) Validate() error { } } + if len(errors) > 0 { + return SchedulerClusterMultiError(errors) + } return nil } +// SchedulerClusterMultiError is an error wrapping multiple validation errors +// returned by SchedulerCluster.ValidateAll() if the designated constraints +// aren't met. +type SchedulerClusterMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m SchedulerClusterMultiError) 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 SchedulerClusterMultiError) AllErrors() []error { return m } + // SchedulerClusterValidationError is the validation error returned by // SchedulerCluster.Validate if the designated constraints aren't met. type SchedulerClusterValidationError struct { @@ -645,12 +949,27 @@ var _ interface { } = SchedulerClusterValidationError{} // Validate checks the field values on Scheduler 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 *Scheduler) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on Scheduler 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 SchedulerMultiError, or nil +// if none found. +func (m *Scheduler) ValidateAll() error { + return m.validate(true) +} + +func (m *Scheduler) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for Id // no validation rules for HostName @@ -671,7 +990,26 @@ func (m *Scheduler) Validate() error { // no validation rules for SchedulerClusterId - if v, ok := interface{}(m.GetSchedulerCluster()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetSchedulerCluster()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, SchedulerValidationError{ + field: "SchedulerCluster", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, SchedulerValidationError{ + field: "SchedulerCluster", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetSchedulerCluster()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return SchedulerValidationError{ field: "SchedulerCluster", @@ -684,7 +1022,26 @@ func (m *Scheduler) Validate() error { for idx, item := range m.GetCdns() { _, _ = 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, SchedulerValidationError{ + field: fmt.Sprintf("Cdns[%v]", idx), + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, SchedulerValidationError{ + field: fmt.Sprintf("Cdns[%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 SchedulerValidationError{ field: fmt.Sprintf("Cdns[%v]", idx), @@ -696,9 +1053,28 @@ func (m *Scheduler) Validate() error { } + if len(errors) > 0 { + return SchedulerMultiError(errors) + } return nil } +// SchedulerMultiError is an error wrapping multiple validation errors returned +// by Scheduler.ValidateAll() if the designated constraints aren't met. +type SchedulerMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m SchedulerMultiError) 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 SchedulerMultiError) AllErrors() []error { return m } + // SchedulerValidationError is the validation error returned by // Scheduler.Validate if the designated constraints aren't met. type SchedulerValidationError struct { @@ -755,34 +1131,63 @@ var _ interface { // Validate checks the field values on GetSchedulerRequest with the rules // defined in the proto definition for this message. If any rules are -// violated, an error is returned. +// violated, the first error encountered is returned, or nil if there are no violations. func (m *GetSchedulerRequest) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on GetSchedulerRequest 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 +// GetSchedulerRequestMultiError, or nil if none found. +func (m *GetSchedulerRequest) ValidateAll() error { + return m.validate(true) +} + +func (m *GetSchedulerRequest) validate(all bool) error { if m == nil { return nil } + var errors []error + if _, ok := SourceType_name[int32(m.GetSourceType())]; !ok { - return GetSchedulerRequestValidationError{ + err := GetSchedulerRequestValidationError{ field: "SourceType", reason: "value must be one of the defined enum values", } + if !all { + return err + } + errors = append(errors, err) } if err := m._validateHostname(m.GetHostName()); err != nil { - return GetSchedulerRequestValidationError{ + err = GetSchedulerRequestValidationError{ field: "HostName", reason: "value must be a valid hostname", cause: err, } + if !all { + return err + } + errors = append(errors, err) } if m.GetSchedulerClusterId() < 1 { - return GetSchedulerRequestValidationError{ + err := GetSchedulerRequestValidationError{ field: "SchedulerClusterId", reason: "value must be greater than or equal to 1", } + if !all { + return err + } + errors = append(errors, err) } + if len(errors) > 0 { + return GetSchedulerRequestMultiError(errors) + } return nil } @@ -816,6 +1221,23 @@ func (m *GetSchedulerRequest) _validateHostname(host string) error { return nil } +// GetSchedulerRequestMultiError is an error wrapping multiple validation +// errors returned by GetSchedulerRequest.ValidateAll() if the designated +// constraints aren't met. +type GetSchedulerRequestMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m GetSchedulerRequestMultiError) 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 GetSchedulerRequestMultiError) AllErrors() []error { return m } + // GetSchedulerRequestValidationError is the validation error returned by // GetSchedulerRequest.Validate if the designated constraints aren't met. type GetSchedulerRequestValidationError struct { @@ -874,34 +1296,60 @@ var _ interface { // Validate checks the field values on UpdateSchedulerRequest with the rules // defined in the proto definition for this message. If any rules are -// violated, an error is returned. +// violated, the first error encountered is returned, or nil if there are no violations. func (m *UpdateSchedulerRequest) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on UpdateSchedulerRequest 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 +// UpdateSchedulerRequestMultiError, or nil if none found. +func (m *UpdateSchedulerRequest) ValidateAll() error { + return m.validate(true) +} + +func (m *UpdateSchedulerRequest) validate(all bool) error { if m == nil { return nil } + var errors []error + if _, ok := SourceType_name[int32(m.GetSourceType())]; !ok { - return UpdateSchedulerRequestValidationError{ + err := UpdateSchedulerRequestValidationError{ field: "SourceType", reason: "value must be one of the defined enum values", } + if !all { + return err + } + errors = append(errors, err) } if err := m._validateHostname(m.GetHostName()); err != nil { - return UpdateSchedulerRequestValidationError{ + err = UpdateSchedulerRequestValidationError{ field: "HostName", reason: "value must be a valid hostname", cause: err, } + if !all { + return err + } + errors = append(errors, err) } if m.GetVips() != "" { if l := utf8.RuneCountInString(m.GetVips()); l < 1 || l > 1024 { - return UpdateSchedulerRequestValidationError{ + err := UpdateSchedulerRequestValidationError{ field: "Vips", reason: "value length must be between 1 and 1024 runes, inclusive", } + if !all { + return err + } + errors = append(errors, err) } } @@ -909,10 +1357,14 @@ func (m *UpdateSchedulerRequest) Validate() error { if m.GetIdc() != "" { if l := utf8.RuneCountInString(m.GetIdc()); l < 1 || l > 1024 { - return UpdateSchedulerRequestValidationError{ + err := UpdateSchedulerRequestValidationError{ field: "Idc", reason: "value length must be between 1 and 1024 runes, inclusive", } + if !all { + return err + } + errors = append(errors, err) } } @@ -920,10 +1372,14 @@ func (m *UpdateSchedulerRequest) Validate() error { if m.GetLocation() != "" { if utf8.RuneCountInString(m.GetLocation()) > 1024 { - return UpdateSchedulerRequestValidationError{ + err := UpdateSchedulerRequestValidationError{ field: "Location", reason: "value length must be at most 1024 runes", } + if !all { + return err + } + errors = append(errors, err) } } @@ -931,35 +1387,54 @@ func (m *UpdateSchedulerRequest) Validate() error { if len(m.GetNetConfig()) > 0 { if len(m.GetNetConfig()) < 1 { - return UpdateSchedulerRequestValidationError{ + err := UpdateSchedulerRequestValidationError{ field: "NetConfig", reason: "value length must be at least 1 bytes", } + if !all { + return err + } + errors = append(errors, err) } } if ip := net.ParseIP(m.GetIp()); ip == nil { - return UpdateSchedulerRequestValidationError{ + err := UpdateSchedulerRequestValidationError{ field: "Ip", reason: "value must be a valid IP address", } + if !all { + return err + } + errors = append(errors, err) } if val := m.GetPort(); val < 1024 || val >= 65535 { - return UpdateSchedulerRequestValidationError{ + err := UpdateSchedulerRequestValidationError{ field: "Port", reason: "value must be inside range [1024, 65535)", } + if !all { + return err + } + errors = append(errors, err) } if m.GetSchedulerClusterId() < 1 { - return UpdateSchedulerRequestValidationError{ + err := UpdateSchedulerRequestValidationError{ field: "SchedulerClusterId", reason: "value must be greater than or equal to 1", } + if !all { + return err + } + errors = append(errors, err) } + if len(errors) > 0 { + return UpdateSchedulerRequestMultiError(errors) + } return nil } @@ -993,6 +1468,23 @@ func (m *UpdateSchedulerRequest) _validateHostname(host string) error { return nil } +// UpdateSchedulerRequestMultiError is an error wrapping multiple validation +// errors returned by UpdateSchedulerRequest.ValidateAll() if the designated +// constraints aren't met. +type UpdateSchedulerRequestMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m UpdateSchedulerRequestMultiError) 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 UpdateSchedulerRequestMultiError) AllErrors() []error { return m } + // UpdateSchedulerRequestValidationError is the validation error returned by // UpdateSchedulerRequest.Validate if the designated constraints aren't met. type UpdateSchedulerRequestValidationError struct { @@ -1051,38 +1543,67 @@ var _ interface { // Validate checks the field values on ListSchedulersRequest with the rules // defined in the proto definition for this message. If any rules are -// violated, an error is returned. +// violated, the first error encountered is returned, or nil if there are no violations. func (m *ListSchedulersRequest) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on ListSchedulersRequest 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 +// ListSchedulersRequestMultiError, or nil if none found. +func (m *ListSchedulersRequest) ValidateAll() error { + return m.validate(true) +} + +func (m *ListSchedulersRequest) validate(all bool) error { if m == nil { return nil } + var errors []error + if _, ok := SourceType_name[int32(m.GetSourceType())]; !ok { - return ListSchedulersRequestValidationError{ + err := ListSchedulersRequestValidationError{ field: "SourceType", reason: "value must be one of the defined enum values", } + if !all { + return err + } + errors = append(errors, err) } if err := m._validateHostname(m.GetHostName()); err != nil { - return ListSchedulersRequestValidationError{ + err = ListSchedulersRequestValidationError{ field: "HostName", reason: "value must be a valid hostname", cause: err, } + if !all { + return err + } + errors = append(errors, err) } if ip := net.ParseIP(m.GetIp()); ip == nil { - return ListSchedulersRequestValidationError{ + err := ListSchedulersRequestValidationError{ field: "Ip", reason: "value must be a valid IP address", } + if !all { + return err + } + errors = append(errors, err) } if len(m.GetHostInfo()) > 0 { } + if len(errors) > 0 { + return ListSchedulersRequestMultiError(errors) + } return nil } @@ -1116,6 +1637,23 @@ func (m *ListSchedulersRequest) _validateHostname(host string) error { return nil } +// ListSchedulersRequestMultiError is an error wrapping multiple validation +// errors returned by ListSchedulersRequest.ValidateAll() if the designated +// constraints aren't met. +type ListSchedulersRequestMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m ListSchedulersRequestMultiError) 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 ListSchedulersRequestMultiError) AllErrors() []error { return m } + // ListSchedulersRequestValidationError is the validation error returned by // ListSchedulersRequest.Validate if the designated constraints aren't met. type ListSchedulersRequestValidationError struct { @@ -1174,16 +1712,49 @@ var _ interface { // Validate checks the field values on ListSchedulersResponse with the rules // defined in the proto definition for this message. If any rules are -// violated, an error is returned. +// violated, the first error encountered is returned, or nil if there are no violations. func (m *ListSchedulersResponse) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on ListSchedulersResponse 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 +// ListSchedulersResponseMultiError, or nil if none found. +func (m *ListSchedulersResponse) ValidateAll() error { + return m.validate(true) +} + +func (m *ListSchedulersResponse) validate(all bool) error { if m == nil { return nil } + var errors []error + for idx, item := range m.GetSchedulers() { _, _ = 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, ListSchedulersResponseValidationError{ + field: fmt.Sprintf("Schedulers[%v]", idx), + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, ListSchedulersResponseValidationError{ + field: fmt.Sprintf("Schedulers[%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 ListSchedulersResponseValidationError{ field: fmt.Sprintf("Schedulers[%v]", idx), @@ -1195,9 +1766,29 @@ func (m *ListSchedulersResponse) Validate() error { } + if len(errors) > 0 { + return ListSchedulersResponseMultiError(errors) + } return nil } +// ListSchedulersResponseMultiError is an error wrapping multiple validation +// errors returned by ListSchedulersResponse.ValidateAll() if the designated +// constraints aren't met. +type ListSchedulersResponseMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m ListSchedulersResponseMultiError) 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 ListSchedulersResponseMultiError) AllErrors() []error { return m } + // ListSchedulersResponseValidationError is the validation error returned by // ListSchedulersResponse.Validate if the designated constraints aren't met. type ListSchedulersResponseValidationError struct { @@ -1255,35 +1846,64 @@ var _ interface { } = ListSchedulersResponseValidationError{} // Validate checks the field values on KeepAliveRequest 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 *KeepAliveRequest) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on KeepAliveRequest 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 +// KeepAliveRequestMultiError, or nil if none found. +func (m *KeepAliveRequest) ValidateAll() error { + return m.validate(true) +} + +func (m *KeepAliveRequest) validate(all bool) error { if m == nil { return nil } + var errors []error + if _, ok := SourceType_name[int32(m.GetSourceType())]; !ok { - return KeepAliveRequestValidationError{ + err := KeepAliveRequestValidationError{ field: "SourceType", reason: "value must be one of the defined enum values", } + if !all { + return err + } + errors = append(errors, err) } if err := m._validateHostname(m.GetHostName()); err != nil { - return KeepAliveRequestValidationError{ + err = KeepAliveRequestValidationError{ field: "HostName", reason: "value must be a valid hostname", cause: err, } + if !all { + return err + } + errors = append(errors, err) } if m.GetClusterId() < 1 { - return KeepAliveRequestValidationError{ + err := KeepAliveRequestValidationError{ field: "ClusterId", reason: "value must be greater than or equal to 1", } + if !all { + return err + } + errors = append(errors, err) } + if len(errors) > 0 { + return KeepAliveRequestMultiError(errors) + } return nil } @@ -1317,6 +1937,23 @@ func (m *KeepAliveRequest) _validateHostname(host string) error { return nil } +// KeepAliveRequestMultiError is an error wrapping multiple validation errors +// returned by KeepAliveRequest.ValidateAll() if the designated constraints +// aren't met. +type KeepAliveRequestMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m KeepAliveRequestMultiError) 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 KeepAliveRequestMultiError) AllErrors() []error { return m } + // KeepAliveRequestValidationError is the validation error returned by // KeepAliveRequest.Validate if the designated constraints aren't met. type KeepAliveRequestValidationError struct { diff --git a/pkg/rpc/manager/manager_grpc.pb.go b/pkg/rpc/manager/manager_grpc.pb.go index 4024ff24e..0f9f3d52e 100644 --- a/pkg/rpc/manager/manager_grpc.pb.go +++ b/pkg/rpc/manager/manager_grpc.pb.go @@ -12,7 +12,6 @@ import ( // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -// Requires gRPC-Go v1.32.0 or later. const _ = grpc.SupportPackageIsVersion7 // ManagerClient is the client API for Manager service. @@ -87,7 +86,7 @@ func (c *managerClient) ListSchedulers(ctx context.Context, in *ListSchedulersRe } func (c *managerClient) KeepAlive(ctx context.Context, opts ...grpc.CallOption) (Manager_KeepAliveClient, error) { - stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[0], "/manager.Manager/KeepAlive", opts...) + stream, err := c.cc.NewStream(ctx, &_Manager_serviceDesc.Streams[0], "/manager.Manager/KeepAlive", opts...) if err != nil { return nil, err } @@ -171,7 +170,7 @@ type UnsafeManagerServer interface { } func RegisterManagerServer(s grpc.ServiceRegistrar, srv ManagerServer) { - s.RegisterService(&Manager_ServiceDesc, srv) + s.RegisterService(&_Manager_serviceDesc, srv) } func _Manager_GetCDN_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { @@ -290,10 +289,7 @@ func (x *managerKeepAliveServer) Recv() (*KeepAliveRequest, error) { return m, nil } -// Manager_ServiceDesc is the grpc.ServiceDesc for Manager service. -// It's only intended for direct use with grpc.RegisterService, -// and not to be introspected or modified (even as a copy) -var Manager_ServiceDesc = grpc.ServiceDesc{ +var _Manager_serviceDesc = grpc.ServiceDesc{ ServiceName: "manager.Manager", HandlerType: (*ManagerServer)(nil), Methods: []grpc.MethodDesc{ diff --git a/pkg/rpc/scheduler/client/client.go b/pkg/rpc/scheduler/client/client.go index f600b55b2..971af7327 100644 --- a/pkg/rpc/scheduler/client/client.go +++ b/pkg/rpc/scheduler/client/client.go @@ -148,7 +148,9 @@ func (sc *schedulerClient) retryRegisterPeerTask(ctx context.Context, hashKey st func (sc *schedulerClient) ReportPieceResult(ctx context.Context, taskID string, ptr *scheduler.PeerTaskRequest, opts ...grpc.CallOption) (PeerPacketStream, error) { pps, err := newPeerPacketStream(ctx, sc, taskID, ptr, opts) - + if err != nil { + return pps, err + } logger.With("peerId", ptr.PeerId, "errMsg", err).Infof("start to report piece result for taskID: %s", taskID) // trigger scheduling diff --git a/pkg/rpc/scheduler/client/peer_packet_stream.go b/pkg/rpc/scheduler/client/peer_packet_stream.go index dc15efdfa..015746285 100644 --- a/pkg/rpc/scheduler/client/peer_packet_stream.go +++ b/pkg/rpc/scheduler/client/peer_packet_stream.go @@ -18,9 +18,10 @@ package client import ( "context" - "io" + "d7y.io/dragonfly/v2/internal/dferrors" logger "d7y.io/dragonfly/v2/internal/dflog" + "github.com/pkg/errors" "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -60,9 +61,9 @@ func newPeerPacketStream(ctx context.Context, sc *schedulerClient, hashKey strin ptr: ptr, opts: opts, retryMeta: rpc.RetryMeta{ - MaxAttempts: 5, - InitBackoff: 0.5, - MaxBackOff: 4.0, + MaxAttempts: 3, + InitBackoff: 0.2, + MaxBackOff: 2.0, }, } @@ -77,14 +78,13 @@ func (pps *peerPacketStream) Send(pr *scheduler.PieceResult) (err error) { pps.sc.UpdateAccessNodeMapByHashKey(pps.hashKey) err = pps.stream.Send(pr) - if pr.PieceNum == common.EndOfPiece { + if pr.PieceInfo.PieceNum == common.EndOfPiece { pps.closeSend() return } if err != nil { pps.closeSend() - err = pps.retrySend(pr, err) } return @@ -96,10 +96,7 @@ func (pps *peerPacketStream) closeSend() error { func (pps *peerPacketStream) Recv() (pp *scheduler.PeerPacket, err error) { pps.sc.UpdateAccessNodeMapByHashKey(pps.hashKey) - if pp, err = pps.stream.Recv(); err != nil && err != io.EOF { - pp, err = pps.retryRecv(err) - } - return + return pps.stream.Recv() } func (pps *peerPacketStream) retrySend(pr *scheduler.PieceResult, cause error) error { @@ -108,7 +105,7 @@ func (pps *peerPacketStream) retrySend(pr *scheduler.PieceResult, cause error) e } if err := pps.replaceStream(cause); err != nil { - return err + return cause } return pps.Send(pr) @@ -159,6 +156,9 @@ func (pps *peerPacketStream) initStream() error { return client.ReportPieceResult(pps.ctx, pps.opts...) }, pps.retryMeta.InitBackoff, pps.retryMeta.MaxBackOff, pps.retryMeta.MaxAttempts, nil) if err != nil { + if errors.Cause(err) == dferrors.ErrNoCandidateNode { + return errors.Wrapf(err, "get grpc server instance failed") + } logger.WithTaskID(pps.hashKey).Infof("initStream: invoke scheduler node %s ReportPieceResult failed: %v", target, err) return pps.replaceClient(err) } diff --git a/pkg/rpc/scheduler/scheduler.go b/pkg/rpc/scheduler/scheduler.go index 8215d10f8..bcfcc20c8 100644 --- a/pkg/rpc/scheduler/scheduler.go +++ b/pkg/rpc/scheduler/scheduler.go @@ -17,14 +17,22 @@ package scheduler import ( + "d7y.io/dragonfly/v2/pkg/rpc/base" "d7y.io/dragonfly/v2/pkg/rpc/base/common" ) func NewZeroPieceResult(taskID, peerID string) *PieceResult { return &PieceResult{ - TaskId: taskID, - SrcPid: peerID, - PieceNum: common.ZeroOfPiece, + TaskId: taskID, + SrcPid: peerID, + PieceInfo: &base.PieceInfo{ + PieceNum: common.ZeroOfPiece, + RangeStart: 0, + RangeSize: 0, + PieceMd5: "", + PieceOffset: 0, + PieceStyle: 0, + }, } } @@ -32,7 +40,14 @@ func NewEndPieceResult(taskID, peerID string, finishedCount int32) *PieceResult return &PieceResult{ TaskId: taskID, SrcPid: peerID, - PieceNum: common.EndOfPiece, FinishedCount: finishedCount, + PieceInfo: &base.PieceInfo{ + PieceNum: common.EndOfPiece, + RangeStart: 0, + RangeSize: 0, + PieceMd5: "", + PieceOffset: 0, + PieceStyle: 0, + }, } } diff --git a/pkg/rpc/scheduler/scheduler.pb.go b/pkg/rpc/scheduler/scheduler.pb.go index c59b9a3ec..b148ef075 100644 --- a/pkg/rpc/scheduler/scheduler.pb.go +++ b/pkg/rpc/scheduler/scheduler.pb.go @@ -15,14 +15,15 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.26.0 -// protoc v3.15.8 +// protoc-gen-go v1.25.0 +// protoc v3.17.3 // source: pkg/rpc/scheduler/scheduler.proto package scheduler import ( base "d7y.io/dragonfly/v2/pkg/rpc/base" + proto "github.com/golang/protobuf/proto" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" emptypb "google.golang.org/protobuf/types/known/emptypb" @@ -37,6 +38,10 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +// This is a compile-time assertion that a sufficiently up-to-date version +// of the legacy proto package is being used. +const _ = proto.ProtoPackageIsVersion4 + type PeerTaskRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -429,8 +434,8 @@ type PieceResult struct { SrcPid string `protobuf:"bytes,2,opt,name=src_pid,json=srcPid,proto3" json:"src_pid,omitempty"` // dest peer id DstPid string `protobuf:"bytes,3,opt,name=dst_pid,json=dstPid,proto3" json:"dst_pid,omitempty"` - // piece number - PieceNum int32 `protobuf:"varint,4,opt,name=piece_num,json=pieceNum,proto3" json:"piece_num,omitempty"` + // piece info + PieceInfo *base.PieceInfo `protobuf:"bytes,4,opt,name=piece_info,json=pieceInfo,proto3" json:"piece_info,omitempty"` // begin time for the piece downloading BeginTime uint64 `protobuf:"varint,5,opt,name=begin_time,json=beginTime,proto3" json:"begin_time,omitempty"` // end time for the piece downloading @@ -498,11 +503,11 @@ func (x *PieceResult) GetDstPid() string { return "" } -func (x *PieceResult) GetPieceNum() int32 { +func (x *PieceResult) GetPieceInfo() *base.PieceInfo { if x != nil { - return x.PieceNum + return x.PieceInfo } - return 0 + return nil } func (x *PieceResult) GetBeginTime() uint64 { @@ -556,7 +561,6 @@ type PeerPacket struct { // source peer id SrcPid string `protobuf:"bytes,3,opt,name=src_pid,json=srcPid,proto3" json:"src_pid,omitempty"` // concurrent downloading count from main peer - // TODO what effect ParallelCount int32 `protobuf:"varint,4,opt,name=parallel_count,json=parallelCount,proto3" json:"parallel_count,omitempty"` MainPeer *PeerPacket_DestPeer `protobuf:"bytes,5,opt,name=main_peer,json=mainPeer,proto3" json:"main_peer,omitempty"` StealPeers []*PeerPacket_DestPeer `protobuf:"bytes,6,rep,name=steal_peers,json=stealPeers,proto3" json:"steal_peers,omitempty"` @@ -658,7 +662,8 @@ type PeerResult struct { // whether peer downloading file is successfully Success bool `protobuf:"varint,10,opt,name=success,proto3" json:"success,omitempty"` // result code - Code base.Code `protobuf:"varint,11,opt,name=code,proto3,enum=base.Code" json:"code,omitempty"` + Code base.Code `protobuf:"varint,11,opt,name=code,proto3,enum=base.Code" json:"code,omitempty"` + TotalPieceCount int32 `protobuf:"varint,12,opt,name=total_piece_count,json=totalPieceCount,proto3" json:"total_piece_count,omitempty"` } func (x *PeerResult) Reset() { @@ -770,6 +775,13 @@ func (x *PeerResult) GetCode() base.Code { return base.Code_X_UNSPECIFIED } +func (x *PeerResult) GetTotalPieceCount() int32 { + if x != nil { + return x.TotalPieceCount + } + return 0 +} + type PeerTarget struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -951,92 +963,96 @@ var file_pkg_rpc_scheduler_scheduler_proto_rawDesc = []byte{ 0x03, 0x69, 0x64, 0x63, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x63, 0x12, 0x21, 0x0a, 0x0c, 0x6e, 0x65, 0x74, 0x5f, 0x74, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, - 0x67, 0x79, 0x22, 0xbd, 0x02, 0x0a, 0x0b, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, + 0x67, 0x79, 0x22, 0xd0, 0x02, 0x0a, 0x0b, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x73, 0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x72, 0x63, 0x50, 0x69, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x1b, 0x0a, - 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, - 0x52, 0x08, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4e, 0x75, 0x6d, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x65, - 0x67, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, - 0x62, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x65, 0x6e, 0x64, - 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x65, 0x6e, 0x64, - 0x54, 0x69, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, - 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1e, - 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, - 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x2b, - 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0e, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x48, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, - 0x64, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x66, - 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0a, 0x20, - 0x01, 0x28, 0x05, 0x52, 0x0d, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x43, 0x6f, 0x75, - 0x6e, 0x74, 0x22, 0xd3, 0x02, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x50, 0x61, 0x63, 0x6b, 0x65, - 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x73, 0x72, - 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x72, 0x63, - 0x50, 0x69, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, - 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x70, 0x61, 0x72, - 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x3b, 0x0a, 0x09, 0x6d, 0x61, - 0x69, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, - 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x50, 0x61, - 0x63, 0x6b, 0x65, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, 0x08, 0x6d, - 0x61, 0x69, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x0b, 0x73, 0x74, 0x65, 0x61, 0x6c, - 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x73, - 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x50, 0x61, 0x63, - 0x6b, 0x65, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x74, - 0x65, 0x61, 0x6c, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x1e, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, - 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, - 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x1a, 0x4e, 0x0a, 0x08, 0x44, 0x65, 0x73, 0x74, - 0x50, 0x65, 0x65, 0x72, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x02, 0x69, 0x70, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x70, 0x63, 0x5f, 0x70, 0x6f, 0x72, 0x74, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x72, 0x70, 0x63, 0x50, 0x6f, 0x72, 0x74, 0x12, - 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x22, 0xb1, 0x02, 0x0a, 0x0a, 0x50, 0x65, 0x65, - 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, - 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x15, 0x0a, 0x06, 0x73, 0x72, 0x63, - 0x5f, 0x69, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x72, 0x63, 0x49, 0x70, - 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, 0x64, 0x6f, 0x6d, - 0x61, 0x69, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x65, 0x63, 0x75, 0x72, - 0x69, 0x74, 0x79, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x63, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x63, 0x12, 0x10, 0x0a, 0x03, 0x75, - 0x72, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x25, 0x0a, - 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, - 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x4c, 0x65, - 0x6e, 0x67, 0x74, 0x68, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x72, 0x61, 0x66, 0x66, 0x69, 0x63, 0x18, - 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x74, 0x72, 0x61, 0x66, 0x66, 0x69, 0x63, 0x12, 0x12, - 0x0a, 0x04, 0x63, 0x6f, 0x73, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x63, 0x6f, - 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x0a, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1e, 0x0a, 0x04, - 0x63, 0x6f, 0x64, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61, 0x73, - 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3e, 0x0a, 0x0a, - 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x2e, 0x0a, + 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x09, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x0a, + 0x0a, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x09, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, + 0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, + 0x65, 0x6e, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, + 0x73, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x12, 0x1e, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x0a, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, + 0x65, 0x12, 0x2b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x48, 0x6f, 0x73, 0x74, + 0x4c, 0x6f, 0x61, 0x64, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x25, + 0x0a, 0x0e, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, + 0x18, 0x0a, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, + 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xd3, 0x02, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x50, 0x61, + 0x63, 0x6b, 0x65, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a, + 0x07, 0x73, 0x72, 0x63, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, + 0x73, 0x72, 0x63, 0x50, 0x69, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, + 0x65, 0x6c, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, + 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x3b, 0x0a, + 0x09, 0x6d, 0x61, 0x69, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1e, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, + 0x52, 0x08, 0x6d, 0x61, 0x69, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x0b, 0x73, 0x74, + 0x65, 0x61, 0x6c, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1e, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, + 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, + 0x0a, 0x73, 0x74, 0x65, 0x61, 0x6c, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x1e, 0x0a, 0x04, 0x63, + 0x6f, 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x62, 0x61, 0x73, 0x65, + 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x1a, 0x4e, 0x0a, 0x08, 0x44, + 0x65, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x70, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x70, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x70, 0x63, 0x5f, 0x70, + 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x72, 0x70, 0x63, 0x50, 0x6f, + 0x72, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x22, 0xdd, 0x02, 0x0a, 0x0a, + 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x32, 0x9d, 0x02, 0x0a, - 0x09, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x10, 0x52, 0x65, - 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x1a, - 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54, - 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x73, 0x63, 0x68, - 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x46, 0x0a, 0x11, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, - 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x2e, 0x73, 0x63, 0x68, - 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x1a, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, - 0x65, 0x65, 0x72, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x28, 0x01, 0x30, 0x01, 0x12, 0x41, 0x0a, - 0x10, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x12, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, - 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, - 0x12, 0x3a, 0x0a, 0x09, 0x4c, 0x65, 0x61, 0x76, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x15, 0x2e, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x12, 0x15, 0x0a, 0x06, + 0x73, 0x72, 0x63, 0x5f, 0x69, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x72, + 0x63, 0x49, 0x70, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, + 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x65, + 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, + 0x69, 0x64, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x63, 0x12, 0x10, + 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, + 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x65, 0x6e, 0x67, + 0x74, 0x68, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, + 0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x72, 0x61, 0x66, 0x66, + 0x69, 0x63, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x74, 0x72, 0x61, 0x66, 0x66, 0x69, + 0x63, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x73, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x04, 0x63, 0x6f, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, + 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, + 0x1e, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, + 0x62, 0x61, 0x73, 0x65, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, + 0x2a, 0x0a, 0x11, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61, + 0x6c, 0x50, 0x69, 0x65, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x3e, 0x0a, 0x0a, 0x50, + 0x65, 0x65, 0x72, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, + 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, + 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x32, 0x9d, 0x02, 0x0a, 0x09, + 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x10, 0x52, 0x65, 0x67, + 0x69, 0x73, 0x74, 0x65, 0x72, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x1a, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, - 0x72, 0x67, 0x65, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x27, 0x5a, 0x25, - 0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, - 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x73, 0x63, 0x68, 0x65, - 0x64, 0x75, 0x6c, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x12, 0x46, 0x0a, 0x11, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, + 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x16, 0x2e, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x1a, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, + 0x65, 0x72, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x28, 0x01, 0x30, 0x01, 0x12, 0x41, 0x0a, 0x10, + 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x12, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, + 0x3a, 0x0a, 0x09, 0x4c, 0x65, 0x61, 0x76, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x15, 0x2e, 0x73, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x27, 0x5a, 0x25, 0x64, + 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f, + 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1076,25 +1092,26 @@ var file_pkg_rpc_scheduler_scheduler_proto_depIdxs = []int32{ 11, // 3: scheduler.RegisterResult.size_scope:type_name -> base.SizeScope 2, // 4: scheduler.RegisterResult.single_piece:type_name -> scheduler.SinglePiece 12, // 5: scheduler.SinglePiece.piece_info:type_name -> base.PieceInfo - 13, // 6: scheduler.PieceResult.code:type_name -> base.Code - 10, // 7: scheduler.PieceResult.host_load:type_name -> base.HostLoad - 8, // 8: scheduler.PeerPacket.main_peer:type_name -> scheduler.PeerPacket.DestPeer - 8, // 9: scheduler.PeerPacket.steal_peers:type_name -> scheduler.PeerPacket.DestPeer - 13, // 10: scheduler.PeerPacket.code:type_name -> base.Code - 13, // 11: scheduler.PeerResult.code:type_name -> base.Code - 0, // 12: scheduler.Scheduler.RegisterPeerTask:input_type -> scheduler.PeerTaskRequest - 4, // 13: scheduler.Scheduler.ReportPieceResult:input_type -> scheduler.PieceResult - 6, // 14: scheduler.Scheduler.ReportPeerResult:input_type -> scheduler.PeerResult - 7, // 15: scheduler.Scheduler.LeaveTask:input_type -> scheduler.PeerTarget - 1, // 16: scheduler.Scheduler.RegisterPeerTask:output_type -> scheduler.RegisterResult - 5, // 17: scheduler.Scheduler.ReportPieceResult:output_type -> scheduler.PeerPacket - 14, // 18: scheduler.Scheduler.ReportPeerResult:output_type -> google.protobuf.Empty - 14, // 19: scheduler.Scheduler.LeaveTask:output_type -> google.protobuf.Empty - 16, // [16:20] is the sub-list for method output_type - 12, // [12:16] is the sub-list for method input_type - 12, // [12:12] is the sub-list for extension type_name - 12, // [12:12] is the sub-list for extension extendee - 0, // [0:12] is the sub-list for field type_name + 12, // 6: scheduler.PieceResult.piece_info:type_name -> base.PieceInfo + 13, // 7: scheduler.PieceResult.code:type_name -> base.Code + 10, // 8: scheduler.PieceResult.host_load:type_name -> base.HostLoad + 8, // 9: scheduler.PeerPacket.main_peer:type_name -> scheduler.PeerPacket.DestPeer + 8, // 10: scheduler.PeerPacket.steal_peers:type_name -> scheduler.PeerPacket.DestPeer + 13, // 11: scheduler.PeerPacket.code:type_name -> base.Code + 13, // 12: scheduler.PeerResult.code:type_name -> base.Code + 0, // 13: scheduler.Scheduler.RegisterPeerTask:input_type -> scheduler.PeerTaskRequest + 4, // 14: scheduler.Scheduler.ReportPieceResult:input_type -> scheduler.PieceResult + 6, // 15: scheduler.Scheduler.ReportPeerResult:input_type -> scheduler.PeerResult + 7, // 16: scheduler.Scheduler.LeaveTask:input_type -> scheduler.PeerTarget + 1, // 17: scheduler.Scheduler.RegisterPeerTask:output_type -> scheduler.RegisterResult + 5, // 18: scheduler.Scheduler.ReportPieceResult:output_type -> scheduler.PeerPacket + 14, // 19: scheduler.Scheduler.ReportPeerResult:output_type -> google.protobuf.Empty + 14, // 20: scheduler.Scheduler.LeaveTask:output_type -> google.protobuf.Empty + 17, // [17:21] is the sub-list for method output_type + 13, // [13:17] is the sub-list for method input_type + 13, // [13:13] is the sub-list for extension type_name + 13, // [13:13] is the sub-list for extension extendee + 0, // [0:13] is the sub-list for field type_name } func init() { file_pkg_rpc_scheduler_scheduler_proto_init() } diff --git a/pkg/rpc/scheduler/scheduler.pb.validate.go b/pkg/rpc/scheduler/scheduler.pb.validate.go index c57352633..9e135fff9 100644 --- a/pkg/rpc/scheduler/scheduler.pb.validate.go +++ b/pkg/rpc/scheduler/scheduler.pb.validate.go @@ -44,16 +44,49 @@ var ( ) // Validate checks the field values on PeerTaskRequest with the rules defined -// in the proto definition for this message. If any rules are violated, an -// error is returned. +// in the proto definition for this message. If any rules are violated, the +// first error encountered is returned, or nil if there are no violations. func (m *PeerTaskRequest) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on PeerTaskRequest with the rules +// defined in the proto definition for this message. If any rules are +// violated, the result is a list of violation errors wrapped in +// PeerTaskRequestMultiError, or nil if none found. +func (m *PeerTaskRequest) ValidateAll() error { + return m.validate(true) +} + +func (m *PeerTaskRequest) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for Url - if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetUrlMeta()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, PeerTaskRequestValidationError{ + field: "UrlMeta", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, PeerTaskRequestValidationError{ + field: "UrlMeta", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetUrlMeta()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return PeerTaskRequestValidationError{ field: "UrlMeta", @@ -65,7 +98,26 @@ func (m *PeerTaskRequest) Validate() error { // no validation rules for PeerId - if v, ok := interface{}(m.GetPeerHost()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetPeerHost()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, PeerTaskRequestValidationError{ + field: "PeerHost", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, PeerTaskRequestValidationError{ + field: "PeerHost", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetPeerHost()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return PeerTaskRequestValidationError{ field: "PeerHost", @@ -75,7 +127,26 @@ func (m *PeerTaskRequest) Validate() error { } } - if v, ok := interface{}(m.GetHostLoad()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetHostLoad()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, PeerTaskRequestValidationError{ + field: "HostLoad", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, PeerTaskRequestValidationError{ + field: "HostLoad", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetHostLoad()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return PeerTaskRequestValidationError{ field: "HostLoad", @@ -87,9 +158,29 @@ func (m *PeerTaskRequest) Validate() error { // no validation rules for IsMigrating + if len(errors) > 0 { + return PeerTaskRequestMultiError(errors) + } return nil } +// PeerTaskRequestMultiError is an error wrapping multiple validation errors +// returned by PeerTaskRequest.ValidateAll() if the designated constraints +// aren't met. +type PeerTaskRequestMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m PeerTaskRequestMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m PeerTaskRequestMultiError) AllErrors() []error { return m } + // PeerTaskRequestValidationError is the validation error returned by // PeerTaskRequest.Validate if the designated constraints aren't met. type PeerTaskRequestValidationError struct { @@ -145,13 +236,27 @@ var _ interface { } = PeerTaskRequestValidationError{} // Validate checks the field values on RegisterResult with the rules defined in -// the proto definition for this message. If any rules are violated, an error -// is returned. +// the proto definition for this message. If any rules are violated, the first +// error encountered is returned, or nil if there are no violations. func (m *RegisterResult) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on RegisterResult with the rules defined +// in the proto definition for this message. If any rules are violated, the +// result is a list of violation errors wrapped in RegisterResultMultiError, +// or nil if none found. +func (m *RegisterResult) ValidateAll() error { + return m.validate(true) +} + +func (m *RegisterResult) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for TaskId // no validation rules for SizeScope @@ -160,7 +265,26 @@ func (m *RegisterResult) Validate() error { case *RegisterResult_SinglePiece: - if v, ok := interface{}(m.GetSinglePiece()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetSinglePiece()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, RegisterResultValidationError{ + field: "SinglePiece", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, RegisterResultValidationError{ + field: "SinglePiece", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetSinglePiece()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return RegisterResultValidationError{ field: "SinglePiece", @@ -175,9 +299,29 @@ func (m *RegisterResult) Validate() error { } + if len(errors) > 0 { + return RegisterResultMultiError(errors) + } return nil } +// RegisterResultMultiError is an error wrapping multiple validation errors +// returned by RegisterResult.ValidateAll() if the designated constraints +// aren't met. +type RegisterResultMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m RegisterResultMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m RegisterResultMultiError) AllErrors() []error { return m } + // RegisterResultValidationError is the validation error returned by // RegisterResult.Validate if the designated constraints aren't met. type RegisterResultValidationError struct { @@ -233,18 +377,51 @@ var _ interface { } = RegisterResultValidationError{} // Validate checks the field values on SinglePiece with the rules defined in -// the proto definition for this message. If any rules are violated, an error -// is returned. +// the proto definition for this message. If any rules are violated, the first +// error encountered is returned, or nil if there are no violations. func (m *SinglePiece) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on SinglePiece with the rules defined in +// the proto definition for this message. If any rules are violated, the +// result is a list of violation errors wrapped in SinglePieceMultiError, or +// nil if none found. +func (m *SinglePiece) ValidateAll() error { + return m.validate(true) +} + +func (m *SinglePiece) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for DstPid // no validation rules for DstAddr - if v, ok := interface{}(m.GetPieceInfo()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetPieceInfo()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, SinglePieceValidationError{ + field: "PieceInfo", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, SinglePieceValidationError{ + field: "PieceInfo", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetPieceInfo()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return SinglePieceValidationError{ field: "PieceInfo", @@ -254,9 +431,28 @@ func (m *SinglePiece) Validate() error { } } + if len(errors) > 0 { + return SinglePieceMultiError(errors) + } return nil } +// SinglePieceMultiError is an error wrapping multiple validation errors +// returned by SinglePiece.ValidateAll() if the designated constraints aren't met. +type SinglePieceMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m SinglePieceMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m SinglePieceMultiError) AllErrors() []error { return m } + // SinglePieceValidationError is the validation error returned by // SinglePiece.Validate if the designated constraints aren't met. type SinglePieceValidationError struct { @@ -312,12 +508,27 @@ var _ interface { } = SinglePieceValidationError{} // Validate checks the field values on PeerHost with the rules defined in the -// proto definition for this message. If any rules are violated, an error is returned. +// proto definition for this message. If any rules are violated, the first +// error encountered is returned, or nil if there are no violations. func (m *PeerHost) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on PeerHost with the rules defined in +// the proto definition for this message. If any rules are violated, the +// result is a list of violation errors wrapped in PeerHostMultiError, or nil +// if none found. +func (m *PeerHost) ValidateAll() error { + return m.validate(true) +} + +func (m *PeerHost) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for Uuid // no validation rules for Ip @@ -336,9 +547,28 @@ func (m *PeerHost) Validate() error { // no validation rules for NetTopology + if len(errors) > 0 { + return PeerHostMultiError(errors) + } return nil } +// PeerHostMultiError is an error wrapping multiple validation errors returned +// by PeerHost.ValidateAll() if the designated constraints aren't met. +type PeerHostMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m PeerHostMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m PeerHostMultiError) AllErrors() []error { return m } + // PeerHostValidationError is the validation error returned by // PeerHost.Validate if the designated constraints aren't met. type PeerHostValidationError struct { @@ -394,20 +624,61 @@ var _ interface { } = PeerHostValidationError{} // Validate checks the field values on PieceResult with the rules defined in -// the proto definition for this message. If any rules are violated, an error -// is returned. +// the proto definition for this message. If any rules are violated, the first +// error encountered is returned, or nil if there are no violations. func (m *PieceResult) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on PieceResult with the rules defined in +// the proto definition for this message. If any rules are violated, the +// result is a list of violation errors wrapped in PieceResultMultiError, or +// nil if none found. +func (m *PieceResult) ValidateAll() error { + return m.validate(true) +} + +func (m *PieceResult) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for TaskId // no validation rules for SrcPid // no validation rules for DstPid - // no validation rules for PieceNum + if all { + switch v := interface{}(m.GetPieceInfo()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, PieceResultValidationError{ + field: "PieceInfo", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, PieceResultValidationError{ + field: "PieceInfo", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetPieceInfo()).(interface{ Validate() error }); ok { + if err := v.Validate(); err != nil { + return PieceResultValidationError{ + field: "PieceInfo", + reason: "embedded message failed validation", + cause: err, + } + } + } // no validation rules for BeginTime @@ -417,7 +688,26 @@ func (m *PieceResult) Validate() error { // no validation rules for Code - if v, ok := interface{}(m.GetHostLoad()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetHostLoad()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, PieceResultValidationError{ + field: "HostLoad", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, PieceResultValidationError{ + field: "HostLoad", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetHostLoad()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return PieceResultValidationError{ field: "HostLoad", @@ -429,9 +719,28 @@ func (m *PieceResult) Validate() error { // no validation rules for FinishedCount + if len(errors) > 0 { + return PieceResultMultiError(errors) + } return nil } +// PieceResultMultiError is an error wrapping multiple validation errors +// returned by PieceResult.ValidateAll() if the designated constraints aren't met. +type PieceResultMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m PieceResultMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m PieceResultMultiError) AllErrors() []error { return m } + // PieceResultValidationError is the validation error returned by // PieceResult.Validate if the designated constraints aren't met. type PieceResultValidationError struct { @@ -487,19 +796,53 @@ var _ interface { } = PieceResultValidationError{} // Validate checks the field values on PeerPacket with the rules defined in the -// proto definition for this message. If any rules are violated, an error is returned. +// proto definition for this message. If any rules are violated, the first +// error encountered is returned, or nil if there are no violations. func (m *PeerPacket) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on PeerPacket with the rules defined in +// the proto definition for this message. If any rules are violated, the +// result is a list of violation errors wrapped in PeerPacketMultiError, or +// nil if none found. +func (m *PeerPacket) ValidateAll() error { + return m.validate(true) +} + +func (m *PeerPacket) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for TaskId // no validation rules for SrcPid // no validation rules for ParallelCount - if v, ok := interface{}(m.GetMainPeer()).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(m.GetMainPeer()).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, PeerPacketValidationError{ + field: "MainPeer", + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, PeerPacketValidationError{ + field: "MainPeer", + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(m.GetMainPeer()).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return PeerPacketValidationError{ field: "MainPeer", @@ -512,7 +855,26 @@ func (m *PeerPacket) Validate() error { for idx, item := range m.GetStealPeers() { _, _ = idx, item - if v, ok := interface{}(item).(interface{ Validate() error }); ok { + if all { + switch v := interface{}(item).(type) { + case interface{ ValidateAll() error }: + if err := v.ValidateAll(); err != nil { + errors = append(errors, PeerPacketValidationError{ + field: fmt.Sprintf("StealPeers[%v]", idx), + reason: "embedded message failed validation", + cause: err, + }) + } + case interface{ Validate() error }: + if err := v.Validate(); err != nil { + errors = append(errors, PeerPacketValidationError{ + field: fmt.Sprintf("StealPeers[%v]", idx), + reason: "embedded message failed validation", + cause: err, + }) + } + } + } else if v, ok := interface{}(item).(interface{ Validate() error }); ok { if err := v.Validate(); err != nil { return PeerPacketValidationError{ field: fmt.Sprintf("StealPeers[%v]", idx), @@ -526,9 +888,28 @@ func (m *PeerPacket) Validate() error { // no validation rules for Code + if len(errors) > 0 { + return PeerPacketMultiError(errors) + } return nil } +// PeerPacketMultiError is an error wrapping multiple validation errors +// returned by PeerPacket.ValidateAll() if the designated constraints aren't met. +type PeerPacketMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m PeerPacketMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m PeerPacketMultiError) AllErrors() []error { return m } + // PeerPacketValidationError is the validation error returned by // PeerPacket.Validate if the designated constraints aren't met. type PeerPacketValidationError struct { @@ -584,12 +965,27 @@ var _ interface { } = PeerPacketValidationError{} // Validate checks the field values on PeerResult with the rules defined in the -// proto definition for this message. If any rules are violated, an error is returned. +// proto definition for this message. If any rules are violated, the first +// error encountered is returned, or nil if there are no violations. func (m *PeerResult) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on PeerResult with the rules defined in +// the proto definition for this message. If any rules are violated, the +// result is a list of violation errors wrapped in PeerResultMultiError, or +// nil if none found. +func (m *PeerResult) ValidateAll() error { + return m.validate(true) +} + +func (m *PeerResult) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for TaskId // no validation rules for PeerId @@ -612,9 +1008,30 @@ func (m *PeerResult) Validate() error { // no validation rules for Code + // no validation rules for TotalPieceCount + + if len(errors) > 0 { + return PeerResultMultiError(errors) + } return nil } +// PeerResultMultiError is an error wrapping multiple validation errors +// returned by PeerResult.ValidateAll() if the designated constraints aren't met. +type PeerResultMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m PeerResultMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m PeerResultMultiError) AllErrors() []error { return m } + // PeerResultValidationError is the validation error returned by // PeerResult.Validate if the designated constraints aren't met. type PeerResultValidationError struct { @@ -670,19 +1087,53 @@ var _ interface { } = PeerResultValidationError{} // Validate checks the field values on PeerTarget with the rules defined in the -// proto definition for this message. If any rules are violated, an error is returned. +// proto definition for this message. If any rules are violated, the first +// error encountered is returned, or nil if there are no violations. func (m *PeerTarget) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on PeerTarget with the rules defined in +// the proto definition for this message. If any rules are violated, the +// result is a list of violation errors wrapped in PeerTargetMultiError, or +// nil if none found. +func (m *PeerTarget) ValidateAll() error { + return m.validate(true) +} + +func (m *PeerTarget) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for TaskId // no validation rules for PeerId + if len(errors) > 0 { + return PeerTargetMultiError(errors) + } return nil } +// PeerTargetMultiError is an error wrapping multiple validation errors +// returned by PeerTarget.ValidateAll() if the designated constraints aren't met. +type PeerTargetMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m PeerTargetMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m PeerTargetMultiError) AllErrors() []error { return m } + // PeerTargetValidationError is the validation error returned by // PeerTarget.Validate if the designated constraints aren't met. type PeerTargetValidationError struct { @@ -739,21 +1190,55 @@ var _ interface { // Validate checks the field values on PeerPacket_DestPeer with the rules // defined in the proto definition for this message. If any rules are -// violated, an error is returned. +// violated, the first error encountered is returned, or nil if there are no violations. func (m *PeerPacket_DestPeer) Validate() error { + return m.validate(false) +} + +// ValidateAll checks the field values on PeerPacket_DestPeer with the rules +// defined in the proto definition for this message. If any rules are +// violated, the result is a list of violation errors wrapped in +// PeerPacket_DestPeerMultiError, or nil if none found. +func (m *PeerPacket_DestPeer) ValidateAll() error { + return m.validate(true) +} + +func (m *PeerPacket_DestPeer) validate(all bool) error { if m == nil { return nil } + var errors []error + // no validation rules for Ip // no validation rules for RpcPort // no validation rules for PeerId + if len(errors) > 0 { + return PeerPacket_DestPeerMultiError(errors) + } return nil } +// PeerPacket_DestPeerMultiError is an error wrapping multiple validation +// errors returned by PeerPacket_DestPeer.ValidateAll() if the designated +// constraints aren't met. +type PeerPacket_DestPeerMultiError []error + +// Error returns a concatenation of all the error messages it wraps. +func (m PeerPacket_DestPeerMultiError) Error() string { + var msgs []string + for _, err := range m { + msgs = append(msgs, err.Error()) + } + return strings.Join(msgs, "; ") +} + +// AllErrors returns a list of validation violation errors. +func (m PeerPacket_DestPeerMultiError) AllErrors() []error { return m } + // PeerPacket_DestPeerValidationError is the validation error returned by // PeerPacket_DestPeer.Validate if the designated constraints aren't met. type PeerPacket_DestPeerValidationError struct { diff --git a/pkg/rpc/scheduler/scheduler.proto b/pkg/rpc/scheduler/scheduler.proto index 3d540bfb3..287a1fe46 100644 --- a/pkg/rpc/scheduler/scheduler.proto +++ b/pkg/rpc/scheduler/scheduler.proto @@ -90,8 +90,8 @@ message PieceResult{ string src_pid = 2; // dest peer id string dst_pid = 3; - // piece number - int32 piece_num = 4; + // piece info + base.PieceInfo piece_info = 4; // begin time for the piece downloading uint64 begin_time = 5; // end time for the piece downloading @@ -122,7 +122,6 @@ message PeerPacket{ string src_pid = 3; // concurrent downloading count from main peer - // TODO what effect int32 parallel_count = 4; DestPeer main_peer = 5; repeated DestPeer steal_peers = 6; @@ -147,6 +146,7 @@ message PeerResult{ bool success = 10; // result code base.Code code = 11; + int32 total_piece_count = 12; } message PeerTarget{ diff --git a/pkg/rpc/scheduler/scheduler_grpc.pb.go b/pkg/rpc/scheduler/scheduler_grpc.pb.go index a1b1c4149..2ac297813 100644 --- a/pkg/rpc/scheduler/scheduler_grpc.pb.go +++ b/pkg/rpc/scheduler/scheduler_grpc.pb.go @@ -12,7 +12,6 @@ import ( // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -// Requires gRPC-Go v1.32.0 or later. const _ = grpc.SupportPackageIsVersion7 // SchedulerClient is the client API for Scheduler service. @@ -49,7 +48,7 @@ func (c *schedulerClient) RegisterPeerTask(ctx context.Context, in *PeerTaskRequ } func (c *schedulerClient) ReportPieceResult(ctx context.Context, opts ...grpc.CallOption) (Scheduler_ReportPieceResultClient, error) { - stream, err := c.cc.NewStream(ctx, &Scheduler_ServiceDesc.Streams[0], "/scheduler.Scheduler/ReportPieceResult", opts...) + stream, err := c.cc.NewStream(ctx, &_Scheduler_serviceDesc.Streams[0], "/scheduler.Scheduler/ReportPieceResult", opts...) if err != nil { return nil, err } @@ -140,7 +139,7 @@ type UnsafeSchedulerServer interface { } func RegisterSchedulerServer(s grpc.ServiceRegistrar, srv SchedulerServer) { - s.RegisterService(&Scheduler_ServiceDesc, srv) + s.RegisterService(&_Scheduler_serviceDesc, srv) } func _Scheduler_RegisterPeerTask_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { @@ -223,10 +222,7 @@ func _Scheduler_LeaveTask_Handler(srv interface{}, ctx context.Context, dec func return interceptor(ctx, in, info, handler) } -// Scheduler_ServiceDesc is the grpc.ServiceDesc for Scheduler service. -// It's only intended for direct use with grpc.RegisterService, -// and not to be introspected or modified (even as a copy) -var Scheduler_ServiceDesc = grpc.ServiceDesc{ +var _Scheduler_serviceDesc = grpc.ServiceDesc{ ServiceName: "scheduler.Scheduler", HandlerType: (*SchedulerServer)(nil), Methods: []grpc.MethodDesc{ diff --git a/pkg/rpc/scheduler/server/server.go b/pkg/rpc/scheduler/server/server.go index 85a508189..198daee1d 100644 --- a/pkg/rpc/scheduler/server/server.go +++ b/pkg/rpc/scheduler/server/server.go @@ -44,15 +44,15 @@ type proxy struct { scheduler.UnimplementedSchedulerServer } -// see scheduler.SchedulerServer +// SchedulerServer scheduler.SchedulerServer type SchedulerServer interface { - // RegisterPeerTask + // RegisterPeerTask register a peer to scheduler RegisterPeerTask(context.Context, *scheduler.PeerTaskRequest) (*scheduler.RegisterResult, error) - // ReportPieceResult + // ReportPieceResult report piece result to scheduler ReportPieceResult(scheduler.Scheduler_ReportPieceResultServer) error - // ReportPeerResult + // ReportPeerResult report peer download result to scheduler ReportPeerResult(context.Context, *scheduler.PeerResult) error - // LeaveTask + // LeaveTask leave peer from scheduler LeaveTask(context.Context, *scheduler.PeerTarget) error } diff --git a/scheduler/config/config.go b/scheduler/config/config.go index ea3934895..fa93e0283 100644 --- a/scheduler/config/config.go +++ b/scheduler/config/config.go @@ -35,12 +35,12 @@ type Config struct { Manager *ManagerConfig `yaml:"manager" mapstructure:"manager"` Host *HostConfig `yaml:"host" mapstructure:"host"` Job *JobConfig `yaml:"job" mapstructure:"job"` + DisableCDN bool `yaml:"disableCDN" mapstructure:"disableCDN"` } func New() *Config { return &Config{ Scheduler: &SchedulerConfig{ - DisableCDN: false, ABTest: false, AScheduler: "", BScheduler: "", @@ -53,8 +53,8 @@ func New() *Config { ClientLoad: 10, OpenMonitor: true, GC: &GCConfig{ - PeerGCInterval: 5 * time.Minute, - TaskGCInterval: 5 * time.Minute, + PeerGCInterval: 1 * time.Minute, + TaskGCInterval: 1 * time.Minute, PeerTTL: 10 * time.Minute, PeerTTI: 3 * time.Minute, TaskTTL: 10 * time.Minute, @@ -109,6 +109,7 @@ func New() *Config { BackendDB: 2, }, }, + DisableCDN: false, } } @@ -178,12 +179,11 @@ type DynConfig struct { } type SchedulerConfig struct { - DisableCDN bool `yaml:"disableCDN" mapstructure:"disableCDN"` ABTest bool `yaml:"abtest" mapstructure:"abtest"` AScheduler string `yaml:"ascheduler" mapstructure:"ascheduler"` BScheduler string `yaml:"bscheduler" mapstructure:"bscheduler"` WorkerNum int `yaml:"workerNum" mapstructure:"workerNum"` - BackSourceCount int `yaml:"backSourceCount" mapstructure:"backSourceCount"` + BackSourceCount int32 `yaml:"backSourceCount" mapstructure:"backSourceCount"` // AccessWindow should less than CDN task expireTime AccessWindow time.Duration `yaml:"accessWindow" mapstructure:"accessWindow"` CandidateParentCount int `yaml:"candidateParentCount" mapstructure:"candidateParentCount"` @@ -202,6 +202,7 @@ type ServerConfig struct { type GCConfig struct { PeerGCInterval time.Duration `yaml:"peerGCInterval" mapstructure:"peerGCInterval"` + // PeerTTL is advised to set the time to be smaller than the expire time of a task in the CDN PeerTTL time.Duration `yaml:"peerTTL" mapstructure:"peerTTL"` PeerTTI time.Duration `yaml:"peerTTI" mapstructure:"peerTTI"` TaskGCInterval time.Duration `yaml:"taskGCInterval" mapstructure:"taskGCInterval"` diff --git a/scheduler/config/constants_otel.go b/scheduler/config/constants_otel.go index b718ee589..97cc90eef 100644 --- a/scheduler/config/constants_otel.go +++ b/scheduler/config/constants_otel.go @@ -35,6 +35,10 @@ const ( AttributePeerID = attribute.Key("d7y.peer.id") AttributeCDNSeedRequest = attribute.Key("d7y.cdn.seed.request") AttributeNeedSeedCDN = attribute.Key("d7y.need.seed.cdn") + AttributeTaskStatus = attribute.Key("d7y.task.status") + AttributeLastTriggerTime = attribute.Key("d7y.task.last.trigger.time") + AttributeClientBackSource = attribute.Key("d7y.need.client.back-source") + AttributeTriggerCDNError = attribute.Key("d7y.trigger.cdn.error") ) const ( @@ -46,11 +50,13 @@ const ( ) const ( - EventScheduleParentFail = "fail-schedule-parent" - EventPeerNotFound = "peer-not-found" - EventHostNotFound = "host-not-found" - EventCreateCDNPeer = "create-cdn-peer" - EventPieceReceived = "receive-piece" - EventPeerDownloaded = "downloaded" - EventDownloadTinyFile = "download-tiny-file" + EventSmallTaskSelectParentFail = "small-task-select-parent-fail" + EventPeerNotFound = "peer-not-found" + EventHostNotFound = "host-not-found" + EventCreateCDNPeer = "create-cdn-peer" + EventPieceReceived = "receive-piece" + EventPeerDownloaded = "downloaded" + EventDownloadTinyFile = "download-tiny-file" + EventStartReportPieceResult = "start-report-piece-result" + EventCDNFailBackClientSource = "cdn-fail-back-client-source" ) diff --git a/scheduler/core/evaluator/basic/basic_evaluator.go b/scheduler/core/evaluator/basic/basic_evaluator.go index 63fbae20e..9a255e529 100644 --- a/scheduler/core/evaluator/basic/basic_evaluator.go +++ b/scheduler/core/evaluator/basic/basic_evaluator.go @@ -69,7 +69,7 @@ func (eval *baseEvaluator) NeedAdjustParent(peer *supervisor.Peer) bool { func (eval *baseEvaluator) IsBadNode(peer *supervisor.Peer) bool { if peer.IsBad() { - logger.Debugf("peer %s is bad because status is %s", peer.PeerID, peer.GetStatus()) + logger.Debugf("peer %s is bad because it's status is %s", peer.PeerID, peer.GetStatus()) return true } costHistory := peer.GetCostHistory() diff --git a/scheduler/core/events.go b/scheduler/core/events.go index 9a8c2b4fa..841bb26c1 100644 --- a/scheduler/core/events.go +++ b/scheduler/core/events.go @@ -21,11 +21,12 @@ import ( "time" "d7y.io/dragonfly/v2/internal/dfcodes" + "d7y.io/dragonfly/v2/internal/dferrors" logger "d7y.io/dragonfly/v2/internal/dflog" - "d7y.io/dragonfly/v2/pkg/rpc/base" schedulerRPC "d7y.io/dragonfly/v2/pkg/rpc/scheduler" "d7y.io/dragonfly/v2/pkg/structure/sortedlist" "d7y.io/dragonfly/v2/pkg/synclock" + "d7y.io/dragonfly/v2/scheduler/config" "d7y.io/dragonfly/v2/scheduler/core/scheduler" "d7y.io/dragonfly/v2/scheduler/supervisor" "go.opentelemetry.io/otel/trace" @@ -76,20 +77,37 @@ var _ event = startReportPieceResultEvent{} func (e startReportPieceResultEvent) apply(s *state) { span := trace.SpanFromContext(e.ctx) + span.AddEvent(config.EventStartReportPieceResult) if e.peer.GetParent() != nil { - span.AddEvent("no parent") logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("startReportPieceResultEvent: no need schedule parent because peer already had parent %s", e.peer.GetParent().PeerID) return } + if e.peer.Task.IsBackSourcePeer(e.peer.PeerID) { + logger.WithTaskAndPeerID(e.peer.Task.TaskID, + e.peer.PeerID).Info("startReportPieceResultEvent: no need schedule parent because peer is back source peer") + s.waitScheduleParentPeerQueue.Done(e.peer) + return + } parent, candidates, hasParent := s.sched.ScheduleParent(e.peer) - span.AddEvent("parent") + // No parent node is currently available if !hasParent { - logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("startReportPieceResultEvent: there is no available parent,reschedule it in one second") + if e.peer.Task.NeedClientBackSource() { + span.SetAttributes(config.AttributeClientBackSource.Bool(true)) + if e.peer.CloseChannel(dferrors.Newf(dfcodes.SchedNeedBackSource, "peer %s need back source", e.peer.PeerID)) == nil { + e.peer.Task.IncreaseBackSourcePeer(e.peer.PeerID) + s.waitScheduleParentPeerQueue.Done(e.peer) + } + return + } + logger.WithTaskAndPeerID(e.peer.Task.TaskID, + e.peer.PeerID).Warnf("startReportPieceResultEvent: no parent node is currently available,reschedule it later") s.waitScheduleParentPeerQueue.AddAfter(e.peer, time.Second) return } - e.peer.SendSchedulePacket(constructSuccessPeerPacket(e.peer, parent, candidates)) + if err := e.peer.SendSchedulePacket(constructSuccessPeerPacket(e.peer, parent, candidates)); err != nil { + logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", e.peer.PeerID, err) + } } func (e startReportPieceResultEvent) hashKey() string { @@ -106,28 +124,45 @@ var _ event = peerDownloadPieceSuccessEvent{} func (e peerDownloadPieceSuccessEvent) apply(s *state) { span := trace.SpanFromContext(e.ctx) - span.AddEvent("piece success") - e.peer.AddPieceInfo(e.pr.FinishedCount, int(e.pr.EndTime-e.pr.BeginTime)) - oldParent := e.peer.GetParent() + span.AddEvent(config.EventPieceReceived, trace.WithAttributes(config.AttributePieceReceived.String(e.pr.String()))) + e.peer.UpdateProgress(e.pr.FinishedCount, int(e.pr.EndTime-e.pr.BeginTime)) + if e.peer.Task.IsBackSourcePeer(e.peer.PeerID) { + e.peer.Task.AddPiece(e.pr.PieceInfo) + if !e.peer.Task.CanSchedule() { + logger.WithTaskAndPeerID(e.peer.Task.TaskID, + e.peer.PeerID).Warnf("peerDownloadPieceSuccessEvent: update task status seeding") + e.peer.Task.SetStatus(supervisor.TaskStatusSeeding) + } + return + } var candidates []*supervisor.Peer parentPeer, ok := s.peerManager.Get(e.pr.DstPid) - if !ok || parentPeer.IsLeave() { + if ok { + oldParent := e.peer.GetParent() + if e.pr.DstPid != e.peer.PeerID && (oldParent == nil || oldParent.PeerID != e.pr.DstPid) { + e.peer.ReplaceParent(parentPeer) + } + } else if parentPeer.IsLeave() { + logger.WithTaskAndPeerID(e.peer.Task.TaskID, + e.peer.PeerID).Warnf("peerDownloadPieceSuccessEvent: need reschedule parent for peer because it's parent is leave") e.peer.ReplaceParent(nil) var hasParent bool parentPeer, candidates, hasParent = s.sched.ScheduleParent(e.peer) if !hasParent { - logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("peerDownloadPieceSuccessEvent: there is no available parent, reschedule it in one second") + logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("peerDownloadPieceSuccessEvent: no parent node is currently available, " + + "reschedule it later") s.waitScheduleParentPeerQueue.AddAfter(e.peer, time.Second) return } } parentPeer.Touch() - if oldParent != nil { - candidates = append(candidates, oldParent) + if parentPeer.PeerID == e.pr.DstPid { + return } // TODO if parentPeer is equal with oldParent, need schedule again ? - e.peer.SendSchedulePacket(constructSuccessPeerPacket(e.peer, parentPeer, candidates)) - return + if err := e.peer.SendSchedulePacket(constructSuccessPeerPacket(e.peer, parentPeer, candidates)); err != nil { + logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", e.peer.PeerID, err) + } } func (e peerDownloadPieceSuccessEvent) hashKey() string { @@ -135,6 +170,7 @@ func (e peerDownloadPieceSuccessEvent) hashKey() string { } type peerDownloadPieceFailEvent struct { + ctx context.Context peer *supervisor.Peer pr *schedulerRPC.PieceResult } @@ -142,7 +178,14 @@ type peerDownloadPieceFailEvent struct { var _ event = peerDownloadPieceFailEvent{} func (e peerDownloadPieceFailEvent) apply(s *state) { + span := trace.SpanFromContext(e.ctx) + span.AddEvent(config.EventPieceReceived, trace.WithAttributes(config.AttributePieceReceived.String(e.pr.String()))) + if e.peer.Task.IsBackSourcePeer(e.peer.PeerID) { + return + } switch e.pr.Code { + case dfcodes.ClientWaitPieceReady: + return case dfcodes.PeerTaskNotFound, dfcodes.ClientPieceRequestFail, dfcodes.ClientPieceDownloadFail: // TODO PeerTaskNotFound remove dest peer task, ClientPieceDownloadFail add blank list reScheduleParent(e.peer, s) @@ -152,16 +195,17 @@ func (e peerDownloadPieceFailEvent) apply(s *state) { // TODO synclock.Lock(task.TaskID, false) defer synclock.UnLock(task.TaskID, false) - task.SetStatus(supervisor.TaskStatusRunning) if cdnPeer, err := s.cdnManager.StartSeedTask(context.Background(), task); err != nil { logger.Errorf("start seed task fail: %v", err) - task.SetStatus(supervisor.TaskStatusFailed) + span.AddEvent(config.EventCDNFailBackClientSource, trace.WithAttributes(config.AttributeTriggerCDNError.String(err.Error()))) handleSeedTaskFail(task) } else { logger.Debugf("===== successfully obtain seeds from cdn, task: %+v =====", e.peer.Task) children := s.sched.ScheduleChildren(cdnPeer) for _, child := range children { - child.SendSchedulePacket(constructSuccessPeerPacket(child, cdnPeer, nil)) + if err := child.SendSchedulePacket(constructSuccessPeerPacket(child, cdnPeer, nil)); err != nil { + logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", child.PeerID, err) + } } } }(e.peer.Task) @@ -197,12 +241,16 @@ var _ event = peerDownloadSuccessEvent{} func (e peerDownloadSuccessEvent) apply(s *state) { e.peer.SetStatus(supervisor.PeerStatusSuccess) + if e.peer.Task.IsBackSourcePeer(e.peer.PeerID) { + e.peer.Task.UpdateTaskSuccessResult(e.peerResult.TotalPieceCount, e.peerResult.ContentLength) + } removePeerFromCurrentTree(e.peer, s) children := s.sched.ScheduleChildren(e.peer) for _, child := range children { - child.SendSchedulePacket(constructSuccessPeerPacket(child, e.peer, nil)) + if err := child.SendSchedulePacket(constructSuccessPeerPacket(child, e.peer, nil)); err != nil { + logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", child.PeerID, err) + } } - e.peer.UnBindSendChannel() } func (e peerDownloadSuccessEvent) hashKey() string { @@ -218,16 +266,23 @@ var _ event = peerDownloadFailEvent{} func (e peerDownloadFailEvent) apply(s *state) { e.peer.SetStatus(supervisor.PeerStatusFail) + if e.peer.Task.IsBackSourcePeer(e.peer.PeerID) && !e.peer.Task.IsSuccess() { + e.peer.Task.SetStatus(supervisor.TaskStatusFail) + handleSeedTaskFail(e.peer.Task) + return + } removePeerFromCurrentTree(e.peer, s) e.peer.GetChildren().Range(func(key, value interface{}) bool { child := (value).(*supervisor.Peer) parent, candidates, hasParent := s.sched.ScheduleParent(child) if !hasParent { - logger.WithTaskAndPeerID(child.Task.TaskID, child.PeerID).Warnf("peerDownloadFailEvent: there is no available parent, reschedule it in one second") + logger.WithTaskAndPeerID(child.Task.TaskID, child.PeerID).Warnf("peerDownloadFailEvent: there is no available parent, reschedule it later") s.waitScheduleParentPeerQueue.AddAfter(e.peer, time.Second) return true } - child.SendSchedulePacket(constructSuccessPeerPacket(child, parent, candidates)) + if err := child.SendSchedulePacket(constructSuccessPeerPacket(child, parent, candidates)); err != nil { + logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", child.PeerID, err) + } return true }) } @@ -250,11 +305,13 @@ func (e peerLeaveEvent) apply(s *state) { child := value.(*supervisor.Peer) parent, candidates, hasParent := s.sched.ScheduleParent(child) if !hasParent { - logger.WithTaskAndPeerID(child.Task.TaskID, child.PeerID).Warnf("handlePeerLeave: there is no available parent,reschedule it in one second") + logger.WithTaskAndPeerID(child.Task.TaskID, child.PeerID).Warnf("handlePeerLeave: there is no available parent,reschedule it later") s.waitScheduleParentPeerQueue.AddAfter(child, time.Second) return true } - child.SendSchedulePacket(constructSuccessPeerPacket(child, parent, candidates)) + if err := child.SendSchedulePacket(constructSuccessPeerPacket(child, parent, candidates)); err != nil { + logger.WithTaskAndPeerID(e.peer.Task.TaskID, e.peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", child.PeerID, err) + } return true }) s.peerManager.Delete(e.peer.PeerID) @@ -264,6 +321,7 @@ func (e peerLeaveEvent) hashKey() string { return e.peer.Task.TaskID } +// constructSuccessPeerPacket construct success peer schedule packet func constructSuccessPeerPacket(peer *supervisor.Peer, parent *supervisor.Peer, candidates []*supervisor.Peer) *schedulerRPC.PeerPacket { mainPeer := &schedulerRPC.PeerPacket_DestPeer{ Ip: parent.Host.IP, @@ -290,30 +348,42 @@ func constructSuccessPeerPacket(peer *supervisor.Peer, parent *supervisor.Peer, return peerPacket } -func constructFailPeerPacket(peer *supervisor.Peer, errCode base.Code) *schedulerRPC.PeerPacket { - return &schedulerRPC.PeerPacket{ - TaskId: peer.Task.TaskID, - SrcPid: peer.PeerID, - Code: errCode, - } -} - func reScheduleParent(peer *supervisor.Peer, s *state) { parent, candidates, hasParent := s.sched.ScheduleParent(peer) if !hasParent { - logger.Errorf("handleReplaceParent: failed to schedule parent to peer %s, reschedule it in one second", peer.PeerID) + if peer.Task.NeedClientBackSource() { + if peer.CloseChannel(dferrors.Newf(dfcodes.SchedNeedBackSource, "peer %s need back source", peer.PeerID)) == nil { + peer.Task.IncreaseBackSourcePeer(peer.PeerID) + } + return + } + logger.Errorf("reScheduleParent: failed to schedule parent to peer %s, reschedule it later", peer.PeerID) //peer.PacketChan <- constructFailPeerPacket(peer, dfcodes.SchedWithoutParentPeer) s.waitScheduleParentPeerQueue.AddAfter(peer, time.Second) return } - peer.SendSchedulePacket(constructSuccessPeerPacket(peer, parent, candidates)) + // TODO if parentPeer is equal with oldParent, need schedule again ? + if err := peer.SendSchedulePacket(constructSuccessPeerPacket(peer, parent, candidates)); err != nil { + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", peer.PeerID, err) + } } func handleSeedTaskFail(task *supervisor.Task) { - if task.IsFail() { + if task.NeedClientBackSource() { task.ListPeers().Range(func(data sortedlist.Item) bool { peer := data.(*supervisor.Peer) - peer.SendSchedulePacket(constructFailPeerPacket(peer, dfcodes.SchedNeedBackSource)) + if task.NeedClientBackSource() { + if peer.CloseChannel(dferrors.Newf(dfcodes.SchedNeedBackSource, "peer %s need back source", peer.PeerID)) == nil { + task.IncreaseBackSourcePeer(peer.PeerID) + } + return true + } + return false + }) + } else { + task.ListPeers().Range(func(data sortedlist.Item) bool { + peer := data.(*supervisor.Peer) + peer.CloseChannel(dferrors.New(dfcodes.SchedTaskStatusError, "schedule task status failed")) return true }) } @@ -326,7 +396,9 @@ func removePeerFromCurrentTree(peer *supervisor.Peer, s *state) { if parent != nil { children := s.sched.ScheduleChildren(parent) for _, child := range children { - child.SendSchedulePacket(constructSuccessPeerPacket(child, peer, nil)) + if err := child.SendSchedulePacket(constructSuccessPeerPacket(child, peer, nil)); err != nil { + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Warnf("send schedule packet to peer %s failed: %v", child.PeerID, err) + } } } } diff --git a/scheduler/core/scheduler/basic/basic_scheduler.go b/scheduler/core/scheduler/basic/basic_scheduler.go index 0c1f4589f..0cd476696 100644 --- a/scheduler/core/scheduler/basic/basic_scheduler.go +++ b/scheduler/core/scheduler/basic/basic_scheduler.go @@ -70,19 +70,20 @@ type Scheduler struct { } func (s *Scheduler) ScheduleChildren(peer *supervisor.Peer) (children []*supervisor.Peer) { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debug("start schedule children flow") if s.evaluator.IsBadNode(peer) { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debug("stop schedule children flow because peer is bad node") + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debug("terminate schedule children flow because peer is bad node") return } freeUpload := peer.Host.GetFreeUploadLoad() candidateChildren := s.selectCandidateChildren(peer, freeUpload*2) - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("select num %d candidate children %v", len(candidateChildren), candidateChildren) - evalResult := make(map[float64]*supervisor.Peer) + if len(candidateChildren) == 0 { + return nil + } + evalResult := make(map[float64][]*supervisor.Peer) var evalScore []float64 for _, child := range candidateChildren { score := s.evaluator.Evaluate(peer, child) - evalResult[score] = child + evalResult[score] = append(evalResult[score], child) evalScore = append(evalScore, score) } sort.Float64s(evalScore) @@ -90,22 +91,26 @@ func (s *Scheduler) ScheduleChildren(peer *supervisor.Peer) (children []*supervi if freeUpload <= 0 { break } - child := evalResult[evalScore[len(evalScore)-i-1]] - if child.GetParent() == peer { - continue + peers := evalResult[evalScore[len(evalScore)-i-1]] + for _, child := range peers { + if freeUpload <= 0 { + break + } + if child.GetParent() == peer { + continue + } + children = append(children, child) + freeUpload-- } - children = append(children, child) - freeUpload-- } for _, child := range children { child.ReplaceParent(peer) } - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("final schedule children list %v", children) + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("schedule children result: %v", children) return } func (s *Scheduler) ScheduleParent(peer *supervisor.Peer) (*supervisor.Peer, []*supervisor.Peer, bool) { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debug("start schedule parent flow") //if !s.evaluator.NeedAdjustParent(peer) { // logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("stop schedule parent flow because peer is not need adjust parent", peer.PeerID) // if peer.GetParent() == nil { @@ -114,24 +119,21 @@ func (s *Scheduler) ScheduleParent(peer *supervisor.Peer) (*supervisor.Peer, []* // return peer.GetParent(), []*types.Peer{peer.GetParent()}, true //} candidateParents := s.selectCandidateParents(peer, s.cfg.CandidateParentCount) - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("select num %d candidates parents, current task tree node count %d ", - len(candidateParents), peer.Task.ListPeers().Size()) if len(candidateParents) == 0 { return nil, nil, false } - evalResult := make(map[float64]*supervisor.Peer) + evalResult := make(map[float64][]*supervisor.Peer) var evalScore []float64 - for _, candidate := range candidateParents { - score := s.evaluator.Evaluate(candidate, peer) - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("evaluate score candidate %s is %f", candidate.PeerID, score) - evalResult[score] = candidate + for _, parent := range candidateParents { + score := s.evaluator.Evaluate(parent, peer) + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("evaluate score candidate %s is %f", parent.PeerID, score) + evalResult[score] = append(evalResult[score], parent) evalScore = append(evalScore, score) } sort.Float64s(evalScore) var parents = make([]*supervisor.Peer, 0, len(candidateParents)) for i := range evalScore { - parent := evalResult[evalScore[len(evalScore)-i-1]] - parents = append(parents, parent) + parents = append(parents, evalResult[evalScore[len(evalScore)-i-1]]...) } if parents[0] != peer.GetParent() { peer.ReplaceParent(parents[0]) @@ -140,108 +142,132 @@ func (s *Scheduler) ScheduleParent(peer *supervisor.Peer) (*supervisor.Peer, []* return parents[0], parents[1:], true } -func (s *Scheduler) selectCandidateChildren(peer *supervisor.Peer, limit int) (list []*supervisor.Peer) { - return s.peerManager.Pick(peer.Task, limit, func(candidateNode *supervisor.Peer) bool { +func (s *Scheduler) selectCandidateChildren(peer *supervisor.Peer, limit int) (candidateChildren []*supervisor.Peer) { + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debug("start schedule children flow") + defer logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("finish schedule parent flow, select num %d candidate children, "+ + "current task tree node count %d, back source peers: %s", len(candidateChildren), peer.Task.ListPeers().Size(), peer.Task.GetBackSourcePeers()) + candidateChildren = peer.Task.Pick(limit, func(candidateNode *supervisor.Peer) bool { if candidateNode == nil { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer is not selected because it is nil") + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer is not selected because it is nil******") return false } - // TODO IsWaiting if candidateNode.IsDone() { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it has done", + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it has done******", candidateNode.PeerID) return false } if candidateNode.IsLeave() { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it has left", + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it has left******", + candidateNode.PeerID) + return false + } + if candidateNode.IsWaiting() { + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it's status is Waiting******", candidateNode.PeerID) return false } if candidateNode == peer { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it and peer are the same", + logger.WithTaskAndPeerID(peer.Task.TaskID, + peer.PeerID).Debugf("******candidate child peer %s is not selected because it and peer are the same******", candidateNode.PeerID) return false } if candidateNode.IsAncestorOf(peer) { logger.WithTaskAndPeerID(peer.Task.TaskID, - peer.PeerID).Debugf("******candidate child peer %s is not selected because peer's ancestor is candidate peer", candidateNode.PeerID) + peer.PeerID).Debugf("******candidate child peer %s is not selected because peer's ancestor is candidate peer******", candidateNode.PeerID) return false } - if candidateNode.GetFinishedNum() > peer.GetFinishedNum() { + if candidateNode.GetFinishedNum() >= peer.GetFinishedNum() { logger.WithTaskAndPeerID(peer.Task.TaskID, - peer.PeerID).Debugf("******candidate child peer %s is not selected because it finished number of download is more than peer's", + peer.PeerID).Debugf("******candidate child peer %s is not selected because it finished number of download is equal to or greater than peer's"+ + "******", candidateNode.PeerID) return false } if candidateNode.Host != nil && candidateNode.Host.CDN { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it is a cdn host", + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it is a cdn host******", + candidateNode.PeerID) + return false + } + if !candidateNode.IsConnected() { + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is not selected because it is not connected******", candidateNode.PeerID) return false } if candidateNode.GetParent() == nil { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is selected because it has not parent", + logger.WithTaskAndPeerID(peer.Task.TaskID, + peer.PeerID).Debugf("******[selected]candidate child peer %s is selected because it has not parent[selected]******", candidateNode.PeerID) return true } if candidateNode.GetParent() != nil && s.evaluator.IsBadNode(candidateNode.GetParent()) { logger.WithTaskAndPeerID(peer.Task.TaskID, - peer.PeerID).Debugf("******candidate child peer %s is selected because it has parent and parent status is not health", candidateNode.PeerID) + peer.PeerID).Debugf("******[selected]candidate child peer %s is selected because parent's status is not health[selected]******", + candidateNode.PeerID) return true } - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******candidate child peer %s is selected", candidateNode.PeerID) - return false + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("******[default]candidate child peer %s is selected[default]******", candidateNode.PeerID) + return true }) + return } -func (s *Scheduler) selectCandidateParents(peer *supervisor.Peer, limit int) (list []*supervisor.Peer) { +func (s *Scheduler) selectCandidateParents(peer *supervisor.Peer, limit int) (candidateParents []*supervisor.Peer) { + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debug("start schedule parent flow") + defer logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("finish schedule parent flow, select num %d candidates parents,"+ + "current task tree node count %d, back source peers: %s", len(candidateParents), peer.Task.ListPeers().Size(), peer.Task.GetBackSourcePeers()) if !peer.Task.CanSchedule() { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++peer %s can not be scheduled because task status", peer.PeerID) + logger.WithTaskAndPeerID(peer.Task.TaskID, + peer.PeerID).Debugf("++++++peer can not be scheduled because task cannot be scheduled at this time,waiting task status become seeding. "+ + "it current status is %s++++++", peer.Task.GetStatus()) return nil } - return s.peerManager.PickReverse(peer.Task, limit, func(candidateNode *supervisor.Peer) bool { + candidateParents = peer.Task.PickReverse(limit, func(candidateNode *supervisor.Peer) bool { if candidateNode == nil { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer is not selected because it is nil") + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer is not selected because it is nil++++++") return false } if s.evaluator.IsBadNode(candidateNode) { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it is badNode", + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it is badNode++++++", candidateNode.PeerID) return false } if candidateNode.IsLeave() { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it has already left", + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it has already left++++++", candidateNode.PeerID) return false } if candidateNode == peer { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it and peer are the same", + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it and peer are the same++++++", candidateNode.PeerID) return false } if candidateNode.IsDescendantOf(peer) { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it's ancestor is peer", + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it's ancestor is peer++++++", candidateNode.PeerID) return false } if candidateNode.Host.GetFreeUploadLoad() <= 0 { logger.WithTaskAndPeerID(peer.Task.TaskID, - peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it's free upload load equal to less than zero", + peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it's free upload load equal to less than zero++++++", candidateNode.PeerID) return false } if candidateNode.IsWaiting() { - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it's status is waiting", + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it's status is waiting++++++", candidateNode.PeerID) return false } - if candidateNode.GetFinishedNum() < peer.GetFinishedNum() { + if candidateNode.GetFinishedNum() <= peer.GetFinishedNum() { logger.WithTaskAndPeerID(peer.Task.TaskID, - peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it finished number of download is smaller than peer's", + peer.PeerID).Debugf("++++++candidate parent peer %s is not selected because it finished number of download is equal to or smaller than peer"+ + "'s++++++", candidateNode.PeerID) return false } - logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++candidate parent peer %s is selected", candidateNode.PeerID) + logger.WithTaskAndPeerID(peer.Task.TaskID, peer.PeerID).Debugf("++++++[default]candidate parent peer %s is selected[default]", candidateNode.PeerID) return true }) + return } diff --git a/scheduler/core/service.go b/scheduler/core/service.go index 5cf659ccc..624d9b497 100644 --- a/scheduler/core/service.go +++ b/scheduler/core/service.go @@ -42,6 +42,25 @@ import ( "k8s.io/client-go/util/workqueue" ) +type Options struct { + openTel bool + disableCDN bool +} + +type Option func(options *Options) + +func WithOpenTel(openTel bool) Option { + return func(options *Options) { + options.openTel = openTel + } +} + +func WithDisableCDN(disableCDN bool) Option { + return func(options *Options) { + options.disableCDN = disableCDN + } +} + type SchedulerService struct { // cdn mgr cdnManager supervisor.CDNMgr @@ -62,22 +81,14 @@ type SchedulerService struct { wg sync.WaitGroup } -func NewSchedulerService(cfg *config.SchedulerConfig, dynConfig config.DynconfigInterface, openTel bool) (*SchedulerService, error) { +func NewSchedulerService(cfg *config.SchedulerConfig, dynConfig config.DynconfigInterface, options ...Option) (*SchedulerService, error) { + ops := &Options{} + for _, op := range options { + op(ops) + } + hostManager := host.NewManager() peerManager := peer.NewManager(cfg.GC, hostManager) - - var opts []grpc.DialOption - if openTel { - opts = append(opts, grpc.WithChainUnaryInterceptor(otelgrpc.UnaryClientInterceptor()), grpc.WithChainStreamInterceptor(otelgrpc.StreamClientInterceptor())) - } - cdnClient, err := cdn.NewRefreshableCDNClient(dynConfig, opts) - if err != nil { - return nil, errors.Wrap(err, "new refreshable cdn client") - } - cdnManager, err := cdn.NewManager(cdnClient, peerManager, hostManager) - if err != nil { - return nil, errors.Wrap(err, "new cdn manager") - } taskManager := task.NewManager(cfg.GC, peerManager) sched, err := scheduler.Get(cfg.Scheduler).Build(cfg, &scheduler.BuildOptions{ PeerManager: peerManager, @@ -89,8 +100,7 @@ func NewSchedulerService(cfg *config.SchedulerConfig, dynConfig config.Dynconfig work := newEventLoopGroup(cfg.WorkerNum) downloadMonitor := newMonitor(cfg.OpenMonitor, peerManager) done := make(chan struct{}) - return &SchedulerService{ - cdnManager: cdnManager, + s := &SchedulerService{ taskManager: taskManager, hostManager: hostManager, peerManager: peerManager, @@ -99,7 +109,23 @@ func NewSchedulerService(cfg *config.SchedulerConfig, dynConfig config.Dynconfig sched: sched, config: cfg, done: done, - }, nil + } + if !ops.disableCDN { + var opts []grpc.DialOption + if ops.openTel { + opts = append(opts, grpc.WithChainUnaryInterceptor(otelgrpc.UnaryClientInterceptor()), grpc.WithChainStreamInterceptor(otelgrpc.StreamClientInterceptor())) + } + cdnClient, err := cdn.NewRefreshableCDNClient(dynConfig, opts) + if err != nil { + return nil, errors.Wrap(err, "new refreshable cdn client") + } + cdnManager, err := cdn.NewManager(cdnClient, peerManager, hostManager) + if err != nil { + return nil, errors.Wrap(err, "new cdn manager") + } + s.cdnManager = cdnManager + } + return s, nil } func (s *SchedulerService) Serve() { @@ -138,6 +164,12 @@ func (s *SchedulerService) runReScheduleParentLoop(wsdq workqueue.DelayingInterf "isLeave %t", peer.GetStatus(), peer.IsLeave()) continue } + if peer.GetParent() != nil { + logger.WithTaskAndPeerID(peer.Task.TaskID, + peer.PeerID).Debugf("runReScheduleLoop: peer has left from waitScheduleParentPeerQueue because peer has parent %s", + peer.GetParent().PeerID) + continue + } s.worker.send(reScheduleParentEvent{peer}) } } @@ -155,6 +187,7 @@ func (s *SchedulerService) Stop() { if s.worker != nil { s.worker.stop() } + s.wg.Wait() }) } @@ -165,9 +198,8 @@ func (s *SchedulerService) GenerateTaskID(url string, meta *base.UrlMeta, peerID return idgen.TaskID(url, meta) } -func (s *SchedulerService) ScheduleParent(peer *supervisor.Peer) (parent *supervisor.Peer, err error) { +func (s *SchedulerService) SelectParent(peer *supervisor.Peer) (parent *supervisor.Peer, err error) { parent, _, hasParent := s.sched.ScheduleParent(peer) - //logger.Debugf("schedule parent result: parent %v, candidates:%v", parent, candidates) if !hasParent || parent == nil { return nil, errors.Errorf("no parent peer available for peer %v", peer.PeerID) } @@ -178,28 +210,27 @@ func (s *SchedulerService) GetPeerTask(peerTaskID string) (peerTask *supervisor. return s.peerManager.Get(peerTaskID) } -func (s *SchedulerService) RegisterPeerTask(req *schedulerRPC.PeerTaskRequest, task *supervisor.Task) (*supervisor.Peer, error) { - var ( - peer *supervisor.Peer - ok bool - peerHost *supervisor.PeerHost - ) +func (s *SchedulerService) RegisterPeerTask(req *schedulerRPC.PeerTaskRequest, task *supervisor.Task) *supervisor.Peer { + // get or create host reqPeerHost := req.PeerHost - if peerHost, ok = s.hostManager.Get(reqPeerHost.Uuid); !ok { + peerHost, ok := s.hostManager.Get(reqPeerHost.Uuid) + if !ok { peerHost = supervisor.NewClientPeerHost(reqPeerHost.Uuid, reqPeerHost.Ip, reqPeerHost.HostName, reqPeerHost.RpcPort, reqPeerHost.DownPort, reqPeerHost.SecurityDomain, reqPeerHost.Location, reqPeerHost.Idc, reqPeerHost.NetTopology, s.config.ClientLoad) s.hostManager.Add(peerHost) } - // get or creat PeerTask - if peer, ok = s.peerManager.Get(req.PeerId); !ok { - peer = supervisor.NewPeer(req.PeerId, task, peerHost) - s.peerManager.Add(peer) + peer, ok := s.peerManager.Get(req.PeerId) + if ok { + logger.Warnf("peer %s has already registered", peer.PeerID) + return peer } - return peer, nil + peer = supervisor.NewPeer(req.PeerId, task, peerHost) + s.peerManager.Add(peer) + return peer } -func (s *SchedulerService) GetOrCreateTask(ctx context.Context, task *supervisor.Task) (*supervisor.Task, error) { +func (s *SchedulerService) GetOrCreateTask(ctx context.Context, task *supervisor.Task) *supervisor.Task { span := trace.SpanFromContext(ctx) synclock.Lock(task.TaskID, true) task, ok := s.taskManager.GetOrAdd(task) @@ -207,46 +238,55 @@ func (s *SchedulerService) GetOrCreateTask(ctx context.Context, task *supervisor if task.GetLastTriggerTime().Add(s.config.AccessWindow).After(time.Now()) || task.IsHealth() { synclock.UnLock(task.TaskID, true) span.SetAttributes(config.AttributeNeedSeedCDN.Bool(false)) - return task, nil + span.SetAttributes(config.AttributeTaskStatus.String(task.GetStatus().String())) + span.SetAttributes(config.AttributeLastTriggerTime.String(task.GetLastTriggerTime().String())) + return task } } synclock.UnLock(task.TaskID, true) // do trigger task.UpdateLastTriggerTime(time.Now()) - // register cdn peer task - // notify peer tasks + synclock.Lock(task.TaskID, false) defer synclock.UnLock(task.TaskID, false) - if task.IsHealth() && task.GetLastTriggerTime().Add(s.config.AccessWindow).After(time.Now()) { - return task, nil - } - if task.IsFrozen() { - task.SetStatus(supervisor.TaskStatusRunning) + if task.IsHealth() { span.SetAttributes(config.AttributeNeedSeedCDN.Bool(false)) + span.SetAttributes(config.AttributeTaskStatus.String(task.GetStatus().String())) + span.SetAttributes(config.AttributeLastTriggerTime.String(task.GetLastTriggerTime().String())) + return task + } + task.SetStatus(supervisor.TaskStatusRunning) + if s.cdnManager == nil { + // client back source + span.SetAttributes(config.AttributeClientBackSource.Bool(true)) + task.SetClientBackSourceStatusAndLimit(s.config.BackSourceCount) + return task } span.SetAttributes(config.AttributeNeedSeedCDN.Bool(true)) go func() { if cdnPeer, err := s.cdnManager.StartSeedTask(ctx, task); err != nil { - if errors.Cause(err) != cdn.ErrCDNInvokeFail { - task.SetStatus(supervisor.TaskStatusFailed) - } - logger.Errorf("failed to seed task: %v", err) + // fall back to client back source + logger.Errorf("seed task failed: %v", err) + span.AddEvent(config.EventCDNFailBackClientSource, trace.WithAttributes(config.AttributeTriggerCDNError.String(err.Error()))) + task.SetClientBackSourceStatusAndLimit(s.config.BackSourceCount) if ok = s.worker.send(taskSeedFailEvent{task}); !ok { - logger.Error("failed to send taskSeed fail event, eventLoop is shutdown") + logger.Error("send taskSeed fail event failed, eventLoop is shutdown") } } else { if ok = s.worker.send(peerDownloadSuccessEvent{cdnPeer, nil}); !ok { - logger.Error("failed to send taskSeed fail event, eventLoop is shutdown") + logger.Error("send taskSeed success event failed, eventLoop is shutdown") } logger.Debugf("===== successfully obtain seeds from cdn, task: %+v ====", task) } }() - return task, nil + return task } func (s *SchedulerService) HandlePieceResult(ctx context.Context, peer *supervisor.Peer, pieceResult *schedulerRPC.PieceResult) error { peer.Touch() - if pieceResult.PieceNum == common.ZeroOfPiece { + if pieceResult.PieceInfo != nil && pieceResult.PieceInfo.PieceNum == common.EndOfPiece { + return nil + } else if pieceResult.PieceInfo != nil && pieceResult.PieceInfo.PieceNum == common.ZeroOfPiece { s.worker.send(startReportPieceResultEvent{ctx, peer}) return nil } else if pieceResult.Success { @@ -258,6 +298,7 @@ func (s *SchedulerService) HandlePieceResult(ctx context.Context, peer *supervis return nil } else if pieceResult.Code != dfcodes.Success { s.worker.send(peerDownloadPieceFailEvent{ + ctx: ctx, peer: peer, pr: pieceResult, }) diff --git a/scheduler/core/worker.go b/scheduler/core/worker.go index 9680abba2..3af042a07 100644 --- a/scheduler/core/worker.go +++ b/scheduler/core/worker.go @@ -31,7 +31,6 @@ type worker interface { type workerGroup struct { workerNum int workerList []*baseWorker - stopCh chan struct{} } var _ worker = (*workerGroup)(nil) @@ -40,7 +39,6 @@ func newEventLoopGroup(workerNum int) worker { return &workerGroup{ workerNum: workerNum, workerList: make([]*baseWorker, 0, workerNum), - stopCh: make(chan struct{}), } } @@ -59,7 +57,6 @@ func (wg *workerGroup) send(e event) bool { } func (wg *workerGroup) stop() { - close(wg.stopCh) for _, worker := range wg.workerList { worker.stop() } diff --git a/scheduler/job/job.go b/scheduler/job/job.go index 99c2aa19c..eb6e91b1c 100644 --- a/scheduler/job/job.go +++ b/scheduler/job/job.go @@ -20,8 +20,6 @@ import ( "context" "time" - "d7y.io/dragonfly/v2/internal/dfcodes" - "d7y.io/dragonfly/v2/internal/dferrors" logger "d7y.io/dragonfly/v2/internal/dflog" "d7y.io/dragonfly/v2/internal/idgen" internaljob "d7y.io/dragonfly/v2/internal/job" @@ -158,11 +156,7 @@ func (t *job) preheat(req string) error { logger.Debugf("ready to preheat \"%s\", taskID = %s", request.URL, taskID) task := supervisor.NewTask(taskID, request.URL, meta) - task, err := t.service.GetOrCreateTask(t.ctx, task) - if err != nil { - return dferrors.Newf(dfcodes.SchedCDNSeedFail, "create task failed: %v", err) - } - + task = t.service.GetOrCreateTask(t.ctx, task) return getPreheatResult(task) } @@ -175,7 +169,7 @@ func getPreheatResult(task *supervisor.Task) error { select { case <-ticker.C: switch task.GetStatus() { - case supervisor.TaskStatusFailed, supervisor.TaskStatusCDNRegisterFail, supervisor.TaskStatusSourceError: + case supervisor.TaskStatusFail: return errors.Errorf("preheat task fail") case supervisor.TaskStatusSuccess: return nil diff --git a/scheduler/rpcserver/rpcserver.go b/scheduler/rpcserver/rpcserver.go index d4a5f8888..af74a7142 100644 --- a/scheduler/rpcserver/rpcserver.go +++ b/scheduler/rpcserver/rpcserver.go @@ -20,13 +20,11 @@ import ( "context" "fmt" "io" - "sync" "d7y.io/dragonfly/v2/internal/dfcodes" "d7y.io/dragonfly/v2/internal/dferrors" logger "d7y.io/dragonfly/v2/internal/dflog" "d7y.io/dragonfly/v2/pkg/rpc/base" - "d7y.io/dragonfly/v2/pkg/rpc/base/common" "d7y.io/dragonfly/v2/pkg/rpc/scheduler" "d7y.io/dragonfly/v2/pkg/rpc/scheduler/server" "d7y.io/dragonfly/v2/pkg/util/net/urlutils" @@ -36,9 +34,6 @@ import ( "d7y.io/dragonfly/v2/scheduler/supervisor" "go.opentelemetry.io/otel" "go.opentelemetry.io/otel/trace" - "golang.org/x/sync/errgroup" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) var tracer trace.Tracer @@ -76,44 +71,29 @@ func (s *SchedulerServer) RegisterPeerTask(ctx context.Context, request *schedul } taskID := s.service.GenerateTaskID(request.Url, request.UrlMeta, request.PeerId) span.SetAttributes(config.AttributeTaskID.String(taskID)) - task := supervisor.NewTask(taskID, request.Url, request.UrlMeta) - task, err = s.service.GetOrCreateTask(ctx, task) - if err != nil { - err = dferrors.Newf(dfcodes.SchedCDNSeedFail, "create task failed: %v", err) - logger.Errorf("get or create task failed: %v", err) - span.RecordError(err) - return - } + task := s.service.GetOrCreateTask(ctx, supervisor.NewTask(taskID, request.Url, request.UrlMeta)) if task.IsFail() { - err = dferrors.Newf(dfcodes.SchedTaskStatusError, "task status is %s", task.GetStatus()) - logger.Errorf("task status is unhealthy, task status is: %s", task.GetStatus()) + err = dferrors.New(dfcodes.SchedTaskStatusError, "task status is fail") + logger.Error("task %s status is fail", task.TaskID) span.RecordError(err) return } resp.SizeScope = getTaskSizeScope(task) + span.SetAttributes(config.AttributeTaskSizeScope.String(resp.SizeScope.String())) resp.TaskId = taskID switch resp.SizeScope { case base.SizeScope_TINY: resp.DirectPiece = &scheduler.RegisterResult_PieceContent{ PieceContent: task.DirectPiece, } - span.SetAttributes(config.AttributeTaskSizeScope.String(resp.SizeScope.String())) return case base.SizeScope_SMALL: - span.SetAttributes(config.AttributeTaskSizeScope.String(resp.SizeScope.String())) - peer, regErr := s.service.RegisterPeerTask(request, task) - if regErr != nil { - err = dferrors.Newf(dfcodes.SchedPeerRegisterFail, "failed to register peer: %v", regErr) - span.RecordError(err) - return - } - parent, schErr := s.service.ScheduleParent(peer) + peer := s.service.RegisterPeerTask(request, task) + parent, schErr := s.service.SelectParent(peer) if schErr != nil { - span.SetAttributes(config.AttributeTaskSizeScope.String(base.SizeScope_NORMAL.String())) - span.AddEvent(config.EventScheduleParentFail) + span.AddEvent(config.EventSmallTaskSelectParentFail) resp.SizeScope = base.SizeScope_NORMAL resp.TaskId = taskID - //err = dferrors.Newf(dfcodes.SchedPeerScheduleFail, "failed to schedule peer %v: %v", peer.PeerID, schErr) return } firstPiece := task.GetPiece(0) @@ -135,12 +115,7 @@ func (s *SchedulerServer) RegisterPeerTask(ctx context.Context, request *schedul span.SetAttributes(config.AttributeSinglePiece.String(singlePiece.String())) return default: - span.SetAttributes(config.AttributeTaskSizeScope.String(resp.SizeScope.String())) - _, regErr := s.service.RegisterPeerTask(request, task) - if regErr != nil { - err = dferrors.Newf(dfcodes.SchedPeerRegisterFail, "failed to register peer: %v", regErr) - span.RecordError(regErr) - } + s.service.RegisterPeerTask(request, task) return } } @@ -149,90 +124,52 @@ func (s *SchedulerServer) ReportPieceResult(stream scheduler.Scheduler_ReportPie var span trace.Span ctx, span := tracer.Start(stream.Context(), config.SpanReportPieceResult, trace.WithSpanKind(trace.SpanKindServer)) defer span.End() - peerPacketChan := make(chan *scheduler.PeerPacket, 1) - var peer *supervisor.Peer - initialized := false - ctx, cancel := context.WithCancel(ctx) - g, ctx := errgroup.WithContext(ctx) - var once sync.Once - g.Go(func() error { - defer func() { - cancel() - once.Do(peer.UnBindSendChannel) - }() - for { - select { - case <-ctx.Done(): - return nil - default: - pieceResult, err := stream.Recv() - if err == io.EOF { - span.AddEvent("report piece process exited because client has terminated sending the request") - return nil - } - if err != nil { - if status.Code(err) == codes.Canceled { - span.AddEvent("report piece process exited because an error exception was received") - if peer != nil { - logger.Info("peer %s canceled", peer.PeerID) - return nil - } - } - return dferrors.Newf(dfcodes.SchedPeerPieceResultReportFail, "peer piece result report error: %v", err) - } - logger.Debugf("report piece result %v of peer %s", pieceResult, pieceResult.SrcPid) - var ok bool - peer, ok = s.service.GetPeerTask(pieceResult.SrcPid) - if !ok { - return dferrors.Newf(dfcodes.SchedPeerNotFound, "peer %s not found", pieceResult.SrcPid) - } - if !initialized { - peer.BindSendChannel(peerPacketChan) - peer.SetStatus(supervisor.PeerStatusRunning) - initialized = true - span.SetAttributes(config.AttributePeerID.String(peer.PeerID)) - span.AddEvent("init") - } - if pieceResult.PieceNum == common.EndOfPiece { - return nil - } - if err := s.service.HandlePieceResult(ctx, peer, pieceResult); err != nil { - logger.Errorf("handle piece result %v fail: %v", pieceResult, err) - } - } + pieceResult, err := stream.Recv() + if err != nil { + if err == io.EOF { + return nil } - }) + err = dferrors.Newf(dfcodes.SchedPeerPieceResultReportFail, "receive an error from peer stream: %v", err) + span.RecordError(err) + return err + } + logger.Debugf("peer %s start report piece result", pieceResult.SrcPid) + peer, ok := s.service.GetPeerTask(pieceResult.SrcPid) + if !ok { + err = dferrors.Newf(dfcodes.SchedPeerNotFound, "peer %s not found", pieceResult.SrcPid) + span.RecordError(err) + return err + } + if peer.Task.IsFail() { + err = dferrors.Newf(dfcodes.SchedTaskStatusError, "peer's task status is fail, task status %s", peer.Task.GetStatus()) + span.RecordError(err) + return err + } + if err := s.service.HandlePieceResult(ctx, peer, pieceResult); err != nil { + logger.Errorf("peer %s handle piece result %v fail: %v", peer.PeerID, pieceResult, err) - g.Go(func() error { - defer func() { - cancel() - once.Do(peer.UnBindSendChannel) - }() - for { - select { - case <-ctx.Done(): - return nil - case pp, ok := <-peerPacketChan: - if !ok { - span.AddEvent("exit report piece process due to send channel has closed") - return nil - } - span.AddEvent("schedule event", trace.WithAttributes(config.AttributeSchedulePacket.String(pp.String()))) - err := stream.Send(pp) - if err != nil { - logger.Errorf("send peer %s schedule packet %v failed: %v", pp.SrcPid, pp, err) - return dferrors.Newf(dfcodes.SchedPeerPieceResultReportFail, "peer piece result report error: %v", err) - } + } + conn := peer.BindNewConn(stream) + logger.Infof("peer %s is connected", peer.PeerID) + defer logger.Infof("peer %s is disconnect", peer.PeerID) + for { + select { + case <-conn.Done(): + return conn.Err() + case piece := <-conn.Receiver(): + if piece == nil { + logger.Infof("peer %s channel has been closed", peer.PeerID) + continue + } + if err := s.service.HandlePieceResult(ctx, peer, piece); err != nil { + logger.Errorf("peer %s handle piece result %v fail: %v", peer.PeerID, piece, err) } } - }) - err := g.Wait() - logger.Debugf("report piece result: %v", err) - return err + } } func (s *SchedulerServer) ReportPeerResult(ctx context.Context, result *scheduler.PeerResult) (err error) { - logger.Debugf("report peer result %+v", result) + logger.Debugf("report peer result %v", result) var span trace.Span ctx, span = tracer.Start(ctx, config.SpanReportPeerResult, trace.WithSpanKind(trace.SpanKindServer)) defer span.End() @@ -244,17 +181,18 @@ func (s *SchedulerServer) ReportPeerResult(ctx context.Context, result *schedule logger.Warnf("report peer result: peer %s is not exists", result.PeerId) err = dferrors.Newf(dfcodes.SchedPeerNotFound, "peer %s not found", result.PeerId) span.RecordError(err) + return err } return s.service.HandlePeerResult(ctx, peer, result) } func (s *SchedulerServer) LeaveTask(ctx context.Context, target *scheduler.PeerTarget) (err error) { + logger.Debugf("leave task %v", target) var span trace.Span ctx, span = tracer.Start(ctx, config.SpanPeerLeave, trace.WithSpanKind(trace.SpanKindServer)) defer span.End() span.SetAttributes(config.AttributeLeavePeerID.String(target.PeerId)) span.SetAttributes(config.AttributeLeaveTaskID.String(target.TaskId)) - logger.Debugf("leave task %+v", target) peer, ok := s.service.GetPeerTask(target.PeerId) if !ok { logger.Warnf("leave task: peer %s is not exists", target.PeerId) diff --git a/scheduler/scheduler.go b/scheduler/scheduler.go index 61e8fc9d9..41deb415c 100644 --- a/scheduler/scheduler.go +++ b/scheduler/scheduler.go @@ -89,7 +89,7 @@ func New(cfg *config.Config) (*Server, error) { if cfg.Options.Telemetry.Jaeger != "" { openTel = true } - schedulerService, err := core.NewSchedulerService(cfg.Scheduler, dynConfig, openTel) + schedulerService, err := core.NewSchedulerService(cfg.Scheduler, dynConfig, core.WithDisableCDN(cfg.DisableCDN), core.WithOpenTel(openTel)) if err != nil { return nil, err } diff --git a/scheduler/supervisor/cdn/manager.go b/scheduler/supervisor/cdn/manager.go index 23cc12637..370151a47 100644 --- a/scheduler/supervisor/cdn/manager.go +++ b/scheduler/supervisor/cdn/manager.go @@ -71,6 +71,8 @@ func NewManager(cdnClient RefreshableCDNClient, peerManager supervisor.PeerMgr, } func (cm *manager) StartSeedTask(ctx context.Context, task *supervisor.Task) (*supervisor.Peer, error) { + logger.Infof("start seed task %s", task.TaskID) + defer logger.Infof("finish seed task %s", task.TaskID) var seedSpan trace.Span ctx, seedSpan = tracer.Start(ctx, config.SpanTriggerCDN) defer seedSpan.End() @@ -161,15 +163,8 @@ func (cm *manager) receivePiece(ctx context.Context, task *supervisor.Task, stre span.SetAttributes(config.AttributeContentLength.Int64(task.ContentLength)) return cdnPeer, nil } - cdnPeer.AddPieceInfo(piece.PieceInfo.PieceNum+1, 0) - task.AddPiece(&supervisor.PieceInfo{ - PieceNum: piece.PieceInfo.PieceNum, - RangeStart: piece.PieceInfo.RangeStart, - RangeSize: piece.PieceInfo.RangeSize, - PieceMd5: piece.PieceInfo.PieceMd5, - PieceOffset: piece.PieceInfo.PieceOffset, - PieceStyle: piece.PieceInfo.PieceStyle, - }) + cdnPeer.UpdateProgress(piece.PieceInfo.PieceNum+1, 0) + task.AddPiece(piece.PieceInfo) } } } diff --git a/scheduler/supervisor/peer.go b/scheduler/supervisor/peer.go index d955aaf48..45afb5188 100644 --- a/scheduler/supervisor/peer.go +++ b/scheduler/supervisor/peer.go @@ -17,10 +17,12 @@ package supervisor import ( + "io" "sync" "time" "d7y.io/dragonfly/v2/pkg/rpc/scheduler" + "github.com/pkg/errors" "go.uber.org/atomic" ) @@ -32,12 +34,12 @@ func (status PeerStatus) String() string { return "Waiting" case PeerStatusRunning: return "Running" - case PeerStatusSuccess: - return "Success" - case PeerStatusFail: - return "Fail" case PeerStatusZombie: return "Zombie" + case PeerStatusFail: + return "Fail" + case PeerStatusSuccess: + return "Success" default: return "unknown" } @@ -46,6 +48,7 @@ func (status PeerStatus) String() string { const ( PeerStatusWaiting PeerStatus = iota PeerStatusRunning + // TODO add Seeding status PeerStatusZombie PeerStatusFail PeerStatusSuccess @@ -59,10 +62,7 @@ type Peer struct { Task *Task // Host specifies Host *PeerHost - // bindPacketChan - bindPacketChan bool - // PacketChan send schedulerPacket to peer client - packetChan chan *scheduler.PeerPacket + conn *Channel // createTime CreateTime time.Time // finishedNum specifies downloaded finished piece number @@ -155,7 +155,7 @@ func (peer *Peer) GetCost() int { return totalCost / len(peer.costHistory) } -func (peer *Peer) AddPieceInfo(finishedCount int32, cost int) { +func (peer *Peer) UpdateProgress(finishedCount int32, cost int) { peer.lock.Lock() if finishedCount > peer.finishedNum.Load() { peer.finishedNum.Store(finishedCount) @@ -254,38 +254,6 @@ func (peer *Peer) GetChildren() *sync.Map { return &peer.children } -func (peer *Peer) BindSendChannel(packetChan chan *scheduler.PeerPacket) { - peer.lock.Lock() - defer peer.lock.Unlock() - peer.bindPacketChan = true - peer.packetChan = packetChan -} - -func (peer *Peer) UnBindSendChannel() { - peer.lock.Lock() - defer peer.lock.Unlock() - if peer.bindPacketChan { - if peer.packetChan != nil { - close(peer.packetChan) - } - peer.bindPacketChan = false - } -} - -func (peer *Peer) IsBindSendChannel() bool { - peer.lock.RLock() - defer peer.lock.RUnlock() - return peer.bindPacketChan -} - -func (peer *Peer) SendSchedulePacket(packet *scheduler.PeerPacket) { - peer.lock.Lock() - defer peer.lock.Unlock() - if peer.bindPacketChan { - peer.packetChan <- packet - } -} - func (peer *Peer) SetStatus(status PeerStatus) { peer.lock.Lock() defer peer.lock.Unlock() @@ -333,3 +301,163 @@ func (peer *Peer) IsBad() bool { func (peer *Peer) IsFail() bool { return peer.status == PeerStatusFail } + +func (peer *Peer) BindNewConn(stream scheduler.Scheduler_ReportPieceResultServer) *Channel { + peer.lock.Lock() + defer peer.lock.Unlock() + if peer.status == PeerStatusWaiting { + peer.status = PeerStatusRunning + } + peer.conn = newChannel(stream) + return peer.conn +} + +func (peer *Peer) IsConnected() bool { + peer.lock.RLock() + defer peer.lock.RUnlock() + if peer.conn == nil { + return false + } + return !peer.conn.IsClosed() +} + +func (peer *Peer) SendSchedulePacket(packet *scheduler.PeerPacket) error { + peer.lock.Lock() + defer peer.lock.Unlock() + if peer.conn != nil { + return peer.conn.Send(packet) + } + return errors.New("client peer is not connected") +} + +func (peer *Peer) CloseChannel(err error) error { + peer.lock.Lock() + defer peer.lock.Unlock() + if peer.conn != nil { + peer.conn.CloseWithError(err) + return nil + } + return errors.New("client peer is not connected") +} + +type Channel struct { + startOnce sync.Once + sender chan *scheduler.PeerPacket + receiver chan *scheduler.PieceResult + stream scheduler.Scheduler_ReportPieceResultServer + closed *atomic.Bool + done chan struct{} + wg sync.WaitGroup + err error +} + +func newChannel(stream scheduler.Scheduler_ReportPieceResultServer) *Channel { + c := &Channel{ + sender: make(chan *scheduler.PeerPacket), + receiver: make(chan *scheduler.PieceResult), + stream: stream, + closed: atomic.NewBool(false), + done: make(chan struct{}), + } + c.start() + return c +} + +func (c *Channel) start() { + c.startOnce.Do(func() { + c.wg.Add(2) + go c.receiveLoop() + go c.sendLoop() + }) +} + +func (c *Channel) Send(packet *scheduler.PeerPacket) error { + select { + case <-c.done: + return errors.New("conn has closed") + case c.sender <- packet: + return nil + default: + return errors.New("send channel is blocking") + } +} + +func (c *Channel) Receiver() <-chan *scheduler.PieceResult { + return c.receiver +} + +func (c *Channel) Close() { + if !c.closed.CAS(false, true) { + return + } + go func() { + close(c.done) + c.wg.Wait() + }() +} + +func (c *Channel) CloseWithError(err error) { + c.err = err + c.Close() +} + +func (c *Channel) Err() error { + err := c.err + return err +} + +func (c *Channel) Done() <-chan struct{} { + if c.done == nil { + c.done = make(chan struct{}) + } + d := c.done + return d +} + +func (c *Channel) IsClosed() bool { + return c.closed.Load() +} + +func (c *Channel) receiveLoop() { + defer func() { + close(c.receiver) + c.wg.Done() + c.Close() + }() + + for { + select { + case <-c.done: + return + default: + pieceResult, err := c.stream.Recv() + if err == io.EOF { + return + } + if err != nil { + c.err = err + return + } + c.receiver <- pieceResult + } + } +} + +func (c *Channel) sendLoop() { + defer func() { + c.wg.Done() + c.Close() + }() + + for { + select { + case <-c.done: + return + case packet := <-c.sender: + if err := c.stream.Send(packet); err != nil { + c.err = err + return + } + } + } +} diff --git a/scheduler/supervisor/peer/manager.go b/scheduler/supervisor/peer/manager.go index 13bfb67e7..af48246ff 100644 --- a/scheduler/supervisor/peer/manager.go +++ b/scheduler/supervisor/peer/manager.go @@ -21,7 +21,6 @@ import ( "time" logger "d7y.io/dragonfly/v2/internal/dflog" - "d7y.io/dragonfly/v2/pkg/structure/sortedlist" "d7y.io/dragonfly/v2/scheduler/config" "d7y.io/dragonfly/v2/scheduler/supervisor" ) @@ -88,69 +87,36 @@ func (m *manager) Delete(peerID string) { if ok { peer.Host.DeletePeer(peerID) peer.Task.DeletePeer(peer) - peer.UnBindSendChannel() peer.ReplaceParent(nil) m.peerMap.Delete(peerID) } return } -func (m *manager) Pick(task *supervisor.Task, limit int, pickFn func(peer *supervisor.Peer) bool) (pickedPeers []*supervisor.Peer) { - return m.pick(task, limit, false, pickFn) -} - -func (m *manager) PickReverse(task *supervisor.Task, limit int, pickFn func(peer *supervisor.Peer) bool) (pickedPeers []*supervisor.Peer) { - return m.pick(task, limit, true, pickFn) -} - -func (m *manager) pick(task *supervisor.Task, limit int, reverse bool, pickFn func(peer *supervisor.Peer) bool) (pickedPeers []*supervisor.Peer) { - if pickFn == nil { - return - } - if !reverse { - task.ListPeers().Range(func(data sortedlist.Item) bool { - if len(pickedPeers) >= limit { - return false - } - peer := data.(*supervisor.Peer) - if pickFn(peer) { - pickedPeers = append(pickedPeers, peer) - } - return true - }) - return - } - task.ListPeers().RangeReverse(func(data sortedlist.Item) bool { - if len(pickedPeers) >= limit { - return false - } - peer := data.(*supervisor.Peer) - if pickFn(peer) { - pickedPeers = append(pickedPeers, peer) - } - return true - }) - return -} - func (m *manager) cleanupPeers() { for range m.cleanupExpiredPeerTicker.C { m.peerMap.Range(func(key, value interface{}) bool { + peerID := key.(string) peer := value.(*supervisor.Peer) elapse := time.Since(peer.GetLastAccessTime()) - if elapse > m.peerTTI && !peer.IsDone() { - if !peer.IsBindSendChannel() { + if elapse > m.peerTTI && !peer.IsDone() && !peer.Host.CDN { + if !peer.IsConnected() { peer.MarkLeave() } logger.Debugf("peer %s has been more than %s since last access, set status to zombie", peer.PeerID, m.peerTTI) peer.SetStatus(supervisor.PeerStatusZombie) } if peer.IsLeave() || peer.IsFail() || elapse > m.peerTTL { - logger.Debugf("delete peer %s because %s have passed since last access", peer.PeerID) - m.Delete(key.(string)) + if elapse > m.peerTTL { + logger.Debugf("delete peer %s because %s have passed since last access", peer.PeerID) + } + m.Delete(peerID) if peer.Host.GetPeerTaskNum() == 0 { m.hostManager.Delete(peer.Host.UUID) } + if peer.Task.ListPeers().Size() == 0 { + peer.Task.SetStatus(supervisor.TaskStatusWaiting) + } } return true }) diff --git a/scheduler/supervisor/peer_mgr.go b/scheduler/supervisor/peer_mgr.go index 753beee0c..8416b25ea 100644 --- a/scheduler/supervisor/peer_mgr.go +++ b/scheduler/supervisor/peer_mgr.go @@ -30,8 +30,4 @@ type PeerMgr interface { ListPeersByTask(taskID string) []*Peer ListPeers() *sync.Map - - Pick(task *Task, limit int, pickFn func(pt *Peer) bool) []*Peer - - PickReverse(task *Task, limit int, pickFn func(peer *Peer) bool) []*Peer } diff --git a/scheduler/supervisor/task.go b/scheduler/supervisor/task.go index 38e6dc0c7..85f0ec299 100644 --- a/scheduler/supervisor/task.go +++ b/scheduler/supervisor/task.go @@ -22,6 +22,7 @@ import ( "d7y.io/dragonfly/v2/pkg/rpc/base" "d7y.io/dragonfly/v2/pkg/structure/sortedlist" + "go.uber.org/atomic" ) type TaskStatus uint8 @@ -32,16 +33,14 @@ func (status TaskStatus) String() string { return "Waiting" case TaskStatusRunning: return "Running" - case TaskStatusZombie: - return "Zombie" + case TaskStatusSeeding: + return "Seeding" case TaskStatusSuccess: return "Success" - case TaskStatusCDNRegisterFail: - return "cdnRegisterFail" - case TaskStatusFailed: - return "fail" - case TaskStatusSourceError: - return "sourceError" + case TaskStatusZombie: + return "Zombie" + case TaskStatusFail: + return "Fail" default: return "unknown" } @@ -50,39 +49,41 @@ func (status TaskStatus) String() string { const ( TaskStatusWaiting TaskStatus = iota TaskStatusRunning - TaskStatusZombie TaskStatusSeeding TaskStatusSuccess - TaskStatusCDNRegisterFail - TaskStatusFailed - TaskStatusSourceError + TaskStatusZombie + TaskStatusFail ) type Task struct { - TaskID string - URL string - URLMeta *base.UrlMeta - DirectPiece []byte - CreateTime time.Time - lastAccessTime time.Time - lastTriggerTime time.Time - lock sync.RWMutex - pieceList map[int32]*PieceInfo - PieceTotal int32 - ContentLength int64 - status TaskStatus - peers *sortedlist.SortedList + lock sync.RWMutex + TaskID string + URL string + URLMeta *base.UrlMeta + DirectPiece []byte + CreateTime time.Time + lastAccessTime time.Time + lastTriggerTime time.Time + pieceList map[int32]*base.PieceInfo + PieceTotal int32 + ContentLength int64 + status TaskStatus + peers *sortedlist.SortedList + backSourceLimit atomic.Int32 + needClientBackSource atomic.Bool + backSourcePeers []string // TODO add cdnPeers } func NewTask(taskID, url string, meta *base.UrlMeta) *Task { return &Task{ - TaskID: taskID, - URL: url, - URLMeta: meta, - pieceList: make(map[int32]*PieceInfo), - peers: sortedlist.NewSortedList(), - status: TaskStatusWaiting, + TaskID: taskID, + URL: url, + URLMeta: meta, + CreateTime: time.Now(), + pieceList: make(map[int32]*base.PieceInfo), + peers: sortedlist.NewSortedList(), + status: TaskStatusWaiting, } } @@ -114,13 +115,31 @@ func (task *Task) GetStatus() TaskStatus { return task.status } -func (task *Task) GetPiece(pieceNum int32) *PieceInfo { +func (task *Task) SetClientBackSourceStatusAndLimit(backSourceLimit int32) { + task.lock.Lock() + defer task.lock.Unlock() + task.backSourcePeers = make([]string, 0, backSourceLimit) + task.needClientBackSource.Store(true) + task.backSourceLimit.Store(backSourceLimit) +} + +func (task *Task) NeedClientBackSource() bool { + return task.needClientBackSource.Load() +} + +func (task *Task) GetPiece(pieceNum int32) *base.PieceInfo { task.lock.RLock() defer task.lock.RUnlock() return task.pieceList[pieceNum] } -func (task *Task) AddPiece(p *PieceInfo) { +func (task *Task) AddPiece(p *base.PieceInfo) { + task.lock.RLock() + if _, ok := task.pieceList[p.PieceNum]; ok { + task.lock.RUnlock() + return + } + task.lock.RUnlock() task.lock.Lock() defer task.lock.Unlock() task.pieceList[p.PieceNum] = p @@ -130,22 +149,32 @@ func (task *Task) GetLastTriggerTime() time.Time { return task.lastTriggerTime } +func (task *Task) UpdateLastTriggerTime(lastTriggerTime time.Time) { + task.lastTriggerTime = lastTriggerTime +} + func (task *Task) Touch() { task.lock.Lock() defer task.lock.Unlock() task.lastAccessTime = time.Now() } -func (task *Task) UpdateLastTriggerTime(lastTriggerTime time.Time) { - task.lastTriggerTime = lastTriggerTime -} - func (task *Task) GetLastAccessTime() time.Time { task.lock.RLock() defer task.lock.RUnlock() return task.lastAccessTime } +func (task *Task) UpdateTaskSuccessResult(pieceTotal int32, contentLength int64) { + task.lock.Lock() + defer task.lock.Unlock() + if task.status != TaskStatusSuccess { + task.status = TaskStatusSuccess + task.PieceTotal = pieceTotal + task.ContentLength = contentLength + } +} + func (task *Task) Lock() { task.lock.Lock() } @@ -164,25 +193,16 @@ func (task *Task) RUnlock() { const TinyFileSize = 128 -type PieceInfo struct { - PieceNum int32 - RangeStart uint64 - RangeSize int32 - PieceMd5 string - PieceOffset uint64 - PieceStyle base.PieceStyle -} - // IsSuccess determines that whether cdn status is success. func (task *Task) IsSuccess() bool { return task.status == TaskStatusSuccess } // IsFrozen determines that whether cdn status is frozen -func (task *Task) IsFrozen() bool { - return task.status == TaskStatusFailed || task.status == TaskStatusWaiting || - task.status == TaskStatusSourceError || task.status == TaskStatusCDNRegisterFail -} +//func (task *Task) IsFrozen() bool { +// return task.status == TaskStatusWaiting || task.status == TaskStatusZombie || task.status == TaskStatusFailed || +// task.status == TaskStatusSourceError || task.status == TaskStatusCDNRegisterFail +//} // CanSchedule determines whether task can be scheduled // only task status is seeding or success can be scheduled @@ -197,10 +217,75 @@ func (task *Task) IsWaiting() bool { // IsHealth determines whether task is health func (task *Task) IsHealth() bool { - return task.status == TaskStatusRunning || task.status == TaskStatusSuccess || task.status == TaskStatusSeeding + return task.status == TaskStatusRunning || task.status == TaskStatusSeeding || task.status == TaskStatusSuccess } // IsFail determines whether task is fail func (task *Task) IsFail() bool { - return task.status == TaskStatusFailed || task.status == TaskStatusSourceError || task.status == TaskStatusCDNRegisterFail + return task.status == TaskStatusFail +} + +func (task *Task) IncreaseBackSourcePeer(peerID string) { + task.lock.Lock() + defer task.lock.Unlock() + task.backSourcePeers = append(task.backSourcePeers, peerID) + if task.backSourceLimit.Dec() <= 0 { + task.needClientBackSource.Store(false) + } +} + +func (task *Task) GetBackSourcePeers() []string { + task.lock.RLock() + defer task.lock.RUnlock() + backSourcePeers := task.backSourcePeers + return backSourcePeers +} + +func (task *Task) IsBackSourcePeer(peerID string) bool { + task.lock.RLock() + defer task.lock.RUnlock() + for i := range task.backSourcePeers { + if task.backSourcePeers[i] == peerID { + return true + } + } + return false +} + +func (task *Task) Pick(limit int, pickFn func(peer *Peer) bool) (pickedPeers []*Peer) { + return task.pick(limit, false, pickFn) +} + +func (task *Task) PickReverse(limit int, pickFn func(peer *Peer) bool) (pickedPeers []*Peer) { + return task.pick(limit, true, pickFn) +} + +func (task *Task) pick(limit int, reverse bool, pickFn func(peer *Peer) bool) (pickedPeers []*Peer) { + if pickFn == nil { + return + } + if !reverse { + task.ListPeers().Range(func(data sortedlist.Item) bool { + if len(pickedPeers) >= limit { + return false + } + peer := data.(*Peer) + if pickFn(peer) { + pickedPeers = append(pickedPeers, peer) + } + return true + }) + return + } + task.ListPeers().RangeReverse(func(data sortedlist.Item) bool { + if len(pickedPeers) >= limit { + return false + } + peer := data.(*Peer) + if pickFn(peer) { + pickedPeers = append(pickedPeers, peer) + } + return true + }) + return } diff --git a/scheduler/supervisor/task/manager.go b/scheduler/supervisor/task/manager.go index 79306048e..27be11e0c 100644 --- a/scheduler/supervisor/task/manager.go +++ b/scheduler/supervisor/task/manager.go @@ -72,19 +72,22 @@ func (m *manager) GetOrAdd(task *supervisor.Task) (actual *supervisor.Task, load func (m *manager) cleanupTasks() { for range m.cleanupExpiredTaskTicker.C { m.taskMap.Range(func(key, value interface{}) bool { + taskID := key.(string) task := value.(*supervisor.Task) elapse := time.Since(task.GetLastAccessTime()) if elapse > m.taskTTI && task.IsSuccess() { task.SetStatus(supervisor.TaskStatusZombie) } + if task.ListPeers().Size() == 0 { + task.SetStatus(supervisor.TaskStatusWaiting) + } if elapse > m.taskTTL { - taskID := key.(string) // TODO lock - m.Delete(taskID) peers := m.peerManager.ListPeersByTask(taskID) for _, peer := range peers { m.peerManager.Delete(peer.PeerID) } + m.Delete(taskID) } return true })