refactor: scheduler evaluator (#805)
* refactor: scheduler evaluator Signed-off-by: Gaius <gaius.qi@gmail.com>
This commit is contained in:
parent
dc2835a454
commit
d01eef6b41
|
|
@ -3,17 +3,13 @@
|
|||
|
||||
# scheduler policy configuration
|
||||
scheduler:
|
||||
# abtest whether to open ab test
|
||||
# default: false
|
||||
abtest: false
|
||||
# aevaluator specify which evaluator is used when taskID suffix with _A
|
||||
# only take effect when abtest is true
|
||||
# default: ""
|
||||
aevaluator: ""
|
||||
# bevaluator specify which evaluator is used when taskID suffix with _B
|
||||
# only take effect when abtest is true
|
||||
# default: ""
|
||||
bevaluator: ""
|
||||
# algorithm configuration to use different scheduling algorithms,
|
||||
# default configuration supports "default" and "ml"
|
||||
# "default" is the rule-based scheduling algorithm, "ml" is the machine learning scheduling algorithm
|
||||
# It also supports user plugin extension, the algorithm value is "plugin",
|
||||
# and the compiled `d7y-scheduler-plugin-evaluator.so` file is added to
|
||||
# the dragonfly working directory plugins
|
||||
algorithm: default
|
||||
# workerNum is the number of goroutines that perform scheduling tasks
|
||||
# default:
|
||||
workerNum: 12
|
||||
|
|
|
|||
|
|
@ -2,17 +2,11 @@
|
|||
|
||||
# scheduler 调度策略配置
|
||||
scheduler:
|
||||
# abtest 是否开启ab测试
|
||||
# default: false
|
||||
abtest: false
|
||||
# aevaluator 指定当开启AB测试时,后缀为 _A 的 taskID 使用的 evaluator 计算方法
|
||||
# 只有当开启了 AB 测试的情况下该配置才生效
|
||||
# default: ""
|
||||
aevaluator: ""
|
||||
# bevaluator 指定当开启AB测试时,后缀为 _B 的 taskID 使用的 evaluator 计算方法
|
||||
# 只有当开启了 AB 测试的情况下该配置才生效
|
||||
# default: ""
|
||||
bevaluator: ""
|
||||
# algorithm 使用不同调度算法配置,当前默认支持 "default" 和 "ml" 两种类型
|
||||
# "default" 为基于规则的调度算法, "ml" 为基于机器学习的调度算法
|
||||
# 也支持用户 plugin 扩展的方式,值为 "plugin"
|
||||
# 并且在 dragonfly 工作目录 plugins 中添加编译好的 `d7y-scheduler-plugin-evaluator.so` 文件
|
||||
algorithm: default
|
||||
# workerNum 执行调度任务处理的 goroutine 数量
|
||||
# default: 默认机器可用的 CPU 数量
|
||||
workerNum: 12
|
||||
|
|
|
|||
|
|
@ -42,8 +42,9 @@ const (
|
|||
type PluginType string
|
||||
|
||||
const (
|
||||
PluginTypeResource = PluginType("resource")
|
||||
PluginTypeManager = PluginType("manager")
|
||||
PluginTypeResource = PluginType("resource")
|
||||
PluginTypeManager = PluginType("manager")
|
||||
PluginTypeScheduler = PluginType("scheduler")
|
||||
)
|
||||
|
||||
type PluginInitFunc func(option map[string]string) (plugin interface{}, meta map[string]string, err error)
|
||||
|
|
|
|||
|
|
@ -17,7 +17,6 @@
|
|||
package idgen
|
||||
|
||||
import (
|
||||
"hash/crc32"
|
||||
"strings"
|
||||
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
|
|
@ -25,12 +24,7 @@ import (
|
|||
"d7y.io/dragonfly/v2/pkg/util/net/urlutils"
|
||||
)
|
||||
|
||||
const (
|
||||
TwinsASuffix = "_A"
|
||||
TwinsBSuffix = "_B"
|
||||
)
|
||||
|
||||
// GenerateTaskID generates a taskId.
|
||||
// TaskID generates a taskId.
|
||||
// filter is separated by & character.
|
||||
func TaskID(url string, meta *base.UrlMeta) string {
|
||||
var data []string
|
||||
|
|
@ -58,16 +52,3 @@ func TaskID(url string, meta *base.UrlMeta) string {
|
|||
|
||||
return digestutils.Sha256(data...)
|
||||
}
|
||||
|
||||
// GenerateTwinsTaskId used A/B testing
|
||||
func TwinsTaskID(url string, meta *base.UrlMeta, peerID string) string {
|
||||
taskID := TaskID(url, meta)
|
||||
|
||||
if crc32.ChecksumIEEE([]byte(peerID))&1 == 0 {
|
||||
taskID += TwinsASuffix
|
||||
} else {
|
||||
taskID += TwinsBSuffix
|
||||
}
|
||||
|
||||
return taskID
|
||||
}
|
||||
|
|
|
|||
|
|
@ -85,78 +85,3 @@ func TestTaskID(t *testing.T) {
|
|||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestTwinsTaskID(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
url string
|
||||
meta *base.UrlMeta
|
||||
peerID string
|
||||
expect func(t *testing.T, d interface{})
|
||||
}{
|
||||
{
|
||||
name: "generate taskID with url",
|
||||
url: "https://example.com",
|
||||
meta: &base.UrlMeta{},
|
||||
peerID: "foo",
|
||||
expect: func(t *testing.T, d interface{}) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal("100680ad546ce6a577f42f52df33b4cfdca756859e664b8d7de329b150d09ce9_B", d)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "generate taskID with meta",
|
||||
url: "https://example.com",
|
||||
meta: &base.UrlMeta{
|
||||
Range: "foo",
|
||||
Digest: "bar",
|
||||
},
|
||||
peerID: "foo",
|
||||
expect: func(t *testing.T, d interface{}) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal("aeee0e0a2a0c75130582641353c539aaf9011a0088b31347f7588e70e449a3e0_B", d)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "generate taskID with filter",
|
||||
url: "https://example.com?foo=foo&bar=bar",
|
||||
meta: &base.UrlMeta{
|
||||
Filter: "foo&bar",
|
||||
},
|
||||
peerID: "foo",
|
||||
expect: func(t *testing.T, d interface{}) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal("100680ad546ce6a577f42f52df33b4cfdca756859e664b8d7de329b150d09ce9_B", d)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "generate taskID with tag",
|
||||
url: "https://example.com",
|
||||
meta: &base.UrlMeta{
|
||||
Tag: "foo",
|
||||
},
|
||||
peerID: "foo",
|
||||
expect: func(t *testing.T, d interface{}) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal("2773851c628744fb7933003195db436ce397c1722920696c4274ff804d86920b_B", d)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "generate twinsA taskID",
|
||||
url: "https://example.com",
|
||||
meta: &base.UrlMeta{},
|
||||
peerID: "bar",
|
||||
expect: func(t *testing.T, d interface{}) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal("100680ad546ce6a577f42f52df33b4cfdca756859e664b8d7de329b150d09ce9_A", d)
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
data := TwinsTaskID(tc.url, tc.meta, tc.peerID)
|
||||
tc.expect(t, data)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -18,9 +18,20 @@
|
|||
package mathutils
|
||||
|
||||
import (
|
||||
"math"
|
||||
"strconv"
|
||||
)
|
||||
|
||||
const float64EqualityThreshold = 1e-9
|
||||
|
||||
func MaxInt(a, b int) int {
|
||||
if a < b {
|
||||
return a
|
||||
}
|
||||
|
||||
return b
|
||||
}
|
||||
|
||||
func MaxInt32(a, b int32) int32 {
|
||||
if a > b {
|
||||
return a
|
||||
|
|
@ -29,6 +40,14 @@ func MaxInt32(a, b int32) int32 {
|
|||
return b
|
||||
}
|
||||
|
||||
func MinInt(a, b int) int {
|
||||
if a < b {
|
||||
return a
|
||||
}
|
||||
|
||||
return b
|
||||
}
|
||||
|
||||
func MinInt32(a, b int32) int32 {
|
||||
if a < b {
|
||||
return a
|
||||
|
|
@ -68,3 +87,7 @@ func IsInteger(value string) bool {
|
|||
|
||||
return false
|
||||
}
|
||||
|
||||
func EqualFloat64(a, b float64) bool {
|
||||
return math.Abs(a-b) <= float64EqualityThreshold
|
||||
}
|
||||
|
|
|
|||
|
|
@ -43,9 +43,7 @@ type Config struct {
|
|||
func New() *Config {
|
||||
return &Config{
|
||||
Scheduler: &SchedulerConfig{
|
||||
ABTest: false,
|
||||
AEvaluator: "",
|
||||
BEvaluator: "",
|
||||
Algorithm: "default",
|
||||
WorkerNum: runtime.GOMAXPROCS(0),
|
||||
BackSourceCount: 3,
|
||||
AccessWindow: 3 * time.Minute,
|
||||
|
|
@ -179,9 +177,7 @@ type DynConfig struct {
|
|||
}
|
||||
|
||||
type SchedulerConfig struct {
|
||||
ABTest bool `yaml:"abtest" mapstructure:"abtest"`
|
||||
AEvaluator string `yaml:"aevaluator" mapstructure:"aevaluator"`
|
||||
BEvaluator string `yaml:"bevaluator" mapstructure:"bevaluator"`
|
||||
Algorithm string `yaml:"algorithm" mapstructure:"algorithm"`
|
||||
WorkerNum int `yaml:"workerNum" mapstructure:"workerNum"`
|
||||
BackSourceCount int32 `yaml:"backSourceCount" mapstructure:"backSourceCount"`
|
||||
// AccessWindow should less than CDN task expireTime
|
||||
|
|
|
|||
|
|
@ -39,10 +39,8 @@ func TestSchedulerConfig_Load(t *testing.T) {
|
|||
CDNDirPath: "tmp",
|
||||
},
|
||||
Scheduler: &SchedulerConfig{
|
||||
ABTest: true,
|
||||
AEvaluator: "a-evaluator",
|
||||
BEvaluator: "b-evaluator",
|
||||
WorkerNum: 8,
|
||||
Algorithm: "default",
|
||||
WorkerNum: 8,
|
||||
},
|
||||
Server: &ServerConfig{
|
||||
IP: "127.0.0.1",
|
||||
|
|
|
|||
|
|
@ -4,13 +4,11 @@ dynconfig:
|
|||
cdnDirPath: tmp
|
||||
|
||||
scheduler:
|
||||
abtest: true
|
||||
aevaluator: "a-evaluator"
|
||||
bevaluator: "b-evaluator"
|
||||
workerNum: 8
|
||||
workerJobPoolSize: 10000
|
||||
senderNum: 10
|
||||
senderJobPoolSize: 10000
|
||||
algorithm: default
|
||||
|
||||
server:
|
||||
ip: "127.0.0.1"
|
||||
|
|
|
|||
|
|
@ -1,146 +0,0 @@
|
|||
/*
|
||||
* Copyright 2020 The Dragonfly Authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package basic
|
||||
|
||||
import (
|
||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||
"d7y.io/dragonfly/v2/scheduler/config"
|
||||
"d7y.io/dragonfly/v2/scheduler/core/evaluator"
|
||||
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
||||
)
|
||||
|
||||
type baseEvaluator struct {
|
||||
cfg *config.SchedulerConfig
|
||||
}
|
||||
|
||||
func NewEvaluator(cfg *config.SchedulerConfig) evaluator.Evaluator {
|
||||
eval := &baseEvaluator{cfg: cfg}
|
||||
logger.Debugf("create basic evaluator successfully")
|
||||
return eval
|
||||
}
|
||||
|
||||
func (eval *baseEvaluator) NeedAdjustParent(peer *supervisor.Peer) bool {
|
||||
if peer.Host.IsCDN {
|
||||
return false
|
||||
}
|
||||
|
||||
parent, ok := peer.GetParent()
|
||||
if !ok && !peer.IsDone() {
|
||||
logger.Debugf("peer %s need adjust parent because it has not parent and status is %s", peer.ID, peer.GetStatus())
|
||||
return true
|
||||
}
|
||||
|
||||
// TODO Check whether the parent node is in the blacklist
|
||||
if ok && eval.IsBadNode(parent) {
|
||||
logger.Debugf("peer %s need adjust parent because it current parent is bad", peer.ID)
|
||||
return true
|
||||
}
|
||||
|
||||
if ok && parent.IsLeave() {
|
||||
logger.Debugf("peer %s need adjust parent because it current parent is status is leave", peer.ID)
|
||||
return true
|
||||
}
|
||||
|
||||
costs := peer.GetPieceCosts()
|
||||
if len(costs) < 4 {
|
||||
return false
|
||||
}
|
||||
|
||||
avgCost, lastCost := getAvgAndLastCost(costs, 4)
|
||||
// TODO adjust policy
|
||||
if (avgCost * 20) < lastCost {
|
||||
logger.Debugf("peer %s need adjust parent because it latest download cost is too time consuming", peer.ID)
|
||||
return true
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
func (eval *baseEvaluator) IsBadNode(peer *supervisor.Peer) bool {
|
||||
if peer.IsBad() {
|
||||
logger.Debugf("peer %s is bad because it's status is %s", peer.ID, peer.GetStatus())
|
||||
return true
|
||||
}
|
||||
|
||||
costs := peer.GetPieceCosts()
|
||||
if len(costs) < 4 {
|
||||
return false
|
||||
}
|
||||
|
||||
avgCost, lastCost := getAvgAndLastCost(costs, 4)
|
||||
if avgCost*40 < lastCost && !peer.Host.IsCDN {
|
||||
logger.Debugf("peer %s is bad because recent pieces have taken too long to download avg[%d] last[%d]", peer.ID, avgCost, lastCost)
|
||||
return true
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
// Evaluate The bigger, the better
|
||||
func (eval *baseEvaluator) Evaluate(parent *supervisor.Peer, child *supervisor.Peer) float64 {
|
||||
profits := getProfits(parent, child)
|
||||
|
||||
load := getHostLoad(parent.Host)
|
||||
|
||||
dist := getAffinity(parent, child)
|
||||
|
||||
return profits * load * dist
|
||||
}
|
||||
|
||||
func getAvgAndLastCost(list []int, splitPos int) (avgCost, lastCost int) {
|
||||
length := len(list)
|
||||
totalCost := 0
|
||||
for i, cost := range list {
|
||||
totalCost += cost
|
||||
if length-i <= splitPos {
|
||||
lastCost += cost
|
||||
}
|
||||
}
|
||||
avgCost = totalCost / length
|
||||
lastCost = lastCost / splitPos
|
||||
return
|
||||
}
|
||||
|
||||
// getProfits 0.0~unlimited larger and better
|
||||
func getProfits(dst *supervisor.Peer, src *supervisor.Peer) float64 {
|
||||
diff := dst.TotalPieceCount.Load() - src.TotalPieceCount.Load()
|
||||
depth := dst.GetTreeDepth()
|
||||
|
||||
return float64(int(diff+1)*src.GetTreeNodeCount()) / float64(depth*depth)
|
||||
}
|
||||
|
||||
// getHostLoad 0.0~1.0 larger and better
|
||||
func getHostLoad(host *supervisor.Host) float64 {
|
||||
return 1.0 - host.GetUploadLoadPercent()
|
||||
}
|
||||
|
||||
// getAffinity 0.0~1.0 larger and better
|
||||
func getAffinity(dst *supervisor.Peer, src *supervisor.Peer) float64 {
|
||||
hostDist := 40.0
|
||||
if dst.Host == src.Host {
|
||||
hostDist = 0.0
|
||||
} else {
|
||||
if src.Host.NetTopology != "" && dst.Host.NetTopology == src.Host.NetTopology {
|
||||
hostDist = 10.0
|
||||
} else if src.Host.IDC != "" && dst.Host.IDC == src.Host.IDC {
|
||||
hostDist = 20.0
|
||||
} else if dst.Host.SecurityDomain != src.Host.SecurityDomain {
|
||||
hostDist = 80.0
|
||||
}
|
||||
}
|
||||
return 1.0 - hostDist/80.0
|
||||
}
|
||||
|
|
@ -17,19 +17,23 @@
|
|||
package evaluator
|
||||
|
||||
import (
|
||||
"sort"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"d7y.io/dragonfly/v2/internal/idgen"
|
||||
"d7y.io/dragonfly/v2/scheduler/config"
|
||||
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
||||
)
|
||||
|
||||
type Evaluator interface {
|
||||
const (
|
||||
// DefaultAlgorithm is a rule-based scheduling algorithm
|
||||
DefaultAlgorithm = "default"
|
||||
|
||||
// MLAlgorithm is a machine learning scheduling algorithm
|
||||
MLAlgorithm = "ml"
|
||||
|
||||
// PluginAlgorithm is a scheduling algorithm based on plugin extension
|
||||
PluginAlgorithm = "plugin"
|
||||
)
|
||||
|
||||
type Evaluator interface {
|
||||
// Evaluate todo Normalization
|
||||
Evaluate(parent *supervisor.Peer, child *supervisor.Peer) float64
|
||||
Evaluate(parent *supervisor.Peer, child *supervisor.Peer, taskPieceCount int32) float64
|
||||
|
||||
// NeedAdjustParent determine whether the peer needs a new parent node
|
||||
NeedAdjustParent(peer *supervisor.Peer) bool
|
||||
|
|
@ -38,170 +42,16 @@ type Evaluator interface {
|
|||
IsBadNode(peer *supervisor.Peer) bool
|
||||
}
|
||||
|
||||
type Factory struct {
|
||||
lock sync.RWMutex
|
||||
evaluators map[string]Evaluator
|
||||
getEvaluatorFuncs map[int]getEvaluatorFunc
|
||||
getEvaluatorFuncPriorityList []getEvaluatorFunc
|
||||
cache map[string]Evaluator
|
||||
cacheClearFunc sync.Once
|
||||
abtest bool
|
||||
aEvaluator string
|
||||
bEvaluator string
|
||||
}
|
||||
|
||||
var _ Evaluator = (*Factory)(nil)
|
||||
|
||||
func (ef *Factory) Evaluate(dst *supervisor.Peer, src *supervisor.Peer) float64 {
|
||||
return ef.get(dst.Task.ID).Evaluate(dst, src)
|
||||
}
|
||||
|
||||
func (ef *Factory) NeedAdjustParent(peer *supervisor.Peer) bool {
|
||||
return ef.get(peer.Task.ID).NeedAdjustParent(peer)
|
||||
}
|
||||
|
||||
func (ef *Factory) IsBadNode(peer *supervisor.Peer) bool {
|
||||
return ef.get(peer.Task.ID).IsBadNode(peer)
|
||||
}
|
||||
|
||||
func NewEvaluatorFactory(cfg *config.SchedulerConfig) *Factory {
|
||||
factory := &Factory{
|
||||
evaluators: make(map[string]Evaluator),
|
||||
getEvaluatorFuncs: map[int]getEvaluatorFunc{},
|
||||
cache: map[string]Evaluator{},
|
||||
abtest: cfg.ABTest,
|
||||
aEvaluator: cfg.AEvaluator,
|
||||
bEvaluator: cfg.BEvaluator,
|
||||
}
|
||||
return factory
|
||||
}
|
||||
|
||||
var (
|
||||
m = make(map[string]Evaluator)
|
||||
)
|
||||
|
||||
func Register(name string, evaluator Evaluator) {
|
||||
m[strings.ToLower(name)] = evaluator
|
||||
}
|
||||
|
||||
func Get(name string) Evaluator {
|
||||
if eval, ok := m[strings.ToLower(name)]; ok {
|
||||
return eval
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type getEvaluatorFunc func(taskID string) (string, bool)
|
||||
|
||||
func (ef *Factory) get(taskID string) Evaluator {
|
||||
ef.lock.RLock()
|
||||
evaluator, ok := ef.cache[taskID]
|
||||
ef.lock.RUnlock()
|
||||
if ok {
|
||||
return evaluator
|
||||
}
|
||||
|
||||
if ef.abtest {
|
||||
name := ""
|
||||
if strings.HasSuffix(taskID, idgen.TwinsBSuffix) {
|
||||
if ef.bEvaluator != "" {
|
||||
name = ef.bEvaluator
|
||||
}
|
||||
} else {
|
||||
if ef.aEvaluator != "" {
|
||||
name = ef.aEvaluator
|
||||
}
|
||||
}
|
||||
if name != "" {
|
||||
ef.lock.RLock()
|
||||
evaluator, ok = ef.evaluators[name]
|
||||
ef.lock.RUnlock()
|
||||
if ok {
|
||||
ef.lock.Lock()
|
||||
ef.cache[taskID] = evaluator
|
||||
ef.lock.Unlock()
|
||||
return evaluator
|
||||
}
|
||||
func New(algorithm string) Evaluator {
|
||||
switch algorithm {
|
||||
case PluginAlgorithm:
|
||||
if plugin, err := LoadPlugin(); err == nil {
|
||||
return plugin
|
||||
}
|
||||
// TODO Implement MLAlgorithm
|
||||
case MLAlgorithm, DefaultAlgorithm:
|
||||
return NewEvaluatorBase()
|
||||
}
|
||||
|
||||
for _, fun := range ef.getEvaluatorFuncPriorityList {
|
||||
name, ok := fun(taskID)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
ef.lock.RLock()
|
||||
evaluator, ok = ef.evaluators[name]
|
||||
ef.lock.RUnlock()
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
||||
ef.lock.Lock()
|
||||
ef.cache[taskID] = evaluator
|
||||
ef.lock.Unlock()
|
||||
return evaluator
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ef *Factory) clearCache() {
|
||||
ef.lock.Lock()
|
||||
ef.cache = make(map[string]Evaluator)
|
||||
ef.lock.Unlock()
|
||||
}
|
||||
|
||||
func (ef *Factory) add(name string, evaluator Evaluator) {
|
||||
ef.lock.Lock()
|
||||
ef.evaluators[name] = evaluator
|
||||
ef.lock.Unlock()
|
||||
}
|
||||
|
||||
func (ef *Factory) addGetEvaluatorFunc(priority int, fun getEvaluatorFunc) {
|
||||
ef.lock.Lock()
|
||||
defer ef.lock.Unlock()
|
||||
_, ok := ef.getEvaluatorFuncs[priority]
|
||||
if ok {
|
||||
return
|
||||
}
|
||||
ef.getEvaluatorFuncs[priority] = fun
|
||||
var priorities []int
|
||||
for p := range ef.getEvaluatorFuncs {
|
||||
priorities = append(priorities, p)
|
||||
}
|
||||
sort.Ints(priorities)
|
||||
ef.getEvaluatorFuncPriorityList = ef.getEvaluatorFuncPriorityList[:0]
|
||||
for i := len(priorities) - 1; i >= 0; i-- {
|
||||
ef.getEvaluatorFuncPriorityList = append(ef.getEvaluatorFuncPriorityList, ef.getEvaluatorFuncs[priorities[i]])
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func (ef *Factory) deleteGetEvaluatorFunc(priority int, fun getEvaluatorFunc) {
|
||||
ef.lock.Lock()
|
||||
|
||||
delete(ef.getEvaluatorFuncs, priority)
|
||||
|
||||
var priorities []int
|
||||
for p := range ef.getEvaluatorFuncs {
|
||||
priorities = append(priorities, p)
|
||||
}
|
||||
sort.Ints(priorities)
|
||||
ef.getEvaluatorFuncPriorityList = ef.getEvaluatorFuncPriorityList[:0]
|
||||
for i := len(priorities) - 1; i >= 0; i-- {
|
||||
ef.getEvaluatorFuncPriorityList = append(ef.getEvaluatorFuncPriorityList, ef.getEvaluatorFuncs[priorities[i]])
|
||||
}
|
||||
|
||||
ef.lock.Unlock()
|
||||
}
|
||||
|
||||
func (ef *Factory) Register(name string, evaluator Evaluator) {
|
||||
ef.add(name, evaluator)
|
||||
ef.clearCache()
|
||||
}
|
||||
|
||||
func (ef *Factory) RegisterGetEvaluatorFunc(priority int, fun getEvaluatorFunc) {
|
||||
ef.addGetEvaluatorFunc(priority, fun)
|
||||
ef.clearCache()
|
||||
return NewEvaluatorBase()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,233 @@
|
|||
/*
|
||||
* Copyright 2020 The Dragonfly Authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package evaluator
|
||||
|
||||
import (
|
||||
"math/big"
|
||||
"strings"
|
||||
|
||||
"github.com/montanaflynn/stats"
|
||||
|
||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||
"d7y.io/dragonfly/v2/pkg/util/mathutils"
|
||||
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
||||
)
|
||||
|
||||
const (
|
||||
// Finished piece weight
|
||||
finishedPieceWeight float64 = 0.4
|
||||
|
||||
// Free load weight
|
||||
freeLoadWeight = 0.3
|
||||
|
||||
// IDC affinity weight
|
||||
idcAffinityWeight = 0.15
|
||||
|
||||
// NetTopology affinity weight
|
||||
netTopologyAffinityWeight = 0.1
|
||||
|
||||
// Location affinity weight
|
||||
locationAffinityWeight = 0.05
|
||||
)
|
||||
|
||||
const (
|
||||
// Maximum score
|
||||
maxScore float64 = 1
|
||||
|
||||
// Minimum score
|
||||
minScore = 0
|
||||
)
|
||||
|
||||
const (
|
||||
// If the number of samples is greater than or equal to 10,
|
||||
// it is close to the normal distribution
|
||||
normalDistributionLen = 10
|
||||
|
||||
// When costs len is greater than or equal to 2,
|
||||
// the last cost can be compared and calculated
|
||||
minAvailableCostLen = 2
|
||||
|
||||
// Maximum number of elements
|
||||
maxElementLen = 5
|
||||
)
|
||||
|
||||
type evaluatorBase struct{}
|
||||
|
||||
func NewEvaluatorBase() Evaluator {
|
||||
return &evaluatorBase{}
|
||||
}
|
||||
|
||||
// The larger the value after evaluation, the higher the priority
|
||||
func (eb *evaluatorBase) Evaluate(parent *supervisor.Peer, child *supervisor.Peer, taskPieceCount int32) float64 {
|
||||
// If the SecurityDomain of hosts exists but is not equal,
|
||||
// it cannot be scheduled as a parent
|
||||
if parent.Host.SecurityDomain != "" &&
|
||||
child.Host.SecurityDomain != "" &&
|
||||
strings.Compare(parent.Host.SecurityDomain, child.Host.SecurityDomain) != 0 {
|
||||
return minScore
|
||||
}
|
||||
|
||||
return finishedPieceWeight*calculatePieceScore(parent, child, taskPieceCount) +
|
||||
freeLoadWeight*calculateFreeLoadScore(parent.Host) +
|
||||
idcAffinityWeight*calculateIDCAffinityScore(parent.Host, child.Host) +
|
||||
netTopologyAffinityWeight*calculateMultiElementAffinityScore(parent.Host.NetTopology, child.Host.NetTopology) +
|
||||
locationAffinityWeight*calculateMultiElementAffinityScore(parent.Host.Location, child.Host.Location)
|
||||
}
|
||||
|
||||
// calculatePieceScore 0.0~unlimited larger and better
|
||||
func calculatePieceScore(parent *supervisor.Peer, child *supervisor.Peer, taskPieceCount int32) float64 {
|
||||
// If the total piece is determined, normalize the number of
|
||||
// pieces downloaded by the parent node
|
||||
if taskPieceCount > 0 {
|
||||
finishedPieceCount := parent.TotalPieceCount.Load()
|
||||
return float64(finishedPieceCount) / float64(taskPieceCount)
|
||||
}
|
||||
|
||||
// Use the difference between the parent node and the child node to
|
||||
// download the piece to roughly represent the piece score
|
||||
parentFinishedPieceCount := parent.TotalPieceCount.Load()
|
||||
childFinishedPieceCount := child.TotalPieceCount.Load()
|
||||
return float64(parentFinishedPieceCount - childFinishedPieceCount)
|
||||
}
|
||||
|
||||
// calculateFreeLoadScore 0.0~1.0 larger and better
|
||||
func calculateFreeLoadScore(host *supervisor.Host) float64 {
|
||||
load := host.CurrentUploadLoad.Load()
|
||||
totalLoad := host.TotalUploadLoad
|
||||
return float64(totalLoad-load) / float64(totalLoad)
|
||||
}
|
||||
|
||||
// calculateIDCAffinityScore 0.0~1.0 larger and better
|
||||
func calculateIDCAffinityScore(dst, src *supervisor.Host) float64 {
|
||||
if dst.IDC != "" && src.IDC != "" && strings.Compare(dst.IDC, src.IDC) == 0 {
|
||||
return maxScore
|
||||
}
|
||||
|
||||
return minScore
|
||||
}
|
||||
|
||||
// calculateMultiElementAffinityScore 0.0~1.0 larger and better
|
||||
func calculateMultiElementAffinityScore(dst, src string) float64 {
|
||||
if dst == "" || src == "" {
|
||||
return minScore
|
||||
}
|
||||
|
||||
if strings.Compare(dst, src) == 0 {
|
||||
return maxScore
|
||||
}
|
||||
|
||||
// Calculate the number of multi-element matches divided by "|"
|
||||
var score, elementLen int
|
||||
dstElements := strings.Split(dst, "|")
|
||||
srcElements := strings.Split(src, "|")
|
||||
elementLen = mathutils.MaxInt(len(dstElements), len(srcElements))
|
||||
|
||||
// Maximum element length is 5
|
||||
if elementLen > maxElementLen {
|
||||
elementLen = maxElementLen
|
||||
}
|
||||
|
||||
for i := 0; i < elementLen; i++ {
|
||||
if strings.Compare(dstElements[i], srcElements[i]) != 0 {
|
||||
break
|
||||
}
|
||||
score++
|
||||
}
|
||||
|
||||
return float64(score) / float64(maxElementLen)
|
||||
}
|
||||
|
||||
func (eb *evaluatorBase) NeedAdjustParent(peer *supervisor.Peer) bool {
|
||||
// CDN is the root node
|
||||
if peer.Host.IsCDN {
|
||||
return false
|
||||
}
|
||||
|
||||
parent, ok := peer.GetParent()
|
||||
// Peer has no parent and is not completed
|
||||
if !ok && !peer.IsDone() {
|
||||
logger.Infof("peer %s need adjust parent because it has not parent and status is %s", peer.ID, peer.GetStatus())
|
||||
return true
|
||||
}
|
||||
|
||||
// Peer has parent but parent can't be scheduled.
|
||||
if ok && (parent.IsLeave() || eb.IsBadNode(parent)) {
|
||||
logger.Infof("peer %s need adjust parent because parent can't be scheduled", peer.ID)
|
||||
return true
|
||||
}
|
||||
|
||||
// Determine whether to adjust parent based on piece download costs
|
||||
rawCosts := peer.GetPieceCosts()
|
||||
costs := stats.LoadRawData(rawCosts)
|
||||
len := len(costs)
|
||||
// Peer has not finished downloading enough piece
|
||||
if len < minAvailableCostLen {
|
||||
logger.Infof("peer %s has not finished downloading enough piece, it can't be adjusted parent", peer.ID)
|
||||
return false
|
||||
}
|
||||
|
||||
lastCost := costs[len-1]
|
||||
mean, _ := stats.Mean(costs[:len-1]) // nolint: errcheck
|
||||
|
||||
// Download costs does not meet the normal distribution,
|
||||
// if the last cost is five times more than mean, it need to be adjusted parent.
|
||||
if len < normalDistributionLen {
|
||||
isNeedAdjustParent := big.NewFloat(lastCost).Cmp(big.NewFloat(mean*5)) > 0
|
||||
logger.Infof("peer %s does not meet the normal distribution and mean is %.2f, peer need adjust parent: %t", peer.ID, mean, isNeedAdjustParent)
|
||||
return isNeedAdjustParent
|
||||
}
|
||||
|
||||
// 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-2]) // nolint: errcheck
|
||||
isNeedAdjustParent := big.NewFloat(lastCost).Cmp(big.NewFloat(mean+3*stdev)) > 0
|
||||
logger.Infof("peer %s meet the normal distribution, costs mean is %.2f and standard deviation is %.2f, peer need adjust parent: %t",
|
||||
peer.ID, mean, stdev, isNeedAdjustParent)
|
||||
return isNeedAdjustParent
|
||||
}
|
||||
|
||||
func (eb *evaluatorBase) IsBadNode(peer *supervisor.Peer) bool {
|
||||
if peer.IsBad() {
|
||||
logger.Infof("peer %s is bad because it's status is %s", peer.ID, peer.GetStatus())
|
||||
return true
|
||||
}
|
||||
|
||||
if peer.Host.IsCDN {
|
||||
logger.Infof("peer %s is cdn can't be bad node", peer.ID)
|
||||
return false
|
||||
}
|
||||
|
||||
// Determine whether to bad node based on piece download costs
|
||||
rawCosts := peer.GetPieceCosts()
|
||||
costs := stats.LoadRawData(rawCosts)
|
||||
len := len(costs)
|
||||
// Peer has not finished downloading enough piece
|
||||
if len < minAvailableCostLen {
|
||||
logger.Infof("peer %s has not finished downloading enough piece, it can't be bad node", peer.ID)
|
||||
return false
|
||||
}
|
||||
|
||||
lastCost := costs[len-1]
|
||||
mean, _ := stats.Mean(costs[:len-1]) // nolint: errcheck
|
||||
|
||||
// Download costs does not meet the normal distribution,
|
||||
// if the last cost is forty times more than mean, it is bad node.
|
||||
isBadNode := big.NewFloat(lastCost).Cmp(big.NewFloat(mean*40)) > 0
|
||||
logger.Infof("peer %s mean is %.2f and it is bad node: %t", peer.ID, mean, isBadNode)
|
||||
return isBadNode
|
||||
}
|
||||
|
|
@ -0,0 +1,667 @@
|
|||
/*
|
||||
* Copyright 2020 The Dragonfly Authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package evaluator
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"d7y.io/dragonfly/v2/internal/idgen"
|
||||
"d7y.io/dragonfly/v2/pkg/util/mathutils"
|
||||
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
||||
)
|
||||
|
||||
const (
|
||||
cdnHostType = "cdn"
|
||||
clientHostType = "client"
|
||||
)
|
||||
|
||||
const (
|
||||
mockIP = "127.0.0.1"
|
||||
mockTaskURL = "https://example.com"
|
||||
)
|
||||
|
||||
type factor struct {
|
||||
hostType string
|
||||
securityDomain string
|
||||
idc string
|
||||
location string
|
||||
netTopology string
|
||||
totalUploadLoad int32
|
||||
currentUploadLoad int32
|
||||
finishedPieceCount int32
|
||||
hostUUID string
|
||||
taskPieceCount int32
|
||||
}
|
||||
|
||||
func TestEvaluatorEvaluate(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
parent *factor
|
||||
child *factor
|
||||
expect func(t *testing.T, v float64)
|
||||
}{
|
||||
{
|
||||
name: "evaluate succeeded with cdn peer",
|
||||
parent: &factor{
|
||||
hostType: cdnHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e",
|
||||
totalUploadLoad: 100,
|
||||
currentUploadLoad: 0,
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
taskPieceCount: 100,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e",
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
},
|
||||
expect: func(t *testing.T, v float64) {
|
||||
assert := assert.New(t)
|
||||
assert.True(mathutils.EqualFloat64(v, float64(0.6)))
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "evaluate with different securityDomain",
|
||||
parent: &factor{
|
||||
hostType: cdnHostType,
|
||||
securityDomain: "foz",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e",
|
||||
totalUploadLoad: 100,
|
||||
currentUploadLoad: 0,
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
taskPieceCount: 100,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e",
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
},
|
||||
expect: func(t *testing.T, v float64) {
|
||||
assert := assert.New(t)
|
||||
assert.True(mathutils.EqualFloat64(v, float64(0)))
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "evaluate with empty securityDomain",
|
||||
parent: &factor{
|
||||
hostType: cdnHostType,
|
||||
securityDomain: "",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e",
|
||||
totalUploadLoad: 100,
|
||||
currentUploadLoad: 0,
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
taskPieceCount: 100,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
securityDomain: "",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e",
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
},
|
||||
expect: func(t *testing.T, v float64) {
|
||||
assert := assert.New(t)
|
||||
assert.True(mathutils.EqualFloat64(v, float64(0.6)))
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "evaluate with different idc",
|
||||
parent: &factor{
|
||||
hostType: cdnHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e",
|
||||
totalUploadLoad: 100,
|
||||
currentUploadLoad: 0,
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
taskPieceCount: 100,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "baz",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e",
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
},
|
||||
expect: func(t *testing.T, v float64) {
|
||||
assert := assert.New(t)
|
||||
assert.True(mathutils.EqualFloat64(v, float64(0.45)))
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "evaluate with different location",
|
||||
parent: &factor{
|
||||
hostType: cdnHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d",
|
||||
netTopology: "a|b|c|d|e",
|
||||
totalUploadLoad: 100,
|
||||
currentUploadLoad: 0,
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
taskPieceCount: 100,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e",
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
},
|
||||
expect: func(t *testing.T, v float64) {
|
||||
assert := assert.New(t)
|
||||
assert.True(mathutils.EqualFloat64(v, float64(0.59)))
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "evaluate with empty location",
|
||||
parent: &factor{
|
||||
hostType: cdnHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "",
|
||||
netTopology: "a|b|c|d|e",
|
||||
totalUploadLoad: 100,
|
||||
currentUploadLoad: 0,
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
taskPieceCount: 100,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e",
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
},
|
||||
expect: func(t *testing.T, v float64) {
|
||||
assert := assert.New(t)
|
||||
assert.True(mathutils.EqualFloat64(v, float64(0.55)))
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "evaluate with excessive location",
|
||||
parent: &factor{
|
||||
hostType: cdnHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e|f",
|
||||
netTopology: "a|b|c|d|e",
|
||||
totalUploadLoad: 100,
|
||||
currentUploadLoad: 0,
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
taskPieceCount: 100,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e|f",
|
||||
netTopology: "a|b|c|d|e",
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
},
|
||||
expect: func(t *testing.T, v float64) {
|
||||
assert := assert.New(t)
|
||||
assert.True(mathutils.EqualFloat64(v, float64(0.6)))
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "evaluate with different netTopology",
|
||||
parent: &factor{
|
||||
hostType: cdnHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d",
|
||||
totalUploadLoad: 100,
|
||||
currentUploadLoad: 0,
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
taskPieceCount: 100,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e",
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
},
|
||||
expect: func(t *testing.T, v float64) {
|
||||
assert := assert.New(t)
|
||||
assert.True(mathutils.EqualFloat64(v, float64(0.58)))
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "evaluate with empty netTopology",
|
||||
parent: &factor{
|
||||
hostType: cdnHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "",
|
||||
totalUploadLoad: 100,
|
||||
currentUploadLoad: 0,
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
taskPieceCount: 100,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e",
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
},
|
||||
expect: func(t *testing.T, v float64) {
|
||||
assert := assert.New(t)
|
||||
assert.True(mathutils.EqualFloat64(v, float64(0.5)))
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "evaluate with excessive netTopology",
|
||||
parent: &factor{
|
||||
hostType: cdnHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e|f",
|
||||
totalUploadLoad: 100,
|
||||
currentUploadLoad: 0,
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
taskPieceCount: 100,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e|f",
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
},
|
||||
expect: func(t *testing.T, v float64) {
|
||||
assert := assert.New(t)
|
||||
assert.True(mathutils.EqualFloat64(v, float64(0.6)))
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "evaluate with task piece count",
|
||||
parent: &factor{
|
||||
hostType: cdnHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e|f",
|
||||
totalUploadLoad: 100,
|
||||
currentUploadLoad: 0,
|
||||
finishedPieceCount: 20,
|
||||
hostUUID: "example",
|
||||
taskPieceCount: 100,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e|f",
|
||||
finishedPieceCount: 0,
|
||||
hostUUID: "example",
|
||||
},
|
||||
expect: func(t *testing.T, v float64) {
|
||||
assert := assert.New(t)
|
||||
assert.True(mathutils.EqualFloat64(v, float64(0.68)))
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "evaluate without task piece count",
|
||||
parent: &factor{
|
||||
hostType: cdnHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e|f",
|
||||
totalUploadLoad: 100,
|
||||
currentUploadLoad: 0,
|
||||
finishedPieceCount: 20,
|
||||
hostUUID: "example",
|
||||
taskPieceCount: 0,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
securityDomain: "foo",
|
||||
idc: "bar",
|
||||
location: "a|b|c|d|e",
|
||||
netTopology: "a|b|c|d|e|f",
|
||||
finishedPieceCount: 10,
|
||||
hostUUID: "example",
|
||||
},
|
||||
expect: func(t *testing.T, v float64) {
|
||||
assert := assert.New(t)
|
||||
assert.True(mathutils.EqualFloat64(v, float64(4.6)))
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
task := supervisor.NewTask(idgen.TaskID(mockTaskURL, nil), mockTaskURL, nil)
|
||||
|
||||
parentHost := supervisor.NewClientHost(
|
||||
tc.parent.hostUUID, "", "", 0, 0,
|
||||
tc.parent.securityDomain,
|
||||
tc.parent.location,
|
||||
tc.parent.idc,
|
||||
supervisor.WithNetTopology(tc.parent.netTopology),
|
||||
supervisor.WithTotalUploadLoad(tc.parent.totalUploadLoad),
|
||||
)
|
||||
parentHost.CurrentUploadLoad.Store(tc.parent.currentUploadLoad)
|
||||
parent := supervisor.NewPeer(idgen.PeerID(mockIP), task, parentHost)
|
||||
parent.TotalPieceCount.Store(tc.parent.finishedPieceCount)
|
||||
|
||||
childHost := supervisor.NewClientHost(
|
||||
tc.parent.hostUUID, "", "", 0, 0,
|
||||
tc.child.securityDomain,
|
||||
tc.child.location,
|
||||
tc.child.idc,
|
||||
supervisor.WithNetTopology(tc.child.netTopology),
|
||||
)
|
||||
child := supervisor.NewPeer(idgen.PeerID(mockIP), task, childHost)
|
||||
child.TotalPieceCount.Store(tc.child.finishedPieceCount)
|
||||
|
||||
e := NewEvaluatorBase()
|
||||
tc.expect(t, e.Evaluate(parent, child, tc.parent.taskPieceCount))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestEvaluatorNeedAdjustParent(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
parent *factor
|
||||
child *factor
|
||||
expect func(t *testing.T, e Evaluator, parent *supervisor.Peer, child *supervisor.Peer)
|
||||
}{
|
||||
{
|
||||
name: "peer is CDN",
|
||||
parent: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: cdnHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, parent *supervisor.Peer, child *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal(e.NeedAdjustParent(child), false)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "peer has no parent",
|
||||
parent: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, parent *supervisor.Peer, child *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal(e.NeedAdjustParent(child), true)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "peer has done",
|
||||
parent: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, parent *supervisor.Peer, child *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal(e.NeedAdjustParent(child), true)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "parent has leaved",
|
||||
parent: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, parent *supervisor.Peer, child *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
child.SetParent(parent)
|
||||
parent.Leave()
|
||||
assert.Equal(e.NeedAdjustParent(child), true)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "empty costs",
|
||||
parent: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, parent *supervisor.Peer, child *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
child.SetParent(parent)
|
||||
assert.Equal(e.NeedAdjustParent(child), false)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "costs are not normal distribution and peer should not be scheduler",
|
||||
parent: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, parent *supervisor.Peer, child *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
child.SetParent(parent)
|
||||
child.SetPieceCosts([]int{1, 2, 3, 4, 5, 6, 7, 8, 9}...)
|
||||
assert.Equal(e.NeedAdjustParent(child), false)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "costs are not normal distribution and peer should be scheduler",
|
||||
parent: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, parent *supervisor.Peer, child *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
child.SetParent(parent)
|
||||
child.SetPieceCosts([]int{1, 2, 3, 4, 5, 6, 7, 8, 23}...)
|
||||
assert.Equal(e.NeedAdjustParent(child), true)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "costs are normal distribution and peer should not be scheduler",
|
||||
parent: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, parent *supervisor.Peer, child *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
child.SetParent(parent)
|
||||
child.SetPieceCosts([]int{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 5}...)
|
||||
assert.Equal(e.NeedAdjustParent(child), false)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "costs are normal distribution and peer should be scheduler",
|
||||
parent: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
child: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, parent *supervisor.Peer, child *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
child.SetParent(parent)
|
||||
child.SetPieceCosts([]int{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 15}...)
|
||||
assert.Equal(e.NeedAdjustParent(child), true)
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
task := supervisor.NewTask(idgen.TaskID(mockTaskURL, nil), mockTaskURL, nil)
|
||||
|
||||
parentHost := supervisor.NewClientHost(uuid.NewString(), "", "", 0, 0, "", "", "")
|
||||
parent := supervisor.NewPeer(idgen.PeerID(mockIP), task, parentHost)
|
||||
|
||||
var child *supervisor.Peer
|
||||
if tc.child.hostType == cdnHostType {
|
||||
childHost := supervisor.NewCDNHost(uuid.NewString(), "", "", 0, 0, "", "", "")
|
||||
child = supervisor.NewPeer(idgen.CDNPeerID(mockIP), task, childHost)
|
||||
} else {
|
||||
childHost := supervisor.NewClientHost(uuid.NewString(), "", "", 0, 0, "", "", "")
|
||||
child = supervisor.NewPeer(idgen.PeerID(mockIP), task, childHost)
|
||||
}
|
||||
|
||||
e := NewEvaluatorBase()
|
||||
tc.expect(t, e, parent, child)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestEvaluatorIsBadNode(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
peer *factor
|
||||
expect func(t *testing.T, e Evaluator, peer *supervisor.Peer)
|
||||
}{
|
||||
{
|
||||
name: "peer is bad",
|
||||
peer: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, peer *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
peer.SetStatus(supervisor.PeerStatusFail)
|
||||
assert.Equal(e.IsBadNode(peer), true)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "peer is CDN",
|
||||
peer: &factor{
|
||||
hostType: cdnHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, peer *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal(e.IsBadNode(peer), false)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "empty costs",
|
||||
peer: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, peer *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal(e.IsBadNode(peer), false)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "costs length is available and peer is not bad node",
|
||||
peer: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, peer *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
peer.SetPieceCosts([]int{1, 2, 3, 4, 5, 6, 7, 8, 9}...)
|
||||
assert.Equal(e.IsBadNode(peer), false)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "costs length is available and peer is bad node",
|
||||
peer: &factor{
|
||||
hostType: clientHostType,
|
||||
},
|
||||
expect: func(t *testing.T, e Evaluator, peer *supervisor.Peer) {
|
||||
assert := assert.New(t)
|
||||
peer.SetPieceCosts([]int{1, 2, 3, 4, 5, 6, 7, 8, 181}...)
|
||||
assert.Equal(e.IsBadNode(peer), true)
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
task := supervisor.NewTask(idgen.TaskID(mockTaskURL, nil), mockTaskURL, nil)
|
||||
|
||||
var peer *supervisor.Peer
|
||||
if tc.peer.hostType == cdnHostType {
|
||||
childHost := supervisor.NewCDNHost(uuid.NewString(), "", "", 0, 0, "", "", "")
|
||||
peer = supervisor.NewPeer(idgen.CDNPeerID(mockIP), task, childHost)
|
||||
} else {
|
||||
childHost := supervisor.NewClientHost(uuid.NewString(), "", "", 0, 0, "", "", "")
|
||||
peer = supervisor.NewPeer(idgen.PeerID(mockIP), task, childHost)
|
||||
}
|
||||
|
||||
e := NewEvaluatorBase()
|
||||
tc.expect(t, e, peer)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,71 @@
|
|||
/*
|
||||
* Copyright 2020 The Dragonfly Authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package evaluator
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestEvaluatorNew(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
algorithm string
|
||||
expect func(t *testing.T, e interface{})
|
||||
}{
|
||||
{
|
||||
name: "new evaluator with default algorithm",
|
||||
algorithm: "default",
|
||||
expect: func(t *testing.T, e interface{}) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal(reflect.TypeOf(e).Elem().Name(), "evaluatorBase")
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "new evaluator with machine learning algorithm",
|
||||
algorithm: "ml",
|
||||
expect: func(t *testing.T, e interface{}) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal(reflect.TypeOf(e).Elem().Name(), "evaluatorBase")
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "new evaluator with plugin",
|
||||
algorithm: "plugin",
|
||||
expect: func(t *testing.T, e interface{}) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal(reflect.TypeOf(e).Elem().Name(), "evaluatorBase")
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "new evaluator with empty string",
|
||||
algorithm: "",
|
||||
expect: func(t *testing.T, e interface{}) {
|
||||
assert := assert.New(t)
|
||||
assert.Equal(reflect.TypeOf(e).Elem().Name(), "evaluatorBase")
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
tc.expect(t, New(tc.algorithm))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,39 @@
|
|||
/*
|
||||
* Copyright 2020 The Dragonfly Authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package evaluator
|
||||
|
||||
import (
|
||||
"errors"
|
||||
|
||||
"d7y.io/dragonfly/v2/internal/dfplugin"
|
||||
)
|
||||
|
||||
const (
|
||||
pluginName = "evaluator"
|
||||
)
|
||||
|
||||
func LoadPlugin() (Evaluator, error) {
|
||||
client, _, err := dfplugin.Load(dfplugin.PluginTypeScheduler, pluginName, map[string]string{})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if rc, ok := client.(Evaluator); ok {
|
||||
return rc, err
|
||||
}
|
||||
return nil, errors.New("invalid client, not a ResourceClient")
|
||||
}
|
||||
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* Copyright 2020 The Dragonfly Authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package evaluator
|
||||
|
||||
import (
|
||||
"os"
|
||||
"os/exec"
|
||||
"path"
|
||||
"testing"
|
||||
|
||||
testifyassert "github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestLoadPlugin(t *testing.T) {
|
||||
assert := testifyassert.New(t)
|
||||
defer func() {
|
||||
os.Remove("./testdata/d7y-scheduler-plugin-evaluator.so")
|
||||
os.Remove("./testdata/test")
|
||||
}()
|
||||
|
||||
var (
|
||||
cmd *exec.Cmd
|
||||
output []byte
|
||||
wd string
|
||||
err error
|
||||
)
|
||||
|
||||
// build plugin
|
||||
cmd = exec.Command("go", "build", "-buildmode=plugin", "-o=./testdata/d7y-scheduler-plugin-evaluator.so", "testdata/plugin/evaluator.go")
|
||||
output, err = cmd.CombinedOutput()
|
||||
assert.Nil(err)
|
||||
if err != nil {
|
||||
t.Fatalf(string(output))
|
||||
return
|
||||
}
|
||||
|
||||
// build test binary
|
||||
cmd = exec.Command("go", "build", "-o=./testdata/test", "testdata/main.go")
|
||||
output, err = cmd.CombinedOutput()
|
||||
assert.Nil(err)
|
||||
if err != nil {
|
||||
t.Fatalf(string(output))
|
||||
return
|
||||
}
|
||||
|
||||
wd, err = os.Getwd()
|
||||
assert.Nil(err)
|
||||
wd = path.Join(wd, "testdata")
|
||||
|
||||
// execute test binary
|
||||
cmd = exec.Command("./testdata/test", "-plugin-dir", wd)
|
||||
output, err = cmd.CombinedOutput()
|
||||
assert.Nil(err)
|
||||
if err != nil {
|
||||
t.Fatalf(string(output))
|
||||
return
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,56 @@
|
|||
/*
|
||||
* Copyright 2020 The Dragonfly Authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package main
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"d7y.io/dragonfly/v2/internal/dfpath"
|
||||
"d7y.io/dragonfly/v2/scheduler/core/evaluator"
|
||||
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
||||
)
|
||||
|
||||
func init() {
|
||||
flag.StringVar(&dfpath.PluginsDir, "plugin-dir", ".", "")
|
||||
}
|
||||
|
||||
func main() {
|
||||
flag.Parse()
|
||||
|
||||
e, err := evaluator.LoadPlugin()
|
||||
if err != nil {
|
||||
fmt.Printf("load plugin error: %s\n", err)
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
if score := e.Evaluate(&supervisor.Peer{}, &supervisor.Peer{}, int32(0)); score != float64(1) {
|
||||
fmt.Println("Evaluate failed")
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
if ok := e.NeedAdjustParent(&supervisor.Peer{}); !ok {
|
||||
fmt.Println("NeedAdjustParent failed")
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
if ok := e.IsBadNode(&supervisor.Peer{}); !ok {
|
||||
fmt.Println("IsBadNode failed")
|
||||
os.Exit(1)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,37 @@
|
|||
/*
|
||||
* Copyright 2020 The Dragonfly Authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package main
|
||||
|
||||
import "d7y.io/dragonfly/v2/scheduler/supervisor"
|
||||
|
||||
type evaluator struct{}
|
||||
|
||||
func (e *evaluator) Evaluate(parent *supervisor.Peer, child *supervisor.Peer, taskPieceCount int32) float64 {
|
||||
return float64(1)
|
||||
}
|
||||
|
||||
func (e *evaluator) NeedAdjustParent(peer *supervisor.Peer) bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (e *evaluator) IsBadNode(peer *supervisor.Peer) bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func DragonflyPluginInit(option map[string]string) (interface{}, map[string]string, error) {
|
||||
return &evaluator{}, map[string]string{"type": "scheduler", "name": "evaluator"}, nil
|
||||
}
|
||||
|
|
@ -24,7 +24,6 @@ import (
|
|||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||
"d7y.io/dragonfly/v2/scheduler/config"
|
||||
"d7y.io/dragonfly/v2/scheduler/core/evaluator"
|
||||
"d7y.io/dragonfly/v2/scheduler/core/evaluator/basic"
|
||||
"d7y.io/dragonfly/v2/scheduler/core/scheduler"
|
||||
"d7y.io/dragonfly/v2/scheduler/supervisor"
|
||||
)
|
||||
|
|
@ -45,15 +44,11 @@ func newBasicSchedulerBuilder() scheduler.Builder {
|
|||
}
|
||||
}
|
||||
|
||||
var _ scheduler.Builder = (*basicSchedulerBuilder)(nil)
|
||||
|
||||
func (builder *basicSchedulerBuilder) Build(cfg *config.SchedulerConfig, opts *scheduler.BuildOptions) (scheduler.Scheduler, error) {
|
||||
logger.Debugf("start create basic scheduler...")
|
||||
evalFactory := evaluator.NewEvaluatorFactory(cfg)
|
||||
evalFactory.Register("default", basic.NewEvaluator(cfg))
|
||||
evalFactory.RegisterGetEvaluatorFunc(0, func(taskID string) (string, bool) { return "default", true })
|
||||
evaluator := evaluator.New(cfg.Algorithm)
|
||||
sched := &Scheduler{
|
||||
evaluator: evalFactory,
|
||||
evaluator: evaluator,
|
||||
peerManager: opts.PeerManager,
|
||||
cfg: cfg,
|
||||
}
|
||||
|
|
@ -83,8 +78,9 @@ func (s *Scheduler) ScheduleChildren(peer *supervisor.Peer, blankChildren sets.S
|
|||
}
|
||||
evalResult := make(map[float64][]*supervisor.Peer)
|
||||
var evalScore []float64
|
||||
taskTotalPieceCount := peer.Task.TotalPieceCount.Load()
|
||||
for _, child := range candidateChildren {
|
||||
score := s.evaluator.Evaluate(peer, child)
|
||||
score := s.evaluator.Evaluate(peer, child, taskTotalPieceCount)
|
||||
evalResult[score] = append(evalResult[score], child)
|
||||
evalScore = append(evalScore, score)
|
||||
}
|
||||
|
|
@ -119,8 +115,9 @@ func (s *Scheduler) ScheduleParent(peer *supervisor.Peer, blankParents sets.Stri
|
|||
}
|
||||
evalResult := make(map[float64][]*supervisor.Peer)
|
||||
var evalScore []float64
|
||||
taskTotalPieceCount := peer.Task.TotalPieceCount.Load()
|
||||
for _, parent := range candidateParents {
|
||||
score := s.evaluator.Evaluate(parent, peer)
|
||||
score := s.evaluator.Evaluate(parent, peer, taskTotalPieceCount)
|
||||
peer.Log().Debugf("evaluate score candidate %s is %f", parent.ID, score)
|
||||
evalResult[score] = append(evalResult[score], parent)
|
||||
evalScore = append(evalScore, score)
|
||||
|
|
|
|||
|
|
@ -31,9 +31,7 @@ import (
|
|||
"d7y.io/dragonfly/v2/internal/dfcodes"
|
||||
"d7y.io/dragonfly/v2/internal/dferrors"
|
||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||
"d7y.io/dragonfly/v2/internal/idgen"
|
||||
"d7y.io/dragonfly/v2/pkg/gc"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/base/common"
|
||||
schedulerRPC "d7y.io/dragonfly/v2/pkg/rpc/scheduler"
|
||||
"d7y.io/dragonfly/v2/pkg/synclock"
|
||||
|
|
@ -208,13 +206,6 @@ func (s *SchedulerService) Stop() {
|
|||
s.wg.Wait()
|
||||
}
|
||||
|
||||
func (s *SchedulerService) GenerateTaskID(url string, meta *base.UrlMeta, peerID string) string {
|
||||
if s.config.ABTest {
|
||||
return idgen.TwinsTaskID(url, meta, peerID)
|
||||
}
|
||||
return idgen.TaskID(url, meta)
|
||||
}
|
||||
|
||||
func (s *SchedulerService) SelectParent(peer *supervisor.Peer) (parent *supervisor.Peer, err error) {
|
||||
parent, _, hasParent := s.sched.ScheduleParent(peer, sets.NewString())
|
||||
if !hasParent || parent == nil {
|
||||
|
|
|
|||
|
|
@ -28,6 +28,7 @@ import (
|
|||
"d7y.io/dragonfly/v2/internal/dfcodes"
|
||||
"d7y.io/dragonfly/v2/internal/dferrors"
|
||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||
"d7y.io/dragonfly/v2/internal/idgen"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
|
||||
schedulerserver "d7y.io/dragonfly/v2/pkg/rpc/scheduler/server"
|
||||
|
|
@ -71,7 +72,8 @@ func (s *server) RegisterPeerTask(ctx context.Context, request *scheduler.PeerTa
|
|||
span.RecordError(err)
|
||||
return
|
||||
}
|
||||
taskID := s.service.GenerateTaskID(request.Url, request.UrlMeta, request.PeerId)
|
||||
|
||||
taskID := idgen.TaskID(request.Url, request.UrlMeta)
|
||||
span.SetAttributes(config.AttributeTaskID.String(taskID))
|
||||
task := s.service.GetOrCreateTask(ctx, supervisor.NewTask(taskID, request.Url, request.UrlMeta))
|
||||
if task.IsFail() {
|
||||
|
|
|
|||
|
|
@ -174,14 +174,6 @@ func (h *Host) GetPeersLen() int {
|
|||
return length
|
||||
}
|
||||
|
||||
func (h *Host) GetUploadLoadPercent() float64 {
|
||||
if h.TotalUploadLoad <= 0 {
|
||||
return 1.0
|
||||
}
|
||||
|
||||
return float64(h.CurrentUploadLoad.Load()) / float64(h.TotalUploadLoad)
|
||||
}
|
||||
|
||||
func (h *Host) GetFreeUploadLoad() int32 {
|
||||
return h.TotalUploadLoad - h.CurrentUploadLoad.Load()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -368,35 +368,20 @@ func (peer *Peer) GetPieceCosts() []int {
|
|||
return peer.pieceCosts
|
||||
}
|
||||
|
||||
func (peer *Peer) GetPieceAverageCost() (int, bool) {
|
||||
costs := peer.GetPieceCosts()
|
||||
if len(costs) < 1 {
|
||||
return 0, false
|
||||
}
|
||||
func (peer *Peer) SetPieceCosts(costs ...int) {
|
||||
peer.lock.Lock()
|
||||
defer peer.lock.Unlock()
|
||||
|
||||
totalCost := 0
|
||||
for _, cost := range costs {
|
||||
totalCost += cost
|
||||
}
|
||||
|
||||
return totalCost / len(costs), true
|
||||
peer.pieceCosts = append(peer.pieceCosts, costs...)
|
||||
}
|
||||
|
||||
func (peer *Peer) UpdateProgress(finishedCount int32, cost int) {
|
||||
if finishedCount > peer.TotalPieceCount.Load() {
|
||||
peer.TotalPieceCount.Store(finishedCount)
|
||||
|
||||
peer.lock.Lock()
|
||||
peer.pieceCosts = append(peer.pieceCosts, cost)
|
||||
if len(peer.pieceCosts) > 20 {
|
||||
peer.pieceCosts = peer.pieceCosts[len(peer.pieceCosts)-20:]
|
||||
}
|
||||
peer.lock.Unlock()
|
||||
|
||||
peer.SetPieceCosts(cost)
|
||||
peer.Task.UpdatePeer(peer)
|
||||
return
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func (peer *Peer) SortedValue() int {
|
||||
|
|
|
|||
|
|
@ -193,71 +193,6 @@ func TestPeer_Tree(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func TestPeer_Cost(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
finishedCount []int32
|
||||
cost []int
|
||||
expect func(t *testing.T, peer *supervisor.Peer, cost []int)
|
||||
}{
|
||||
{
|
||||
name: "normal workflow",
|
||||
finishedCount: []int32{2, 3, 4},
|
||||
cost: []int{3, 4, 5},
|
||||
expect: func(t *testing.T, peer *supervisor.Peer, cost []int) {
|
||||
assert := assert.New(t)
|
||||
|
||||
costFetch := peer.GetPieceCosts()
|
||||
assert.ElementsMatch(costFetch, cost)
|
||||
|
||||
average, success := peer.GetPieceAverageCost()
|
||||
assert.True(success)
|
||||
assert.Equal(4, average)
|
||||
assert.Equal(peer.SortedValue(), 4*HostMaxLoad+100)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "no workflow will be neglected",
|
||||
finishedCount: []int32{},
|
||||
cost: []int{},
|
||||
expect: func(t *testing.T, peer *supervisor.Peer, cost []int) {
|
||||
assert := assert.New(t)
|
||||
|
||||
average, success := peer.GetPieceAverageCost()
|
||||
assert.False(success)
|
||||
assert.Equal(0, average)
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "long workflow will be clipped",
|
||||
finishedCount: []int32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10,
|
||||
11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22},
|
||||
cost: []int{1, 2, 3, 4, 5, 6, 7, 8, 9, 10,
|
||||
11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22},
|
||||
expect: func(t *testing.T, peer *supervisor.Peer, cost []int) {
|
||||
assert := assert.New(t)
|
||||
|
||||
costFetch := peer.GetPieceCosts()
|
||||
assert.ElementsMatch(costFetch, cost[2:])
|
||||
|
||||
average, success := peer.GetPieceAverageCost()
|
||||
assert.True(success)
|
||||
assert.Equal(12, average)
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tc := range tests {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
task := mockATask("task")
|
||||
peer := mockAPeer("peer", task)
|
||||
for i := 0; i < len(tc.finishedCount); i++ {
|
||||
peer.UpdateProgress(tc.finishedCount[i], tc.cost[i])
|
||||
}
|
||||
tc.expect(t, peer, tc.cost)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestPeer_Status(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
|
|
|
|||
Loading…
Reference in New Issue