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:
Pradyumna Kaushik 2018-01-19 21:20:43 +00:00
parent cb71153362
commit 065705d480
24 changed files with 1392 additions and 917 deletions

186
schedulers/MaxGreedyMins.go Normal file
View 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
View 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--
}
}
}

View file

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

View file

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

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

View file

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

View file

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

View file

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

View file

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

View file

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