Merged in experimentation/schedPolicySwitcher (pull request #1)
Experimentation/schedPolicySwitcher 1. Initial commit for consolidated loggers using observer pattern. 2. class factory for schedulers. 3. Using the scheduling policy class factory in schedulers/store.go and the scheduler builder helpers in schedulers/helpers.go, feature to be able to be able to plug a scheduling policy of your choice from the command line (right now only first-fit and bin-packing are possible. Will be updating the class factory to include other scheduling policies as well. 4. Removed TODO for using generic task sorters. Modified TODO for a config file input to run electron. 5. Added other schedulers to the factory 6. Partially retrofitted the other scheduling policies to use the logging library. 7. Retrofitted extrema and progressive to use the consolidated logging library. Fixed parameter issue with s.base.Disconnected(). Formatted project 8. Move statusUpdate(...) into base.go to remove redundant code. 9. Converted the baseScheduler into a state machine where the state is a scheduling policy that defines an approach to consume resource offers. 10. Added another command line argument to be used to enable switching of scheduling policies. Retrofitted scheduling policies to switch only if the particular feature has been enabled. changed argument to coLocated(...) to take base type rather than ElectronScheduler type. Also, prepended the prefix to the directory of the logs so that it would be easier to determine what the files in a directory correspond to without viewing the contents of the directory. Defined methods in ElectronScheduler. Each of these methods corresponds to a type of log that an ElectronScheduler would make. Each of these methods would need to be implemented by the scheduling policy. Electron has only one scheduler that implements the mesos scheduler interface. All the scheduling policies are just different implementations of ways to consume mesos resource offers. Retrofitted scheduling policies to now embed SchedPolicyState instead of baseScheduler. Approved-by: Pradyumna Kaushik <pkaushi1@binghamton.edu>
This commit is contained in:
parent
cb71153362
commit
065705d480
24 changed files with 1392 additions and 917 deletions
186
schedulers/MaxGreedyMins.go
Normal file
186
schedulers/MaxGreedyMins.go
Normal file
|
@ -0,0 +1,186 @@
|
|||
package schedulers
|
||||
|
||||
import (
|
||||
"bitbucket.org/sunybingcloud/elektron/def"
|
||||
"bitbucket.org/sunybingcloud/elektron/utilities/mesosUtils"
|
||||
"bitbucket.org/sunybingcloud/elektron/utilities/offerUtils"
|
||||
"fmt"
|
||||
mesos "github.com/mesos/mesos-go/mesosproto"
|
||||
sched "github.com/mesos/mesos-go/scheduler"
|
||||
"log"
|
||||
"math/rand"
|
||||
)
|
||||
|
||||
// Decides if to take an offer or not
|
||||
func (s *MaxGreedyMins) takeOffer(spc SchedPolicyContext, offer *mesos.Offer, task def.Task,
|
||||
totalCPU, totalRAM, totalWatts float64) bool {
|
||||
baseSchedRef := spc.(*baseScheduler)
|
||||
cpus, mem, watts := offerUtils.OfferAgg(offer)
|
||||
|
||||
//TODO: Insert watts calculation here instead of taking them as a parameter
|
||||
|
||||
wattsConsideration, err := def.WattsToConsider(task, baseSchedRef.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration
|
||||
log.Fatal(err)
|
||||
}
|
||||
if (cpus >= (totalCPU + task.CPU)) && (mem >= (totalRAM + task.RAM)) &&
|
||||
(!baseSchedRef.wattsAsAResource || (watts >= (totalWatts + wattsConsideration))) {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
type MaxGreedyMins struct {
|
||||
SchedPolicyState
|
||||
}
|
||||
|
||||
// Determine if the remaining space inside of the offer is enough for this
|
||||
// the task we need to create. If it is, create a TaskInfo and return it.
|
||||
func (s *MaxGreedyMins) CheckFit(
|
||||
spc SchedPolicyContext,
|
||||
i int,
|
||||
task def.Task,
|
||||
wattsConsideration float64,
|
||||
offer *mesos.Offer,
|
||||
totalCPU *float64,
|
||||
totalRAM *float64,
|
||||
totalWatts *float64) (bool, *mesos.TaskInfo) {
|
||||
|
||||
baseSchedRef := spc.(*baseScheduler)
|
||||
// Does the task fit
|
||||
if s.takeOffer(spc, offer, task, *totalCPU, *totalRAM, *totalWatts) {
|
||||
|
||||
*totalWatts += wattsConsideration
|
||||
*totalCPU += task.CPU
|
||||
*totalRAM += task.RAM
|
||||
baseSchedRef.LogCoLocatedTasks(offer.GetSlaveId().GoString())
|
||||
|
||||
taskToSchedule := baseSchedRef.newTask(offer, task)
|
||||
|
||||
baseSchedRef.LogSchedTrace(taskToSchedule, offer)
|
||||
*task.Instances--
|
||||
|
||||
if *task.Instances <= 0 {
|
||||
// All instances of task have been scheduled, remove it
|
||||
baseSchedRef.tasks = append(baseSchedRef.tasks[:i], baseSchedRef.tasks[i+1:]...)
|
||||
|
||||
if len(baseSchedRef.tasks) <= 0 {
|
||||
baseSchedRef.LogTerminateScheduler()
|
||||
close(baseSchedRef.Shutdown)
|
||||
}
|
||||
}
|
||||
|
||||
return true, taskToSchedule
|
||||
}
|
||||
|
||||
return false, nil
|
||||
}
|
||||
|
||||
func (s *MaxGreedyMins) ConsumeOffers(spc SchedPolicyContext, driver sched.SchedulerDriver, offers []*mesos.Offer) {
|
||||
fmt.Println("Max-GreedyMins scheduling...")
|
||||
baseSchedRef := spc.(*baseScheduler)
|
||||
def.SortTasks(baseSchedRef.tasks, def.SortByWatts)
|
||||
baseSchedRef.LogOffersReceived(offers)
|
||||
|
||||
for _, offer := range offers {
|
||||
offerUtils.UpdateEnvironment(offer)
|
||||
select {
|
||||
case <-baseSchedRef.Shutdown:
|
||||
baseSchedRef.LogNoPendingTasksDeclineOffers(offer)
|
||||
driver.DeclineOffer(offer.Id, mesosUtils.LongFilter)
|
||||
baseSchedRef.LogNumberOfRunningTasks()
|
||||
continue
|
||||
default:
|
||||
}
|
||||
|
||||
tasks := []*mesos.TaskInfo{}
|
||||
|
||||
offerTaken := false
|
||||
totalWatts := 0.0
|
||||
totalCPU := 0.0
|
||||
totalRAM := 0.0
|
||||
|
||||
// Assumes s.tasks is ordered in non-decreasing median max peak order
|
||||
|
||||
// Attempt to schedule a single instance of the heaviest workload available first
|
||||
// Start from the back until one fits
|
||||
for i := len(baseSchedRef.tasks) - 1; i >= 0; i-- {
|
||||
|
||||
task := baseSchedRef.tasks[i]
|
||||
wattsConsideration, err := def.WattsToConsider(task, baseSchedRef.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
// Don't take offer if it doesn't match our task's host requirement
|
||||
if offerUtils.HostMismatch(*offer.Hostname, task.Host) {
|
||||
continue
|
||||
}
|
||||
|
||||
// TODO: Fix this so index doesn't need to be passed
|
||||
taken, taskToSchedule := s.CheckFit(spc, i, task, wattsConsideration, offer,
|
||||
&totalCPU, &totalRAM, &totalWatts)
|
||||
|
||||
if taken {
|
||||
offerTaken = true
|
||||
tasks = append(tasks, taskToSchedule)
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// Pack the rest of the offer with the smallest tasks
|
||||
for i := 0; i < len(baseSchedRef.tasks); i++ {
|
||||
task := baseSchedRef.tasks[i]
|
||||
wattsConsideration, err := def.WattsToConsider(task, baseSchedRef.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
// Don't take offer if it doesn't match our task's host requirement
|
||||
if offerUtils.HostMismatch(*offer.Hostname, task.Host) {
|
||||
continue
|
||||
}
|
||||
|
||||
for *task.Instances > 0 {
|
||||
// TODO: Fix this so index doesn't need to be passed
|
||||
taken, taskToSchedule := s.CheckFit(spc, i, task, wattsConsideration, offer,
|
||||
&totalCPU, &totalRAM, &totalWatts)
|
||||
|
||||
if taken {
|
||||
offerTaken = true
|
||||
tasks = append(tasks, taskToSchedule)
|
||||
} else {
|
||||
break // Continue on to next task
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if offerTaken {
|
||||
baseSchedRef.LogTaskStarting(nil, offer)
|
||||
driver.LaunchTasks([]*mesos.OfferID{offer.Id}, tasks, mesosUtils.DefaultFilter)
|
||||
} else {
|
||||
|
||||
// If there was no match for the task
|
||||
cpus, mem, watts := offerUtils.OfferAgg(offer)
|
||||
baseSchedRef.LogInsufficientResourcesDeclineOffer(offer, cpus, mem, watts)
|
||||
driver.DeclineOffer(offer.Id, mesosUtils.DefaultFilter)
|
||||
}
|
||||
}
|
||||
|
||||
// Switch scheduling policy only if feature enabled from CLI
|
||||
if baseSchedRef.schedPolSwitchEnabled {
|
||||
// Switching to a random scheduling policy.
|
||||
// TODO: Switch based on some criteria.
|
||||
index := rand.Intn(len(SchedPolicies))
|
||||
for _, v := range SchedPolicies {
|
||||
if index == 0 {
|
||||
spc.SwitchSchedPol(v)
|
||||
break
|
||||
}
|
||||
index--
|
||||
}
|
||||
}
|
||||
}
|
180
schedulers/MaxMin.go
Normal file
180
schedulers/MaxMin.go
Normal file
|
@ -0,0 +1,180 @@
|
|||
package schedulers
|
||||
|
||||
import (
|
||||
"bitbucket.org/sunybingcloud/elektron/def"
|
||||
"bitbucket.org/sunybingcloud/elektron/utilities/mesosUtils"
|
||||
"bitbucket.org/sunybingcloud/elektron/utilities/offerUtils"
|
||||
"fmt"
|
||||
mesos "github.com/mesos/mesos-go/mesosproto"
|
||||
sched "github.com/mesos/mesos-go/scheduler"
|
||||
"log"
|
||||
"math/rand"
|
||||
)
|
||||
|
||||
// Decides if to take an offer or not
|
||||
func (s *MaxMin) takeOffer(spc SchedPolicyContext, offer *mesos.Offer, task def.Task,
|
||||
totalCPU, totalRAM, totalWatts float64) bool {
|
||||
baseSchedRef := spc.(*baseScheduler)
|
||||
cpus, mem, watts := offerUtils.OfferAgg(offer)
|
||||
|
||||
//TODO: Insert watts calculation here instead of taking them as a parameter
|
||||
|
||||
wattsConsideration, err := def.WattsToConsider(task, baseSchedRef.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration
|
||||
log.Fatal(err)
|
||||
}
|
||||
if (cpus >= (totalCPU + task.CPU)) && (mem >= (totalRAM + task.RAM)) &&
|
||||
(!baseSchedRef.wattsAsAResource || (watts >= (totalWatts + wattsConsideration))) {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
type MaxMin struct {
|
||||
SchedPolicyState
|
||||
}
|
||||
|
||||
// Determine if the remaining space inside of the offer is enough for this
|
||||
// task that we need to create. If it is, create a TaskInfo and return it.
|
||||
func (s *MaxMin) CheckFit(
|
||||
spc SchedPolicyContext,
|
||||
i int,
|
||||
task def.Task,
|
||||
wattsConsideration float64,
|
||||
offer *mesos.Offer,
|
||||
totalCPU *float64,
|
||||
totalRAM *float64,
|
||||
totalWatts *float64) (bool, *mesos.TaskInfo) {
|
||||
|
||||
baseSchedRef := spc.(*baseScheduler)
|
||||
// Does the task fit.
|
||||
if s.takeOffer(spc, offer, task, *totalCPU, *totalRAM, *totalWatts) {
|
||||
|
||||
*totalWatts += wattsConsideration
|
||||
*totalCPU += task.CPU
|
||||
*totalRAM += task.RAM
|
||||
baseSchedRef.LogCoLocatedTasks(offer.GetSlaveId().GoString())
|
||||
|
||||
taskToSchedule := baseSchedRef.newTask(offer, task)
|
||||
|
||||
baseSchedRef.LogSchedTrace(taskToSchedule, offer)
|
||||
*task.Instances--
|
||||
|
||||
if *task.Instances <= 0 {
|
||||
// All instances of task have been scheduled, remove it.
|
||||
baseSchedRef.tasks = append(baseSchedRef.tasks[:i], baseSchedRef.tasks[i+1:]...)
|
||||
|
||||
if len(baseSchedRef.tasks) <= 0 {
|
||||
baseSchedRef.LogTerminateScheduler()
|
||||
close(baseSchedRef.Shutdown)
|
||||
}
|
||||
}
|
||||
|
||||
return true, taskToSchedule
|
||||
}
|
||||
return false, nil
|
||||
}
|
||||
|
||||
func (s *MaxMin) ConsumeOffers(spc SchedPolicyContext, driver sched.SchedulerDriver, offers []*mesos.Offer) {
|
||||
fmt.Println("Max-Min scheduling...")
|
||||
baseSchedRef := spc.(*baseScheduler)
|
||||
def.SortTasks(baseSchedRef.tasks, def.SortByWatts)
|
||||
baseSchedRef.LogOffersReceived(offers)
|
||||
|
||||
for _, offer := range offers {
|
||||
offerUtils.UpdateEnvironment(offer)
|
||||
select {
|
||||
case <-baseSchedRef.Shutdown:
|
||||
baseSchedRef.LogNoPendingTasksDeclineOffers(offer)
|
||||
driver.DeclineOffer(offer.Id, mesosUtils.LongFilter)
|
||||
baseSchedRef.LogNumberOfRunningTasks()
|
||||
continue
|
||||
default:
|
||||
}
|
||||
|
||||
tasks := []*mesos.TaskInfo{}
|
||||
|
||||
offerTaken := false
|
||||
totalWatts := 0.0
|
||||
totalCPU := 0.0
|
||||
totalRAM := 0.0
|
||||
|
||||
// Assumes s.tasks is ordered in non-decreasing median max-peak order
|
||||
|
||||
// Attempt to schedule a single instance of the heaviest workload available first.
|
||||
// Start from the back until one fits.
|
||||
|
||||
direction := false // True = Min Max, False = Max Min
|
||||
var index int
|
||||
start := true // If false then index has changed and need to keep it that way
|
||||
for i := 0; i < len(baseSchedRef.tasks); i++ {
|
||||
// We need to pick a min task or a max task
|
||||
// depending on the value of direction.
|
||||
if direction && start {
|
||||
index = 0
|
||||
} else if start {
|
||||
index = len(baseSchedRef.tasks) - i - 1
|
||||
}
|
||||
task := baseSchedRef.tasks[index]
|
||||
|
||||
wattsConsideration, err := def.WattsToConsider(task, baseSchedRef.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration.
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
// Don't take offer if it doesn't match our task's host requirement.
|
||||
if offerUtils.HostMismatch(*offer.Hostname, task.Host) {
|
||||
continue
|
||||
}
|
||||
|
||||
// TODO: Fix this so index doesn't need to be passed.
|
||||
taken, taskToSchedule := s.CheckFit(spc, index, task, wattsConsideration, offer,
|
||||
&totalCPU, &totalRAM, &totalWatts)
|
||||
|
||||
if taken {
|
||||
offerTaken = true
|
||||
tasks = append(tasks, taskToSchedule)
|
||||
// Need to change direction and set start to true.
|
||||
// Setting start to true would ensure that index be set accurately again.
|
||||
direction = !direction
|
||||
start = true
|
||||
i--
|
||||
} else {
|
||||
// Need to move index depending on the value of direction.
|
||||
if direction {
|
||||
index++
|
||||
start = false
|
||||
} else {
|
||||
index--
|
||||
start = false
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if offerTaken {
|
||||
baseSchedRef.LogTaskStarting(nil, offer)
|
||||
driver.LaunchTasks([]*mesos.OfferID{offer.Id}, tasks, mesosUtils.DefaultFilter)
|
||||
} else {
|
||||
// If there was no match for the task
|
||||
cpus, mem, watts := offerUtils.OfferAgg(offer)
|
||||
baseSchedRef.LogInsufficientResourcesDeclineOffer(offer, cpus, mem, watts)
|
||||
driver.DeclineOffer(offer.Id, mesosUtils.DefaultFilter)
|
||||
}
|
||||
}
|
||||
|
||||
// Switch scheduling policy only if feature enabled from CLI
|
||||
if baseSchedRef.schedPolSwitchEnabled {
|
||||
// Switching to a random scheduling policy.
|
||||
// TODO: Switch based on some criteria.
|
||||
index := rand.Intn(len(SchedPolicies))
|
||||
for _, v := range SchedPolicies {
|
||||
if index == 0 {
|
||||
spc.SwitchSchedPol(v)
|
||||
break
|
||||
}
|
||||
index--
|
||||
}
|
||||
}
|
||||
}
|
|
@ -2,19 +2,24 @@ package schedulers
|
|||
|
||||
import (
|
||||
"bitbucket.org/sunybingcloud/elektron/def"
|
||||
elecLogDef "bitbucket.org/sunybingcloud/elektron/logging/def"
|
||||
"bytes"
|
||||
"fmt"
|
||||
"github.com/golang/protobuf/proto"
|
||||
mesos "github.com/mesos/mesos-go/mesosproto"
|
||||
"github.com/mesos/mesos-go/mesosutil"
|
||||
sched "github.com/mesos/mesos-go/scheduler"
|
||||
"log"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
|
||||
// Implements mesos scheduler.
|
||||
type ElectronScheduler interface {
|
||||
sched.Scheduler
|
||||
init(opts ...schedPolicyOption)
|
||||
}
|
||||
|
||||
type base struct {
|
||||
type baseScheduler struct {
|
||||
ElectronScheduler
|
||||
SchedPolicyContext
|
||||
// Current scheduling policy used for resource offer consumption.
|
||||
curSchedPolicy SchedPolicyState
|
||||
|
||||
tasksCreated int
|
||||
tasksRunning int
|
||||
tasks []def.Task
|
||||
|
@ -24,7 +29,7 @@ type base struct {
|
|||
classMapWatts bool
|
||||
|
||||
// First set of PCP values are garbage values, signal to logger to start recording when we're
|
||||
// about to schedule a new task.
|
||||
// about to schedule a new task
|
||||
RecordPCP *bool
|
||||
|
||||
// This channel is closed when the program receives an interrupt,
|
||||
|
@ -38,55 +43,309 @@ type base struct {
|
|||
PCPLog chan struct{}
|
||||
|
||||
schedTrace *log.Logger
|
||||
|
||||
// Send the type of the message to be logged
|
||||
logMsgType chan elecLogDef.LogMessageType
|
||||
// Send the message to be logged
|
||||
logMsg chan string
|
||||
|
||||
mutex sync.Mutex
|
||||
|
||||
// Whether switching of scheduling policies at runtime has been enabled
|
||||
schedPolSwitchEnabled bool
|
||||
}
|
||||
|
||||
func (s *base) init(opts ...schedPolicyOption) {
|
||||
func (s *baseScheduler) init(opts ...schedPolicyOption) {
|
||||
for _, opt := range opts {
|
||||
// Applying options.
|
||||
// applying options
|
||||
if err := opt(s); err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
}
|
||||
s.running = make(map[string]map[string]bool)
|
||||
s.mutex = sync.Mutex{}
|
||||
}
|
||||
|
||||
func (s *base) OfferRescinded(_ sched.SchedulerDriver, offerID *mesos.OfferID) {
|
||||
log.Printf("Offer %s rescinded", offerID)
|
||||
func (s *baseScheduler) SwitchSchedPol(newSchedPol SchedPolicyState) {
|
||||
s.curSchedPolicy = newSchedPol
|
||||
}
|
||||
func (s *base) SlaveLost(_ sched.SchedulerDriver, slaveID *mesos.SlaveID) {
|
||||
log.Printf("Slave %s lost", slaveID)
|
||||
|
||||
func (s *baseScheduler) newTask(offer *mesos.Offer, task def.Task) *mesos.TaskInfo {
|
||||
taskName := fmt.Sprintf("%s-%d", task.Name, *task.Instances)
|
||||
s.tasksCreated++
|
||||
|
||||
if !*s.RecordPCP {
|
||||
// Turn on elecLogDef
|
||||
*s.RecordPCP = true
|
||||
time.Sleep(1 * time.Second) // Make sure we're recording by the time the first task starts
|
||||
}
|
||||
|
||||
// If this is our first time running into this Agent
|
||||
if _, ok := s.running[offer.GetSlaveId().GoString()]; !ok {
|
||||
s.running[offer.GetSlaveId().GoString()] = make(map[string]bool)
|
||||
}
|
||||
|
||||
// Add task to list of tasks running on node
|
||||
s.running[offer.GetSlaveId().GoString()][taskName] = true
|
||||
|
||||
resources := []*mesos.Resource{
|
||||
mesosutil.NewScalarResource("cpus", task.CPU),
|
||||
mesosutil.NewScalarResource("mem", task.RAM),
|
||||
}
|
||||
|
||||
if s.wattsAsAResource {
|
||||
if wattsToConsider, err := def.WattsToConsider(task, s.classMapWatts, offer); err == nil {
|
||||
s.LogTaskWattsConsideration(task, *offer.Hostname, wattsToConsider)
|
||||
resources = append(resources, mesosutil.NewScalarResource("watts", wattsToConsider))
|
||||
} else {
|
||||
// Error in determining wattsConsideration
|
||||
s.LogElectronError(err)
|
||||
}
|
||||
}
|
||||
|
||||
return &mesos.TaskInfo{
|
||||
Name: proto.String(taskName),
|
||||
TaskId: &mesos.TaskID{
|
||||
Value: proto.String("electron-" + taskName),
|
||||
},
|
||||
SlaveId: offer.SlaveId,
|
||||
Resources: resources,
|
||||
Command: &mesos.CommandInfo{
|
||||
Value: proto.String(task.CMD),
|
||||
},
|
||||
Container: &mesos.ContainerInfo{
|
||||
Type: mesos.ContainerInfo_DOCKER.Enum(),
|
||||
Docker: &mesos.ContainerInfo_DockerInfo{
|
||||
Image: proto.String(task.Image),
|
||||
Network: mesos.ContainerInfo_DockerInfo_BRIDGE.Enum(), // Run everything isolated
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
func (s *base) ExecutorLost(_ sched.SchedulerDriver,
|
||||
executorID *mesos.ExecutorID,
|
||||
|
||||
func (s *baseScheduler) OfferRescinded(_ sched.SchedulerDriver, offerID *mesos.OfferID) {
|
||||
s.LogOfferRescinded(offerID)
|
||||
}
|
||||
func (s *baseScheduler) SlaveLost(_ sched.SchedulerDriver, slaveID *mesos.SlaveID) {
|
||||
s.LogSlaveLost(slaveID)
|
||||
}
|
||||
func (s *baseScheduler) ExecutorLost(_ sched.SchedulerDriver, executorID *mesos.ExecutorID,
|
||||
slaveID *mesos.SlaveID, status int) {
|
||||
log.Printf("Executor %s on slave %s was lost", executorID, slaveID)
|
||||
s.LogExecutorLost(executorID, slaveID)
|
||||
}
|
||||
|
||||
func (s *base) Error(_ sched.SchedulerDriver, err string) {
|
||||
log.Printf("Receiving an error: %s", err)
|
||||
func (s *baseScheduler) Error(_ sched.SchedulerDriver, err string) {
|
||||
s.LogMesosError(err)
|
||||
}
|
||||
|
||||
func (s *base) FrameworkMessage(
|
||||
func (s *baseScheduler) FrameworkMessage(
|
||||
driver sched.SchedulerDriver,
|
||||
executorID *mesos.ExecutorID,
|
||||
slaveID *mesos.SlaveID,
|
||||
message string) {
|
||||
|
||||
log.Println("Getting a framework message: ", message)
|
||||
log.Printf("Received a framework message from some unknown source: %s", *executorID.Value)
|
||||
s.LogFrameworkMessage(executorID, slaveID, message)
|
||||
}
|
||||
|
||||
func (s *base) Registered(
|
||||
func (s *baseScheduler) Registered(
|
||||
_ sched.SchedulerDriver,
|
||||
frameworkID *mesos.FrameworkID,
|
||||
masterInfo *mesos.MasterInfo) {
|
||||
log.Printf("Framework %s registered with master %s", frameworkID, masterInfo)
|
||||
s.LogFrameworkRegistered(frameworkID, masterInfo)
|
||||
}
|
||||
|
||||
func (s *base) Reregistered(_ sched.SchedulerDriver, masterInfo *mesos.MasterInfo) {
|
||||
log.Printf("Framework re-registered with master %s", masterInfo)
|
||||
func (s *baseScheduler) Reregistered(_ sched.SchedulerDriver, masterInfo *mesos.MasterInfo) {
|
||||
s.LogFrameworkReregistered(masterInfo)
|
||||
}
|
||||
|
||||
func (s *base) Disconnected(sched.SchedulerDriver) {
|
||||
log.Println("Framework disconnected with master")
|
||||
func (s *baseScheduler) Disconnected(sched.SchedulerDriver) {
|
||||
s.LogDisconnected()
|
||||
}
|
||||
|
||||
func (s *baseScheduler) ResourceOffers(driver sched.SchedulerDriver, offers []*mesos.Offer) {
|
||||
s.curSchedPolicy.ConsumeOffers(s, driver, offers)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) StatusUpdate(driver sched.SchedulerDriver, status *mesos.TaskStatus) {
|
||||
s.LogTaskStatusUpdate(status)
|
||||
if *status.State == mesos.TaskState_TASK_RUNNING {
|
||||
s.tasksRunning++
|
||||
} else if IsTerminal(status.State) {
|
||||
delete(s.running[status.GetSlaveId().GoString()], *status.TaskId.Value)
|
||||
s.tasksRunning--
|
||||
if s.tasksRunning == 0 {
|
||||
select {
|
||||
case <-s.Shutdown:
|
||||
close(s.Done)
|
||||
default:
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *baseScheduler) Log(lmt elecLogDef.LogMessageType, msg string) {
|
||||
s.mutex.Lock()
|
||||
s.logMsgType <- lmt
|
||||
s.logMsg <- msg
|
||||
s.mutex.Unlock()
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogTaskStarting(ts *def.Task, offer *mesos.Offer) {
|
||||
lmt := elecLogDef.GENERAL
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
var msg string
|
||||
if ts == nil {
|
||||
msg = msgColor.Sprintf("TASKS STARTING... host = [%s]", offer.GetHostname())
|
||||
} else {
|
||||
msg = msgColor.Sprintf("TASK STARTING... task = [%s], Instance = %d, host = [%s]",
|
||||
ts.Name, *ts.Instances, offer.GetHostname())
|
||||
}
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogTaskWattsConsideration(ts def.Task, host string, wattsToConsider float64) {
|
||||
lmt := elecLogDef.GENERAL
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprintf("Watts considered for task[%s] and host[%s] = %f Watts",
|
||||
ts.Name, host, wattsToConsider)
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogOffersReceived(offers []*mesos.Offer) {
|
||||
lmt := elecLogDef.GENERAL
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprintf("Received %d resource offers", len(offers))
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogNoPendingTasksDeclineOffers(offer *mesos.Offer) {
|
||||
lmt := elecLogDef.WARNING
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprintf("DECLINING OFFER for host[%s]... "+
|
||||
"No tasks left to schedule", offer.GetHostname())
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogNumberOfRunningTasks() {
|
||||
lmt := elecLogDef.GENERAL
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprintf("Number of tasks still running = %d", s.tasksRunning)
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogCoLocatedTasks(slaveID string) {
|
||||
lmt := elecLogDef.GENERAL
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
buffer := bytes.Buffer{}
|
||||
buffer.WriteString(fmt.Sprintln("Colocated with:"))
|
||||
for taskName := range s.running[slaveID] {
|
||||
buffer.WriteString(fmt.Sprintln(taskName))
|
||||
}
|
||||
msg := msgColor.Sprintf(buffer.String())
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogSchedTrace(taskToSchedule *mesos.TaskInfo, offer *mesos.Offer) {
|
||||
msg := fmt.Sprint(offer.GetHostname() + ":" + taskToSchedule.GetTaskId().GetValue())
|
||||
s.Log(elecLogDef.SCHED_TRACE, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogTerminateScheduler() {
|
||||
lmt := elecLogDef.GENERAL
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprint("Done scheduling all tasks!")
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogInsufficientResourcesDeclineOffer(offer *mesos.Offer,
|
||||
offerResources ...interface{}) {
|
||||
lmt := elecLogDef.WARNING
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
buffer := bytes.Buffer{}
|
||||
buffer.WriteString(fmt.Sprintln("DECLINING OFFER... Offer has insufficient resources to launch a task"))
|
||||
buffer.WriteString(fmt.Sprintf("Offer Resources <CPU: %f, RAM: %f, Watts: %f>", offerResources...))
|
||||
msg := msgColor.Sprint(buffer.String())
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogOfferRescinded(offerID *mesos.OfferID) {
|
||||
lmt := elecLogDef.ERROR
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprintf("OFFER RESCINDED: OfferID = %s", offerID)
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogSlaveLost(slaveID *mesos.SlaveID) {
|
||||
lmt := elecLogDef.ERROR
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprintf("SLAVE LOST: SlaveID = %s", slaveID)
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogExecutorLost(executorID *mesos.ExecutorID, slaveID *mesos.SlaveID) {
|
||||
lmt := elecLogDef.ERROR
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprintf("EXECUTOR LOST: ExecutorID = %s, SlaveID = %s", executorID, slaveID)
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogFrameworkMessage(executorID *mesos.ExecutorID,
|
||||
slaveID *mesos.SlaveID, message string) {
|
||||
lmt := elecLogDef.GENERAL
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprintf("Received Framework message from executor [%s]: %s", executorID, message)
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogMesosError(err string) {
|
||||
lmt := elecLogDef.ERROR
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprintf("MESOS ERROR: %s", err)
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogElectronError(err error) {
|
||||
lmt := elecLogDef.ERROR
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprintf("ELECTRON ERROR: %v", err)
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogFrameworkRegistered(frameworkID *mesos.FrameworkID,
|
||||
masterInfo *mesos.MasterInfo) {
|
||||
lmt := elecLogDef.SUCCESS
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprintf("FRAMEWORK REGISTERED! frameworkID = %s, master = %s",
|
||||
frameworkID, masterInfo)
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogFrameworkReregistered(masterInfo *mesos.MasterInfo) {
|
||||
lmt := elecLogDef.GENERAL
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprintf("Framework re-registered with master %s", masterInfo)
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogDisconnected() {
|
||||
lmt := elecLogDef.WARNING
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := msgColor.Sprint("Framework disconnected with master")
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
||||
func (s *baseScheduler) LogTaskStatusUpdate(status *mesos.TaskStatus) {
|
||||
var lmt elecLogDef.LogMessageType
|
||||
switch *status.State {
|
||||
case mesos.TaskState_TASK_ERROR, mesos.TaskState_TASK_FAILED,
|
||||
mesos.TaskState_TASK_KILLED, mesos.TaskState_TASK_LOST:
|
||||
lmt = elecLogDef.ERROR
|
||||
case mesos.TaskState_TASK_FINISHED:
|
||||
lmt = elecLogDef.SUCCESS
|
||||
default:
|
||||
lmt = elecLogDef.GENERAL
|
||||
}
|
||||
msgColor := elecLogDef.LogMessageColors[lmt]
|
||||
msg := elecLogDef.LogMessageColors[elecLogDef.GENERAL].Sprintf("Task Status received for task [%s] --> %s",
|
||||
*status.TaskId.Value, msgColor.Sprint(NameFor(status.State)))
|
||||
s.Log(lmt, msg)
|
||||
}
|
||||
|
|
|
@ -5,108 +5,49 @@ import (
|
|||
"bitbucket.org/sunybingcloud/elektron/utilities/mesosUtils"
|
||||
"bitbucket.org/sunybingcloud/elektron/utilities/offerUtils"
|
||||
"fmt"
|
||||
"github.com/golang/protobuf/proto"
|
||||
mesos "github.com/mesos/mesos-go/mesosproto"
|
||||
"github.com/mesos/mesos-go/mesosutil"
|
||||
sched "github.com/mesos/mesos-go/scheduler"
|
||||
"log"
|
||||
"time"
|
||||
"math/rand"
|
||||
)
|
||||
|
||||
// Decides if to take an offer or not.
|
||||
func (s *BinPacking) takeOffer(offer *mesos.Offer, task def.Task, totalCPU, totalRAM, totalWatts float64) bool {
|
||||
// Decides if to take an offer or not
|
||||
func (s *BinPackSortedWatts) takeOffer(spc SchedPolicyContext, offer *mesos.Offer, task def.Task, totalCPU, totalRAM, totalWatts float64) bool {
|
||||
|
||||
baseSchedRef := spc.(*baseScheduler)
|
||||
cpus, mem, watts := offerUtils.OfferAgg(offer)
|
||||
|
||||
//TODO: Insert watts calculation here instead of taking them as a parameter
|
||||
|
||||
wattsConsideration, err := def.WattsToConsider(task, s.classMapWatts, offer)
|
||||
wattsConsideration, err := def.WattsToConsider(task, baseSchedRef.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration.
|
||||
log.Fatal(err)
|
||||
}
|
||||
if (cpus >= (totalCPU + task.CPU)) && (mem >= (totalRAM + task.RAM)) &&
|
||||
(!s.wattsAsAResource || (watts >= (totalWatts + wattsConsideration))) {
|
||||
(!baseSchedRef.wattsAsAResource || (watts >= (totalWatts + wattsConsideration))) {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
type BinPacking struct {
|
||||
base // Type embedded to inherit common functions.
|
||||
type BinPackSortedWatts struct {
|
||||
SchedPolicyState
|
||||
}
|
||||
|
||||
// Initialization.
|
||||
func (s *BinPacking) init(opts ...schedPolicyOption) {
|
||||
s.base.init(opts...)
|
||||
// Sorting the tasks based on watts.
|
||||
def.SortTasks(s.tasks, def.SortByWatts)
|
||||
}
|
||||
|
||||
func (s *BinPacking) newTask(offer *mesos.Offer, task def.Task) *mesos.TaskInfo {
|
||||
taskName := fmt.Sprintf("%s-%d", task.Name, *task.Instances)
|
||||
s.tasksCreated++
|
||||
|
||||
if !*s.RecordPCP {
|
||||
// Turn on logging.
|
||||
*s.RecordPCP = true
|
||||
time.Sleep(1 * time.Second) // Make sure we're recording by the time the first task starts
|
||||
}
|
||||
|
||||
// If this is our first time running into this Agent.
|
||||
if _, ok := s.running[offer.GetSlaveId().GoString()]; !ok {
|
||||
s.running[offer.GetSlaveId().GoString()] = make(map[string]bool)
|
||||
}
|
||||
|
||||
// Add task to list of tasks running on node.
|
||||
s.running[offer.GetSlaveId().GoString()][taskName] = true
|
||||
|
||||
resources := []*mesos.Resource{
|
||||
mesosutil.NewScalarResource("cpus", task.CPU),
|
||||
mesosutil.NewScalarResource("mem", task.RAM),
|
||||
}
|
||||
|
||||
if s.wattsAsAResource {
|
||||
if wattsToConsider, err := def.WattsToConsider(task, s.classMapWatts, offer); err == nil {
|
||||
log.Printf("Watts considered for host[%s] and task[%s] = %f", *offer.Hostname, task.Name, wattsToConsider)
|
||||
resources = append(resources, mesosutil.NewScalarResource("watts", wattsToConsider))
|
||||
} else {
|
||||
// Error in determining wattsToConsider.
|
||||
log.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
return &mesos.TaskInfo{
|
||||
Name: proto.String(taskName),
|
||||
TaskId: &mesos.TaskID{
|
||||
Value: proto.String("elektron-" + taskName),
|
||||
},
|
||||
SlaveId: offer.SlaveId,
|
||||
Resources: resources,
|
||||
Command: &mesos.CommandInfo{
|
||||
Value: proto.String(task.CMD),
|
||||
},
|
||||
Container: &mesos.ContainerInfo{
|
||||
Type: mesos.ContainerInfo_DOCKER.Enum(),
|
||||
Docker: &mesos.ContainerInfo_DockerInfo{
|
||||
Image: proto.String(task.Image),
|
||||
Network: mesos.ContainerInfo_DockerInfo_BRIDGE.Enum(), // Run everything isolated
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (s *BinPacking) ResourceOffers(driver sched.SchedulerDriver, offers []*mesos.Offer) {
|
||||
log.Printf("Received %d resource offers", len(offers))
|
||||
func (s *BinPackSortedWatts) ConsumeOffers(spc SchedPolicyContext, driver sched.SchedulerDriver, offers []*mesos.Offer) {
|
||||
fmt.Println("BPSW scheduling...")
|
||||
baseSchedRef := spc.(*baseScheduler)
|
||||
def.SortTasks(baseSchedRef.tasks, def.SortByWatts)
|
||||
baseSchedRef.LogOffersReceived(offers)
|
||||
|
||||
for _, offer := range offers {
|
||||
offerUtils.UpdateEnvironment(offer)
|
||||
select {
|
||||
case <-s.Shutdown:
|
||||
log.Println("Done scheduling tasks: declining offer on [", offer.GetHostname(), "]")
|
||||
case <-baseSchedRef.Shutdown:
|
||||
baseSchedRef.LogNoPendingTasksDeclineOffers(offer)
|
||||
driver.DeclineOffer(offer.Id, mesosUtils.LongFilter)
|
||||
|
||||
log.Println("Number of tasks still running: ", s.tasksRunning)
|
||||
baseSchedRef.LogNumberOfRunningTasks()
|
||||
continue
|
||||
default:
|
||||
}
|
||||
|
@ -117,9 +58,9 @@ func (s *BinPacking) ResourceOffers(driver sched.SchedulerDriver, offers []*meso
|
|||
totalWatts := 0.0
|
||||
totalCPU := 0.0
|
||||
totalRAM := 0.0
|
||||
for i := 0; i < len(s.tasks); i++ {
|
||||
task := s.tasks[i]
|
||||
wattsConsideration, err := def.WattsToConsider(task, s.classMapWatts, offer)
|
||||
for i := 0; i < len(baseSchedRef.tasks); i++ {
|
||||
task := baseSchedRef.tasks[i]
|
||||
wattsConsideration, err := def.WattsToConsider(task, baseSchedRef.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration.
|
||||
log.Fatal(err)
|
||||
|
@ -131,29 +72,28 @@ func (s *BinPacking) ResourceOffers(driver sched.SchedulerDriver, offers []*meso
|
|||
}
|
||||
|
||||
for *task.Instances > 0 {
|
||||
// Does the task fit.
|
||||
if s.takeOffer(offer, task, totalCPU, totalRAM, totalWatts) {
|
||||
// Does the task fit
|
||||
if s.takeOffer(spc, offer, task, totalCPU, totalRAM, totalWatts) {
|
||||
|
||||
offerTaken = true
|
||||
totalWatts += wattsConsideration
|
||||
totalCPU += task.CPU
|
||||
totalRAM += task.RAM
|
||||
log.Println("Co-Located with: ")
|
||||
coLocated(s.running[offer.GetSlaveId().GoString()])
|
||||
taskToSchedule := s.newTask(offer, task)
|
||||
baseSchedRef.LogCoLocatedTasks(offer.GetSlaveId().GoString())
|
||||
taskToSchedule := baseSchedRef.newTask(offer, task)
|
||||
tasks = append(tasks, taskToSchedule)
|
||||
|
||||
fmt.Println("Inst: ", *task.Instances)
|
||||
s.schedTrace.Print(offer.GetHostname() + ":" + taskToSchedule.GetTaskId().GetValue())
|
||||
baseSchedRef.LogSchedTrace(taskToSchedule, offer)
|
||||
*task.Instances--
|
||||
|
||||
if *task.Instances <= 0 {
|
||||
// All instances of task have been scheduled, remove it.
|
||||
s.tasks = append(s.tasks[:i], s.tasks[i+1:]...)
|
||||
// All instances of task have been scheduled, remove it
|
||||
baseSchedRef.tasks = append(baseSchedRef.tasks[:i],
|
||||
baseSchedRef.tasks[i+1:]...)
|
||||
|
||||
if len(s.tasks) <= 0 {
|
||||
log.Println("Done scheduling all tasks")
|
||||
close(s.Shutdown)
|
||||
if len(baseSchedRef.tasks) <= 0 {
|
||||
baseSchedRef.LogTerminateScheduler()
|
||||
close(baseSchedRef.Shutdown)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
|
@ -163,35 +103,28 @@ func (s *BinPacking) ResourceOffers(driver sched.SchedulerDriver, offers []*meso
|
|||
}
|
||||
|
||||
if offerTaken {
|
||||
log.Printf("Starting on [%s]\n", offer.GetHostname())
|
||||
baseSchedRef.LogTaskStarting(nil, offer)
|
||||
driver.LaunchTasks([]*mesos.OfferID{offer.Id}, tasks, mesosUtils.DefaultFilter)
|
||||
} else {
|
||||
|
||||
// If there was no match for the task.
|
||||
fmt.Println("There is not enough resources to launch a task:")
|
||||
// If there was no match for the task
|
||||
cpus, mem, watts := offerUtils.OfferAgg(offer)
|
||||
|
||||
log.Printf("<CPU: %f, RAM: %f, Watts: %f>\n", cpus, mem, watts)
|
||||
baseSchedRef.LogInsufficientResourcesDeclineOffer(offer, cpus, mem, watts)
|
||||
driver.DeclineOffer(offer.Id, mesosUtils.DefaultFilter)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *BinPacking) StatusUpdate(driver sched.SchedulerDriver, status *mesos.TaskStatus) {
|
||||
log.Printf("Received task status [%s] for task [%s]", NameFor(status.State), *status.TaskId.Value)
|
||||
|
||||
if *status.State == mesos.TaskState_TASK_RUNNING {
|
||||
s.tasksRunning++
|
||||
} else if IsTerminal(status.State) {
|
||||
delete(s.running[status.GetSlaveId().GoString()], *status.TaskId.Value)
|
||||
s.tasksRunning--
|
||||
if s.tasksRunning == 0 {
|
||||
select {
|
||||
case <-s.Shutdown:
|
||||
close(s.Done)
|
||||
default:
|
||||
// Switch scheduling policy only if feature enabled from CLI
|
||||
if baseSchedRef.schedPolSwitchEnabled {
|
||||
// Switching to a random scheduling policy.
|
||||
// TODO: Switch based on some criteria.
|
||||
index := rand.Intn(len(SchedPolicies))
|
||||
for _, v := range SchedPolicies {
|
||||
if index == 0 {
|
||||
spc.SwitchSchedPol(v)
|
||||
break
|
||||
}
|
||||
index--
|
||||
}
|
||||
}
|
||||
log.Printf("DONE: Task status [%s] for task [%s]", NameFor(status.State), *status.TaskId.Value)
|
||||
}
|
||||
|
|
73
schedulers/electronScheduler.go
Normal file
73
schedulers/electronScheduler.go
Normal file
|
@ -0,0 +1,73 @@
|
|||
package schedulers
|
||||
|
||||
import (
|
||||
"bitbucket.org/sunybingcloud/elektron/def"
|
||||
elecLogDef "bitbucket.org/sunybingcloud/elektron/logging/def"
|
||||
mesos "github.com/mesos/mesos-go/mesosproto"
|
||||
sched "github.com/mesos/mesos-go/scheduler"
|
||||
)
|
||||
|
||||
// Implements mesos scheduler.
|
||||
type ElectronScheduler interface {
|
||||
sched.Scheduler
|
||||
init(opts ...schedPolicyOption)
|
||||
|
||||
// Interface for log messages.
|
||||
// Every ElectronScheduler implementer should provide definitions for these functions.
|
||||
// This interface serves as a template to maintain consistent log messages.
|
||||
// Each of these functions are supposed to call the Log(...) that sends the
|
||||
// log message type, and the log message to the corresponding channels.
|
||||
|
||||
// Pass the logMessageType and the logMessage to the loggers for logging.
|
||||
Log(logMType elecLogDef.LogMessageType, logMsg string)
|
||||
// To be called when about to launch a task.
|
||||
// Log message indicating that a task is about to start executing.
|
||||
// Also, log the host on which the task is going to be launched.
|
||||
LogTaskStarting(ts *def.Task, offer *mesos.Offer)
|
||||
// To be called when an offer is taken.
|
||||
// Log the chosen watts attribute for the task that has fit an offer.
|
||||
LogTaskWattsConsideration(ts def.Task, host string, wattsToConsider float64)
|
||||
// To be called when offers are received from Mesos.
|
||||
// Log the number of offers received and/or information about the received offers.
|
||||
LogOffersReceived(offers []*mesos.Offer)
|
||||
// To be called when a scheduling policy declines Mesos offers, as
|
||||
// there are no tasks pending to be scheduled.
|
||||
// Log the host information corresponding to the offers that were declined.
|
||||
LogNoPendingTasksDeclineOffers(offers *mesos.Offer)
|
||||
// Log the number of tasks that are currently executing on the cluster.
|
||||
LogNumberOfRunningTasks()
|
||||
// To be called when a task fits a Mesos offer.
|
||||
// Log information on the tasks that the new task is going to be coLocated with.
|
||||
// Uses the coLocated(...) utility in helpers.go.
|
||||
LogCoLocatedTasks(slaveID string)
|
||||
// Log the scheduled trace of task.
|
||||
// The schedTrace includes the TaskID and the hostname of the node
|
||||
// where is the task is going to be launched.
|
||||
LogSchedTrace(taskToSchedule *mesos.TaskInfo, offer *mesos.Offer)
|
||||
// To be called when all the tasks have completed executing.
|
||||
// Log message indicating that Electron has scheduled all the tasks.
|
||||
LogTerminateScheduler()
|
||||
// To be called when the offer is not consumed.
|
||||
// Log message to indicate that the offer had insufficient resources.
|
||||
LogInsufficientResourcesDeclineOffer(offer *mesos.Offer, offerResources ...interface{})
|
||||
// To be called when offer is rescinded by Mesos.
|
||||
LogOfferRescinded(offerID *mesos.OfferID)
|
||||
// To be called when Mesos agent is lost
|
||||
LogSlaveLost(slaveID *mesos.SlaveID)
|
||||
// To be called when executor lost.
|
||||
LogExecutorLost(executorID *mesos.ExecutorID, slaveID *mesos.SlaveID)
|
||||
// Log a mesos error
|
||||
LogMesosError(err string)
|
||||
// Log an Electron error
|
||||
LogElectronError(err error)
|
||||
// Log Framework message
|
||||
LogFrameworkMessage(executorID *mesos.ExecutorID, slaveID *mesos.SlaveID, message string)
|
||||
// Log Framework has been registered
|
||||
LogFrameworkRegistered(frameworkID *mesos.FrameworkID, masterInfo *mesos.MasterInfo)
|
||||
// Log Framework has been re-registered
|
||||
LogFrameworkReregistered(masterInfo *mesos.MasterInfo)
|
||||
// Log Framework has been disconnected from the Mesos master
|
||||
LogDisconnected()
|
||||
// Log Status update of a task
|
||||
LogTaskStatusUpdate(status *mesos.TaskStatus)
|
||||
}
|
|
@ -5,27 +5,24 @@ import (
|
|||
"bitbucket.org/sunybingcloud/elektron/utilities/mesosUtils"
|
||||
"bitbucket.org/sunybingcloud/elektron/utilities/offerUtils"
|
||||
"fmt"
|
||||
"github.com/golang/protobuf/proto"
|
||||
mesos "github.com/mesos/mesos-go/mesosproto"
|
||||
"github.com/mesos/mesos-go/mesosutil"
|
||||
sched "github.com/mesos/mesos-go/scheduler"
|
||||
"log"
|
||||
"time"
|
||||
"math/rand"
|
||||
)
|
||||
|
||||
// Decides if to take an offer or not.
|
||||
func (s *FirstFit) takeOffer(offer *mesos.Offer, task def.Task) bool {
|
||||
|
||||
// Decides if to take an offer or not
|
||||
func (s *FirstFit) takeOffer(spc SchedPolicyContext, offer *mesos.Offer, task def.Task) bool {
|
||||
baseSchedRef := spc.(*baseScheduler)
|
||||
cpus, mem, watts := offerUtils.OfferAgg(offer)
|
||||
|
||||
//TODO: Insert watts calculation here instead of taking them as a parameter
|
||||
|
||||
wattsConsideration, err := def.WattsToConsider(task, s.classMapWatts, offer)
|
||||
wattsConsideration, err := def.WattsToConsider(task, baseSchedRef.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration.
|
||||
log.Fatal(err)
|
||||
// Error in determining wattsConsideration
|
||||
baseSchedRef.LogElectronError(err)
|
||||
}
|
||||
if cpus >= task.CPU && mem >= task.RAM && (!s.wattsAsAResource || watts >= wattsConsideration) {
|
||||
if cpus >= task.CPU && mem >= task.RAM && (!baseSchedRef.wattsAsAResource || watts >= wattsConsideration) {
|
||||
return true
|
||||
}
|
||||
|
||||
|
@ -34,120 +31,61 @@ func (s *FirstFit) takeOffer(offer *mesos.Offer, task def.Task) bool {
|
|||
|
||||
// Elektron scheduler implements the Scheduler interface.
|
||||
type FirstFit struct {
|
||||
base // Type embedded to inherit common functions
|
||||
SchedPolicyState
|
||||
}
|
||||
|
||||
// Initialization.
|
||||
func (s *FirstFit) init(opts ...schedPolicyOption) {
|
||||
s.base.init(opts...)
|
||||
}
|
||||
|
||||
func (s *FirstFit) newTask(offer *mesos.Offer, task def.Task) *mesos.TaskInfo {
|
||||
taskName := fmt.Sprintf("%s-%d", task.Name, *task.Instances)
|
||||
s.tasksCreated++
|
||||
|
||||
if !*s.RecordPCP {
|
||||
// Turn on logging.
|
||||
*s.RecordPCP = true
|
||||
time.Sleep(1 * time.Second) // Make sure we're recording by the time the first task starts.
|
||||
}
|
||||
|
||||
// If this is our first time running into this Agent.
|
||||
if _, ok := s.running[offer.GetSlaveId().GoString()]; !ok {
|
||||
s.running[offer.GetSlaveId().GoString()] = make(map[string]bool)
|
||||
}
|
||||
|
||||
// Add task to list of tasks running on node.
|
||||
s.running[offer.GetSlaveId().GoString()][taskName] = true
|
||||
|
||||
resources := []*mesos.Resource{
|
||||
mesosutil.NewScalarResource("cpus", task.CPU),
|
||||
mesosutil.NewScalarResource("mem", task.RAM),
|
||||
}
|
||||
|
||||
if s.wattsAsAResource {
|
||||
if wattsToConsider, err := def.WattsToConsider(task, s.classMapWatts, offer); err == nil {
|
||||
log.Printf("Watts considered for host[%s] and task[%s] = %f", *offer.Hostname, task.Name, wattsToConsider)
|
||||
resources = append(resources, mesosutil.NewScalarResource("watts", wattsToConsider))
|
||||
} else {
|
||||
// Error in determining wattsConsideration.
|
||||
log.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
return &mesos.TaskInfo{
|
||||
Name: proto.String(taskName),
|
||||
TaskId: &mesos.TaskID{
|
||||
Value: proto.String("elektron-" + taskName),
|
||||
},
|
||||
SlaveId: offer.SlaveId,
|
||||
Resources: resources,
|
||||
Command: &mesos.CommandInfo{
|
||||
Value: proto.String(task.CMD),
|
||||
},
|
||||
Container: &mesos.ContainerInfo{
|
||||
Type: mesos.ContainerInfo_DOCKER.Enum(),
|
||||
Docker: &mesos.ContainerInfo_DockerInfo{
|
||||
Image: proto.String(task.Image),
|
||||
Network: mesos.ContainerInfo_DockerInfo_BRIDGE.Enum(), // Run everything isolated
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (s *FirstFit) ResourceOffers(driver sched.SchedulerDriver, offers []*mesos.Offer) {
|
||||
log.Printf("Received %d resource offers", len(offers))
|
||||
func (s *FirstFit) ConsumeOffers(spc SchedPolicyContext, driver sched.SchedulerDriver, offers []*mesos.Offer) {
|
||||
fmt.Println("FirstFit scheduling...")
|
||||
baseSchedRef := spc.(*baseScheduler)
|
||||
baseSchedRef.LogOffersReceived(offers)
|
||||
|
||||
for _, offer := range offers {
|
||||
offerUtils.UpdateEnvironment(offer)
|
||||
select {
|
||||
case <-s.Shutdown:
|
||||
log.Println("Done scheduling tasks: declining offer on [", offer.GetHostname(), "]")
|
||||
case <-baseSchedRef.Shutdown:
|
||||
baseSchedRef.LogNoPendingTasksDeclineOffers(offer)
|
||||
driver.DeclineOffer(offer.Id, mesosUtils.LongFilter)
|
||||
|
||||
log.Println("Number of tasks still running: ", s.tasksRunning)
|
||||
baseSchedRef.LogNumberOfRunningTasks()
|
||||
continue
|
||||
default:
|
||||
}
|
||||
|
||||
tasks := []*mesos.TaskInfo{}
|
||||
|
||||
// First fit strategy.
|
||||
// First fit strategy
|
||||
offerTaken := false
|
||||
for i := 0; i < len(s.tasks); i++ {
|
||||
task := s.tasks[i]
|
||||
for i := 0; i < len(baseSchedRef.tasks); i++ {
|
||||
task := baseSchedRef.tasks[i]
|
||||
|
||||
// Don't take offer if it doesn't match our task's host requirement.
|
||||
if offerUtils.HostMismatch(*offer.Hostname, task.Host) {
|
||||
continue
|
||||
}
|
||||
|
||||
// Decision to take the offer or not.
|
||||
if s.takeOffer(offer, task) {
|
||||
// Decision to take the offer or not
|
||||
if s.takeOffer(spc, offer, task) {
|
||||
|
||||
log.Println("Co-Located with: ")
|
||||
coLocated(s.running[offer.GetSlaveId().GoString()])
|
||||
baseSchedRef.LogCoLocatedTasks(offer.GetSlaveId().GoString())
|
||||
|
||||
taskToSchedule := s.newTask(offer, task)
|
||||
taskToSchedule := baseSchedRef.newTask(offer, task)
|
||||
tasks = append(tasks, taskToSchedule)
|
||||
|
||||
log.Printf("Starting %s on [%s]\n", task.Name, offer.GetHostname())
|
||||
baseSchedRef.LogTaskStarting(&task, offer)
|
||||
driver.LaunchTasks([]*mesos.OfferID{offer.Id}, tasks, mesosUtils.DefaultFilter)
|
||||
|
||||
offerTaken = true
|
||||
|
||||
fmt.Println("Inst: ", *task.Instances)
|
||||
s.schedTrace.Print(offer.GetHostname() + ":" + taskToSchedule.GetTaskId().GetValue())
|
||||
baseSchedRef.LogSchedTrace(taskToSchedule, offer)
|
||||
*task.Instances--
|
||||
|
||||
if *task.Instances <= 0 {
|
||||
// All instances of task have been scheduled, remove it.
|
||||
s.tasks[i] = s.tasks[len(s.tasks)-1]
|
||||
s.tasks = s.tasks[:len(s.tasks)-1]
|
||||
// All instances of task have been scheduled, remove it
|
||||
baseSchedRef.tasks[i] = baseSchedRef.tasks[len(baseSchedRef.tasks)-1]
|
||||
baseSchedRef.tasks = baseSchedRef.tasks[:len(baseSchedRef.tasks)-1]
|
||||
|
||||
if len(s.tasks) <= 0 {
|
||||
log.Println("Done scheduling all tasks")
|
||||
close(s.Shutdown)
|
||||
if len(baseSchedRef.tasks) <= 0 {
|
||||
baseSchedRef.LogTerminateScheduler()
|
||||
close(baseSchedRef.Shutdown)
|
||||
}
|
||||
}
|
||||
break // Offer taken, move on.
|
||||
|
@ -156,31 +94,23 @@ func (s *FirstFit) ResourceOffers(driver sched.SchedulerDriver, offers []*mesos.
|
|||
|
||||
// If there was no match for the task.
|
||||
if !offerTaken {
|
||||
fmt.Println("There is not enough resources to launch a task:")
|
||||
cpus, mem, watts := offerUtils.OfferAgg(offer)
|
||||
|
||||
log.Printf("<CPU: %f, RAM: %f, Watts: %f>\n", cpus, mem, watts)
|
||||
baseSchedRef.LogInsufficientResourcesDeclineOffer(offer, cpus, mem, watts)
|
||||
driver.DeclineOffer(offer.Id, mesosUtils.DefaultFilter)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
func (s *FirstFit) StatusUpdate(driver sched.SchedulerDriver, status *mesos.TaskStatus) {
|
||||
log.Printf("Received task status [%s] for task [%s]", NameFor(status.State), *status.TaskId.Value)
|
||||
|
||||
if *status.State == mesos.TaskState_TASK_RUNNING {
|
||||
s.tasksRunning++
|
||||
} else if IsTerminal(status.State) {
|
||||
delete(s.running[status.GetSlaveId().GoString()], *status.TaskId.Value)
|
||||
s.tasksRunning--
|
||||
if s.tasksRunning == 0 {
|
||||
select {
|
||||
case <-s.Shutdown:
|
||||
close(s.Done)
|
||||
default:
|
||||
// Switch scheduling policy only if feature enabled from CLI
|
||||
if baseSchedRef.schedPolSwitchEnabled {
|
||||
// Switching to a random scheduling policy.
|
||||
// TODO: Switch based on some criteria.
|
||||
index := rand.Intn(len(SchedPolicies))
|
||||
for _, v := range SchedPolicies {
|
||||
if index == 0 {
|
||||
spc.SwitchSchedPol(v)
|
||||
break
|
||||
}
|
||||
index--
|
||||
}
|
||||
}
|
||||
log.Printf("DONE: Task status [%s] for task [%s]", NameFor(status.State), *status.TaskId.Value)
|
||||
}
|
||||
|
|
|
@ -4,18 +4,16 @@ import (
|
|||
"bitbucket.org/sunybingcloud/elektron/constants"
|
||||
"bitbucket.org/sunybingcloud/elektron/def"
|
||||
"errors"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
elecLogDef "bitbucket.org/sunybingcloud/elektron/logging/def"
|
||||
)
|
||||
|
||||
func coLocated(tasks map[string]bool) {
|
||||
func coLocated(tasks map[string]bool, s baseScheduler) {
|
||||
|
||||
for task := range tasks {
|
||||
log.Println(task)
|
||||
s.Log(elecLogDef.GENERAL, task)
|
||||
}
|
||||
|
||||
fmt.Println("---------------------")
|
||||
s.Log(elecLogDef.GENERAL, "---------------------")
|
||||
}
|
||||
|
||||
// Get the powerClass of the given hostname.
|
||||
|
@ -28,15 +26,26 @@ func hostToPowerClass(hostName string) string {
|
|||
return ""
|
||||
}
|
||||
|
||||
// Scheduler policy options to help initialize schedulers.
|
||||
// scheduler policy options to help initialize schedulers
|
||||
type schedPolicyOption func(e ElectronScheduler) error
|
||||
|
||||
func WithSchedPolicy(schedPolicyName string) schedPolicyOption {
|
||||
return func(s ElectronScheduler) error {
|
||||
if schedPolicy, ok := SchedPolicies[schedPolicyName]; !ok {
|
||||
return errors.New("Incorrect scheduling policy.")
|
||||
} else {
|
||||
s.(*baseScheduler).curSchedPolicy = schedPolicy
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func WithTasks(ts []def.Task) schedPolicyOption {
|
||||
return func(s ElectronScheduler) error {
|
||||
if ts == nil {
|
||||
return errors.New("Task[] is empty.")
|
||||
} else {
|
||||
s.(*base).tasks = ts
|
||||
s.(*baseScheduler).tasks = ts
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
@ -44,43 +53,31 @@ func WithTasks(ts []def.Task) schedPolicyOption {
|
|||
|
||||
func WithWattsAsAResource(waar bool) schedPolicyOption {
|
||||
return func(s ElectronScheduler) error {
|
||||
s.(*base).wattsAsAResource = waar
|
||||
s.(*baseScheduler).wattsAsAResource = waar
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func WithClassMapWatts(cmw bool) schedPolicyOption {
|
||||
return func(s ElectronScheduler) error {
|
||||
s.(*base).classMapWatts = cmw
|
||||
s.(*baseScheduler).classMapWatts = cmw
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func WithRecordPCP(recordPCP *bool) schedPolicyOption {
|
||||
return func(s ElectronScheduler) error {
|
||||
s.(*base).RecordPCP = recordPCP
|
||||
s.(*baseScheduler).RecordPCP = recordPCP
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func WithSchedTracePrefix(schedTracePrefix string) schedPolicyOption {
|
||||
return func(s ElectronScheduler) error {
|
||||
logFile, err := os.Create("./" + schedTracePrefix + "_schedTrace.log")
|
||||
if err != nil {
|
||||
return err
|
||||
} else {
|
||||
s.(*base).schedTrace = log.New(logFile, "", log.LstdFlags)
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func WithShutdown(shutdown chan struct{}) schedPolicyOption {
|
||||
return func(s ElectronScheduler) error {
|
||||
if shutdown == nil {
|
||||
return errors.New("Shutdown channel is nil.")
|
||||
} else {
|
||||
s.(*base).Shutdown = shutdown
|
||||
s.(*baseScheduler).Shutdown = shutdown
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
@ -91,7 +88,7 @@ func WithDone(done chan struct{}) schedPolicyOption {
|
|||
if done == nil {
|
||||
return errors.New("Done channel is nil.")
|
||||
} else {
|
||||
s.(*base).Done = done
|
||||
s.(*baseScheduler).Done = done
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
@ -102,8 +99,23 @@ func WithPCPLog(pcpLog chan struct{}) schedPolicyOption {
|
|||
if pcpLog == nil {
|
||||
return errors.New("PCPLog channel is nil.")
|
||||
} else {
|
||||
s.(*base).PCPLog = pcpLog
|
||||
s.(*baseScheduler).PCPLog = pcpLog
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func WithLoggingChannels(lmt chan elecLogDef.LogMessageType, msg chan string) schedPolicyOption {
|
||||
return func(s ElectronScheduler) error {
|
||||
s.(*baseScheduler).logMsgType = lmt
|
||||
s.(*baseScheduler).logMsg = msg
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func WithSchedPolSwitchEnabled(enableSchedPolicySwitch bool) schedPolicyOption {
|
||||
return func(s ElectronScheduler) error {
|
||||
s.(*baseScheduler).schedPolSwitchEnabled = enableSchedPolicySwitch
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,255 +0,0 @@
|
|||
package schedulers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"log"
|
||||
"time"
|
||||
|
||||
"bitbucket.org/sunybingcloud/elektron/def"
|
||||
"bitbucket.org/sunybingcloud/elektron/utilities/mesosUtils"
|
||||
"bitbucket.org/sunybingcloud/elektron/utilities/offerUtils"
|
||||
"github.com/golang/protobuf/proto"
|
||||
mesos "github.com/mesos/mesos-go/mesosproto"
|
||||
"github.com/mesos/mesos-go/mesosutil"
|
||||
sched "github.com/mesos/mesos-go/scheduler"
|
||||
)
|
||||
|
||||
// Decides if to take an offer or not.
|
||||
func (s *MaxGreedyMins) takeOffer(offer *mesos.Offer, task def.Task,
|
||||
totalCPU, totalRAM, totalWatts float64) bool {
|
||||
|
||||
cpus, mem, watts := offerUtils.OfferAgg(offer)
|
||||
|
||||
//TODO: Insert watts calculation here instead of taking them as a parameter
|
||||
|
||||
wattsConsideration, err := def.WattsToConsider(task, s.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration.
|
||||
log.Fatal(err)
|
||||
}
|
||||
if (cpus >= (totalCPU + task.CPU)) && (mem >= (totalRAM + task.RAM)) &&
|
||||
(!s.wattsAsAResource || (watts >= (totalWatts + wattsConsideration))) {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
type MaxGreedyMins struct {
|
||||
base //Type embedding to inherit common functions.
|
||||
}
|
||||
|
||||
// Initialization.
|
||||
func (s *MaxGreedyMins) init(opts ...schedPolicyOption) {
|
||||
s.base.init(opts...)
|
||||
// Sorting the tasks based on watts.
|
||||
def.SortTasks(s.tasks, def.SortByWatts)
|
||||
}
|
||||
|
||||
func (s *MaxGreedyMins) newTask(offer *mesos.Offer, task def.Task) *mesos.TaskInfo {
|
||||
taskName := fmt.Sprintf("%s-%d", task.Name, *task.Instances)
|
||||
s.tasksCreated++
|
||||
|
||||
// Start recording only when we're creating the first task.
|
||||
if !*s.RecordPCP {
|
||||
// Turn on logging
|
||||
*s.RecordPCP = true
|
||||
time.Sleep(1 * time.Second) // Make sure we're recording by the time the first task starts.
|
||||
}
|
||||
|
||||
// If this is our first time running into this Agent.
|
||||
if _, ok := s.running[offer.GetSlaveId().GoString()]; !ok {
|
||||
s.running[offer.GetSlaveId().GoString()] = make(map[string]bool)
|
||||
}
|
||||
|
||||
// Add task to list of tasks running on node.
|
||||
s.running[offer.GetSlaveId().GoString()][taskName] = true
|
||||
|
||||
resources := []*mesos.Resource{
|
||||
mesosutil.NewScalarResource("cpus", task.CPU),
|
||||
mesosutil.NewScalarResource("mem", task.RAM),
|
||||
}
|
||||
|
||||
if s.wattsAsAResource {
|
||||
if wattsToConsider, err := def.WattsToConsider(task, s.classMapWatts, offer); err == nil {
|
||||
log.Printf("Watts considered for host[%s] and task[%s] = %f", *offer.Hostname, task.Name, wattsToConsider)
|
||||
resources = append(resources, mesosutil.NewScalarResource("watts", wattsToConsider))
|
||||
} else {
|
||||
// Error in determining wattsConsideration.
|
||||
log.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
return &mesos.TaskInfo{
|
||||
Name: proto.String(taskName),
|
||||
TaskId: &mesos.TaskID{
|
||||
Value: proto.String("elektron-" + taskName),
|
||||
},
|
||||
SlaveId: offer.SlaveId,
|
||||
Resources: resources,
|
||||
Command: &mesos.CommandInfo{
|
||||
Value: proto.String(task.CMD),
|
||||
},
|
||||
Container: &mesos.ContainerInfo{
|
||||
Type: mesos.ContainerInfo_DOCKER.Enum(),
|
||||
Docker: &mesos.ContainerInfo_DockerInfo{
|
||||
Image: proto.String(task.Image),
|
||||
Network: mesos.ContainerInfo_DockerInfo_BRIDGE.Enum(), // Run everything isolated.
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// Determine if the remaining space inside of the offer is enough for this
|
||||
// the task we need to create. If it is, create a TaskInfo and return it.
|
||||
func (s *MaxGreedyMins) CheckFit(
|
||||
i int,
|
||||
task def.Task,
|
||||
wattsConsideration float64,
|
||||
offer *mesos.Offer,
|
||||
totalCPU *float64,
|
||||
totalRAM *float64,
|
||||
totalWatts *float64) (bool, *mesos.TaskInfo) {
|
||||
|
||||
// Does the task fit.
|
||||
if s.takeOffer(offer, task, *totalCPU, *totalRAM, *totalWatts) {
|
||||
|
||||
*totalWatts += wattsConsideration
|
||||
*totalCPU += task.CPU
|
||||
*totalRAM += task.RAM
|
||||
log.Println("Co-Located with: ")
|
||||
coLocated(s.running[offer.GetSlaveId().GoString()])
|
||||
|
||||
taskToSchedule := s.newTask(offer, task)
|
||||
|
||||
fmt.Println("Inst: ", *task.Instances)
|
||||
s.schedTrace.Print(offer.GetHostname() + ":" + taskToSchedule.GetTaskId().GetValue())
|
||||
*task.Instances--
|
||||
|
||||
if *task.Instances <= 0 {
|
||||
// All instances of task have been scheduled, remove it.
|
||||
s.tasks = append(s.tasks[:i], s.tasks[i+1:]...)
|
||||
|
||||
if len(s.tasks) <= 0 {
|
||||
log.Println("Done scheduling all tasks")
|
||||
close(s.Shutdown)
|
||||
}
|
||||
}
|
||||
|
||||
return true, taskToSchedule
|
||||
}
|
||||
|
||||
return false, nil
|
||||
}
|
||||
|
||||
func (s *MaxGreedyMins) ResourceOffers(driver sched.SchedulerDriver, offers []*mesos.Offer) {
|
||||
log.Printf("Received %d resource offers", len(offers))
|
||||
|
||||
for _, offer := range offers {
|
||||
offerUtils.UpdateEnvironment(offer)
|
||||
select {
|
||||
case <-s.Shutdown:
|
||||
log.Println("Done scheduling tasks: declining offer on [", offer.GetHostname(), "]")
|
||||
driver.DeclineOffer(offer.Id, mesosUtils.LongFilter)
|
||||
|
||||
log.Println("Number of tasks still running: ", s.tasksRunning)
|
||||
continue
|
||||
default:
|
||||
}
|
||||
|
||||
tasks := []*mesos.TaskInfo{}
|
||||
|
||||
offerTaken := false
|
||||
totalWatts := 0.0
|
||||
totalCPU := 0.0
|
||||
totalRAM := 0.0
|
||||
|
||||
// Assumes s.tasks is ordered in non-decreasing median max peak order.
|
||||
|
||||
// Attempt to schedule a single instance of the heaviest workload available first.
|
||||
// Start from the back until one fits.
|
||||
for i := len(s.tasks) - 1; i >= 0; i-- {
|
||||
|
||||
task := s.tasks[i]
|
||||
wattsConsideration, err := def.WattsToConsider(task, s.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration.
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
// Don't take offer if it doesn't match our task's host requirement.
|
||||
if offerUtils.HostMismatch(*offer.Hostname, task.Host) {
|
||||
continue
|
||||
}
|
||||
|
||||
// TODO: Fix this so index doesn't need to be passed
|
||||
taken, taskToSchedule := s.CheckFit(i, task, wattsConsideration, offer,
|
||||
&totalCPU, &totalRAM, &totalWatts)
|
||||
|
||||
if taken {
|
||||
offerTaken = true
|
||||
tasks = append(tasks, taskToSchedule)
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// Pack the rest of the offer with the smallest tasks.
|
||||
for i := 0; i < len(s.tasks); i++ {
|
||||
task := s.tasks[i]
|
||||
wattsConsideration, err := def.WattsToConsider(task, s.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration.
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
// Don't take offer if it doesn't match our task's host requirement.
|
||||
if offerUtils.HostMismatch(*offer.Hostname, task.Host) {
|
||||
continue
|
||||
}
|
||||
|
||||
for *task.Instances > 0 {
|
||||
// TODO: Fix this so index doesn't need to be passed
|
||||
taken, taskToSchedule := s.CheckFit(i, task, wattsConsideration, offer,
|
||||
&totalCPU, &totalRAM, &totalWatts)
|
||||
|
||||
if taken {
|
||||
offerTaken = true
|
||||
tasks = append(tasks, taskToSchedule)
|
||||
} else {
|
||||
break // Continue on to next task.
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if offerTaken {
|
||||
log.Printf("Starting on [%s]\n", offer.GetHostname())
|
||||
driver.LaunchTasks([]*mesos.OfferID{offer.Id}, tasks, mesosUtils.DefaultFilter)
|
||||
} else {
|
||||
|
||||
// If there was no match for the task.
|
||||
fmt.Println("There is not enough resources to launch a task:")
|
||||
cpus, mem, watts := offerUtils.OfferAgg(offer)
|
||||
|
||||
log.Printf("<CPU: %f, RAM: %f, Watts: %f>\n", cpus, mem, watts)
|
||||
driver.DeclineOffer(offer.Id, mesosUtils.DefaultFilter)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *MaxGreedyMins) StatusUpdate(driver sched.SchedulerDriver, status *mesos.TaskStatus) {
|
||||
log.Printf("Received task status [%s] for task [%s]", NameFor(status.State), *status.TaskId.Value)
|
||||
|
||||
if *status.State == mesos.TaskState_TASK_RUNNING {
|
||||
s.tasksRunning++
|
||||
} else if IsTerminal(status.State) {
|
||||
delete(s.running[status.GetSlaveId().GoString()], *status.TaskId.Value)
|
||||
s.tasksRunning--
|
||||
if s.tasksRunning == 0 {
|
||||
select {
|
||||
case <-s.Shutdown:
|
||||
close(s.Done)
|
||||
default:
|
||||
}
|
||||
}
|
||||
}
|
||||
log.Printf("DONE: Task status [%s] for task [%s]", NameFor(status.State), *status.TaskId.Value)
|
||||
}
|
|
@ -1,251 +0,0 @@
|
|||
package schedulers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"log"
|
||||
"time"
|
||||
|
||||
"bitbucket.org/sunybingcloud/elektron/def"
|
||||
"bitbucket.org/sunybingcloud/elektron/utilities/mesosUtils"
|
||||
"bitbucket.org/sunybingcloud/elektron/utilities/offerUtils"
|
||||
"github.com/golang/protobuf/proto"
|
||||
mesos "github.com/mesos/mesos-go/mesosproto"
|
||||
"github.com/mesos/mesos-go/mesosutil"
|
||||
sched "github.com/mesos/mesos-go/scheduler"
|
||||
)
|
||||
|
||||
// Decides if to take an offer or not.
|
||||
func (s *MaxMin) takeOffer(offer *mesos.Offer, task def.Task,
|
||||
totalCPU, totalRAM, totalWatts float64) bool {
|
||||
|
||||
cpus, mem, watts := offerUtils.OfferAgg(offer)
|
||||
|
||||
//TODO: Insert watts calculation here instead of taking them as a parameter
|
||||
|
||||
wattsConsideration, err := def.WattsToConsider(task, s.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration.
|
||||
log.Fatal(err)
|
||||
}
|
||||
if (cpus >= (totalCPU + task.CPU)) && (mem >= (totalRAM + task.RAM)) &&
|
||||
(!s.wattsAsAResource || (watts >= (totalWatts + wattsConsideration))) {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
type MaxMin struct {
|
||||
base //Type embedding to inherit common functions.
|
||||
}
|
||||
|
||||
// Initialization.
|
||||
func (s *MaxMin) init(opts ...schedPolicyOption) {
|
||||
s.base.init(opts...)
|
||||
// Sorting the tasks based on Watts.
|
||||
def.SortTasks(s.tasks, def.SortByWatts)
|
||||
}
|
||||
|
||||
func (s *MaxMin) newTask(offer *mesos.Offer, task def.Task) *mesos.TaskInfo {
|
||||
taskName := fmt.Sprintf("%s-%d", task.Name, *task.Instances)
|
||||
s.tasksCreated++
|
||||
|
||||
// Start recording only when we're creating the first task.
|
||||
if !*s.RecordPCP {
|
||||
// Turn on logging.
|
||||
*s.RecordPCP = true
|
||||
time.Sleep(1 * time.Second) // Make sure we're recording by the time the first task starts.
|
||||
}
|
||||
|
||||
// If this is our first time running into this Agent.
|
||||
if _, ok := s.running[offer.GetSlaveId().GoString()]; !ok {
|
||||
s.running[offer.GetSlaveId().GoString()] = make(map[string]bool)
|
||||
}
|
||||
|
||||
// Add task to list of tasks running on node.
|
||||
s.running[offer.GetSlaveId().GoString()][taskName] = true
|
||||
|
||||
resources := []*mesos.Resource{
|
||||
mesosutil.NewScalarResource("cpus", task.CPU),
|
||||
mesosutil.NewScalarResource("mem", task.RAM),
|
||||
}
|
||||
|
||||
if s.wattsAsAResource {
|
||||
if wattsToConsider, err := def.WattsToConsider(task, s.classMapWatts, offer); err == nil {
|
||||
log.Printf("Watts considered for host[%s] and task[%s] = %f", *offer.Hostname, task.Name, wattsToConsider)
|
||||
resources = append(resources, mesosutil.NewScalarResource("watts", wattsToConsider))
|
||||
} else {
|
||||
// Error in determining wattsConsideration.
|
||||
log.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
return &mesos.TaskInfo{
|
||||
Name: proto.String(taskName),
|
||||
TaskId: &mesos.TaskID{
|
||||
Value: proto.String("elektron-" + taskName),
|
||||
},
|
||||
SlaveId: offer.SlaveId,
|
||||
Resources: resources,
|
||||
Command: &mesos.CommandInfo{
|
||||
Value: proto.String(task.CMD),
|
||||
},
|
||||
Container: &mesos.ContainerInfo{
|
||||
Type: mesos.ContainerInfo_DOCKER.Enum(),
|
||||
Docker: &mesos.ContainerInfo_DockerInfo{
|
||||
Image: proto.String(task.Image),
|
||||
Network: mesos.ContainerInfo_DockerInfo_BRIDGE.Enum(), // Run everything isolated.
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// Determine if the remaining space inside of the offer is enough for this
|
||||
// the task we need to create. If it is, create a TaskInfo and return it.
|
||||
func (s *MaxMin) CheckFit(
|
||||
i int,
|
||||
task def.Task,
|
||||
wattsConsideration float64,
|
||||
offer *mesos.Offer,
|
||||
totalCPU *float64,
|
||||
totalRAM *float64,
|
||||
totalWatts *float64) (bool, *mesos.TaskInfo) {
|
||||
|
||||
// Does the task fit.
|
||||
if s.takeOffer(offer, task, *totalCPU, *totalRAM, *totalWatts) {
|
||||
|
||||
*totalWatts += wattsConsideration
|
||||
*totalCPU += task.CPU
|
||||
*totalRAM += task.RAM
|
||||
log.Println("Co-Located with: ")
|
||||
coLocated(s.running[offer.GetSlaveId().GoString()])
|
||||
|
||||
taskToSchedule := s.newTask(offer, task)
|
||||
|
||||
fmt.Println("Inst: ", *task.Instances)
|
||||
s.schedTrace.Print(offer.GetHostname() + ":" + taskToSchedule.GetTaskId().GetValue())
|
||||
*task.Instances--
|
||||
|
||||
if *task.Instances <= 0 {
|
||||
// All instances of task have been scheduled, remove it.
|
||||
s.tasks = append(s.tasks[:i], s.tasks[i+1:]...)
|
||||
|
||||
if len(s.tasks) <= 0 {
|
||||
log.Println("Done scheduling all tasks")
|
||||
close(s.Shutdown)
|
||||
}
|
||||
}
|
||||
|
||||
return true, taskToSchedule
|
||||
}
|
||||
|
||||
return false, nil
|
||||
}
|
||||
|
||||
func (s *MaxMin) ResourceOffers(driver sched.SchedulerDriver, offers []*mesos.Offer) {
|
||||
log.Printf("Received %d resource offers", len(offers))
|
||||
|
||||
for _, offer := range offers {
|
||||
offerUtils.UpdateEnvironment(offer)
|
||||
select {
|
||||
case <-s.Shutdown:
|
||||
log.Println("Done scheduling tasks: declining offer on [", offer.GetHostname(), "]")
|
||||
driver.DeclineOffer(offer.Id, mesosUtils.LongFilter)
|
||||
|
||||
log.Println("Number of tasks still running: ", s.tasksRunning)
|
||||
continue
|
||||
default:
|
||||
}
|
||||
|
||||
tasks := []*mesos.TaskInfo{}
|
||||
|
||||
offerTaken := false
|
||||
totalWatts := 0.0
|
||||
totalCPU := 0.0
|
||||
totalRAM := 0.0
|
||||
|
||||
// Assumes s.tasks is ordered in non-decreasing median max peak order.
|
||||
|
||||
// Attempt to schedule a single instance of the heaviest workload available first.
|
||||
// Start from the back until one fits.
|
||||
|
||||
direction := false // True = Min Max, False = Max Min.
|
||||
var index int
|
||||
start := true // If false then index has changed and need to keep it that way.
|
||||
for i := 0; i < len(s.tasks); i++ {
|
||||
// We need to pick a min task or a max task
|
||||
// depending on the value of direction.
|
||||
if direction && start {
|
||||
index = 0
|
||||
} else if start {
|
||||
index = len(s.tasks) - i - 1
|
||||
}
|
||||
task := s.tasks[index]
|
||||
|
||||
wattsConsideration, err := def.WattsToConsider(task, s.classMapWatts, offer)
|
||||
if err != nil {
|
||||
// Error in determining wattsConsideration.
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
// Don't take offer it is doesn't match our task's host requirement.
|
||||
if offerUtils.HostMismatch(*offer.Hostname, task.Host) {
|
||||
continue
|
||||
}
|
||||
|
||||
// TODO: Fix this so index doesn't need to be passed
|
||||
taken, taskToSchedule := s.CheckFit(index, task, wattsConsideration, offer,
|
||||
&totalCPU, &totalRAM, &totalWatts)
|
||||
|
||||
if taken {
|
||||
offerTaken = true
|
||||
tasks = append(tasks, taskToSchedule)
|
||||
// Need to change direction and set start to true.
|
||||
// Setting start to true would ensure that index be set accurately again.
|
||||
direction = !direction
|
||||
start = true
|
||||
i--
|
||||
} else {
|
||||
// Need to move index depending on the value of direction.
|
||||
if direction {
|
||||
index++
|
||||
start = false
|
||||
} else {
|
||||
index--
|
||||
start = false
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if offerTaken {
|
||||
log.Printf("Starting on [%s]\n", offer.GetHostname())
|
||||
driver.LaunchTasks([]*mesos.OfferID{offer.Id}, tasks, mesosUtils.DefaultFilter)
|
||||
} else {
|
||||
|
||||
// If there was no match for the task
|
||||
fmt.Println("There is not enough resources to launch a task:")
|
||||
cpus, mem, watts := offerUtils.OfferAgg(offer)
|
||||
|
||||
log.Printf("<CPU: %f, RAM: %f, Watts: %f>\n", cpus, mem, watts)
|
||||
driver.DeclineOffer(offer.Id, mesosUtils.DefaultFilter)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *MaxMin) StatusUpdate(driver sched.SchedulerDriver, status *mesos.TaskStatus) {
|
||||
log.Printf("Received task status [%s] for task [%s]", NameFor(status.State), *status.TaskId.Value)
|
||||
|
||||
if *status.State == mesos.TaskState_TASK_RUNNING {
|
||||
s.tasksRunning++
|
||||
} else if IsTerminal(status.State) {
|
||||
delete(s.running[status.GetSlaveId().GoString()], *status.TaskId.Value)
|
||||
s.tasksRunning--
|
||||
if s.tasksRunning == 0 {
|
||||
select {
|
||||
case <-s.Shutdown:
|
||||
close(s.Done)
|
||||
default:
|
||||
}
|
||||
}
|
||||
}
|
||||
log.Printf("DONE: Task status [%s] for task [%s]", NameFor(status.State), *status.TaskId.Value)
|
||||
}
|
16
schedulers/schedPolicy.go
Normal file
16
schedulers/schedPolicy.go
Normal file
|
@ -0,0 +1,16 @@
|
|||
package schedulers
|
||||
|
||||
import (
|
||||
mesos "github.com/mesos/mesos-go/mesosproto"
|
||||
sched "github.com/mesos/mesos-go/scheduler"
|
||||
)
|
||||
|
||||
type SchedPolicyContext interface {
|
||||
// Change the state of scheduling.
|
||||
SwitchSchedPol(s SchedPolicyState)
|
||||
}
|
||||
|
||||
type SchedPolicyState interface {
|
||||
// Define the particular scheduling policy's methodology of resource offer consumption.
|
||||
ConsumeOffers(SchedPolicyContext, sched.SchedulerDriver, []*mesos.Offer)
|
||||
}
|
|
@ -1,30 +1,32 @@
|
|||
package schedulers
|
||||
|
||||
import "github.com/mesos/mesos-go/scheduler"
|
||||
import (
|
||||
sched "github.com/mesos/mesos-go/scheduler"
|
||||
)
|
||||
|
||||
// Names of different scheduling policies.
|
||||
const (
|
||||
ff = "first-fit"
|
||||
bp = "bin-packing"
|
||||
mgm = "max-greedymins"
|
||||
mm = "max-min"
|
||||
ff = "first-fit"
|
||||
bp = "bin-packing"
|
||||
mgm = "max-greedymins"
|
||||
mm = "max-min"
|
||||
)
|
||||
|
||||
// Scheduler class factory.
|
||||
var Schedulers map[string]scheduler.Scheduler = map[string]scheduler.Scheduler{
|
||||
ff: &FirstFit{base: base{}},
|
||||
bp: &BinPacking{base: base{}},
|
||||
mgm: &MaxGreedyMins{base: base{}},
|
||||
mm: &MaxMin{base: base{}},
|
||||
// Scheduling policy factory
|
||||
var SchedPolicies map[string]SchedPolicyState = map[string]SchedPolicyState{
|
||||
ff: &FirstFit{},
|
||||
bp: &BinPackSortedWatts{},
|
||||
mgm: &MaxGreedyMins{},
|
||||
mm: &MaxMin{},
|
||||
}
|
||||
|
||||
// Build the scheduling policy with the options being applied.
|
||||
func BuildSchedPolicy(s scheduler.Scheduler, opts ...schedPolicyOption) {
|
||||
// build the scheduling policy with the options being applied
|
||||
func buildScheduler(s sched.Scheduler, opts ...schedPolicyOption) {
|
||||
s.(ElectronScheduler).init(opts...)
|
||||
}
|
||||
|
||||
func SchedFactory(schedPolicyName string, opts ...schedPolicyOption) scheduler.Scheduler {
|
||||
s := Schedulers[schedPolicyName]
|
||||
BuildSchedPolicy(s, opts...)
|
||||
func SchedFactory(opts ...schedPolicyOption) sched.Scheduler {
|
||||
s := &baseScheduler{}
|
||||
buildScheduler(s, opts...)
|
||||
return s
|
||||
}
|
||||
|
|
Reference in a new issue