refactor: scheduler evaluator (#805)

* refactor: scheduler evaluator

Signed-off-by: Gaius <gaius.qi@gmail.com>
This commit is contained in:
Gaius 2021-11-18 19:48:09 +08:00
parent dc2835a454
commit d01eef6b41
No known key found for this signature in database
GPG Key ID: 8B4E5D1290FA2FFB
24 changed files with 1254 additions and 561 deletions

View File

@ -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

View File

@ -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

View File

@ -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)

View File

@ -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
}

View File

@ -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)
})
}
}

View File

@ -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
}

View File

@ -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

View File

@ -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",

View File

@ -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"

View File

@ -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
}

View File

@ -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()
}

View File

@ -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
}

View File

@ -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)
})
}
}

View File

@ -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))
})
}
}

View File

@ -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")
}

View File

@ -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
}
}

View File

@ -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)
}
}

View File

@ -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
}

View File

@ -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)

View File

@ -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 {

View File

@ -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() {

View File

@ -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()
}

View File

@ -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 {

View File

@ -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