From fcbf9a0bcb540415fc4e6572926b0d67d5bfd728 Mon Sep 17 00:00:00 2001 From: Gaius Date: Thu, 16 Jan 2025 22:07:23 +0800 Subject: [PATCH] feat: implement FindCandidatePersistentCacheParents for scheduling persistent cache task Signed-off-by: Gaius --- scheduler/metrics/metrics.go | 56 ++ scheduler/resource/persistentcache/host.go | 93 +-- .../resource/persistentcache/host_manager.go | 33 +- scheduler/resource/persistentcache/peer.go | 26 +- scheduler/resource/persistentcache/task.go | 35 + scheduler/scheduler.go | 2 +- scheduler/scheduling/evaluator/evaluator.go | 48 +- .../scheduling/evaluator/evaluator_base.go | 64 +- .../evaluator/evaluator_base_test.go | 56 +- .../evaluator/testdata/plugin/evaluator.go | 2 +- scheduler/scheduling/mocks/scheduling_mock.go | 31 + scheduler/scheduling/scheduling.go | 183 +++-- scheduler/scheduling/scheduling_test.go | 745 +++++++++--------- scheduler/service/service_v2.go | 617 ++++++++++++++- scheduler/service/service_v2_test.go | 64 +- 15 files changed, 1413 insertions(+), 642 deletions(-) diff --git a/scheduler/metrics/metrics.go b/scheduler/metrics/metrics.go index 00bf4bcdfd9..428c2e846ac 100644 --- a/scheduler/metrics/metrics.go +++ b/scheduler/metrics/metrics.go @@ -401,6 +401,62 @@ var ( Help: "Counter of the number of failed of the delete cache task.", }) + RegisterPersistentCachePeerCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: types.MetricsNamespace, + Subsystem: types.SchedulerMetricsName, + Name: "register_persistent_cache_peer_total", + Help: "Counter of the number of the register persistent cache peer.", + }, []string{"host_type"}) + + RegisterPersistentCachePeerFailureCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: types.MetricsNamespace, + Subsystem: types.SchedulerMetricsName, + Name: "register_persistent_cache_peer_failure_total", + Help: "Counter of the number of failed of the register persistent cache peer.", + }, []string{"host_type"}) + + DownloadPersistentCachePeerStartedCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: types.MetricsNamespace, + Subsystem: types.SchedulerMetricsName, + Name: "download_persistent_cache_peer_started_total", + Help: "Counter of the number of the download persistent cache peer started.", + }, []string{"host_type"}) + + DownloadPersistentCachePeerStartedFailureCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: types.MetricsNamespace, + Subsystem: types.SchedulerMetricsName, + Name: "download_persistent_cache_peer_started_failure_total", + Help: "Counter of the number of failed of the download persistent cache peer started.", + }, []string{"host_type"}) + + DownloadPersistentCachePeerCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: types.MetricsNamespace, + Subsystem: types.SchedulerMetricsName, + Name: "download_persistent_cache_peer_finished_total", + Help: "Counter of the number of the download persistent cache peer.", + }, []string{"host_type"}) + + DownloadPersistentCachePeerFailureCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: types.MetricsNamespace, + Subsystem: types.SchedulerMetricsName, + Name: "download_persistent_cache_peer_finished_failure_total", + Help: "Counter of the number of failed of the download persistent cache peer.", + }, []string{"host_type"}) + + DownloadPersistentCachePieceCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: types.MetricsNamespace, + Subsystem: types.SchedulerMetricsName, + Name: "download_persistent_cache_piece_finished_total", + Help: "Counter of the number of the download persistent cache piece.", + }, []string{"host_type"}) + + DownloadPersistentCachePieceFailureCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: types.MetricsNamespace, + Subsystem: types.SchedulerMetricsName, + Name: "download_persistent_cache_piece_finished_failure_total", + Help: "Counter of the number of failed of the download persistent cache piece.", + }, []string{"host_type"}) + VersionGauge = promauto.NewGaugeVec(prometheus.GaugeOpts{ Namespace: types.MetricsNamespace, Subsystem: types.SchedulerMetricsName, diff --git a/scheduler/resource/persistentcache/host.go b/scheduler/resource/persistentcache/host.go index c4dc7122884..1b849db9cfd 100644 --- a/scheduler/resource/persistentcache/host.go +++ b/scheduler/resource/persistentcache/host.go @@ -21,19 +21,8 @@ import ( logger "d7y.io/dragonfly/v2/internal/dflog" "d7y.io/dragonfly/v2/pkg/types" - "d7y.io/dragonfly/v2/scheduler/config" ) -// HostOption is a functional option for configuring the persistent cache host. -type HostOption func(h *Host) - -// WithConcurrentUploadLimit sets persistent cache host's ConcurrentUploadLimit. -func WithConcurrentUploadLimit(limit int32) HostOption { - return func(h *Host) { - h.ConcurrentUploadLimit = limit - } -} - // Host contains content for host. type Host struct { // ID is host id. @@ -88,21 +77,12 @@ type Host struct { // Build information. Build Build + // SchedulerClusterID is the scheduler cluster id matched by scopes. + SchedulerClusterID uint64 + // AnnounceInterval is the interval between host announces to scheduler. AnnounceInterval time.Duration - // ConcurrentUploadLimit is concurrent upload limit count. - ConcurrentUploadLimit int32 - - // ConcurrentUploadCount is concurrent upload count. - ConcurrentUploadCount int32 - - // UploadCount is total upload count. - UploadCount int64 - - // UploadFailedCount is upload failed count. - UploadFailedCount int64 - // CreatedAt is host create time. CreatedAt time.Time @@ -265,47 +245,32 @@ type Disk struct { // New host instance. func NewHost( - id, hostname, ip, os, platform, platformFamily, platformVersion, kernelVersion string, port, downloadPort, concurrentUploadCount int32, - UploadCount, UploadFailedCount int64, disableShared bool, typ types.HostType, cpu CPU, memory Memory, network Network, disk Disk, - build Build, announceInterval time.Duration, createdAt, updatedAt time.Time, log *logger.SugaredLoggerOnWith, options ...HostOption, + id, hostname, ip, os, platform, platformFamily, platformVersion, kernelVersion string, port, downloadPort int32, + schedulerClusterId uint64, disableShared bool, typ types.HostType, cpu CPU, memory Memory, network Network, disk Disk, + build Build, announceInterval time.Duration, createdAt, updatedAt time.Time, log *logger.SugaredLoggerOnWith, ) *Host { - // Calculate default of the concurrent upload limit by host type. - concurrentUploadLimit := config.DefaultSeedPeerConcurrentUploadLimit - if typ == types.HostTypeNormal { - concurrentUploadLimit = config.DefaultPeerConcurrentUploadLimit - } - - h := &Host{ - ID: id, - Type: types.HostType(typ), - Hostname: hostname, - IP: ip, - Port: port, - DownloadPort: downloadPort, - DisableShared: disableShared, - OS: os, - Platform: platform, - PlatformFamily: platformFamily, - PlatformVersion: platformVersion, - KernelVersion: kernelVersion, - CPU: cpu, - Memory: memory, - Network: network, - Disk: disk, - Build: build, - AnnounceInterval: announceInterval, - ConcurrentUploadLimit: int32(concurrentUploadLimit), - ConcurrentUploadCount: concurrentUploadCount, - UploadCount: UploadCount, - UploadFailedCount: UploadFailedCount, - CreatedAt: createdAt, - UpdatedAt: updatedAt, - Log: logger.WithHost(id, hostname, ip), + return &Host{ + ID: id, + Type: types.HostType(typ), + Hostname: hostname, + IP: ip, + Port: port, + DownloadPort: downloadPort, + DisableShared: disableShared, + OS: os, + Platform: platform, + PlatformFamily: platformFamily, + PlatformVersion: platformVersion, + KernelVersion: kernelVersion, + CPU: cpu, + Memory: memory, + Network: network, + Disk: disk, + Build: build, + SchedulerClusterID: schedulerClusterId, + AnnounceInterval: announceInterval, + CreatedAt: createdAt, + UpdatedAt: updatedAt, + Log: logger.WithHost(id, hostname, ip), } - - for _, opt := range options { - opt(h) - } - - return h } diff --git a/scheduler/resource/persistentcache/host_manager.go b/scheduler/resource/persistentcache/host_manager.go index 07779e07802..8ea2d449721 100644 --- a/scheduler/resource/persistentcache/host_manager.go +++ b/scheduler/resource/persistentcache/host_manager.go @@ -106,27 +106,10 @@ func (h *hostManager) Load(ctx context.Context, hostID string) (*Host, bool) { return nil, false } - concurrentUploadLimit, err := strconv.ParseInt(rawHost["concurrent_upload_limit"], 10, 32) - if err != nil { - log.Errorf("parsing concurrent upload limit failed: %v", err) - return nil, false - } - - concurrentUploadCount, err := strconv.ParseInt(rawHost["concurrent_upload_count"], 10, 32) - if err != nil { - log.Errorf("parsing concurrent upload count failed: %v", err) - return nil, false - } - - uploadCount, err := strconv.ParseInt(rawHost["upload_count"], 10, 64) - if err != nil { - log.Errorf("parsing upload count failed: %v", err) - return nil, false - } - - uploadFailedCount, err := strconv.ParseInt(rawHost["upload_failed_count"], 10, 64) + // Set cpu fields from raw host. + schedulerClusterID, err := strconv.ParseUint(rawHost["scheduler_cluster_id"], 10, 64) if err != nil { - log.Errorf("parsing upload failed count failed: %v", err) + log.Errorf("parsing scheduler cluster id failed: %v", err) return nil, false } @@ -446,9 +429,7 @@ func (h *hostManager) Load(ctx context.Context, hostID string) (*Host, bool) { rawHost["kernel_version"], int32(port), int32(downloadPort), - int32(concurrentUploadCount), - uploadCount, - uploadFailedCount, + uint64(schedulerClusterID), diableShared, pkgtypes.ParseHostType(rawHost["type"]), cpu, @@ -460,7 +441,6 @@ func (h *hostManager) Load(ctx context.Context, hostID string) (*Host, bool) { createdAt, updatedAt, logger.WithHost(rawHost["id"], rawHost["hostname"], rawHost["ip"]), - WithConcurrentUploadLimit(int32(concurrentUploadLimit)), ), true } @@ -522,11 +502,8 @@ func (h *hostManager) Store(ctx context.Context, host *Host) error { "build_git_commit", host.Build.GitCommit, "build_go_version", host.Build.GoVersion, "build_platform", host.Build.Platform, + "scheduler_cluster_id", host.SchedulerClusterID, "announce_interval", host.AnnounceInterval, - "concurrent_upload_limit", host.ConcurrentUploadLimit, - "concurrent_upload_count", host.ConcurrentUploadCount, - "upload_count", host.UploadCount, - "upload_failed_count", host.UploadFailedCount, "created_at", host.CreatedAt.Format(time.RFC3339), "updated_at", host.UpdatedAt.Format(time.RFC3339)).Result() diff --git a/scheduler/resource/persistentcache/peer.go b/scheduler/resource/persistentcache/peer.go index 6abcb0ebf7e..f36e5890e08 100644 --- a/scheduler/resource/persistentcache/peer.go +++ b/scheduler/resource/persistentcache/peer.go @@ -33,8 +33,11 @@ const ( // Peer is uploading resources for p2p cluster. PeerStateUploading = "Uploading" - // Peer successfully registered and perpared to download. - PeerStateReceived = "Received" + // Peer successfully registered as empty scope size. + PeerStateReceivedEmpty = "ReceivedEmpty" + + // Peer successfully registered as normal scope size. + PeerStateReceivedNormal = "ReceivedNormal" // Peer is downloading resources from peer. PeerStateRunning = "Running" @@ -48,10 +51,13 @@ const ( const ( // Peer is uploding. - PeerEventUpload = "Uploda" + PeerEventUpload = "Upload" + + // Peer is registered as empty scope size. + PeerEventRegisterEmpty = "RegisterEmpty" - // Peer is registered and perpared to download. - PeerEventRegister = "Register" + // Peer is registered as normal scope size. + PeerEventRegisterNormal = "RegisterNormal" // Peer is downloading. PeerEventDownload = "Download" @@ -119,8 +125,9 @@ func NewPeer(id, state string, persistent bool, finishedPieces *bitset.BitSet, b PeerStatePending, fsm.Events{ fsm.EventDesc{Name: PeerEventUpload, Src: []string{PeerStatePending, PeerStateFailed}, Dst: PeerStateUploading}, - fsm.EventDesc{Name: PeerEventRegister, Src: []string{PeerStatePending, PeerStateFailed}, Dst: PeerStateReceived}, - fsm.EventDesc{Name: PeerEventDownload, Src: []string{PeerStateReceived}, Dst: PeerStateRunning}, + fsm.EventDesc{Name: PeerEventRegisterEmpty, Src: []string{PeerStatePending, PeerStateFailed}, Dst: PeerStateReceivedEmpty}, + fsm.EventDesc{Name: PeerEventRegisterNormal, Src: []string{PeerStatePending, PeerStateFailed}, Dst: PeerStateReceivedNormal}, + fsm.EventDesc{Name: PeerEventDownload, Src: []string{PeerStateReceivedEmpty, PeerStateReceivedNormal}, Dst: PeerStateRunning}, fsm.EventDesc{Name: PeerEventSucceeded, Src: []string{PeerStateUploading, PeerStateRunning}, Dst: PeerStateSucceeded}, fsm.EventDesc{Name: PeerEventFailed, Src: []string{PeerStateUploading, PeerStateRunning}, Dst: PeerStateFailed}, }, @@ -128,7 +135,10 @@ func NewPeer(id, state string, persistent bool, finishedPieces *bitset.BitSet, b PeerEventUpload: func(ctx context.Context, e *fsm.Event) { p.Log.Infof("peer state is %s", e.FSM.Current()) }, - PeerEventRegister: func(ctx context.Context, e *fsm.Event) { + PeerEventRegisterEmpty: func(ctx context.Context, e *fsm.Event) { + p.Log.Infof("peer state is %s", e.FSM.Current()) + }, + PeerEventRegisterNormal: func(ctx context.Context, e *fsm.Event) { p.Log.Infof("peer state is %s", e.FSM.Current()) }, PeerEventDownload: func(ctx context.Context, e *fsm.Event) { diff --git a/scheduler/resource/persistentcache/task.go b/scheduler/resource/persistentcache/task.go index edc86206b34..00bca080d49 100644 --- a/scheduler/resource/persistentcache/task.go +++ b/scheduler/resource/persistentcache/task.go @@ -22,10 +22,20 @@ import ( "github.com/looplab/fsm" + commonv2 "d7y.io/api/v2/pkg/apis/common/v2" + logger "d7y.io/dragonfly/v2/internal/dflog" "d7y.io/dragonfly/v2/pkg/digest" ) +const ( + // Tiny file size is 128 bytes. + TinyFileSize = 128 + + // Empty file size is 0 bytes. + EmptyFileSize = 0 +) + const ( // Task has been created but did not start uploading. TaskStatePending = "Pending" @@ -137,3 +147,28 @@ func NewTask(id, tag, application, state string, persistentReplicaCount uint64, return t } + +// SizeScope return task size scope type. +func (t *Task) SizeScope() commonv2.SizeScope { + if t.ContentLength < 0 { + return commonv2.SizeScope_UNKNOW + } + + if t.TotalPieceCount < 0 { + return commonv2.SizeScope_UNKNOW + } + + if t.ContentLength == EmptyFileSize { + return commonv2.SizeScope_EMPTY + } + + if t.ContentLength <= TinyFileSize { + return commonv2.SizeScope_TINY + } + + if t.TotalPieceCount == 1 { + return commonv2.SizeScope_SMALL + } + + return commonv2.SizeScope_NORMAL +} diff --git a/scheduler/scheduler.go b/scheduler/scheduler.go index cbde6f63d74..b9ecd3bbaf0 100644 --- a/scheduler/scheduler.go +++ b/scheduler/scheduler.go @@ -194,7 +194,7 @@ func New(ctx context.Context, cfg *config.Config, d dfpath.Dfpath) (*Server, err } // Initialize scheduling. - scheduling := scheduling.New(&cfg.Scheduler, dynconfig, d.PluginDir()) + scheduling := scheduling.New(&cfg.Scheduler, s.persistentCacheResource, dynconfig, d.PluginDir()) // Initialize server options of scheduler grpc server. schedulerServerOptions := []grpc.ServerOption{} diff --git a/scheduler/scheduling/evaluator/evaluator.go b/scheduler/scheduling/evaluator/evaluator.go index 7cab68fe983..17621cb848a 100644 --- a/scheduler/scheduling/evaluator/evaluator.go +++ b/scheduler/scheduling/evaluator/evaluator.go @@ -22,7 +22,8 @@ import ( "github.com/montanaflynn/stats" logger "d7y.io/dragonfly/v2/internal/dflog" - resource "d7y.io/dragonfly/v2/scheduler/resource/standard" + "d7y.io/dragonfly/v2/scheduler/resource/persistentcache" + "d7y.io/dragonfly/v2/scheduler/resource/standard" ) const ( @@ -60,10 +61,16 @@ const ( // Evaluator is an interface that evaluates the parents. type Evaluator interface { // EvaluateParents sort parents by evaluating multiple feature scores. - EvaluateParents(parents []*resource.Peer, child *resource.Peer, taskPieceCount int32) []*resource.Peer + EvaluateParents(parents []*standard.Peer, child *standard.Peer, taskPieceCount int32) []*standard.Peer - // IsBadNode determine if peer is a failed node. - IsBadNode(peer *resource.Peer) bool + // IsBadParent determine if peer is a bad parent, it can not be selected as a parent. + IsBadParent(peer *standard.Peer) bool + + // EvaluatePersistentCacheParents sort persistent cache parents by evaluating multiple feature scores. + EvaluatePersistentCacheParents(parents []*persistentcache.Peer, child *persistentcache.Peer, taskPieceCount int32) []*persistentcache.Peer + + // IsBadParent determine if persistent cache peer is a bad parent, it can not be selected as a parent. + IsBadPersistentCacheParent(peer *persistentcache.Peer) bool } // evaluator is an implementation of Evaluator. @@ -84,11 +91,11 @@ func New(algorithm string, pluginDir string) Evaluator { return newEvaluatorBase() } -// IsBadNode determine if peer is a failed node. -func (e *evaluator) IsBadNode(peer *resource.Peer) bool { - if peer.FSM.Is(resource.PeerStateFailed) || peer.FSM.Is(resource.PeerStateLeave) || peer.FSM.Is(resource.PeerStatePending) || - peer.FSM.Is(resource.PeerStateReceivedTiny) || peer.FSM.Is(resource.PeerStateReceivedSmall) || - peer.FSM.Is(resource.PeerStateReceivedNormal) || peer.FSM.Is(resource.PeerStateReceivedEmpty) { +// IsBadParent determine if peer is a bad parent, it can not be selected as a parent. +func (e *evaluator) IsBadParent(peer *standard.Peer) bool { + if peer.FSM.Is(standard.PeerStateFailed) || peer.FSM.Is(standard.PeerStateLeave) || peer.FSM.Is(standard.PeerStatePending) || + peer.FSM.Is(standard.PeerStateReceivedTiny) || peer.FSM.Is(standard.PeerStateReceivedSmall) || + peer.FSM.Is(standard.PeerStateReceivedNormal) || peer.FSM.Is(standard.PeerStateReceivedEmpty) { peer.Log.Debugf("peer is bad node because peer status is %s", peer.FSM.Current()) return true } @@ -108,17 +115,28 @@ func (e *evaluator) IsBadNode(peer *resource.Peer) bool { // Download costs does not meet the normal distribution, // if the last cost is twenty times more than mean, it is bad node. if len < normalDistributionLen { - isBadNode := big.NewFloat(lastCost).Cmp(big.NewFloat(mean*20)) > 0 - logger.Debugf("peer %s mean is %.2f and it is bad node: %t", peer.ID, mean, isBadNode) - return isBadNode + isBadParent := big.NewFloat(lastCost).Cmp(big.NewFloat(mean*20)) > 0 + logger.Debugf("peer %s mean is %.2f and it is bad node: %t", peer.ID, mean, isBadParent) + return isBadParent } // Download costs satisfies the normal distribution, // last cost falling outside of three-sigma effect need to be adjusted parent, // refer to https://en.wikipedia.org/wiki/68%E2%80%9395%E2%80%9399.7_rule. stdev, _ := stats.StandardDeviation(costs[:len-1]) // nolint: errcheck - isBadNode := big.NewFloat(lastCost).Cmp(big.NewFloat(mean+3*stdev)) > 0 + isBadParent := big.NewFloat(lastCost).Cmp(big.NewFloat(mean+3*stdev)) > 0 logger.Debugf("peer %s meet the normal distribution, costs mean is %.2f and standard deviation is %.2f, peer is bad node: %t", - peer.ID, mean, stdev, isBadNode) - return isBadNode + peer.ID, mean, stdev, isBadParent) + return isBadParent +} + +// IsBadParent determine if peer is a bad parent, it can not be selected as a parent. +func (e *evaluator) IsBadPersistentCacheParent(peer *persistentcache.Peer) bool { + if peer.FSM.Is(persistentcache.PeerStatePending) || peer.FSM.Is(persistentcache.PeerStateUploading) || peer.FSM.Is(persistentcache.PeerStateReceivedEmpty) || + peer.FSM.Is(persistentcache.PeerStateReceivedNormal) || peer.FSM.Is(persistentcache.PeerStateFailed) { + peer.Log.Debugf("persistent cache peer is bad node because peer status is %s", peer.FSM.Current()) + return true + } + + return false } diff --git a/scheduler/scheduling/evaluator/evaluator_base.go b/scheduler/scheduling/evaluator/evaluator_base.go index d2024275376..e7f204b7d2e 100644 --- a/scheduler/scheduling/evaluator/evaluator_base.go +++ b/scheduler/scheduling/evaluator/evaluator_base.go @@ -22,7 +22,8 @@ import ( "d7y.io/dragonfly/v2/pkg/math" "d7y.io/dragonfly/v2/pkg/types" - resource "d7y.io/dragonfly/v2/scheduler/resource/standard" + "d7y.io/dragonfly/v2/scheduler/resource/persistentcache" + "d7y.io/dragonfly/v2/scheduler/resource/standard" ) const ( @@ -56,66 +57,85 @@ func newEvaluatorBase() Evaluator { } // EvaluateParents sort parents by evaluating multiple feature scores. -func (e *evaluatorBase) EvaluateParents(parents []*resource.Peer, child *resource.Peer, totalPieceCount int32) []*resource.Peer { +func (e *evaluatorBase) EvaluateParents(parents []*standard.Peer, child *standard.Peer, totalPieceCount int32) []*standard.Peer { sort.Slice( parents, func(i, j int) bool { - return e.evaluate(parents[i], child, totalPieceCount) > e.evaluate(parents[j], child, totalPieceCount) + return e.evaluateParents(parents[i], child, totalPieceCount) > e.evaluateParents(parents[j], child, totalPieceCount) }, ) return parents } -// The larger the value, the higher the priority. -func (e *evaluatorBase) evaluate(parent *resource.Peer, child *resource.Peer, totalPieceCount int32) float64 { +// evaluateParents sort parents by evaluating multiple feature scores. +func (e *evaluatorBase) evaluateParents(parent *standard.Peer, child *standard.Peer, totalPieceCount int32) float64 { parentLocation := parent.Host.Network.Location parentIDC := parent.Host.Network.IDC childLocation := child.Host.Network.Location childIDC := child.Host.Network.IDC - return finishedPieceWeight*e.calculatePieceScore(parent, child, totalPieceCount) + - parentHostUploadSuccessWeight*e.calculateParentHostUploadSuccessScore(parent) + + return finishedPieceWeight*e.calculatePieceScore(parent.FinishedPieces.Count(), child.FinishedPieces.Count(), totalPieceCount) + + parentHostUploadSuccessWeight*e.calculateParentHostUploadSuccessScore(parent.Host.UploadCount.Load(), parent.Host.UploadFailedCount.Load()) + freeUploadWeight*e.calculateFreeUploadScore(parent.Host) + hostTypeWeight*e.calculateHostTypeScore(parent) + idcAffinityWeight*e.calculateIDCAffinityScore(parentIDC, childIDC) + locationAffinityWeight*e.calculateMultiElementAffinityScore(parentLocation, childLocation) } +// EvaluateParents sort parents by evaluating multiple feature scores. +func (e *evaluatorBase) EvaluatePersistentCacheParents(parents []*persistentcache.Peer, child *persistentcache.Peer, totalPieceCount int32) []*persistentcache.Peer { + sort.Slice( + parents, + func(i, j int) bool { + return e.evaluatePersistentCacheParents(parents[i], child, totalPieceCount) > e.evaluatePersistentCacheParents(parents[j], child, totalPieceCount) + }, + ) + + return parents +} + +// evaluatePersistentCacheParents sort persistent cache parents by evaluating multiple feature scores. +func (e *evaluatorBase) evaluatePersistentCacheParents(parent *persistentcache.Peer, child *persistentcache.Peer, totalPieceCount int32) float64 { + parentLocation := parent.Host.Network.Location + parentIDC := parent.Host.Network.IDC + childLocation := child.Host.Network.Location + childIDC := child.Host.Network.IDC + + return finishedPieceWeight*e.calculatePieceScore(parent.FinishedPieces.Count(), child.FinishedPieces.Count(), totalPieceCount) + + idcAffinityWeight*e.calculateIDCAffinityScore(parentIDC, childIDC) + + locationAffinityWeight*e.calculateMultiElementAffinityScore(parentLocation, childLocation) +} + // calculatePieceScore 0.0~unlimited larger and better. -func (e *evaluatorBase) calculatePieceScore(parent *resource.Peer, child *resource.Peer, totalPieceCount int32) float64 { +func (e *evaluatorBase) calculatePieceScore(parentFinishedPieceCount uint, childFinishedPieceCount uint, totalPieceCount int32) float64 { // If the total piece is determined, normalize the number of // pieces downloaded by the parent node. if totalPieceCount > 0 { - finishedPieceCount := parent.FinishedPieces.Count() - return float64(finishedPieceCount) / float64(totalPieceCount) + return float64(parentFinishedPieceCount) / float64(totalPieceCount) } // Use the difference between the parent node and the child node to // download the piece to roughly represent the piece score. - parentFinishedPieceCount := parent.FinishedPieces.Count() - childFinishedPieceCount := child.FinishedPieces.Count() return float64(parentFinishedPieceCount) - float64(childFinishedPieceCount) } // calculateParentHostUploadSuccessScore 0.0~unlimited larger and better. -func (e *evaluatorBase) calculateParentHostUploadSuccessScore(peer *resource.Peer) float64 { - uploadCount := peer.Host.UploadCount.Load() - uploadFailedCount := peer.Host.UploadFailedCount.Load() - if uploadCount < uploadFailedCount { +func (e *evaluatorBase) calculateParentHostUploadSuccessScore(parentUploadCount int64, parentUploadFailedCount int64) float64 { + if parentUploadCount < parentUploadFailedCount { return minScore } // Host has not been scheduled, then it is scheduled first. - if uploadCount == 0 && uploadFailedCount == 0 { + if parentUploadCount == 0 && parentUploadFailedCount == 0 { return maxScore } - return float64(uploadCount-uploadFailedCount) / float64(uploadCount) + return float64(parentUploadCount-parentUploadFailedCount) / float64(parentUploadCount) } // calculateFreeUploadScore 0.0~1.0 larger and better. -func (e *evaluatorBase) calculateFreeUploadScore(host *resource.Host) float64 { +func (e *evaluatorBase) calculateFreeUploadScore(host *standard.Host) float64 { ConcurrentUploadLimit := host.ConcurrentUploadLimit.Load() freeUploadCount := host.FreeUploadCount() if ConcurrentUploadLimit > 0 && freeUploadCount > 0 { @@ -126,13 +146,13 @@ func (e *evaluatorBase) calculateFreeUploadScore(host *resource.Host) float64 { } // calculateHostTypeScore 0.0~1.0 larger and better. -func (e *evaluatorBase) calculateHostTypeScore(peer *resource.Peer) float64 { +func (e *evaluatorBase) calculateHostTypeScore(peer *standard.Peer) float64 { // When the task is downloaded for the first time, // peer will be scheduled to seed peer first, // otherwise it will be scheduled to dfdaemon first. if peer.Host.Type != types.HostTypeNormal { - if peer.FSM.Is(resource.PeerStateReceivedNormal) || - peer.FSM.Is(resource.PeerStateRunning) { + if peer.FSM.Is(standard.PeerStateReceivedNormal) || + peer.FSM.Is(standard.PeerStateRunning) { return maxScore } diff --git a/scheduler/scheduling/evaluator/evaluator_base_test.go b/scheduler/scheduling/evaluator/evaluator_base_test.go index 5b72b6368bb..20b12f66f30 100644 --- a/scheduler/scheduling/evaluator/evaluator_base_test.go +++ b/scheduler/scheduling/evaluator/evaluator_base_test.go @@ -391,7 +391,7 @@ func TestEvaluatorBase_evaluate(t *testing.T) { t.Run(tc.name, func(t *testing.T) { e := newEvaluatorBase() tc.mock(tc.parent, tc.child) - tc.expect(t, e.(*evaluatorBase).evaluate(tc.parent, tc.child, tc.totalPieceCount)) + tc.expect(t, e.(*evaluatorBase).evaluateParents(tc.parent, tc.child, tc.totalPieceCount)) }) } } @@ -511,7 +511,7 @@ func TestEvaluatorBase_calculatePieceScore(t *testing.T) { t.Run(tc.name, func(t *testing.T) { e := newEvaluatorBase() tc.mock(tc.parent, tc.child) - tc.expect(t, e.(*evaluatorBase).calculatePieceScore(tc.parent, tc.child, tc.totalPieceCount)) + tc.expect(t, e.(*evaluatorBase).calculatePieceScore(tc.parent.FinishedPieces.Count(), tc.child.FinishedPieces.Count(), tc.totalPieceCount)) }) } } @@ -566,7 +566,7 @@ func TestEvaluatorBase_calculatehostUploadSuccessScore(t *testing.T) { mockPeer := resource.NewPeer(mockPeerID, mockTask, host) e := newEvaluatorBase() tc.mock(host) - tc.expect(t, e.(*evaluatorBase).calculateParentHostUploadSuccessScore(mockPeer)) + tc.expect(t, e.(*evaluatorBase).calculateParentHostUploadSuccessScore(mockPeer.Host.UploadCount.Load(), mockPeer.Host.UploadFailedCount.Load())) }) } } @@ -874,7 +874,7 @@ func TestEvaluatorBase_calculateMultiElementAffinityScore(t *testing.T) { } } -func TestEvaluatorBase_IsBadNode(t *testing.T) { +func TestEvaluatorBase_IsBadParent(t *testing.T) { mockHost := resource.NewHost( mockRawHost.ID, mockRawHost.IP, mockRawHost.Hostname, mockRawHost.Port, mockRawHost.DownloadPort, mockRawHost.Type) @@ -885,7 +885,7 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { peer *resource.Peer totalPieceCount int32 mock func(peer *resource.Peer) - expect func(t *testing.T, isBadNode bool) + expect func(t *testing.T, isBadParent bool) }{ { name: "peer state is PeerStateFailed", @@ -894,9 +894,9 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { mock: func(peer *resource.Peer) { peer.FSM.SetState(resource.PeerStateFailed) }, - expect: func(t *testing.T, isBadNode bool) { + expect: func(t *testing.T, isBadParent bool) { assert := assert.New(t) - assert.True(isBadNode) + assert.True(isBadParent) }, }, { @@ -906,9 +906,9 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { mock: func(peer *resource.Peer) { peer.FSM.SetState(resource.PeerStateLeave) }, - expect: func(t *testing.T, isBadNode bool) { + expect: func(t *testing.T, isBadParent bool) { assert := assert.New(t) - assert.True(isBadNode) + assert.True(isBadParent) }, }, { @@ -918,9 +918,9 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { mock: func(peer *resource.Peer) { peer.FSM.SetState(resource.PeerStatePending) }, - expect: func(t *testing.T, isBadNode bool) { + expect: func(t *testing.T, isBadParent bool) { assert := assert.New(t) - assert.True(isBadNode) + assert.True(isBadParent) }, }, { @@ -930,9 +930,9 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { mock: func(peer *resource.Peer) { peer.FSM.SetState(resource.PeerStateReceivedTiny) }, - expect: func(t *testing.T, isBadNode bool) { + expect: func(t *testing.T, isBadParent bool) { assert := assert.New(t) - assert.True(isBadNode) + assert.True(isBadParent) }, }, { @@ -942,9 +942,9 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { mock: func(peer *resource.Peer) { peer.FSM.SetState(resource.PeerStateReceivedSmall) }, - expect: func(t *testing.T, isBadNode bool) { + expect: func(t *testing.T, isBadParent bool) { assert := assert.New(t) - assert.True(isBadNode) + assert.True(isBadParent) }, }, { @@ -954,9 +954,9 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { mock: func(peer *resource.Peer) { peer.FSM.SetState(resource.PeerStateReceivedNormal) }, - expect: func(t *testing.T, isBadNode bool) { + expect: func(t *testing.T, isBadParent bool) { assert := assert.New(t) - assert.True(isBadNode) + assert.True(isBadParent) }, }, { @@ -968,9 +968,9 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { peer.AppendPieceCost(10) peer.AppendPieceCost(201) }, - expect: func(t *testing.T, isBadNode bool) { + expect: func(t *testing.T, isBadParent bool) { assert := assert.New(t) - assert.True(isBadNode) + assert.True(isBadParent) }, }, { @@ -982,9 +982,9 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { peer.AppendPieceCost(10) peer.AppendPieceCost(200) }, - expect: func(t *testing.T, isBadNode bool) { + expect: func(t *testing.T, isBadParent bool) { assert := assert.New(t) - assert.False(isBadNode) + assert.False(isBadParent) }, }, { @@ -998,9 +998,9 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { } peer.AppendPieceCost(50) }, - expect: func(t *testing.T, isBadNode bool) { + expect: func(t *testing.T, isBadParent bool) { assert := assert.New(t) - assert.True(isBadNode) + assert.True(isBadParent) }, }, { @@ -1014,9 +1014,9 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { } peer.AppendPieceCost(18) }, - expect: func(t *testing.T, isBadNode bool) { + expect: func(t *testing.T, isBadParent bool) { assert := assert.New(t) - assert.False(isBadNode) + assert.False(isBadParent) }, }, { @@ -1030,9 +1030,9 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { } peer.AppendPieceCost(0) }, - expect: func(t *testing.T, isBadNode bool) { + expect: func(t *testing.T, isBadParent bool) { assert := assert.New(t) - assert.False(isBadNode) + assert.False(isBadParent) }, }, } @@ -1041,7 +1041,7 @@ func TestEvaluatorBase_IsBadNode(t *testing.T) { t.Run(tc.name, func(t *testing.T) { e := newEvaluatorBase() tc.mock(tc.peer) - tc.expect(t, e.IsBadNode(tc.peer)) + tc.expect(t, e.IsBadParent(tc.peer)) }) } } diff --git a/scheduler/scheduling/evaluator/testdata/plugin/evaluator.go b/scheduler/scheduling/evaluator/testdata/plugin/evaluator.go index b9247b6476b..2c6580292b9 100644 --- a/scheduler/scheduling/evaluator/testdata/plugin/evaluator.go +++ b/scheduler/scheduling/evaluator/testdata/plugin/evaluator.go @@ -24,7 +24,7 @@ func (e *evaluator) EvaluateParents(parents []*resource.Peer, child *resource.Pe return []*resource.Peer{&resource.Peer{}} } -func (e *evaluator) IsBadNode(peer *resource.Peer) bool { +func (e *evaluator) IsBadParent(peer *resource.Peer) bool { return true } diff --git a/scheduler/scheduling/mocks/scheduling_mock.go b/scheduler/scheduling/mocks/scheduling_mock.go index e42272737c5..cb6e864cb63 100644 --- a/scheduler/scheduling/mocks/scheduling_mock.go +++ b/scheduler/scheduling/mocks/scheduling_mock.go @@ -14,6 +14,7 @@ import ( reflect "reflect" set "d7y.io/dragonfly/v2/pkg/container/set" + persistentcache "d7y.io/dragonfly/v2/scheduler/resource/persistentcache" standard "d7y.io/dragonfly/v2/scheduler/resource/standard" gomock "go.uber.org/mock/gomock" ) @@ -57,6 +58,21 @@ func (mr *MockSchedulingMockRecorder) FindCandidateParents(arg0, arg1, arg2 any) return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FindCandidateParents", reflect.TypeOf((*MockScheduling)(nil).FindCandidateParents), arg0, arg1, arg2) } +// FindCandidatePersistentCacheParents mocks base method. +func (m *MockScheduling) FindCandidatePersistentCacheParents(arg0 context.Context, arg1 *persistentcache.Peer, arg2 set.SafeSet[string]) ([]*persistentcache.Peer, bool) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "FindCandidatePersistentCacheParents", arg0, arg1, arg2) + ret0, _ := ret[0].([]*persistentcache.Peer) + ret1, _ := ret[1].(bool) + return ret0, ret1 +} + +// FindCandidatePersistentCacheParents indicates an expected call of FindCandidatePersistentCacheParents. +func (mr *MockSchedulingMockRecorder) FindCandidatePersistentCacheParents(arg0, arg1, arg2 any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FindCandidatePersistentCacheParents", reflect.TypeOf((*MockScheduling)(nil).FindCandidatePersistentCacheParents), arg0, arg1, arg2) +} + // FindParentAndCandidateParents mocks base method. func (m *MockScheduling) FindParentAndCandidateParents(arg0 context.Context, arg1 *standard.Peer, arg2 set.SafeSet[string]) ([]*standard.Peer, bool) { m.ctrl.T.Helper() @@ -72,6 +88,21 @@ func (mr *MockSchedulingMockRecorder) FindParentAndCandidateParents(arg0, arg1, return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FindParentAndCandidateParents", reflect.TypeOf((*MockScheduling)(nil).FindParentAndCandidateParents), arg0, arg1, arg2) } +// FindReplicatePersistentCacheParents mocks base method. +func (m *MockScheduling) FindReplicatePersistentCacheParents(arg0 context.Context, arg1 *persistentcache.Task, arg2 set.SafeSet[string]) ([]*persistentcache.Peer, bool) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "FindReplicatePersistentCacheParents", arg0, arg1, arg2) + ret0, _ := ret[0].([]*persistentcache.Peer) + ret1, _ := ret[1].(bool) + return ret0, ret1 +} + +// FindReplicatePersistentCacheParents indicates an expected call of FindReplicatePersistentCacheParents. +func (mr *MockSchedulingMockRecorder) FindReplicatePersistentCacheParents(arg0, arg1, arg2 any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FindReplicatePersistentCacheParents", reflect.TypeOf((*MockScheduling)(nil).FindReplicatePersistentCacheParents), arg0, arg1, arg2) +} + // FindSuccessParent mocks base method. func (m *MockScheduling) FindSuccessParent(arg0 context.Context, arg1 *standard.Peer, arg2 set.SafeSet[string]) (*standard.Peer, bool) { m.ctrl.T.Helper() diff --git a/scheduler/scheduling/scheduling.go b/scheduler/scheduling/scheduling.go index ad64d0a1d49..0ee61c130c0 100644 --- a/scheduler/scheduling/scheduling.go +++ b/scheduler/scheduling/scheduling.go @@ -36,29 +36,36 @@ import ( "d7y.io/dragonfly/v2/pkg/container/set" "d7y.io/dragonfly/v2/pkg/types" "d7y.io/dragonfly/v2/scheduler/config" - resource "d7y.io/dragonfly/v2/scheduler/resource/standard" + "d7y.io/dragonfly/v2/scheduler/resource/persistentcache" + "d7y.io/dragonfly/v2/scheduler/resource/standard" "d7y.io/dragonfly/v2/scheduler/scheduling/evaluator" ) type Scheduling interface { - // ScheduleCandidateParents schedules candidate parents to the normal peer. + // ScheduleCandidateParents schedules candidate parents to the normal peer to download the task. // Used only in v2 version of the grpc. - ScheduleCandidateParents(context.Context, *resource.Peer, set.SafeSet[string]) error + ScheduleCandidateParents(context.Context, *standard.Peer, set.SafeSet[string]) error - // ScheduleParentAndCandidateParents schedules a parent and candidate parents to the normal peer. + // ScheduleParentAndCandidateParents schedules a parent and candidate parents to the normal peer to download the task. // Used only in v1 version of the grpc. - ScheduleParentAndCandidateParents(context.Context, *resource.Peer, set.SafeSet[string]) + ScheduleParentAndCandidateParents(context.Context, *standard.Peer, set.SafeSet[string]) - // FindCandidateParents finds candidate parents for the peer. + // FindCandidateParents finds candidate parents for the peer to download the task. // Used only in v2 version of the grpc. - FindCandidateParents(context.Context, *resource.Peer, set.SafeSet[string]) ([]*resource.Peer, bool) + FindCandidateParents(context.Context, *standard.Peer, set.SafeSet[string]) ([]*standard.Peer, bool) - // FindParentAndCandidateParents finds a parent and candidate parents for the peer. + // FindParentAndCandidateParents finds a parent and candidate parents for the peer to download the task. // Used only in v1 version of the grpc. - FindParentAndCandidateParents(context.Context, *resource.Peer, set.SafeSet[string]) ([]*resource.Peer, bool) + FindParentAndCandidateParents(context.Context, *standard.Peer, set.SafeSet[string]) ([]*standard.Peer, bool) - // FindSuccessParent finds success parent for the peer. - FindSuccessParent(context.Context, *resource.Peer, set.SafeSet[string]) (*resource.Peer, bool) + // FindSuccessParent finds success parent for the peer to download the task. + FindSuccessParent(context.Context, *standard.Peer, set.SafeSet[string]) (*standard.Peer, bool) + + // FindReplicatePersistentCacheParents finds replicate persistent cache parents for the peer to replicate the task. + FindReplicatePersistentCacheParents(context.Context, *persistentcache.Task, set.SafeSet[string]) ([]*persistentcache.Peer, bool) + + // FindCandidatePersistentCacheParents finds candidate persistent cache parents for the peer to download the task. + FindCandidatePersistentCacheParents(context.Context, *persistentcache.Peer, set.SafeSet[string]) ([]*persistentcache.Peer, bool) } type scheduling struct { @@ -68,21 +75,25 @@ type scheduling struct { // Scheduler configuration. config *config.SchedulerConfig + // Persistent cache resource. + persistentCacheResource persistentcache.Resource + // Scheduler dynamic configuration. dynconfig config.DynconfigInterface } -func New(cfg *config.SchedulerConfig, dynconfig config.DynconfigInterface, pluginDir string) Scheduling { +func New(cfg *config.SchedulerConfig, persistentCacheResource persistentcache.Resource, dynconfig config.DynconfigInterface, pluginDir string) Scheduling { return &scheduling{ - evaluator: evaluator.New(cfg.Algorithm, pluginDir), - config: cfg, - dynconfig: dynconfig, + evaluator: evaluator.New(cfg.Algorithm, pluginDir), + config: cfg, + persistentCacheResource: persistentCacheResource, + dynconfig: dynconfig, } } // ScheduleCandidateParents schedules candidate parents to the normal peer. // Used only in v2 version of the grpc. -func (s *scheduling) ScheduleCandidateParents(ctx context.Context, peer *resource.Peer, blocklist set.SafeSet[string]) error { +func (s *scheduling) ScheduleCandidateParents(ctx context.Context, peer *standard.Peer, blocklist set.SafeSet[string]) error { var n int for { select { @@ -193,7 +204,7 @@ func (s *scheduling) ScheduleCandidateParents(ctx context.Context, peer *resourc // Send NormalTaskResponse to peer. peer.Log.Info("send NormalTaskResponse") if err := stream.Send(&schedulerv2.AnnouncePeerResponse{ - Response: ConstructSuccessNormalTaskResponse(candidateParents), + Response: constructSuccessNormalTaskResponse(candidateParents), }); err != nil { peer.Log.Error(err) return status.Error(codes.FailedPrecondition, err.Error()) @@ -214,7 +225,7 @@ func (s *scheduling) ScheduleCandidateParents(ctx context.Context, peer *resourc // ScheduleParentAndCandidateParents schedules a parent and candidate parents to a peer. // Used only in v1 version of the grpc. -func (s *scheduling) ScheduleParentAndCandidateParents(ctx context.Context, peer *resource.Peer, blocklist set.SafeSet[string]) { +func (s *scheduling) ScheduleParentAndCandidateParents(ctx context.Context, peer *standard.Peer, blocklist set.SafeSet[string]) { var n int for { select { @@ -245,15 +256,15 @@ func (s *scheduling) ScheduleParentAndCandidateParents(ctx context.Context, peer } peer.Log.Infof("send Code_SchedNeedBackSource to peer, because of peer's NeedBackToSource is %t", peer.NeedBackToSource.Load()) - if err := peer.FSM.Event(ctx, resource.PeerEventDownloadBackToSource); err != nil { + if err := peer.FSM.Event(ctx, standard.PeerEventDownloadBackToSource); err != nil { peer.Log.Errorf("peer fsm event failed: %s", err.Error()) return } // If the task state is TaskStateFailed, // peer back-to-source and reset task state to TaskStateRunning. - if peer.Task.FSM.Is(resource.TaskStateFailed) { - if err := peer.Task.FSM.Event(ctx, resource.TaskEventDownload); err != nil { + if peer.Task.FSM.Is(standard.TaskStateFailed) { + if err := peer.Task.FSM.Event(ctx, standard.TaskEventDownload); err != nil { peer.Task.Log.Errorf("task fsm event failed: %s", err.Error()) return } @@ -278,15 +289,15 @@ func (s *scheduling) ScheduleParentAndCandidateParents(ctx context.Context, peer } peer.Log.Infof("send Code_SchedNeedBackSource to peer, because of scheduling exceeded RetryBackToSourceLimit %d", s.config.RetryBackToSourceLimit) - if err := peer.FSM.Event(ctx, resource.PeerEventDownloadBackToSource); err != nil { + if err := peer.FSM.Event(ctx, standard.PeerEventDownloadBackToSource); err != nil { peer.Log.Errorf("peer fsm event failed: %s", err.Error()) return } // If the task state is TaskStateFailed, // peer back-to-source and reset task state to TaskStateRunning. - if peer.Task.FSM.Is(resource.TaskStateFailed) { - if err := peer.Task.FSM.Event(ctx, resource.TaskEventDownload); err != nil { + if peer.Task.FSM.Is(standard.TaskStateFailed) { + if err := peer.Task.FSM.Event(ctx, standard.TaskEventDownload); err != nil { peer.Task.Log.Errorf("task fsm event failed: %s", err.Error()) return } @@ -355,7 +366,7 @@ func (s *scheduling) ScheduleParentAndCandidateParents(ctx context.Context, peer // Send PeerPacket to peer. peer.Log.Info("send PeerPacket to peer") - if err := stream.Send(ConstructSuccessPeerPacket(peer, candidateParents[0], candidateParents[1:])); err != nil { + if err := stream.Send(constructSuccessPeerPacket(peer, candidateParents[0], candidateParents[1:])); err != nil { n++ peer.Log.Errorf("scheduling failed in %d times, because of %s", n, err.Error()) @@ -381,20 +392,20 @@ func (s *scheduling) ScheduleParentAndCandidateParents(ctx context.Context, peer } // FindCandidateParents finds candidate parents for the peer. -func (s *scheduling) FindCandidateParents(ctx context.Context, peer *resource.Peer, blocklist set.SafeSet[string]) ([]*resource.Peer, bool) { +func (s *scheduling) FindCandidateParents(ctx context.Context, peer *standard.Peer, blocklist set.SafeSet[string]) ([]*standard.Peer, bool) { // Only PeerStateReceivedNormal and PeerStateRunning peers need to be rescheduled, // and other states including the PeerStateBackToSource indicate that // they have been scheduled. - if !(peer.FSM.Is(resource.PeerStateReceivedNormal) || peer.FSM.Is(resource.PeerStateRunning)) { + if !(peer.FSM.Is(standard.PeerStateReceivedNormal) || peer.FSM.Is(standard.PeerStateRunning)) { peer.Log.Infof("peer state is %s, can not schedule parent", peer.FSM.Current()) - return []*resource.Peer{}, false + return []*standard.Peer{}, false } // Find the candidate parent that can be scheduled. candidateParents := s.filterCandidateParents(peer, blocklist) if len(candidateParents) == 0 { peer.Log.Info("can not find candidate parents") - return []*resource.Peer{}, false + return []*standard.Peer{}, false } // Sort candidate parents by evaluation score. @@ -423,20 +434,20 @@ func (s *scheduling) FindCandidateParents(ctx context.Context, peer *resource.Pe } // FindParentAndCandidateParents finds a parent and candidate parents for the peer. -func (s *scheduling) FindParentAndCandidateParents(ctx context.Context, peer *resource.Peer, blocklist set.SafeSet[string]) ([]*resource.Peer, bool) { +func (s *scheduling) FindParentAndCandidateParents(ctx context.Context, peer *standard.Peer, blocklist set.SafeSet[string]) ([]*standard.Peer, bool) { // Only PeerStateRunning peers need to be rescheduled, // and other states including the PeerStateBackToSource indicate that // they have been scheduled. - if !peer.FSM.Is(resource.PeerStateRunning) { + if !peer.FSM.Is(standard.PeerStateRunning) { peer.Log.Infof("peer state is %s, can not schedule parent", peer.FSM.Current()) - return []*resource.Peer{}, false + return []*standard.Peer{}, false } // Find the candidate parent that can be scheduled. candidateParents := s.filterCandidateParents(peer, blocklist) if len(candidateParents) == 0 { peer.Log.Info("can not find candidate parents") - return []*resource.Peer{}, false + return []*standard.Peer{}, false } // Sort candidate parents by evaluation score. @@ -465,11 +476,11 @@ func (s *scheduling) FindParentAndCandidateParents(ctx context.Context, peer *re } // FindSuccessParent finds success parent for the peer. -func (s *scheduling) FindSuccessParent(ctx context.Context, peer *resource.Peer, blocklist set.SafeSet[string]) (*resource.Peer, bool) { +func (s *scheduling) FindSuccessParent(ctx context.Context, peer *standard.Peer, blocklist set.SafeSet[string]) (*standard.Peer, bool) { // Only PeerStateRunning peers need to be rescheduled, // and other states including the PeerStateBackToSource indicate that // they have been scheduled. - if !peer.FSM.Is(resource.PeerStateRunning) { + if !peer.FSM.Is(standard.PeerStateRunning) { peer.Log.Infof("peer state is %s, can not schedule parent", peer.FSM.Current()) return nil, false } @@ -481,9 +492,9 @@ func (s *scheduling) FindSuccessParent(ctx context.Context, peer *resource.Peer, return nil, false } - var successParents []*resource.Peer + var successParents []*standard.Peer for _, candidateParent := range candidateParents { - if candidateParent.FSM.Is(resource.PeerStateSucceeded) { + if candidateParent.FSM.Is(standard.PeerStateSucceeded) { successParents = append(successParents, candidateParent) } } @@ -497,7 +508,7 @@ func (s *scheduling) FindSuccessParent(ctx context.Context, peer *resource.Peer, } // filterCandidateParents filters the candidate parents that can be scheduled. -func (s *scheduling) filterCandidateParents(peer *resource.Peer, blocklist set.SafeSet[string]) []*resource.Peer { +func (s *scheduling) filterCandidateParents(peer *standard.Peer, blocklist set.SafeSet[string]) []*standard.Peer { filterParentLimit := config.DefaultSchedulerFilterParentLimit if config, err := s.dynconfig.GetSchedulerClusterConfig(); err == nil { if config.FilterParentLimit > 0 { @@ -506,7 +517,7 @@ func (s *scheduling) filterCandidateParents(peer *resource.Peer, blocklist set.S } var ( - candidateParents []*resource.Peer + candidateParents []*standard.Peer candidateParentIDs []string ) for _, candidateParent := range peer.Task.LoadRandomPeers(uint(filterParentLimit)) { @@ -542,15 +553,15 @@ func (s *scheduling) filterCandidateParents(peer *resource.Peer, blocklist set.S // Condition 2: Parent has been back-to-source. // Condition 3: Parent has been succeeded. // Condition 4: Parent is seed peer. - if candidateParent.Host.Type == types.HostTypeNormal && inDegree == 0 && !candidateParent.FSM.Is(resource.PeerStateBackToSource) && - !candidateParent.FSM.Is(resource.PeerStateSucceeded) { + if candidateParent.Host.Type == types.HostTypeNormal && inDegree == 0 && !candidateParent.FSM.Is(standard.PeerStateBackToSource) && + !candidateParent.FSM.Is(standard.PeerStateSucceeded) { peer.Log.Debugf("parent %s host %s is not selected, because its download state is %d %d %s", candidateParent.ID, candidateParent.Host.ID, inDegree, int(candidateParent.Host.Type), candidateParent.FSM.Current()) continue } - // Candidate parent is bad node. - if s.evaluator.IsBadNode(candidateParent) { + // Candidate parent is bad parent. + if s.evaluator.IsBadParent(candidateParent) { peer.Log.Debugf("parent %s host %s is not selected because it is bad node", candidateParent.ID, candidateParent.Host.ID) continue } @@ -576,9 +587,87 @@ func (s *scheduling) filterCandidateParents(peer *resource.Peer, blocklist set.S return candidateParents } -// ConstructSuccessNormalTaskResponse constructs scheduling successful response of the normal task. +// TODO(Gaius) Implement the following methods. +// FindReplicatePersistentCacheParents finds replicate persistent cache parents for the peer to replicate the task. +func (s *scheduling) FindReplicatePersistentCacheParents(ctx context.Context, task *persistentcache.Task, blocklist set.SafeSet[string]) ([]*persistentcache.Peer, bool) { + return nil, false +} + +// FindCandidatePersistentCacheParents finds candidate persistent cache parents for the peer to download the task. +func (s *scheduling) FindCandidatePersistentCacheParents(ctx context.Context, peer *persistentcache.Peer, blocklist set.SafeSet[string]) ([]*persistentcache.Peer, bool) { + // Find the candidate parent that can be scheduled. + candidateParents := s.filterCandidatePersistentCacheParents(ctx, peer, blocklist) + if len(candidateParents) == 0 { + peer.Log.Info("can not find candidate persistent cache parents") + return candidateParents, false + } + + // Sort candidate parents by evaluation score. + candidateParents = s.evaluator.EvaluatePersistentCacheParents(candidateParents, peer, peer.Task.TotalPieceCount) + + // Get the parents with candidateParentLimit. + candidateParentLimit := config.DefaultSchedulerCandidateParentLimit + if config, err := s.dynconfig.GetSchedulerClusterConfig(); err == nil { + if config.CandidateParentLimit > 0 { + candidateParentLimit = int(config.CandidateParentLimit) + } + } + + if len(candidateParents) > candidateParentLimit { + candidateParents = candidateParents[:candidateParentLimit] + } + + var parentIDs []string + for _, candidateParent := range candidateParents { + parentIDs = append(parentIDs, candidateParent.ID) + } + + peer.Log.Infof("scheduling candidate persistent cache parents is %#v", parentIDs) + return candidateParents, true +} + +// filterCandidateParents filters the candidate parents that can be scheduled. +func (s *scheduling) filterCandidatePersistentCacheParents(ctx context.Context, peer *persistentcache.Peer, blocklist set.SafeSet[string]) []*persistentcache.Peer { + parents, err := s.persistentCacheResource.PeerManager().LoadAllByTaskID(ctx, peer.Task.ID) + if err != nil { + peer.Log.Errorf("load all persistent cache parents failed: %s", err.Error()) + return nil + } + + var ( + candidateParents []*persistentcache.Peer + candidateParentIDs []string + ) + for _, candidateParent := range parents { + // Candidate persistent cache parent is in blocklist. + if blocklist.Contains(candidateParent.ID) { + peer.Log.Debugf("persistent cache parent %s host %s is not selected because it is in blocklist", candidateParent.ID, candidateParent.Host.ID) + continue + } + + // Candidate persistent cache parent host is not allowed to be the same as the peer host, + if peer.Host.ID == candidateParent.Host.ID { + peer.Log.Debugf("persistent cache parent %s host %s is the same as peer host", candidateParent.ID, candidateParent.Host.ID) + continue + } + + // Candidate persistent cache parent is bad parent. + if s.evaluator.IsBadPersistentCacheParent(candidateParent) { + peer.Log.Debugf("persistent cache parent %s host %s is not selected because it is bad node", candidateParent.ID, candidateParent.Host.ID) + continue + } + + candidateParents = append(candidateParents, candidateParent) + candidateParentIDs = append(candidateParentIDs, candidateParent.ID) + } + + peer.Log.Infof("filter candidate persistent cache parents is %#v", candidateParentIDs) + return candidateParents +} + +// constructSuccessNormalTaskResponse constructs scheduling successful response of the normal task. // Used only in v2 version of the grpc. -func ConstructSuccessNormalTaskResponse(candidateParents []*resource.Peer) *schedulerv2.AnnouncePeerResponse_NormalTaskResponse { +func constructSuccessNormalTaskResponse(candidateParents []*standard.Peer) *schedulerv2.AnnouncePeerResponse_NormalTaskResponse { var parents []*commonv2.Peer for _, candidateParent := range candidateParents { parent := &commonv2.Peer{ @@ -703,9 +792,9 @@ func ConstructSuccessNormalTaskResponse(candidateParents []*resource.Peer) *sche } } -// ConstructSuccessPeerPacket constructs peer successful packet. +// constructSuccessPeerPacket constructs peer successful packet. // Used only in v1 version of the grpc. -func ConstructSuccessPeerPacket(peer *resource.Peer, parent *resource.Peer, candidateParents []*resource.Peer) *schedulerv1.PeerPacket { +func constructSuccessPeerPacket(peer *standard.Peer, parent *standard.Peer, candidateParents []*standard.Peer) *schedulerv1.PeerPacket { var parents []*schedulerv1.PeerPacket_DestPeer for _, candidateParent := range candidateParents { parents = append(parents, &schedulerv1.PeerPacket_DestPeer{ diff --git a/scheduler/scheduling/scheduling_test.go b/scheduler/scheduling/scheduling_test.go index 47b44d5f182..5f97b01ad5a 100644 --- a/scheduler/scheduling/scheduling_test.go +++ b/scheduler/scheduling/scheduling_test.go @@ -48,7 +48,8 @@ import ( pkgtypes "d7y.io/dragonfly/v2/pkg/types" "d7y.io/dragonfly/v2/scheduler/config" configmocks "d7y.io/dragonfly/v2/scheduler/config/mocks" - resource "d7y.io/dragonfly/v2/scheduler/resource/standard" + "d7y.io/dragonfly/v2/scheduler/resource/persistentcache" + "d7y.io/dragonfly/v2/scheduler/resource/standard" "d7y.io/dragonfly/v2/scheduler/scheduling/evaluator" ) @@ -62,7 +63,7 @@ var ( Algorithm: evaluator.DefaultAlgorithm, } - mockRawHost = resource.Host{ + mockRawHost = standard.Host{ ID: mockHostID, Type: pkgtypes.HostTypeNormal, Hostname: "foo", @@ -83,7 +84,7 @@ var ( UpdatedAt: atomic.NewTime(time.Now()), } - mockRawSeedHost = resource.Host{ + mockRawSeedHost = standard.Host{ ID: mockSeedHostID, Type: pkgtypes.HostTypeSuperSeed, Hostname: "bar", @@ -104,12 +105,12 @@ var ( UpdatedAt: atomic.NewTime(time.Now()), } - mockCPU = resource.CPU{ + mockCPU = standard.CPU{ LogicalCount: 4, PhysicalCount: 2, Percent: 1, ProcessPercent: 0.5, - Times: resource.CPUTimes{ + Times: standard.CPUTimes{ User: 240662.2, System: 317950.1, Idle: 3393691.3, @@ -123,7 +124,7 @@ var ( }, } - mockMemory = resource.Memory{ + mockMemory = standard.Memory{ Total: 17179869184, Available: 5962813440, Used: 11217055744, @@ -132,7 +133,7 @@ var ( Free: 2749598908, } - mockNetwork = resource.Network{ + mockNetwork = standard.Network{ TCPConnectionCount: 10, UploadTCPConnectionCount: 1, Location: mockHostLocation, @@ -143,7 +144,7 @@ var ( UploadRateLimit: 200, } - mockDisk = resource.Disk{ + mockDisk = standard.Disk{ Total: 499963174912, Free: 37226479616, Used: 423809622016, @@ -154,7 +155,7 @@ var ( InodesUsedPercent: 0.1604884305611568, } - mockBuild = resource.Build{ + mockBuild = standard.Build{ GitVersion: "v1.0.0", GitCommit: "221176b117c6d59366d68f2b34d38be50c935883", GoVersion: "1.18", @@ -176,7 +177,7 @@ var ( mockHostIDC = "bas" mockPeerID = idgen.PeerIDV2() mockSeedPeerID = idgen.PeerIDV2() - mockPiece = resource.Piece{ + mockPiece = standard.Piece{ Number: 1, ParentID: "foo", Offset: 2, @@ -217,8 +218,9 @@ func TestScheduling_New(t *testing.T) { ctl := gomock.NewController(t) defer ctl.Finish() dynconfig := configmocks.NewMockDynconfigInterface(ctl) + persistentCacheResource := persistentcache.NewMockResource(ctl) - tc.expect(t, New(mockSchedulerConfig, dynconfig, tc.pluginDir)) + tc.expect(t, New(mockSchedulerConfig, persistentCacheResource, dynconfig, tc.pluginDir)) }) } } @@ -229,45 +231,45 @@ func TestScheduling_ScheduleCandidateParents(t *testing.T) { tests := []struct { name string - mock func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) - expect func(t *testing.T, peer *resource.Peer, err error) + mock func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) + expect func(t *testing.T, peer *standard.Peer, err error) }{ { name: "context was done", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) cancel() }, - expect: func(t *testing.T, peer *resource.Peer, err error) { + expect: func(t *testing.T, peer *standard.Peer, err error) { assert := assert.New(t) assert.ErrorIs(err, context.Canceled) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "peer needs back-to-source and peer stream load failed", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) peer.NeedBackToSource.Store(true) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) }, - expect: func(t *testing.T, peer *resource.Peer, err error) { + expect: func(t *testing.T, peer *standard.Peer, err error) { assert := assert.New(t) assert.ErrorIs(err, status.Error(codes.FailedPrecondition, "load stream failed")) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "peer needs back-to-source and send NeedBackToSourceResponse failed", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) peer.NeedBackToSource.Store(true) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) peer.StoreAnnouncePeerStream(stream) ma.Send(gomock.Eq(&schedulerv2.AnnouncePeerResponse{ @@ -278,21 +280,21 @@ func TestScheduling_ScheduleCandidateParents(t *testing.T) { }, })).Return(errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, err error) { + expect: func(t *testing.T, peer *standard.Peer, err error) { assert := assert.New(t) assert.ErrorIs(err, status.Error(codes.FailedPrecondition, "foo")) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "peer needs back-to-source and send NeedBackToSourceResponse success", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) peer.NeedBackToSource.Store(true) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) peer.StoreAnnouncePeerStream(stream) ma.Send(gomock.Eq(&schedulerv2.AnnouncePeerResponse{ @@ -303,36 +305,36 @@ func TestScheduling_ScheduleCandidateParents(t *testing.T) { }, })).Return(nil).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, err error) { + expect: func(t *testing.T, peer *standard.Peer, err error) { assert := assert.New(t) assert.NoError(err) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "schedule exceeds RetryBackToSourceLimit and peer stream load failed", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, err error) { + expect: func(t *testing.T, peer *standard.Peer, err error) { assert := assert.New(t) assert.ErrorIs(err, status.Error(codes.FailedPrecondition, "load stream failed")) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "schedule exceeds RetryBackToSourceLimit and send NeedBackToSourceResponse failed", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) peer.StoreAnnouncePeerStream(stream) gomock.InOrder( @@ -346,20 +348,20 @@ func TestScheduling_ScheduleCandidateParents(t *testing.T) { })).Return(errors.New("foo")).Times(1), ) }, - expect: func(t *testing.T, peer *resource.Peer, err error) { + expect: func(t *testing.T, peer *standard.Peer, err error) { assert := assert.New(t) assert.ErrorIs(err, status.Error(codes.FailedPrecondition, "foo")) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "schedule exceeds RetryBackToSourceLimit and send NeedBackToSourceResponse success", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) peer.StoreAnnouncePeerStream(stream) gomock.InOrder( @@ -373,20 +375,20 @@ func TestScheduling_ScheduleCandidateParents(t *testing.T) { })).Return(nil).Times(1), ) }, - expect: func(t *testing.T, peer *resource.Peer, err error) { + expect: func(t *testing.T, peer *standard.Peer, err error) { assert := assert.New(t) assert.NoError(err) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "schedule exceeds RetryLimit", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) peer.Task.BackToSourceLimit.Store(-1) peer.StoreAnnouncePeerStream(stream) @@ -394,34 +396,34 @@ func TestScheduling_ScheduleCandidateParents(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2), ) }, - expect: func(t *testing.T, peer *resource.Peer, err error) { + expect: func(t *testing.T, peer *standard.Peer, err error) { assert := assert.New(t) assert.ErrorIs(err, status.Error(codes.FailedPrecondition, "scheduling exceeded RetryLimit")) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "schedule succeeded", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv2.Scheduler_AnnouncePeerServer, ma *schedulerv2mocks.MockScheduler_AnnouncePeerServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) task.StorePeer(seedPeer) - peer.FSM.SetState(resource.PeerStateRunning) - seedPeer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) + seedPeer.FSM.SetState(standard.PeerStateRunning) peer.StoreAnnouncePeerStream(stream) gomock.InOrder( md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2), ma.Send(gomock.Any()).Return(nil).Times(1), ) }, - expect: func(t *testing.T, peer *resource.Peer, err error) { + expect: func(t *testing.T, peer *standard.Peer, err error) { assert := assert.New(t) assert.NoError(err) assert.Equal(len(peer.Parents()), 1) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, } @@ -431,21 +433,22 @@ func TestScheduling_ScheduleCandidateParents(t *testing.T) { ctl := gomock.NewController(t) defer ctl.Finish() stream := schedulerv2mocks.NewMockScheduler_AnnouncePeerServer(ctl) + persistentCacheResource := persistentcache.NewMockResource(ctl) dynconfig := configmocks.NewMockDynconfigInterface(ctl) ctx, cancel := context.WithCancel(context.Background()) - mockHost := resource.NewHost( + mockHost := standard.NewHost( mockRawHost.ID, mockRawHost.IP, mockRawHost.Hostname, mockRawHost.Port, mockRawHost.DownloadPort, mockRawHost.Type) - mockTask := resource.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, resource.WithDigest(mockTaskDigest), resource.WithPieceLength(mockTaskPieceLength)) - peer := resource.NewPeer(mockPeerID, mockTask, mockHost) - mockSeedHost := resource.NewHost( + mockTask := standard.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, standard.WithDigest(mockTaskDigest), standard.WithPieceLength(mockTaskPieceLength)) + peer := standard.NewPeer(mockPeerID, mockTask, mockHost) + mockSeedHost := standard.NewHost( mockRawSeedHost.ID, mockRawSeedHost.IP, mockRawSeedHost.Hostname, mockRawSeedHost.Port, mockRawSeedHost.DownloadPort, mockRawSeedHost.Type) - seedPeer := resource.NewPeer(mockSeedPeerID, mockTask, mockSeedHost) + seedPeer := standard.NewPeer(mockSeedPeerID, mockTask, mockSeedHost) blocklist := set.NewSafeSet[string]() tc.mock(cancel, peer, seedPeer, blocklist, stream, stream.EXPECT(), dynconfig.EXPECT()) - scheduling := New(mockSchedulerConfig, dynconfig, mockPluginDir) + scheduling := New(mockSchedulerConfig, persistentCacheResource, dynconfig, mockPluginDir) tc.expect(t, peer, scheduling.ScheduleCandidateParents(ctx, peer, blocklist)) }) } @@ -454,112 +457,112 @@ func TestScheduling_ScheduleCandidateParents(t *testing.T) { func TestScheduling_ScheduleParentAndCandidateParents(t *testing.T) { tests := []struct { name string - mock func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) - expect func(t *testing.T, peer *resource.Peer) + mock func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) + expect func(t *testing.T, peer *standard.Peer) }{ { name: "context was done", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) cancel() }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "peer needs back-to-source and peer stream load failed", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) peer.NeedBackToSource.Store(true) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "peer needs back-to-source and send Code_SchedNeedBackSource failed", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) peer.NeedBackToSource.Store(true) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) peer.StoreReportPieceResultStream(stream) mr.Send(gomock.Eq(&schedulerv1.PeerPacket{Code: commonv1.Code_SchedNeedBackSource})).Return(errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "peer needs back-to-source and send Code_SchedNeedBackSource success", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) peer.NeedBackToSource.Store(true) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) peer.StoreReportPieceResultStream(stream) mr.Send(gomock.Eq(&schedulerv1.PeerPacket{Code: commonv1.Code_SchedNeedBackSource})).Return(nil).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateBackToSource)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateBackToSource)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "peer needs back-to-source and task state is TaskStateFailed", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) peer.NeedBackToSource.Store(true) - peer.FSM.SetState(resource.PeerStateRunning) - task.FSM.SetState(resource.TaskStateFailed) + peer.FSM.SetState(standard.PeerStateRunning) + task.FSM.SetState(standard.TaskStateFailed) peer.StoreReportPieceResultStream(stream) mr.Send(gomock.Eq(&schedulerv1.PeerPacket{Code: commonv1.Code_SchedNeedBackSource})).Return(nil).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateBackToSource)) - assert.True(peer.Task.FSM.Is(resource.TaskStateRunning)) + assert.True(peer.FSM.Is(standard.PeerStateBackToSource)) + assert.True(peer.Task.FSM.Is(standard.TaskStateRunning)) }, }, { name: "schedule exceeds RetryBackToSourceLimit and peer stream load failed", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "schedule exceeds RetryBackToSourceLimit and send Code_SchedNeedBackSource failed", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) peer.StoreReportPieceResultStream(stream) gomock.InOrder( @@ -567,19 +570,19 @@ func TestScheduling_ScheduleParentAndCandidateParents(t *testing.T) { mr.Send(gomock.Eq(&schedulerv1.PeerPacket{Code: commonv1.Code_SchedNeedBackSource})).Return(errors.New("foo")).Times(1), ) }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "schedule exceeds RetryBackToSourceLimit and send Code_SchedNeedBackSource success", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) peer.StoreReportPieceResultStream(stream) gomock.InOrder( @@ -587,20 +590,20 @@ func TestScheduling_ScheduleParentAndCandidateParents(t *testing.T) { mr.Send(gomock.Eq(&schedulerv1.PeerPacket{Code: commonv1.Code_SchedNeedBackSource})).Return(nil).Times(1), ) }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateBackToSource)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateBackToSource)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "schedule exceeds RetryBackToSourceLimit and task state is TaskStateFailed", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) - peer.FSM.SetState(resource.PeerStateRunning) - task.FSM.SetState(resource.TaskStateFailed) + peer.FSM.SetState(standard.PeerStateRunning) + task.FSM.SetState(standard.TaskStateFailed) peer.StoreReportPieceResultStream(stream) gomock.InOrder( @@ -608,35 +611,35 @@ func TestScheduling_ScheduleParentAndCandidateParents(t *testing.T) { mr.Send(gomock.Eq(&schedulerv1.PeerPacket{Code: commonv1.Code_SchedNeedBackSource})).Return(nil).Times(1), ) }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateBackToSource)) - assert.True(peer.Task.FSM.Is(resource.TaskStateRunning)) + assert.True(peer.FSM.Is(standard.PeerStateBackToSource)) + assert.True(peer.Task.FSM.Is(standard.TaskStateRunning)) }, }, { name: "schedule exceeds RetryLimit and peer stream load failed", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) peer.Task.BackToSourceLimit.Store(-1) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "schedule exceeds RetryLimit and send Code_SchedTaskStatusError failed", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) peer.Task.BackToSourceLimit.Store(-1) peer.StoreReportPieceResultStream(stream) @@ -645,19 +648,19 @@ func TestScheduling_ScheduleParentAndCandidateParents(t *testing.T) { mr.Send(gomock.Eq(&schedulerv1.PeerPacket{Code: commonv1.Code_SchedTaskStatusError})).Return(errors.New("foo")).Times(1), ) }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "schedule exceeds RetryLimit and send Code_SchedTaskStatusError success", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) - peer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) peer.Task.BackToSourceLimit.Store(-1) peer.StoreReportPieceResultStream(stream) @@ -666,32 +669,32 @@ func TestScheduling_ScheduleParentAndCandidateParents(t *testing.T) { mr.Send(gomock.Eq(&schedulerv1.PeerPacket{Code: commonv1.Code_SchedTaskStatusError})).Return(nil).Times(1), ) }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) assert.Equal(len(peer.Parents()), 0) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, { name: "schedule succeeded", - mock: func(cancel context.CancelFunc, peer *resource.Peer, seedPeer *resource.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { + mock: func(cancel context.CancelFunc, peer *standard.Peer, seedPeer *standard.Peer, blocklist set.SafeSet[string], stream schedulerv1.Scheduler_ReportPieceResultServer, mr *schedulerv1mocks.MockScheduler_ReportPieceResultServerMockRecorder, md *configmocks.MockDynconfigInterfaceMockRecorder) { task := peer.Task task.StorePeer(peer) task.StorePeer(seedPeer) - peer.FSM.SetState(resource.PeerStateRunning) - seedPeer.FSM.SetState(resource.PeerStateRunning) + peer.FSM.SetState(standard.PeerStateRunning) + seedPeer.FSM.SetState(standard.PeerStateRunning) peer.StoreReportPieceResultStream(stream) gomock.InOrder( md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2), mr.Send(gomock.Any()).Return(nil).Times(1), ) }, - expect: func(t *testing.T, peer *resource.Peer) { + expect: func(t *testing.T, peer *standard.Peer) { assert := assert.New(t) assert.Equal(len(peer.Parents()), 1) - assert.True(peer.FSM.Is(resource.PeerStateRunning)) - assert.True(peer.Task.FSM.Is(resource.TaskStatePending)) + assert.True(peer.FSM.Is(standard.PeerStateRunning)) + assert.True(peer.Task.FSM.Is(standard.TaskStatePending)) }, }, } @@ -701,21 +704,22 @@ func TestScheduling_ScheduleParentAndCandidateParents(t *testing.T) { ctl := gomock.NewController(t) defer ctl.Finish() stream := schedulerv1mocks.NewMockScheduler_ReportPieceResultServer(ctl) + persistentCacheResource := persistentcache.NewMockResource(ctl) dynconfig := configmocks.NewMockDynconfigInterface(ctl) ctx, cancel := context.WithCancel(context.Background()) - mockHost := resource.NewHost( + mockHost := standard.NewHost( mockRawHost.ID, mockRawHost.IP, mockRawHost.Hostname, mockRawHost.Port, mockRawHost.DownloadPort, mockRawHost.Type) - mockTask := resource.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, resource.WithDigest(mockTaskDigest), resource.WithPieceLength(mockTaskPieceLength)) - peer := resource.NewPeer(mockPeerID, mockTask, mockHost) - mockSeedHost := resource.NewHost( + mockTask := standard.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, standard.WithDigest(mockTaskDigest), standard.WithPieceLength(mockTaskPieceLength)) + peer := standard.NewPeer(mockPeerID, mockTask, mockHost) + mockSeedHost := standard.NewHost( mockRawSeedHost.ID, mockRawSeedHost.IP, mockRawSeedHost.Hostname, mockRawSeedHost.Port, mockRawSeedHost.DownloadPort, mockRawSeedHost.Type) - seedPeer := resource.NewPeer(mockSeedPeerID, mockTask, mockSeedHost) + seedPeer := standard.NewPeer(mockSeedPeerID, mockTask, mockSeedHost) blocklist := set.NewSafeSet[string]() tc.mock(cancel, peer, seedPeer, blocklist, stream, stream.EXPECT(), dynconfig.EXPECT()) - scheduling := New(mockSchedulerConfig, dynconfig, mockPluginDir) + scheduling := New(mockSchedulerConfig, persistentCacheResource, dynconfig, mockPluginDir) scheduling.ScheduleParentAndCandidateParents(ctx, peer, blocklist) tc.expect(t, peer) }) @@ -725,15 +729,15 @@ func TestScheduling_ScheduleParentAndCandidateParents(t *testing.T) { func TestScheduling_FindCandidateParents(t *testing.T) { tests := []struct { name string - mock func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) - expect func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) + mock func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) + expect func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) }{ { name: "task peers state is failed", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateFailed) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateFailed) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.Equal(len(parents), 0) assert.False(ok) @@ -741,64 +745,64 @@ func TestScheduling_FindCandidateParents(t *testing.T) { }, { name: "task peers is empty", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "task contains only one peer and peer is itself", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateReceivedNormal) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateReceivedNormal) peer.Task.StorePeer(peer) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "peer is in blocklist", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) blocklist.Add(mockPeers[0].ID) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "peer is bad node", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateReceivedNormal) - mockPeers[0].FSM.SetState(resource.PeerStateFailed) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateReceivedNormal) + mockPeers[0].FSM.SetState(standard.PeerStateFailed) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "parent is peer's descendant", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) if err := peer.Task.AddPeerEdge(peer, mockPeers[0]); err != nil { @@ -807,56 +811,56 @@ func TestScheduling_FindCandidateParents(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "parent free upload load is zero", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateReceivedNormal) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateReceivedNormal) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) mockPeers[0].Host.ConcurrentUploadLimit.Store(0) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "parent is disabled share data with other peers", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateReceivedNormal) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateReceivedNormal) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) mockPeers[0].Host.DisableShared = true md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "find back-to-source parent", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) - mockPeers[1].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) + mockPeers[1].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) peer.Task.BackToSourcePeers.Add(mockPeers[0].ID) peer.Task.BackToSourcePeers.Add(mockPeers[1].ID) - mockPeers[0].FSM.SetState(resource.PeerStateBackToSource) - mockPeers[1].FSM.SetState(resource.PeerStateBackToSource) + mockPeers[0].FSM.SetState(standard.PeerStateBackToSource) + mockPeers[1].FSM.SetState(standard.PeerStateBackToSource) mockPeers[0].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(1) @@ -864,7 +868,7 @@ func TestScheduling_FindCandidateParents(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[1].ID, parents[0].ID) @@ -872,10 +876,10 @@ func TestScheduling_FindCandidateParents(t *testing.T) { }, { name: "find seed peer parent", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) - mockPeers[1].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) + mockPeers[1].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) @@ -889,7 +893,7 @@ func TestScheduling_FindCandidateParents(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[1].ID, parents[0].ID) @@ -897,10 +901,10 @@ func TestScheduling_FindCandidateParents(t *testing.T) { }, { name: "parent state is PeerStateSucceeded", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateSucceeded) - mockPeers[1].FSM.SetState(resource.PeerStateSucceeded) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateSucceeded) + mockPeers[1].FSM.SetState(standard.PeerStateSucceeded) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) @@ -911,7 +915,7 @@ func TestScheduling_FindCandidateParents(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[1].ID, parents[0].ID) @@ -919,10 +923,10 @@ func TestScheduling_FindCandidateParents(t *testing.T) { }, { name: "find parent with ancestor", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) - mockPeers[1].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) + mockPeers[1].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) @@ -942,7 +946,7 @@ func TestScheduling_FindCandidateParents(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[1].ID, parents[0].ID) @@ -950,18 +954,18 @@ func TestScheduling_FindCandidateParents(t *testing.T) { }, { name: "find parent with same host", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) - mockPeers[1].FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateBackToSource) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) + mockPeers[1].FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateBackToSource) mockPeers[1].Host = peer.Host peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[0].ID, parents[0].ID) @@ -969,17 +973,17 @@ func TestScheduling_FindCandidateParents(t *testing.T) { }, { name: "find parent and fetch candidateParentLimit from manager dynconfig", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) - mockPeers[1].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) + mockPeers[1].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) peer.Task.BackToSourcePeers.Add(mockPeers[0].ID) peer.Task.BackToSourcePeers.Add(mockPeers[1].ID) - mockPeers[0].FSM.SetState(resource.PeerStateBackToSource) - mockPeers[1].FSM.SetState(resource.PeerStateBackToSource) + mockPeers[0].FSM.SetState(standard.PeerStateBackToSource) + mockPeers[1].FSM.SetState(standard.PeerStateBackToSource) mockPeers[0].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(1) @@ -989,7 +993,7 @@ func TestScheduling_FindCandidateParents(t *testing.T) { CandidateParentLimit: 3, }, nil).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Contains([]string{mockPeers[0].ID, mockPeers[1].ID, peer.ID}, parents[0].ID) @@ -997,17 +1001,17 @@ func TestScheduling_FindCandidateParents(t *testing.T) { }, { name: "candidateParents is longer than candidateParentLimit", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) - mockPeers[1].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) + mockPeers[1].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) peer.Task.BackToSourcePeers.Add(mockPeers[0].ID) peer.Task.BackToSourcePeers.Add(mockPeers[1].ID) - mockPeers[0].FSM.SetState(resource.PeerStateBackToSource) - mockPeers[1].FSM.SetState(resource.PeerStateBackToSource) + mockPeers[0].FSM.SetState(standard.PeerStateBackToSource) + mockPeers[1].FSM.SetState(standard.PeerStateBackToSource) mockPeers[0].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(1) @@ -1017,7 +1021,7 @@ func TestScheduling_FindCandidateParents(t *testing.T) { CandidateParentLimit: 1, }, nil).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(len(parents), 1) @@ -1031,24 +1035,25 @@ func TestScheduling_FindCandidateParents(t *testing.T) { ctl := gomock.NewController(t) defer ctl.Finish() dynconfig := configmocks.NewMockDynconfigInterface(ctl) - mockHost := resource.NewHost( + persistentCacheResource := persistentcache.NewMockResource(ctl) + mockHost := standard.NewHost( mockRawHost.ID, mockRawHost.IP, mockRawHost.Hostname, mockRawHost.Port, mockRawHost.DownloadPort, mockRawHost.Type) - mockTask := resource.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, resource.WithDigest(mockTaskDigest), resource.WithPieceLength(mockTaskPieceLength)) - peer := resource.NewPeer(mockPeerID, mockTask, mockHost) + mockTask := standard.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, standard.WithDigest(mockTaskDigest), standard.WithPieceLength(mockTaskPieceLength)) + peer := standard.NewPeer(mockPeerID, mockTask, mockHost) - var mockPeers []*resource.Peer + var mockPeers []*standard.Peer for i := 0; i < 11; i++ { - mockHost := resource.NewHost( + mockHost := standard.NewHost( idgen.HostIDV2("127.0.0.1", uuid.New().String(), false), mockRawHost.IP, mockRawHost.Hostname, mockRawHost.Port, mockRawHost.DownloadPort, mockRawHost.Type) - peer := resource.NewPeer(idgen.PeerIDV1(fmt.Sprintf("127.0.0.%d", i)), mockTask, mockHost) + peer := standard.NewPeer(idgen.PeerIDV1(fmt.Sprintf("127.0.0.%d", i)), mockTask, mockHost) mockPeers = append(mockPeers, peer) } blocklist := set.NewSafeSet[string]() tc.mock(peer, mockPeers, blocklist, dynconfig.EXPECT()) - scheduling := New(mockSchedulerConfig, dynconfig, mockPluginDir) + scheduling := New(mockSchedulerConfig, persistentCacheResource, dynconfig, mockPluginDir) parents, found := scheduling.FindCandidateParents(context.Background(), peer, blocklist) tc.expect(t, peer, mockPeers, parents, found) }) @@ -1058,15 +1063,15 @@ func TestScheduling_FindCandidateParents(t *testing.T) { func TestScheduling_FindParentAndCandidateParents(t *testing.T) { tests := []struct { name string - mock func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) - expect func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) + mock func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) + expect func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) }{ { name: "task peers state is failed", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateFailed) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateFailed) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.Equal(len(parents), 0) assert.False(ok) @@ -1074,64 +1079,64 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { }, { name: "task peers is empty", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "task contains only one peer and peer is itself", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "peer is in blocklist", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) blocklist.Add(mockPeers[0].ID) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "peer is bad node", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateFailed) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateFailed) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "parent is peer's descendant", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) if err := peer.Task.AddPeerEdge(peer, mockPeers[0]); err != nil { @@ -1140,40 +1145,40 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "parent free upload load is zero", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) mockPeers[0].Host.ConcurrentUploadLimit.Store(0) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "find back-to-source parent", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) - mockPeers[1].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) + mockPeers[1].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) peer.Task.BackToSourcePeers.Add(mockPeers[0].ID) peer.Task.BackToSourcePeers.Add(mockPeers[1].ID) - mockPeers[0].FSM.SetState(resource.PeerStateBackToSource) - mockPeers[1].FSM.SetState(resource.PeerStateBackToSource) + mockPeers[0].FSM.SetState(standard.PeerStateBackToSource) + mockPeers[1].FSM.SetState(standard.PeerStateBackToSource) mockPeers[0].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(1) @@ -1181,7 +1186,7 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[1].ID, parents[0].ID) @@ -1189,10 +1194,10 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { }, { name: "find seed peer parent", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) - mockPeers[1].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) + mockPeers[1].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) @@ -1206,7 +1211,7 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[1].ID, parents[0].ID) @@ -1214,10 +1219,10 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { }, { name: "parent state is PeerStateSucceeded", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateSucceeded) - mockPeers[1].FSM.SetState(resource.PeerStateSucceeded) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateSucceeded) + mockPeers[1].FSM.SetState(standard.PeerStateSucceeded) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) @@ -1228,7 +1233,7 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[1].ID, parents[0].ID) @@ -1236,10 +1241,10 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { }, { name: "find parent with ancestor", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) - mockPeers[1].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) + mockPeers[1].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) @@ -1259,7 +1264,7 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[1].ID, parents[0].ID) @@ -1267,18 +1272,18 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { }, { name: "find parent with same host", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) - mockPeers[1].FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateBackToSource) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) + mockPeers[1].FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateBackToSource) mockPeers[1].Host = peer.Host peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[0].ID, parents[0].ID) @@ -1286,17 +1291,17 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { }, { name: "find parent and fetch candidateParentLimit from manager dynconfig", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) - mockPeers[1].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) + mockPeers[1].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) peer.Task.BackToSourcePeers.Add(mockPeers[0].ID) peer.Task.BackToSourcePeers.Add(mockPeers[1].ID) - mockPeers[0].FSM.SetState(resource.PeerStateBackToSource) - mockPeers[1].FSM.SetState(resource.PeerStateBackToSource) + mockPeers[0].FSM.SetState(standard.PeerStateBackToSource) + mockPeers[1].FSM.SetState(standard.PeerStateBackToSource) mockPeers[0].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(1) @@ -1306,7 +1311,7 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { CandidateParentLimit: 3, }, nil).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Contains([]string{mockPeers[0].ID, mockPeers[1].ID, peer.ID}, parents[0].ID) @@ -1314,17 +1319,17 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { }, { name: "candidateParents is longer than candidateParentLimit", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) - mockPeers[1].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) + mockPeers[1].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) peer.Task.BackToSourcePeers.Add(mockPeers[0].ID) peer.Task.BackToSourcePeers.Add(mockPeers[1].ID) - mockPeers[0].FSM.SetState(resource.PeerStateBackToSource) - mockPeers[1].FSM.SetState(resource.PeerStateBackToSource) + mockPeers[0].FSM.SetState(standard.PeerStateBackToSource) + mockPeers[1].FSM.SetState(standard.PeerStateBackToSource) mockPeers[0].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(1) @@ -1334,7 +1339,7 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { CandidateParentLimit: 1, }, nil).Times(2) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parents []*resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parents []*standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(len(parents), 1) @@ -1348,24 +1353,25 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { ctl := gomock.NewController(t) defer ctl.Finish() dynconfig := configmocks.NewMockDynconfigInterface(ctl) - mockHost := resource.NewHost( + persistentCacheResource := persistentcache.NewMockResource(ctl) + mockHost := standard.NewHost( mockRawHost.ID, mockRawHost.IP, mockRawHost.Hostname, mockRawHost.Port, mockRawHost.DownloadPort, mockRawHost.Type) - mockTask := resource.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, resource.WithDigest(mockTaskDigest), resource.WithPieceLength(mockTaskPieceLength)) - peer := resource.NewPeer(mockPeerID, mockTask, mockHost) + mockTask := standard.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, standard.WithDigest(mockTaskDigest), standard.WithPieceLength(mockTaskPieceLength)) + peer := standard.NewPeer(mockPeerID, mockTask, mockHost) - var mockPeers []*resource.Peer + var mockPeers []*standard.Peer for i := 0; i < 11; i++ { - mockHost := resource.NewHost( + mockHost := standard.NewHost( idgen.HostIDV2("127.0.0.1", uuid.New().String(), false), mockRawHost.IP, mockRawHost.Hostname, mockRawHost.Port, mockRawHost.DownloadPort, mockRawHost.Type) - peer := resource.NewPeer(idgen.PeerIDV1(fmt.Sprintf("127.0.0.%d", i)), mockTask, mockHost) + peer := standard.NewPeer(idgen.PeerIDV1(fmt.Sprintf("127.0.0.%d", i)), mockTask, mockHost) mockPeers = append(mockPeers, peer) } blocklist := set.NewSafeSet[string]() tc.mock(peer, mockPeers, blocklist, dynconfig.EXPECT()) - scheduling := New(mockSchedulerConfig, dynconfig, mockPluginDir) + scheduling := New(mockSchedulerConfig, persistentCacheResource, dynconfig, mockPluginDir) parents, found := scheduling.FindParentAndCandidateParents(context.Background(), peer, blocklist) tc.expect(t, peer, mockPeers, parents, found) }) @@ -1375,15 +1381,15 @@ func TestScheduling_FindParentAndCandidateParents(t *testing.T) { func TestScheduling_FindSuccessParent(t *testing.T) { tests := []struct { name string - mock func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) - expect func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) + mock func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) + expect func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) }{ { name: "task peers state is failed", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateFailed) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateFailed) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) { assert := assert.New(t) assert.Nil(parent) assert.False(ok) @@ -1391,64 +1397,64 @@ func TestScheduling_FindSuccessParent(t *testing.T) { }, { name: "task peers is empty", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "task contains only one peer and peer is itself", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "peer is in blocklist", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) blocklist.Add(mockPeers[0].ID) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "peer is bad node", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateFailed) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateFailed) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "parent is peer's descendant", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) if err := peer.Task.AddPeerEdge(peer, mockPeers[0]); err != nil { @@ -1457,38 +1463,38 @@ func TestScheduling_FindSuccessParent(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "parent free upload load is zero", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) mockPeers[0].Host.ConcurrentUploadLimit.Store(0) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) { assert := assert.New(t) assert.False(ok) }, }, { name: "find back-to-source parent", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) peer.Task.BackToSourcePeers.Add(mockPeers[0].ID) peer.Task.BackToSourcePeers.Add(mockPeers[1].ID) - mockPeers[0].FSM.SetState(resource.PeerStateSucceeded) - mockPeers[1].FSM.SetState(resource.PeerStateSucceeded) + mockPeers[0].FSM.SetState(standard.PeerStateSucceeded) + mockPeers[1].FSM.SetState(standard.PeerStateSucceeded) mockPeers[0].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(1) @@ -1496,7 +1502,7 @@ func TestScheduling_FindSuccessParent(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[1].ID, parent.ID) @@ -1504,10 +1510,10 @@ func TestScheduling_FindSuccessParent(t *testing.T) { }, { name: "find seed peer parent", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateSucceeded) - mockPeers[1].FSM.SetState(resource.PeerStateSucceeded) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateSucceeded) + mockPeers[1].FSM.SetState(standard.PeerStateSucceeded) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) @@ -1521,7 +1527,7 @@ func TestScheduling_FindSuccessParent(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[1].ID, parent.ID) @@ -1529,10 +1535,10 @@ func TestScheduling_FindSuccessParent(t *testing.T) { }, { name: "find parent with ancestor", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateSucceeded) - mockPeers[1].FSM.SetState(resource.PeerStateSucceeded) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateSucceeded) + mockPeers[1].FSM.SetState(standard.PeerStateSucceeded) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) @@ -1552,7 +1558,7 @@ func TestScheduling_FindSuccessParent(t *testing.T) { md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[1].ID, parent.ID) @@ -1560,17 +1566,17 @@ func TestScheduling_FindSuccessParent(t *testing.T) { }, { name: "find parent with same host", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) - mockPeers[0].FSM.SetState(resource.PeerStateSucceeded) - mockPeers[1].FSM.SetState(resource.PeerStateSucceeded) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) + mockPeers[0].FSM.SetState(standard.PeerStateSucceeded) + mockPeers[1].FSM.SetState(standard.PeerStateSucceeded) mockPeers[1].Host = peer.Host peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) md.GetSchedulerClusterConfig().Return(types.SchedulerClusterConfig{}, errors.New("foo")).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Equal(mockPeers[0].ID, parent.ID) @@ -1578,15 +1584,15 @@ func TestScheduling_FindSuccessParent(t *testing.T) { }, { name: "find parent and fetch candidateParentLimit from manager dynconfig", - mock: func(peer *resource.Peer, mockPeers []*resource.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { - peer.FSM.SetState(resource.PeerStateRunning) + mock: func(peer *standard.Peer, mockPeers []*standard.Peer, blocklist set.SafeSet[string], md *configmocks.MockDynconfigInterfaceMockRecorder) { + peer.FSM.SetState(standard.PeerStateRunning) peer.Task.StorePeer(peer) peer.Task.StorePeer(mockPeers[0]) peer.Task.StorePeer(mockPeers[1]) peer.Task.BackToSourcePeers.Add(mockPeers[0].ID) peer.Task.BackToSourcePeers.Add(mockPeers[1].ID) - mockPeers[0].FSM.SetState(resource.PeerStateSucceeded) - mockPeers[1].FSM.SetState(resource.PeerStateSucceeded) + mockPeers[0].FSM.SetState(standard.PeerStateSucceeded) + mockPeers[1].FSM.SetState(standard.PeerStateSucceeded) mockPeers[0].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(0) mockPeers[1].FinishedPieces.Set(1) @@ -1596,7 +1602,7 @@ func TestScheduling_FindSuccessParent(t *testing.T) { FilterParentLimit: 3, }, nil).Times(1) }, - expect: func(t *testing.T, peer *resource.Peer, mockPeers []*resource.Peer, parent *resource.Peer, ok bool) { + expect: func(t *testing.T, peer *standard.Peer, mockPeers []*standard.Peer, parent *standard.Peer, ok bool) { assert := assert.New(t) assert.True(ok) assert.Contains([]string{mockPeers[0].ID, mockPeers[1].ID, peer.ID}, parent.ID) @@ -1609,38 +1615,39 @@ func TestScheduling_FindSuccessParent(t *testing.T) { ctl := gomock.NewController(t) defer ctl.Finish() dynconfig := configmocks.NewMockDynconfigInterface(ctl) - mockHost := resource.NewHost( + persistentCacheResource := persistentcache.NewMockResource(ctl) + mockHost := standard.NewHost( mockRawHost.ID, mockRawHost.IP, mockRawHost.Hostname, mockRawHost.Port, mockRawHost.DownloadPort, mockRawHost.Type) - mockTask := resource.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, resource.WithDigest(mockTaskDigest), resource.WithPieceLength(mockTaskPieceLength)) - peer := resource.NewPeer(mockPeerID, mockTask, mockHost) + mockTask := standard.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, standard.WithDigest(mockTaskDigest), standard.WithPieceLength(mockTaskPieceLength)) + peer := standard.NewPeer(mockPeerID, mockTask, mockHost) - var mockPeers []*resource.Peer + var mockPeers []*standard.Peer for i := 0; i < 11; i++ { - mockHost := resource.NewHost( + mockHost := standard.NewHost( idgen.HostIDV2("127.0.0.1", uuid.New().String(), false), mockRawHost.IP, mockRawHost.Hostname, mockRawHost.Port, mockRawHost.DownloadPort, mockRawHost.Type) - peer := resource.NewPeer(idgen.PeerIDV1(fmt.Sprintf("127.0.0.%d", i)), mockTask, mockHost) + peer := standard.NewPeer(idgen.PeerIDV1(fmt.Sprintf("127.0.0.%d", i)), mockTask, mockHost) mockPeers = append(mockPeers, peer) } blocklist := set.NewSafeSet[string]() tc.mock(peer, mockPeers, blocklist, dynconfig.EXPECT()) - scheduling := New(mockSchedulerConfig, dynconfig, mockPluginDir) + scheduling := New(mockSchedulerConfig, persistentCacheResource, dynconfig, mockPluginDir) parent, found := scheduling.FindSuccessParent(context.Background(), peer, blocklist) tc.expect(t, peer, mockPeers, parent, found) }) } } -func TestScheduling_ConstructSuccessNormalTaskResponse(t *testing.T) { +func TestScheduling_constructSuccessNormalTaskResponse(t *testing.T) { tests := []struct { name string - expect func(t *testing.T, resp *schedulerv2.AnnouncePeerResponse_NormalTaskResponse, candidateParents []*resource.Peer) + expect func(t *testing.T, resp *schedulerv2.AnnouncePeerResponse_NormalTaskResponse, candidateParents []*standard.Peer) }{ { name: "construct success normal task response", - expect: func(t *testing.T, resp *schedulerv2.AnnouncePeerResponse_NormalTaskResponse, candidateParents []*resource.Peer) { + expect: func(t *testing.T, resp *schedulerv2.AnnouncePeerResponse_NormalTaskResponse, candidateParents []*standard.Peer) { dgst := candidateParents[0].Task.Digest.String() assert := assert.New(t) @@ -1754,30 +1761,30 @@ func TestScheduling_ConstructSuccessNormalTaskResponse(t *testing.T) { t.Run(tc.name, func(t *testing.T) { ctl := gomock.NewController(t) defer ctl.Finish() - mockHost := resource.NewHost( + mockHost := standard.NewHost( mockRawHost.ID, mockRawHost.IP, mockRawHost.Hostname, mockRawHost.Port, mockRawHost.DownloadPort, mockRawHost.Type) - mockTask := resource.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, resource.WithDigest(mockTaskDigest), resource.WithPieceLength(mockTaskPieceLength)) - candidateParents := []*resource.Peer{resource.NewPeer(idgen.PeerIDV1("127.0.0.1"), mockTask, mockHost, resource.WithRange(nethttp.Range{ + mockTask := standard.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, standard.WithDigest(mockTaskDigest), standard.WithPieceLength(mockTaskPieceLength)) + candidateParents := []*standard.Peer{standard.NewPeer(idgen.PeerIDV1("127.0.0.1"), mockTask, mockHost, standard.WithRange(nethttp.Range{ Start: 1, Length: 10, }))} candidateParents[0].StorePiece(&mockPiece) candidateParents[0].Task.StorePiece(&mockPiece) - tc.expect(t, ConstructSuccessNormalTaskResponse(candidateParents), candidateParents) + tc.expect(t, constructSuccessNormalTaskResponse(candidateParents), candidateParents) }) } } -func TestScheduling_ConstructSuccessPeerPacket(t *testing.T) { +func TestScheduling_constructSuccessPeerPacket(t *testing.T) { tests := []struct { name string - expect func(t *testing.T, packet *schedulerv1.PeerPacket, parent *resource.Peer, candidateParents []*resource.Peer) + expect func(t *testing.T, packet *schedulerv1.PeerPacket, parent *standard.Peer, candidateParents []*standard.Peer) }{ { name: "construct success peer packet", - expect: func(t *testing.T, packet *schedulerv1.PeerPacket, parent *resource.Peer, candidateParents []*resource.Peer) { + expect: func(t *testing.T, packet *schedulerv1.PeerPacket, parent *standard.Peer, candidateParents []*standard.Peer) { assert := assert.New(t) assert.EqualValues(packet, &schedulerv1.PeerPacket{ TaskId: mockTaskID, @@ -1804,16 +1811,16 @@ func TestScheduling_ConstructSuccessPeerPacket(t *testing.T) { t.Run(tc.name, func(t *testing.T) { ctl := gomock.NewController(t) defer ctl.Finish() - mockHost := resource.NewHost( + mockHost := standard.NewHost( mockRawHost.ID, mockRawHost.IP, mockRawHost.Hostname, mockRawHost.Port, mockRawHost.DownloadPort, mockRawHost.Type) - mockTask := resource.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, resource.WithDigest(mockTaskDigest), resource.WithPieceLength(mockTaskPieceLength)) + mockTask := standard.NewTask(mockTaskID, mockTaskURL, mockTaskTag, mockTaskApplication, commonv2.TaskType_STANDARD, mockTaskFilteredQueryParams, mockTaskHeader, mockTaskBackToSourceLimit, standard.WithDigest(mockTaskDigest), standard.WithPieceLength(mockTaskPieceLength)) - peer := resource.NewPeer(mockPeerID, mockTask, mockHost) - parent := resource.NewPeer(idgen.PeerIDV1("127.0.0.1"), mockTask, mockHost) - candidateParents := []*resource.Peer{resource.NewPeer(idgen.PeerIDV1("127.0.0.1"), mockTask, mockHost)} + peer := standard.NewPeer(mockPeerID, mockTask, mockHost) + parent := standard.NewPeer(idgen.PeerIDV1("127.0.0.1"), mockTask, mockHost) + candidateParents := []*standard.Peer{standard.NewPeer(idgen.PeerIDV1("127.0.0.1"), mockTask, mockHost)} - tc.expect(t, ConstructSuccessPeerPacket(peer, parent, candidateParents), parent, candidateParents) + tc.expect(t, constructSuccessPeerPacket(peer, parent, candidateParents), parent, candidateParents) }) } } diff --git a/scheduler/service/service_v2.go b/scheduler/service/service_v2.go index 23256ef03e6..e30acbaca63 100644 --- a/scheduler/service/service_v2.go +++ b/scheduler/service/service_v2.go @@ -708,14 +708,9 @@ func (v *V2) AnnounceHost(ctx context.Context, req *schedulerv2.AnnounceHostRequ persistentCacheHost, loaded := v.persistentCacheResource.HostManager().Load(ctx, req.Host.GetId()) if !loaded { - options := []persistentcache.HostOption{} - if concurrentUploadLimit > 0 { - options = append(options, persistentcache.WithConcurrentUploadLimit(concurrentUploadLimit)) - } - persistentCacheHost = persistentcache.NewHost(req.Host.GetId(), req.Host.GetHostname(), req.Host.GetIp(), req.Host.GetOs(), req.Host.GetPlatform(), req.Host.GetPlatformFamily(), req.Host.GetPlatformVersion(), req.Host.GetKernelVersion(), req.Host.GetPort(), - req.Host.GetDownloadPort(), 0, 0, 0, req.Host.GetDisableShared(), types.HostType(req.Host.GetType()), + req.Host.GetDownloadPort(), req.Host.GetSchedulerClusterId(), req.Host.GetDisableShared(), types.HostType(req.Host.GetType()), persistentcache.CPU{ LogicalCount: req.Host.Cpu.GetLogicalCount(), PhysicalCount: req.Host.Cpu.GetPhysicalCount(), @@ -774,7 +769,6 @@ func (v *V2) AnnounceHost(ctx context.Context, req *schedulerv2.AnnounceHostRequ time.Now(), time.Now(), logger.WithHostID(req.Host.GetId()), - options..., ) persistentCacheHost.Log.Infof("announce new persistent cache host: %#v", req) @@ -788,6 +782,7 @@ func (v *V2) AnnounceHost(ctx context.Context, req *schedulerv2.AnnounceHostRequ persistentCacheHost.DownloadPort = req.Host.GetDownloadPort() persistentCacheHost.Type = types.HostType(req.Host.GetType()) persistentCacheHost.DisableShared = req.Host.GetDisableShared() + persistentCacheHost.SchedulerClusterID = req.Host.GetSchedulerClusterId() persistentCacheHost.OS = req.Host.GetOs() persistentCacheHost.Platform = req.Host.GetPlatform() persistentCacheHost.PlatformFamily = req.Host.GetPlatformFamily() @@ -795,10 +790,6 @@ func (v *V2) AnnounceHost(ctx context.Context, req *schedulerv2.AnnounceHostRequ persistentCacheHost.KernelVersion = req.Host.GetKernelVersion() persistentCacheHost.UpdatedAt = time.Now() - if concurrentUploadLimit > 0 { - persistentCacheHost.ConcurrentUploadLimit = concurrentUploadLimit - } - if req.Host.GetCpu() != nil { persistentCacheHost.CPU = persistentcache.CPU{ LogicalCount: req.Host.Cpu.GetLogicalCount(), @@ -1244,9 +1235,6 @@ func (v *V2) handleDownloadPeerFailedRequest(ctx context.Context, peerID string) return status.Error(codes.Internal, err.Error()) } - // Handle task with peer failed request. - peer.Task.UpdatedAt.Store(time.Now()) - // Collect DownloadPeerCount and DownloadPeerFailureCount metrics. priority := peer.CalculatePriority(v.dynconfig) metrics.DownloadPeerCount.WithLabelValues(priority.String(), peer.Task.Type.String(), @@ -1575,16 +1563,615 @@ func (v *V2) downloadTaskBySeedPeer(ctx context.Context, taskID string, download return nil } -// TODO Implement the following methods. // AnnouncePersistentCachePeer announces persistent cache peer to scheduler. func (v *V2) AnnouncePersistentCachePeer(stream schedulerv2.Scheduler_AnnouncePersistentCachePeerServer) error { if v.persistentCacheResource == nil { return status.Error(codes.FailedPrecondition, "redis is not enabled") } + ctx, cancel := context.WithCancel(stream.Context()) + defer cancel() + + for { + select { + case <-ctx.Done(): + logger.Info("context was done") + return ctx.Err() + default: + } + + req, err := stream.Recv() + if err != nil { + if err == io.EOF { + return nil + } + + logger.Errorf("receive error: %s", err.Error()) + return err + } + + log := logger.WithPeer(req.GetHostId(), req.GetTaskId(), req.GetPeerId()) + switch announcePersistentCachePeerRequest := req.GetRequest().(type) { + case *schedulerv2.AnnouncePersistentCachePeerRequest_RegisterPersistentCachePeerRequest: + registerPersistentCachePeerRequest := announcePersistentCachePeerRequest.RegisterPersistentCachePeerRequest + + log.Info("receive RegisterPersistentCachePeerRequest") + if err := v.handleRegisterPersistentCachePeerRequest(ctx, stream, req.GetHostId(), req.GetTaskId(), req.GetPeerId(), registerPersistentCachePeerRequest); err != nil { + log.Error(err) + return err + } + case *schedulerv2.AnnouncePersistentCachePeerRequest_DownloadPersistentCachePeerStartedRequest: + downloadPersistentCachePeerStartedRequest := announcePersistentCachePeerRequest.DownloadPersistentCachePeerStartedRequest + + log.Info("receive DownloadPersistentCachePeerStartedRequest") + if err := v.handleDownloadPersistentCachePeerStartedRequest(ctx, req.GetPeerId(), downloadPersistentCachePeerStartedRequest); err != nil { + log.Error(err) + return err + } + case *schedulerv2.AnnouncePersistentCachePeerRequest_ReschedulePersistentCachePeerRequest: + reschedulePersistentCachePeerRequest := announcePersistentCachePeerRequest.ReschedulePersistentCachePeerRequest + + log.Info("receive ReschedulePersistentCachePeerRequest") + if err := v.handleReschedulePersistentCachePeerRequest(ctx, stream, req.GetTaskId(), req.GetPeerId(), reschedulePersistentCachePeerRequest); err != nil { + log.Error(err) + return err + } + case *schedulerv2.AnnouncePersistentCachePeerRequest_DownloadPersistentCachePeerFinishedRequest: + downloadPersistentCachePeerFinishedRequest := announcePersistentCachePeerRequest.DownloadPersistentCachePeerFinishedRequest + + log.Info("receive DownloadPersistentCachePeerFinishedRequest") + if err := v.handleDownloadPersistentCachePeerFinishedRequest(ctx, req.GetPeerId(), downloadPersistentCachePeerFinishedRequest); err != nil { + log.Error(err) + return err + } + case *schedulerv2.AnnouncePersistentCachePeerRequest_DownloadPersistentCachePeerFailedRequest: + downloadPersistentCachePeerFailedRequest := announcePersistentCachePeerRequest.DownloadPersistentCachePeerFailedRequest + + log.Info("receive DownloadPersistentCachePeerFailedRequest") + if err := v.handleDownloadPersistentCachePeerFailedRequest(ctx, req.GetPeerId(), downloadPersistentCachePeerFailedRequest); err != nil { + log.Error(err) + return err + } + case *schedulerv2.AnnouncePersistentCachePeerRequest_DownloadPieceFinishedRequest: + downloadPieceFinishedRequest := announcePersistentCachePeerRequest.DownloadPieceFinishedRequest + + log.Info("receive DownloadPieceFinishedRequest") + if err := v.handleDownloadPersistentCachePieceFinishedRequest(ctx, req.GetPeerId(), downloadPieceFinishedRequest); err != nil { + log.Error(err) + return err + } + case *schedulerv2.AnnouncePersistentCachePeerRequest_DownloadPieceFailedRequest: + downloadPieceFailedRequest := announcePersistentCachePeerRequest.DownloadPieceFailedRequest + + log.Info("receive DownloadPieceFailedRequest") + if err := v.handleDownloadPersistentCachePieceFailedRequest(ctx, req.GetPeerId(), downloadPieceFailedRequest); err != nil { + log.Error(err) + return err + } + default: + msg := fmt.Sprintf("receive unknow request: %#v", announcePersistentCachePeerRequest) + log.Error(msg) + return status.Error(codes.FailedPrecondition, msg) + } + } +} + +// handleRegisterPersistentCachePeerRequest handles RegisterPersistentCachePeerRequest of AnnouncePersistentCachePeerRequest. +func (v *V2) handleRegisterPersistentCachePeerRequest(ctx context.Context, stream schedulerv2.Scheduler_AnnouncePersistentCachePeerServer, hostID, taskID, peerID string, req *schedulerv2.RegisterPersistentCachePeerRequest) error { + host, loaded := v.persistentCacheResource.HostManager().Load(ctx, hostID) + if !loaded { + return status.Errorf(codes.NotFound, "host %s not found", hostID) + } + + task, loaded := v.persistentCacheResource.TaskManager().Load(ctx, taskID) + if !loaded { + return status.Errorf(codes.NotFound, "task %s not found", taskID) + } + + peer := persistentcache.NewPeer(peerID, persistentcache.PeerStatePending, false, &bitset.BitSet{}, []string{}, task, host, 0, time.Now(), time.Now(), logger.WithPeer(hostID, taskID, peerID)) + + // Collect RegisterPersistentCachePeerCount metrics. + metrics.RegisterPersistentCachePeerCount.WithLabelValues(peer.Host.Type.Name()).Inc() + + // Handle task with persistent cache peer register request. + if !peer.Task.FSM.Is(persistentcache.TaskStateSucceeded) { + // Collect RegisterPersistentCachePeerFailureCount metrics. + metrics.RegisterPersistentCachePeerFailureCount.WithLabelValues(peer.Host.Type.Name()).Inc() + return status.Error(codes.Internal, "persistent cache task is not succeeded") + } else { + peer.Task.UpdatedAt = time.Now() + } + + // FSM event state transition by size scope. + sizeScope := peer.Task.SizeScope() + switch sizeScope { + case commonv2.SizeScope_EMPTY: + // Return an EmptyTaskResponse directly. + peer.Log.Info("scheduling as SizeScope_EMPTY") + if err := peer.FSM.Event(ctx, persistentcache.PeerEventRegisterEmpty); err != nil { + return status.Errorf(codes.Internal, err.Error()) + } + + // Update metadata of the persistent cache task. + if err := v.persistentCacheResource.TaskManager().Store(ctx, peer.Task); err != nil { + return status.Errorf(codes.Internal, err.Error()) + } + + // Update metadata of the persistent cache peer. + if err := v.persistentCacheResource.PeerManager().Store(ctx, peer); err != nil { + return status.Errorf(codes.Internal, err.Error()) + } + + if err := stream.Send(&schedulerv2.AnnouncePersistentCachePeerResponse{ + Response: &schedulerv2.AnnouncePersistentCachePeerResponse_EmptyPersistentCacheTaskResponse{ + EmptyPersistentCacheTaskResponse: &schedulerv2.EmptyPersistentCacheTaskResponse{}, + }, + }); err != nil { + peer.Log.Error(err) + return status.Error(codes.Internal, err.Error()) + } + + return nil + case commonv2.SizeScope_NORMAL, commonv2.SizeScope_TINY, commonv2.SizeScope_SMALL, commonv2.SizeScope_UNKNOW: + peer.Log.Info("scheduling as SizeScope_NORMAL") + if err := peer.FSM.Event(ctx, persistentcache.PeerEventRegisterNormal); err != nil { + return status.Error(codes.Internal, err.Error()) + } + + // Scheduling parent for the peer. + blocklist := set.NewSafeSet[string]() + blocklist.Add(peer.ID) + + parents, found := v.scheduling.FindCandidatePersistentCacheParents(ctx, peer, blocklist) + if !found { + // Collect RegisterPersistentCachePeerFailureCount metrics. + metrics.RegisterPersistentCachePeerFailureCount.WithLabelValues(peer.Host.Type.Name()).Inc() + return status.Error(codes.FailedPrecondition, "no candidate parents found") + } + + currentPersistentReplicaCount, err := v.persistentCacheResource.TaskManager().LoadCurrentPersistentReplicaCount(ctx, taskID) + if err != nil { + // Collect RegisterPersistentCachePeerFailureCount metrics. + metrics.RegisterPersistentCachePeerFailureCount.WithLabelValues(peer.Host.Type.Name()).Inc() + return status.Errorf(codes.Internal, err.Error()) + } + + currentReplicaCount, err := v.persistentCacheResource.TaskManager().LoadCorrentReplicaCount(ctx, taskID) + if err != nil { + // Collect RegisterPersistentCachePeerFailureCount metrics. + metrics.RegisterPersistentCachePeerFailureCount.WithLabelValues(peer.Host.Type.Name()).Inc() + return status.Errorf(codes.Internal, err.Error()) + } + + candidateParents := make([]*commonv2.PersistentCachePeer, 0, len(parents)) + for _, parent := range parents { + candidateParents = append(candidateParents, &commonv2.PersistentCachePeer{ + Id: parent.ID, + Persistent: parent.Persistent, + Cost: durationpb.New(parent.Cost), + State: parent.FSM.Current(), + Task: &commonv2.PersistentCacheTask{ + Id: parent.Task.ID, + PersistentReplicaCount: parent.Task.PersistentReplicaCount, + CurrentPersistentReplicaCount: uint64(currentPersistentReplicaCount), + CurrentReplicaCount: uint64(currentReplicaCount), + Digest: parent.Task.Digest.String(), + Tag: &parent.Task.Tag, + Application: &parent.Task.Application, + PieceLength: uint64(parent.Task.PieceLength), + ContentLength: uint64(parent.Task.ContentLength), + PieceCount: uint32(parent.Task.TotalPieceCount), + State: parent.Task.FSM.Current(), + CreatedAt: timestamppb.New(parent.Task.CreatedAt), + UpdatedAt: timestamppb.New(parent.Task.UpdatedAt), + }, + Host: &commonv2.Host{ + Id: parent.Host.ID, + Type: uint32(parent.Host.Type), + Hostname: parent.Host.Hostname, + Ip: parent.Host.IP, + Port: parent.Host.Port, + DownloadPort: parent.Host.DownloadPort, + Os: parent.Host.OS, + Platform: parent.Host.Platform, + PlatformFamily: parent.Host.PlatformFamily, + PlatformVersion: parent.Host.PlatformVersion, + KernelVersion: parent.Host.KernelVersion, + Cpu: &commonv2.CPU{ + LogicalCount: parent.Host.CPU.LogicalCount, + PhysicalCount: parent.Host.CPU.PhysicalCount, + Percent: parent.Host.CPU.Percent, + ProcessPercent: parent.Host.CPU.ProcessPercent, + Times: &commonv2.CPUTimes{ + User: parent.Host.CPU.Times.User, + System: parent.Host.CPU.Times.System, + Idle: parent.Host.CPU.Times.Idle, + Nice: parent.Host.CPU.Times.Nice, + Iowait: parent.Host.CPU.Times.Iowait, + Irq: parent.Host.CPU.Times.Irq, + Softirq: parent.Host.CPU.Times.Softirq, + Steal: parent.Host.CPU.Times.Steal, + Guest: parent.Host.CPU.Times.Guest, + GuestNice: parent.Host.CPU.Times.GuestNice, + }, + }, + Memory: &commonv2.Memory{ + Total: parent.Host.Memory.Total, + Available: parent.Host.Memory.Available, + Used: parent.Host.Memory.Used, + UsedPercent: parent.Host.Memory.UsedPercent, + ProcessUsedPercent: parent.Host.Memory.ProcessUsedPercent, + Free: parent.Host.Memory.Free, + }, + Network: &commonv2.Network{ + TcpConnectionCount: parent.Host.Network.TCPConnectionCount, + UploadTcpConnectionCount: parent.Host.Network.UploadTCPConnectionCount, + Location: &parent.Host.Network.Location, + Idc: &parent.Host.Network.IDC, + DownloadRate: parent.Host.Network.DownloadRate, + DownloadRateLimit: parent.Host.Network.DownloadRateLimit, + UploadRate: parent.Host.Network.UploadRate, + UploadRateLimit: parent.Host.Network.UploadRateLimit, + }, + Disk: &commonv2.Disk{ + Total: parent.Host.Disk.Total, + Free: parent.Host.Disk.Free, + Used: parent.Host.Disk.Used, + UsedPercent: parent.Host.Disk.UsedPercent, + InodesTotal: parent.Host.Disk.InodesTotal, + InodesUsed: parent.Host.Disk.InodesUsed, + InodesFree: parent.Host.Disk.InodesFree, + InodesUsedPercent: parent.Host.Disk.InodesUsedPercent, + ReadBandwidth: parent.Host.Disk.ReadBandwidth, + WriteBandwidth: parent.Host.Disk.WriteBandwidth, + }, + Build: &commonv2.Build{ + GitVersion: parent.Host.Build.GitVersion, + GitCommit: &parent.Host.Build.GitCommit, + GoVersion: &parent.Host.Build.GoVersion, + RustVersion: &parent.Host.Build.RustVersion, + Platform: &parent.Host.Build.Platform, + }, + SchedulerClusterId: parent.Host.SchedulerClusterID, + DisableShared: parent.Host.DisableShared, + }, + CreatedAt: timestamppb.New(parent.CreatedAt), + UpdatedAt: timestamppb.New(parent.UpdatedAt), + }) + } + + // Update metadata of the persistent cache task. + if err := v.persistentCacheResource.TaskManager().Store(ctx, peer.Task); err != nil { + return status.Errorf(codes.Internal, err.Error()) + } + + // Update metadata of the persistent cache peer. + if err := v.persistentCacheResource.PeerManager().Store(ctx, peer); err != nil { + return status.Errorf(codes.Internal, err.Error()) + } + + if err := stream.Send(&schedulerv2.AnnouncePersistentCachePeerResponse{ + Response: &schedulerv2.AnnouncePersistentCachePeerResponse_NormalPersistentCacheTaskResponse{ + NormalPersistentCacheTaskResponse: &schedulerv2.NormalPersistentCacheTaskResponse{ + CandidateParents: candidateParents, + }, + }, + }); err != nil { + peer.Log.Error(err) + return status.Error(codes.Internal, err.Error()) + } + + return nil + default: + return status.Errorf(codes.FailedPrecondition, "invalid size cope %#v", sizeScope) + } +} + +// handleDownloadPersistentCachePeerStartedRequest handles DownloadPersistentCachePeerStartedRequest of AnnouncePersistentCachePeerRequest. +func (v *V2) handleDownloadPersistentCachePeerStartedRequest(ctx context.Context, peerID string, req *schedulerv2.DownloadPersistentCachePeerStartedRequest) error { + peer, loaded := v.persistentCacheResource.PeerManager().Load(ctx, peerID) + if !loaded { + return status.Errorf(codes.NotFound, "peer %s not found", peerID) + } + + // Collect DownloadPersistentCachePeerStartedCount metrics. + metrics.DownloadPersistentCachePeerStartedCount.WithLabelValues(peer.Host.Type.Name()).Inc() + + // Handle peer with peer started request. + if err := peer.FSM.Event(ctx, persistentcache.PeerEventDownload); err != nil { + // Collect DownloadPersistentCachePeerStartedFailureCount metrics. + metrics.DownloadPersistentCachePeerStartedFailureCount.WithLabelValues(peer.Host.Type.Name()).Inc() + return status.Error(codes.Internal, err.Error()) + } + + // Update metadata of the persistent cache peer. + if err := v.persistentCacheResource.PeerManager().Store(ctx, peer); err != nil { + // Collect DownloadPersistentCachePeerStartedFailureCount metrics. + metrics.DownloadPersistentCachePeerStartedFailureCount.WithLabelValues(peer.Host.Type.Name()).Inc() + return status.Errorf(codes.Internal, err.Error()) + } + + return nil +} + +// handleReschedulePersistentCachePeerRequest handles ReschedulePersistentCachePeerRequest of AnnouncePersistentCachePeerRequest. +func (v *V2) handleReschedulePersistentCachePeerRequest(ctx context.Context, stream schedulerv2.Scheduler_AnnouncePersistentCachePeerServer, taskID, peerID string, req *schedulerv2.ReschedulePersistentCachePeerRequest) error { + peer, loaded := v.persistentCacheResource.PeerManager().Load(ctx, peerID) + if !loaded { + return status.Errorf(codes.NotFound, "peer %s not found", peerID) + } + + // Scheduling parent for the peer. + blocklist := set.NewSafeSet[string]() + blocklist.Add(peer.ID) + + // Add block parents to blocklist. + for _, parent := range peer.BlockParents { + blocklist.Add(parent) + } + + // Add block parents to blocklist by client report. + for _, parent := range req.GetCandidateParents() { + blocklist.Add(parent.GetId()) + } + peer.BlockParents = blocklist.Values() + + parents, found := v.scheduling.FindCandidatePersistentCacheParents(ctx, peer, blocklist) + if !found { + // Collect RegisterPersistentCachePeerFailureCount metrics. + metrics.RegisterPersistentCachePeerFailureCount.WithLabelValues(peer.Host.Type.Name()).Inc() + return status.Error(codes.FailedPrecondition, "no candidate parents found") + } + + currentPersistentReplicaCount, err := v.persistentCacheResource.TaskManager().LoadCurrentPersistentReplicaCount(ctx, taskID) + if err != nil { + // Collect RegisterPersistentCachePeerFailureCount metrics. + metrics.RegisterPersistentCachePeerFailureCount.WithLabelValues(peer.Host.Type.Name()).Inc() + return status.Errorf(codes.Internal, err.Error()) + } + + currentReplicaCount, err := v.persistentCacheResource.TaskManager().LoadCorrentReplicaCount(ctx, taskID) + if err != nil { + // Collect RegisterPersistentCachePeerFailureCount metrics. + metrics.RegisterPersistentCachePeerFailureCount.WithLabelValues(peer.Host.Type.Name()).Inc() + return status.Errorf(codes.Internal, err.Error()) + } + + candidateParents := make([]*commonv2.PersistentCachePeer, 0, len(parents)) + for _, parent := range parents { + candidateParents = append(candidateParents, &commonv2.PersistentCachePeer{ + Id: parent.ID, + Persistent: parent.Persistent, + Cost: durationpb.New(parent.Cost), + State: parent.FSM.Current(), + Task: &commonv2.PersistentCacheTask{ + Id: parent.Task.ID, + PersistentReplicaCount: parent.Task.PersistentReplicaCount, + CurrentPersistentReplicaCount: uint64(currentPersistentReplicaCount), + CurrentReplicaCount: uint64(currentReplicaCount), + Digest: parent.Task.Digest.String(), + Tag: &parent.Task.Tag, + Application: &parent.Task.Application, + PieceLength: uint64(parent.Task.PieceLength), + ContentLength: uint64(parent.Task.ContentLength), + PieceCount: uint32(parent.Task.TotalPieceCount), + State: parent.Task.FSM.Current(), + CreatedAt: timestamppb.New(parent.Task.CreatedAt), + UpdatedAt: timestamppb.New(parent.Task.UpdatedAt), + }, + Host: &commonv2.Host{ + Id: parent.Host.ID, + Type: uint32(parent.Host.Type), + Hostname: parent.Host.Hostname, + Ip: parent.Host.IP, + Port: parent.Host.Port, + DownloadPort: parent.Host.DownloadPort, + Os: parent.Host.OS, + Platform: parent.Host.Platform, + PlatformFamily: parent.Host.PlatformFamily, + PlatformVersion: parent.Host.PlatformVersion, + KernelVersion: parent.Host.KernelVersion, + Cpu: &commonv2.CPU{ + LogicalCount: parent.Host.CPU.LogicalCount, + PhysicalCount: parent.Host.CPU.PhysicalCount, + Percent: parent.Host.CPU.Percent, + ProcessPercent: parent.Host.CPU.ProcessPercent, + Times: &commonv2.CPUTimes{ + User: parent.Host.CPU.Times.User, + System: parent.Host.CPU.Times.System, + Idle: parent.Host.CPU.Times.Idle, + Nice: parent.Host.CPU.Times.Nice, + Iowait: parent.Host.CPU.Times.Iowait, + Irq: parent.Host.CPU.Times.Irq, + Softirq: parent.Host.CPU.Times.Softirq, + Steal: parent.Host.CPU.Times.Steal, + Guest: parent.Host.CPU.Times.Guest, + GuestNice: parent.Host.CPU.Times.GuestNice, + }, + }, + Memory: &commonv2.Memory{ + Total: parent.Host.Memory.Total, + Available: parent.Host.Memory.Available, + Used: parent.Host.Memory.Used, + UsedPercent: parent.Host.Memory.UsedPercent, + ProcessUsedPercent: parent.Host.Memory.ProcessUsedPercent, + Free: parent.Host.Memory.Free, + }, + Network: &commonv2.Network{ + TcpConnectionCount: parent.Host.Network.TCPConnectionCount, + UploadTcpConnectionCount: parent.Host.Network.UploadTCPConnectionCount, + Location: &parent.Host.Network.Location, + Idc: &parent.Host.Network.IDC, + DownloadRate: parent.Host.Network.DownloadRate, + DownloadRateLimit: parent.Host.Network.DownloadRateLimit, + UploadRate: parent.Host.Network.UploadRate, + UploadRateLimit: parent.Host.Network.UploadRateLimit, + }, + Disk: &commonv2.Disk{ + Total: parent.Host.Disk.Total, + Free: parent.Host.Disk.Free, + Used: parent.Host.Disk.Used, + UsedPercent: parent.Host.Disk.UsedPercent, + InodesTotal: parent.Host.Disk.InodesTotal, + InodesUsed: parent.Host.Disk.InodesUsed, + InodesFree: parent.Host.Disk.InodesFree, + InodesUsedPercent: parent.Host.Disk.InodesUsedPercent, + ReadBandwidth: parent.Host.Disk.ReadBandwidth, + WriteBandwidth: parent.Host.Disk.WriteBandwidth, + }, + Build: &commonv2.Build{ + GitVersion: parent.Host.Build.GitVersion, + GitCommit: &parent.Host.Build.GitCommit, + GoVersion: &parent.Host.Build.GoVersion, + RustVersion: &parent.Host.Build.RustVersion, + Platform: &parent.Host.Build.Platform, + }, + SchedulerClusterId: parent.Host.SchedulerClusterID, + DisableShared: parent.Host.DisableShared, + }, + CreatedAt: timestamppb.New(parent.CreatedAt), + UpdatedAt: timestamppb.New(parent.UpdatedAt), + }) + } + + // Update metadata of the persistent cache peer. + if err := v.persistentCacheResource.PeerManager().Store(ctx, peer); err != nil { + return status.Errorf(codes.Internal, err.Error()) + } + + if err := stream.Send(&schedulerv2.AnnouncePersistentCachePeerResponse{ + Response: &schedulerv2.AnnouncePersistentCachePeerResponse_NormalPersistentCacheTaskResponse{ + NormalPersistentCacheTaskResponse: &schedulerv2.NormalPersistentCacheTaskResponse{ + CandidateParents: candidateParents, + }, + }, + }); err != nil { + peer.Log.Error(err) + return status.Error(codes.Internal, err.Error()) + } + return nil } +// handleDownloadPersistentCachePeerFinishedRequest handles DownloadPersistentCachePeerFinishedRequest of AnnouncePersistentCachePeerRequest. +func (v *V2) handleDownloadPersistentCachePeerFinishedRequest(ctx context.Context, peerID string, req *schedulerv2.DownloadPersistentCachePeerFinishedRequest) error { + peer, loaded := v.persistentCacheResource.PeerManager().Load(ctx, peerID) + if !loaded { + return status.Errorf(codes.NotFound, "peer %s not found", peerID) + } + + // Handle peer with peer finished request. + peer.Cost = time.Since(peer.CreatedAt) + if err := peer.FSM.Event(ctx, persistentcache.PeerEventSucceeded); err != nil { + return status.Error(codes.Internal, err.Error()) + } + + // Update metadata of the persistent cache peer. + if err := v.persistentCacheResource.PeerManager().Store(ctx, peer); err != nil { + return status.Errorf(codes.Internal, err.Error()) + } + + // Collect DownloadPersistentCachePeerCount metrics. + metrics.DownloadPersistentCachePeerCount.WithLabelValues(peer.Host.Type.Name()).Inc() + return nil +} + +// handleDownloadPersistentCachePeerFailedRequest handles DownloadPersistentCachePeerFailedRequest of AnnouncePersistentCachePeerRequest. +func (v *V2) handleDownloadPersistentCachePeerFailedRequest(ctx context.Context, peerID string, req *schedulerv2.DownloadPersistentCachePeerFailedRequest) error { + peer, loaded := v.persistentCacheResource.PeerManager().Load(ctx, peerID) + if !loaded { + return status.Errorf(codes.NotFound, "peer %s not found", peerID) + } + + // Handle peer with peer failed request. + if err := peer.FSM.Event(ctx, persistentcache.PeerEventFailed); err != nil { + return status.Error(codes.Internal, err.Error()) + } + + // Update metadata of the persistent cache peer. + if err := v.persistentCacheResource.PeerManager().Store(ctx, peer); err != nil { + return status.Errorf(codes.Internal, err.Error()) + } + + // Collect DownloadPersistentCachePeerCount and DownloadPersistentCachePeerFailureCount metrics. + metrics.DownloadPersistentCachePeerCount.WithLabelValues(peer.Host.Type.Name()).Inc() + metrics.DownloadPersistentCachePeerFailureCount.WithLabelValues(peer.Host.Type.Name()).Inc() + return nil +} + +// handleDownloadPersistentCachePieceFinishedRequest handles DownloadPersistentCachePieceFinishedRequest of AnnouncePersistentCachePeerRequest. +func (v *V2) handleDownloadPersistentCachePieceFinishedRequest(ctx context.Context, peerID string, req *schedulerv2.DownloadPieceFinishedRequest) error { + peer, loaded := v.persistentCacheResource.PeerManager().Load(ctx, peerID) + if !loaded { + return status.Errorf(codes.NotFound, "peer %s not found", peerID) + } + + piece := req.GetPiece() + peer.FinishedPieces.Set(uint(piece.GetNumber())) + peer.UpdatedAt = time.Now() + + parent, loadedParent := v.persistentCacheResource.PeerManager().Load(ctx, piece.GetParentId()) + if loadedParent { + parent.UpdatedAt = time.Now() + parent.Host.UpdatedAt = time.Now() + } + + // Update metadata of the persistent cache peer. + if err := v.persistentCacheResource.PeerManager().Store(ctx, peer); err != nil { + return status.Errorf(codes.Internal, err.Error()) + } + + // Update metadata of the persistent cache peer's parent. + if err := v.persistentCacheResource.PeerManager().Store(ctx, parent); err != nil { + return status.Errorf(codes.Internal, err.Error()) + } + + // Collect DownloadPersistentCachePieceCount metrics. + metrics.DownloadPersistentCachePieceCount.WithLabelValues(peer.Host.Type.Name()).Inc() + return nil +} + +// handleDownloadPersistentCachePieceFailedRequest handles DownloadPersistentCachePieceFailedRequest of AnnouncePersistentCachePeerRequest. +func (v *V2) handleDownloadPersistentCachePieceFailedRequest(ctx context.Context, peerID string, req *schedulerv2.DownloadPieceFailedRequest) error { + peer, loaded := v.persistentCacheResource.PeerManager().Load(ctx, peerID) + if !loaded { + return status.Errorf(codes.NotFound, "peer %s not found", peerID) + } + + // Collect DownloadPersistentCachePieceCount and DownloadPersistentCachePieceFailureCount metrics. + metrics.DownloadPersistentCachePieceCount.WithLabelValues(peer.Host.Type.Name()).Inc() + metrics.DownloadPersistentCachePieceFailureCount.WithLabelValues(peer.Host.Type.Name()).Inc() + + if req.Temporary { + // Handle peer with piece temporary failed request. + peer.UpdatedAt = time.Now() + blocklist := set.NewSafeSet[string]() + blocklist.Add(peer.ID) + + // Add block parents to blocklist. + for _, parent := range peer.BlockParents { + blocklist.Add(parent) + } + + // Add parent to blocklist, because download piece from parent failed. + blocklist.Add(req.GetParentId()) + + peer.BlockParents = blocklist.Values() + if parent, loaded := v.resource.PeerManager().Load(req.GetParentId()); loaded { + parent.Host.UploadFailedCount.Inc() + } + + // Update metadata of the persistent cache peer. + if err := v.persistentCacheResource.PeerManager().Store(ctx, peer); err != nil { + return status.Errorf(codes.Internal, err.Error()) + } + + return nil + } + + return status.Error(codes.FailedPrecondition, "download piece failed") +} + // StatPersistentCachePeer checks information of persistent cache peer. func (v *V2) StatPersistentCachePeer(ctx context.Context, req *schedulerv2.StatPersistentCachePeerRequest) (*commonv2.PersistentCachePeer, error) { if v.persistentCacheResource == nil { diff --git a/scheduler/service/service_v2_test.go b/scheduler/service/service_v2_test.go index f010a0244d8..1701af57917 100644 --- a/scheduler/service/service_v2_test.go +++ b/scheduler/service/service_v2_test.go @@ -152,24 +152,25 @@ var ( mockInterval = durationpb.New(5 * time.Minute).AsDuration() mockRawPersistentCacheHost = persistentcache.Host{ - ID: mockHostID, - Type: pkgtypes.HostTypeNormal, - Hostname: "foo", - IP: "127.0.0.1", - Port: 8003, - DownloadPort: 8001, - OS: "darwin", - Platform: "darwin", - PlatformFamily: "Standalone Workstation", - PlatformVersion: "11.1", - KernelVersion: "20.2.0", - CPU: mockPersistentCacheCPU, - Memory: mockPersistentCacheMemory, - Network: mockPersistentCacheNetwork, - Disk: mockPersistentCacheDisk, - Build: mockPersistentCacheBuild, - CreatedAt: time.Now(), - UpdatedAt: time.Now(), + ID: mockHostID, + Type: pkgtypes.HostTypeNormal, + Hostname: "foo", + IP: "127.0.0.1", + Port: 8003, + DownloadPort: 8001, + OS: "darwin", + Platform: "darwin", + PlatformFamily: "Standalone Workstation", + PlatformVersion: "11.1", + KernelVersion: "20.2.0", + CPU: mockPersistentCacheCPU, + Memory: mockPersistentCacheMemory, + Network: mockPersistentCacheNetwork, + Disk: mockPersistentCacheDisk, + Build: mockPersistentCacheBuild, + SchedulerClusterID: 1, + CreatedAt: time.Now(), + UpdatedAt: time.Now(), } mockPersistentCacheCPU = persistentcache.CPU{ @@ -727,10 +728,6 @@ func TestServiceV2_AnnounceHost(t *testing.T) { assert.EqualValues(host.Disk, mockPersistentCacheDisk) assert.EqualValues(host.Build, mockPersistentCacheBuild) assert.EqualValues(host.AnnounceInterval, mockPersistentCacheInterval) - assert.Equal(host.ConcurrentUploadLimit, int32(10)) - assert.Equal(host.ConcurrentUploadCount, int32(0)) - assert.Equal(host.UploadCount, int64(0)) - assert.Equal(host.UploadFailedCount, int64(0)) assert.NotEqual(host.CreatedAt.Nanosecond(), 0) assert.NotEqual(host.UpdatedAt.Nanosecond(), 0) assert.NotNil(host.Log) @@ -869,10 +866,6 @@ func TestServiceV2_AnnounceHost(t *testing.T) { assert.EqualValues(host.Disk, mockPersistentCacheDisk) assert.EqualValues(host.Build, mockPersistentCacheBuild) assert.EqualValues(host.AnnounceInterval, mockPersistentCacheInterval) - assert.Equal(host.ConcurrentUploadLimit, int32(200)) - assert.Equal(host.ConcurrentUploadCount, int32(0)) - assert.Equal(host.UploadCount, int64(0)) - assert.Equal(host.UploadFailedCount, int64(0)) assert.NotEqual(host.CreatedAt.Nanosecond(), 0) assert.NotEqual(host.UpdatedAt.Nanosecond(), 0) assert.NotNil(host.Log) @@ -981,10 +974,6 @@ func TestServiceV2_AnnounceHost(t *testing.T) { assert.EqualValues(host.Disk, mockPersistentCacheDisk) assert.EqualValues(host.Build, mockPersistentCacheBuild) assert.EqualValues(host.AnnounceInterval, mockPersistentCacheInterval) - assert.Equal(host.ConcurrentUploadLimit, int32(10)) - assert.Equal(host.ConcurrentUploadCount, int32(0)) - assert.Equal(host.UploadCount, int64(0)) - assert.Equal(host.UploadFailedCount, int64(0)) assert.NotEqual(host.CreatedAt.Nanosecond(), 0) assert.NotEqual(host.UpdatedAt.Nanosecond(), 0) assert.NotNil(host.Log) @@ -1119,10 +1108,6 @@ func TestServiceV2_AnnounceHost(t *testing.T) { assert.EqualValues(host.Disk, mockPersistentCacheDisk) assert.EqualValues(host.Build, mockPersistentCacheBuild) assert.EqualValues(host.AnnounceInterval, mockPersistentCacheInterval) - assert.Equal(host.ConcurrentUploadLimit, int32(200)) - assert.Equal(host.ConcurrentUploadCount, int32(0)) - assert.Equal(host.UploadCount, int64(0)) - assert.Equal(host.UploadFailedCount, int64(0)) assert.NotEqual(host.CreatedAt.Nanosecond(), 0) assert.NotEqual(host.UpdatedAt.Nanosecond(), 0) assert.NotNil(host.Log) @@ -1287,10 +1272,6 @@ func TestServiceV2_AnnounceHost(t *testing.T) { assert.EqualValues(host.Disk, mockPersistentCacheDisk) assert.EqualValues(host.Build, mockPersistentCacheBuild) assert.EqualValues(host.AnnounceInterval, mockPersistentCacheInterval) - assert.Equal(host.ConcurrentUploadLimit, int32(10)) - assert.Equal(host.ConcurrentUploadCount, int32(0)) - assert.Equal(host.UploadCount, int64(0)) - assert.Equal(host.UploadFailedCount, int64(0)) assert.NotEqual(host.CreatedAt.Nanosecond(), 0) assert.NotEqual(host.UpdatedAt.Nanosecond(), 0) assert.NotNil(host.Log) @@ -1399,10 +1380,6 @@ func TestServiceV2_AnnounceHost(t *testing.T) { assert.EqualValues(host.Disk, mockPersistentCacheDisk) assert.EqualValues(host.Build, mockPersistentCacheBuild) assert.EqualValues(host.AnnounceInterval, mockPersistentCacheInterval) - assert.Equal(host.ConcurrentUploadLimit, int32(200)) - assert.Equal(host.ConcurrentUploadCount, int32(0)) - assert.Equal(host.UploadCount, int64(0)) - assert.Equal(host.UploadFailedCount, int64(0)) assert.NotEqual(host.CreatedAt.Nanosecond(), 0) assert.NotEqual(host.UpdatedAt.Nanosecond(), 0) assert.NotNil(host.Log) @@ -1432,8 +1409,7 @@ func TestServiceV2_AnnounceHost(t *testing.T) { persistentCacheHost := persistentcache.NewHost( mockRawPersistentCacheHost.ID, mockRawPersistentCacheHost.Hostname, mockRawPersistentCacheHost.IP, mockRawPersistentCacheHost.OS, mockRawPersistentCacheHost.Platform, mockRawPersistentCacheHost.PlatformFamily, mockRawPersistentCacheHost.PlatformVersion, mockRawPersistentCacheHost.KernelVersion, - mockRawPersistentCacheHost.Port, mockRawPersistentCacheHost.DownloadPort, mockRawPersistentCacheHost.ConcurrentUploadCount, - mockRawPersistentCacheHost.UploadCount, mockRawPersistentCacheHost.UploadFailedCount, mockRawPersistentCacheHost.DisableShared, pkgtypes.HostType(mockRawPersistentCacheHost.Type), + mockRawPersistentCacheHost.Port, mockRawPersistentCacheHost.DownloadPort, mockRawPersistentCacheHost.SchedulerClusterID, mockRawPersistentCacheHost.DisableShared, pkgtypes.HostType(mockRawPersistentCacheHost.Type), mockRawPersistentCacheHost.CPU, mockRawPersistentCacheHost.Memory, mockRawPersistentCacheHost.Network, mockRawPersistentCacheHost.Disk, mockRawPersistentCacheHost.Build, mockRawPersistentCacheHost.AnnounceInterval, mockRawPersistentCacheHost.CreatedAt, mockRawPersistentCacheHost.UpdatedAt, mockRawHost.Log)