objects

package
v1.0.0 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Apr 26, 2022 License: Apache-2.0 Imports: 26 Imported by: 0

Documentation

Index

Constants

View Source
const (
	None allocationResult = iota
	Allocated
	AllocatedReserved
	Reserved
	Unreserved
	Replaced
)
View Source
const (
	Soft                    string = "Soft"
	Hard                    string = "Hard"
	AppTagStateAwareDisable string = "application.stateaware.disable"
)
View Source
const (
	RunApplication applicationEvent = iota
	RejectApplication
	CompleteApplication
	FailApplication
	ExpireApplication
	ResumeApplication
)
View Source
const (
	New applicationState = iota
	Accepted
	Starting
	Running
	Rejected
	Completing
	Completed
	Failing
	Failed
	Expired
	Resuming
)
View Source
const AppTagNamespaceResourceQuota = "namespace.resourcequota"
View Source
const ReadyFlag = "ready"

Variables

This section is empty.

Functions

func NewAppState

func NewAppState() *fsm.FSM

func NewObjectState

func NewObjectState() *fsm.FSM

func SetCompletingTimeout

func SetCompletingTimeout(duration time.Duration)

test only

func SetReservationDelay

func SetReservationDelay(delay time.Duration)

Set the reservation delay. Set when the cluster context is created to disable reservation.

Types

type Allocation

type Allocation struct {
	Ask               *AllocationAsk
	ApplicationID     string
	AllocationKey     string
	QueueName         string // CLEANUP: why do we need this? the app is linked to the queue
	NodeID            string
	ReservedNodeID    string
	PartitionName     string
	UUID              string
	Tags              map[string]string
	Priority          int32
	AllocatedResource *resources.Resource
	Result            allocationResult
	Releases          []*Allocation

	sync.RWMutex
	// contains filtered or unexported fields
}

Related to Allocation

func NewAllocation

func NewAllocation(uuid, nodeID string, ask *AllocationAsk) *Allocation

func NewAllocationFromSI

func NewAllocationFromSI(alloc *si.Allocation) *Allocation

Create a new Allocation from a node recovered allocation. Also creates an AllocationAsk to maintain backward compatible behaviour This returns a nil Allocation on nil input or errors

func (*Allocation) GetCreateTime

func (a *Allocation) GetCreateTime() time.Time

Return the time this alloc was created Should be treated as read only not to be modified

func (*Allocation) GetPlaceholderCreateTime

func (a *Allocation) GetPlaceholderCreateTime() time.Time

Return the placeholder's create time for this alloc, if applicable

func (*Allocation) GetPlaceholderUsed

func (a *Allocation) GetPlaceholderUsed() bool

Return whether this alloc is replacing a placeholder or not

func (*Allocation) IsPlaceholder

func (a *Allocation) IsPlaceholder() bool

IsPlaceholder returns true if the allocation is a placeholder, false otherwise.

func (*Allocation) IsReleased

func (a *Allocation) IsReleased() bool

IsReleased returns the release status of the allocation.

func (*Allocation) NewSIFromAllocation

func (a *Allocation) NewSIFromAllocation() *si.Allocation

Convert the Allocation into a SI object. This is a limited set of values that gets copied into the SI. We only use this to communicate *back* to the RM. All other fields are considered incoming fields from the RM into the core. The limited set of fields link the Allocation to an Application, Node and AllocationAsk.

func (*Allocation) String

func (a *Allocation) String() string

type AllocationAsk

type AllocationAsk struct {
	// Extracted info
	AllocationKey     string
	AllocatedResource *resources.Resource
	ApplicationID     string
	PartitionName     string
	QueueName         string // CLEANUP: why do we need this? the app is linked to the queue
	Tags              map[string]string

	sync.RWMutex
	// contains filtered or unexported fields
}

func NewAllocationAsk

func NewAllocationAsk(ask *si.AllocationAsk) *AllocationAsk

func (*AllocationAsk) GetCreateTime

func (aa *AllocationAsk) GetCreateTime() time.Time

Return the time this ask was created Should be treated as read only not te be modified

func (*AllocationAsk) GetPendingAskRepeat

func (aa *AllocationAsk) GetPendingAskRepeat() int32

Get the pending ask repeat

func (*AllocationAsk) GetRequiredNode

func (aa *AllocationAsk) GetRequiredNode() string

func (*AllocationAsk) SetPendingAskRepeat

func (aa *AllocationAsk) SetPendingAskRepeat(pendingRepeatAsk int32)

test only

func (*AllocationAsk) String

func (aa *AllocationAsk) String() string

type Application

type Application struct {
	ApplicationID  string
	Partition      string
	SubmissionTime time.Time

	sync.RWMutex
	// contains filtered or unexported fields
}

func NewApplication

func NewApplication(siApp *si.AddApplicationRequest, ugi security.UserGroup, eventHandler handler.EventHandler, rmID string) *Application

func (*Application) AddAllocation

func (sa *Application) AddAllocation(info *Allocation)

Add a new Allocation to the application

func (*Application) AddAllocationAsk

func (sa *Application) AddAllocationAsk(ask *AllocationAsk) error

Add an allocation ask to this application If the ask already exist update the existing info

func (*Application) CurrentState

func (sa *Application) CurrentState() string

Return the current state or a checked specific state for the application. The state machine handles the locking.

func (*Application) FailApplication

func (sa *Application) FailApplication(failureMessage string) error

FailApplication fails this application.

func (*Application) FinishedTime

func (sa *Application) FinishedTime() time.Time

func (*Application) GetAllAllocations

func (sa *Application) GetAllAllocations() []*Allocation

get a copy of all allocations of the application

func (*Application) GetAllPlaceholderData

func (sa *Application) GetAllPlaceholderData() []*PlaceholderData

func (*Application) GetAllocatedResource

func (sa *Application) GetAllocatedResource() *resources.Resource

Return the allocated resources for this application

func (*Application) GetAllocationAsk

func (sa *Application) GetAllocationAsk(allocationKey string) *AllocationAsk

Return the allocation ask for the key, nil if not found

func (*Application) GetAskReservations

func (sa *Application) GetAskReservations(allocKey string) []string

Return the allocation reservations on any node. The returned array is 0 or more keys into the reservations map. No locking must be called while holding the lock

func (*Application) GetMaxAllocatedResource

func (sa *Application) GetMaxAllocatedResource() *resources.Resource

func (*Application) GetPendingResource

func (sa *Application) GetPendingResource() *resources.Resource

Return the pending resources for this application

func (*Application) GetPlaceholderAsk

func (sa *Application) GetPlaceholderAsk() *resources.Resource

Return the total placeholder ask for this application Is only set on app creation and used when app is added to a queue

func (*Application) GetPlaceholderResource

func (sa *Application) GetPlaceholderResource() *resources.Resource

Return the allocated placeholder resources for this application

func (*Application) GetQueue

func (sa *Application) GetQueue() *Queue

func (*Application) GetQueuePath

func (sa *Application) GetQueuePath() string

func (*Application) GetRejectedMessage

func (sa *Application) GetRejectedMessage() string

func (*Application) GetReservations

func (sa *Application) GetReservations() []string

Return an array of all reservation keys for the app. This will return an empty array if there are no reservations. Visible for tests

func (*Application) GetStateLog

func (sa *Application) GetStateLog() []*StateLogEntry

func (*Application) GetTag

func (sa *Application) GetTag(tag string) string

Get a tag from the application Note: Tags are not case sensitive

func (*Application) GetUser

func (sa *Application) GetUser() security.UserGroup

get a copy of the user details for the application

func (*Application) HandleApplicationEvent

func (sa *Application) HandleApplicationEvent(event applicationEvent) error

HandleApplicationEvent handles the state event for the application. The application lock is expected to be held.

func (*Application) HandleApplicationEventWithInfo

func (sa *Application) HandleApplicationEventWithInfo(event applicationEvent, eventInfo string) error

HandleApplicationEventWithInfo handles the state event for the application with associated info object. The application lock is expected to be held.

func (*Application) IsAccepted

func (sa *Application) IsAccepted() bool

func (*Application) IsAllocationAssignedToApp

func (sa *Application) IsAllocationAssignedToApp(alloc *Allocation) bool

func (*Application) IsCompleted

func (sa *Application) IsCompleted() bool

func (*Application) IsCompleting

func (sa *Application) IsCompleting() bool

func (*Application) IsExpired

func (sa *Application) IsExpired() bool

func (*Application) IsFailed

func (sa *Application) IsFailed() bool

func (*Application) IsFailing

func (sa *Application) IsFailing() bool

func (*Application) IsNew

func (sa *Application) IsNew() bool

func (*Application) IsRejected

func (sa *Application) IsRejected() bool

func (*Application) IsReservedOnNode

func (sa *Application) IsReservedOnNode(nodeID string) bool

IsReservedOnNode returns true if and only if the node has been reserved by the application An empty nodeID is never reserved.

func (*Application) IsResuming

func (sa *Application) IsResuming() bool

func (*Application) IsRunning

func (sa *Application) IsRunning() bool

func (*Application) IsStarting

func (sa *Application) IsStarting() bool

func (*Application) OnStateChange

func (sa *Application) OnStateChange(event *fsm.Event, eventInfo string)

OnStatChange every time the application enters a new state. It sends an event about the state change to the shim as an application update. The only state that does not generate an event is Rejected.

func (*Application) RecoverAllocationAsk

func (sa *Application) RecoverAllocationAsk(ask *AllocationAsk)

Add the ask when a node allocation is recovered. Maintaining the rule that an Allocation always has a link to an AllocationAsk. Safeguarded against a nil but the recovery generates the ask and should never be nil.

func (*Application) RejectApplication

func (sa *Application) RejectApplication(rejectedMessage string) error

RejectApplication rejects this application.

func (*Application) RemoveAllAllocations

func (sa *Application) RemoveAllAllocations() []*Allocation

Remove all allocations from the application. All allocations that have been removed are returned.

func (*Application) RemoveAllocation

func (sa *Application) RemoveAllocation(uuid string) *Allocation

Remove the Allocation from the application. Return the allocation that was removed or nil if not found.

func (*Application) RemoveAllocationAsk

func (sa *Application) RemoveAllocationAsk(allocKey string) int

Remove one or more allocation asks from this application. This also removes any reservations that are linked to the ask. The return value is the number of reservations released

func (*Application) ReplaceAllocation

func (sa *Application) ReplaceAllocation(uuid string) *Allocation

func (*Application) Reserve

func (sa *Application) Reserve(node *Node, ask *AllocationAsk) error

Reserve the application for this node and ask combination. If the reservation fails the function returns false, if the reservation is made it returns true. If the node and ask combination was already reserved for the application this is a noop and returns true.

func (*Application) SetQueue

func (sa *Application) SetQueue(queue *Queue)

Set the leaf queue the application runs in.

func (*Application) SetQueuePath

func (sa *Application) SetQueuePath(queuePath string)

Set the leaf queue the application runs in. The queue will be created when the app is added to the partition. The queue name is set to what the placement rule returned.

func (*Application) SetState

func (sa *Application) SetState(state string)

func (*Application) SetTerminatedCallback

func (sa *Application) SetTerminatedCallback(callback func(appID string))

func (*Application) String

func (sa *Application) String() string

func (*Application) UnReserve

func (sa *Application) UnReserve(node *Node, ask *AllocationAsk) (int, error)

UnReserve the application for this node and ask combination. This first removes the reservation from the node. If the reservation does not exist it returns 0 for reservations removed, if the reservation is removed it returns 1. The error is set if the reservation key cannot be removed from the app or node.

func (*Application) UnSetQueue

func (sa *Application) UnSetQueue()

remove the leaf queue the application runs in, used when completing the app

func (*Application) UpdateAskRepeat

func (sa *Application) UpdateAskRepeat(allocKey string, delta int32) (*resources.Resource, error)

type Node

type Node struct {
	// Fields for fast access These fields are considered read only.
	// Values should only be set when creating a new node and never changed.
	NodeID    string
	Hostname  string
	Rackname  string
	Partition string

	sync.RWMutex
	// contains filtered or unexported fields
}

func NewNode

func NewNode(proto *si.NodeInfo) *Node

func (*Node) AddAllocation

func (sn *Node) AddAllocation(alloc *Allocation) bool

Add the allocation to the node. Used resources will increase available will decrease. A nil Allocation makes no changes. Pre-empted resources must have been released already. Do a sanity check to make sure it still fits in the node and nothing has changed

func (*Node) AddListener

func (sn *Node) AddListener(listener NodeListener)

func (*Node) CanAllocate

func (sn *Node) CanAllocate(res *resources.Resource, preemptionPhase bool) bool

Check if the proposed allocation fits in the available resources. Taking into account resources marked for preemption if applicable. If the proposed allocation does not fit false is returned. TODO: remove when updating preemption

func (*Node) FitInNode

func (sn *Node) FitInNode(resRequest *resources.Resource) bool

func (*Node) GetAllAllocations

func (sn *Node) GetAllAllocations() []*Allocation

Get a copy of the allocations on this node

func (*Node) GetAllocatedResource

func (sn *Node) GetAllocatedResource() *resources.Resource

Get the allocated resource on this node.

func (*Node) GetAllocation

func (sn *Node) GetAllocation(uuid string) *Allocation

Return the allocation based on the uuid of the allocation. returns nil if the allocation is not found

func (*Node) GetAttribute

func (sn *Node) GetAttribute(key string) string

Get an attribute by name. The most used attributes can be directly accessed via the fields: HostName, RackName and Partition. This is a lock free call. All attributes are considered read only

func (*Node) GetAvailableResource

func (sn *Node) GetAvailableResource() *resources.Resource

Get the available resource on this node.

func (*Node) GetCapacity

func (sn *Node) GetCapacity() *resources.Resource

func (*Node) GetOccupiedResource

func (sn *Node) GetOccupiedResource() *resources.Resource

func (*Node) GetReservations

func (sn *Node) GetReservations() []string

Return an array of all reservation keys for the node. This will return an empty array if there are no reservations. Visible for tests

func (*Node) GetResourceUsageShares

func (sn *Node) GetResourceUsageShares() map[string]float64

Gets map of name -> resource usages per type in shares (0 to 1). Can return NaN.

func (*Node) GetUtilizedResource

func (sn *Node) GetUtilizedResource() *resources.Resource

Get the utilized resource on this node.

func (*Node) IncPreemptingResource

func (sn *Node) IncPreemptingResource(preempting *resources.Resource)

Update the number of resource tagged for preemption on this node

func (*Node) IsReady

func (sn *Node) IsReady() bool

func (*Node) IsReserved

func (sn *Node) IsReserved() bool

Return if the node has been reserved by any application

func (*Node) IsSchedulable

func (sn *Node) IsSchedulable() bool

Can this node be used in scheduling.

func (*Node) IsValidFor

func (sn *Node) IsValidFor(ask *AllocationAsk) error

IsValidFor checks if the node is valid for this allocationAsk

func (*Node) RemoveAllocation

func (sn *Node) RemoveAllocation(uuid string) *Allocation

Remove the allocation to the node. Returns nil if the allocation was not found and no changes are made. If the allocation is found the Allocation removed is returned. Used resources will decrease available will increase as per the allocation removed.

func (*Node) RemoveListener

func (sn *Node) RemoveListener(listener NodeListener)

func (*Node) ReplaceAllocation

func (sn *Node) ReplaceAllocation(uuid string, replace *Allocation, delta *resources.Resource)

ReplaceAllocation replaces the placeholder with the real allocation on the node. The delta passed in is the difference in resource usage between placeholder and real allocation. It should always be a negative value or zero: it is a decrease in usage or no change

func (*Node) Reserve

func (sn *Node) Reserve(app *Application, ask *AllocationAsk) error

Reserve the node for this application and ask combination, if not reserved yet. The reservation is checked against the node resources. If the reservation fails the function returns false, if the reservation is made it returns true.

func (*Node) SetCapacity

func (sn *Node) SetCapacity(newCapacity *resources.Resource) *resources.Resource

func (*Node) SetOccupiedResource

func (sn *Node) SetOccupiedResource(occupiedResource *resources.Resource)

func (*Node) SetReady

func (sn *Node) SetReady(ready bool)

func (*Node) SetSchedulable

func (sn *Node) SetSchedulable(schedulable bool)

Set the node to unschedulable. This will cause the node to be skipped during the scheduling cycle. Visible for testing only

func (*Node) String

func (sn *Node) String() string

func (*Node) UnReserveApps

func (sn *Node) UnReserveApps() ([]string, []int)

Remove all reservation made on this node from the app. This is an unlocked function, it does not use a copy of the map when calling unReserve. That call will via the app call unReserve on the node which is locked and modifies the original map. However deleting an entry from a map while iterating over the map is perfectly safe based on the Go Specs. It must only be called when removing the node under a partition lock. It returns a list of all apps that have been checked on the node regardless of the result of the app unReserve call. The corresponding integers show the number of reservations removed for each app entry

type NodeCollection

type NodeCollection interface {
	AddNode(node *Node) error
	RemoveNode(nodeID string) *Node
	GetNode(nodeID string) *Node
	GetNodeCount() int
	GetNodes() []*Node
	GetNodeIterator() NodeIterator
	SetNodeSortingPolicy(policy NodeSortingPolicy)
	GetNodeSortingPolicy() NodeSortingPolicy
}

func NewNodeCollection

func NewNodeCollection(partition string) NodeCollection

Create a new collection for the given partition.

type NodeIterator

type NodeIterator interface {
	// returns true if there are more values to iterate over
	HasNext() bool
	// returns the next node from the iterator
	Next() *Node
	// reset the iterator to a clean state
	Reset()
}

NodeIterator iterates over a list of nodes based on the defined policy

func NewDefaultNodeIterator

func NewDefaultNodeIterator(schedulerNodes []*Node) NodeIterator

Create a new default iterator

func NewRoundRobinNodeIterator

func NewRoundRobinNodeIterator(schedulerNodes []*Node) NodeIterator

The starting point is randomised in the slice.

type NodeListener

type NodeListener interface {
	NodeUpdated(sn *Node)
}

type NodeSortingPolicy

type NodeSortingPolicy interface {
	PolicyType() policies.SortingPolicy
	ScoreNode(node *Node) float64
	ResourceWeights() map[string]float64
}

func NewNodeSortingPolicy

func NewNodeSortingPolicy(policyType string, resourceWeights map[string]float64) NodeSortingPolicy

type ObjectEvent

type ObjectEvent int

---------------------------------- object events these events are used for: partitions and managed queues ----------------------------------

const (
	Remove ObjectEvent = iota
	Start
	Stop
)

func (ObjectEvent) String

func (oe ObjectEvent) String() string

type ObjectState

type ObjectState int

---------------------------------- object states these states are used by: partitions and managed queues ----------------------------------

const (
	Active ObjectState = iota
	Draining
	Stopped
)

func (ObjectState) String

func (os ObjectState) String() string

type PlaceholderData

type PlaceholderData struct {
	TaskGroupName string
	Count         int64
	MinResource   *resources.Resource
	RequiredNode  string
	Replaced      int64
}

type Queue

type Queue struct {
	QueuePath string // Fully qualified path for the queue
	Name      string // Queue name as in the config etc.

	sync.RWMutex
	// contains filtered or unexported fields
}

Queue structure inside Scheduler

func NewConfiguredQueue

func NewConfiguredQueue(conf configs.QueueConfig, parent *Queue) (*Queue, error)

NewConfiguredQueue creates a new queue from scratch based on the configuration lock free as it cannot be referenced yet

func NewDynamicQueue

func NewDynamicQueue(name string, leaf bool, parent *Queue) (*Queue, error)

NewDynamicQueue creates a new queue to be added to the system based on the placement rules A dynamically added queue can never be the root queue so parent must be set lock free as it cannot be referenced yet

func (*Queue) AddApplication

func (sq *Queue) AddApplication(app *Application)

AddApplication adds the application to the queue. All checks are assumed to have passed before we get here. No update of pending resource is needed as it should not have any requests yet. Replaces the existing application without further checks.

func (*Queue) ApplyConf

func (sq *Queue) ApplyConf(conf configs.QueueConfig) error

ApplyConf is the locked version of applyConf

func (*Queue) CheckAdminAccess

func (sq *Queue) CheckAdminAccess(user security.UserGroup) bool

CheckAdminAccess checks if the user has access to the queue to perform administrative actions. The check is performed recursively: i.e. access to the parent allows access to this queue.

func (*Queue) CheckSubmitAccess

func (sq *Queue) CheckSubmitAccess(user security.UserGroup) bool

CheckSubmitAccess checks if the user has access to the queue to submit an application. The check is performed recursively: i.e. access to the parent allows access to this queue. This will check both submitACL and adminACL.

func (*Queue) CurrentState

func (sq *Queue) CurrentState() string

CurrentState returns the current state of the queue in string form.

func (*Queue) DecAllocatedResource

func (sq *Queue) DecAllocatedResource(alloc *resources.Resource) error

DecAllocatedResource decrement the allocated resources for this queue (recursively) Guard against going below zero resources.

func (*Queue) GetAllocatedResource

func (sq *Queue) GetAllocatedResource() *resources.Resource

GetAllocatedResource returns a clone of the allocated resources for this queue.

func (*Queue) GetChildQueue

func (sq *Queue) GetChildQueue(name string) *Queue

GetChildQueue returns a queue if the name exists in the child map as a key.

func (*Queue) GetCopyOfApps

func (sq *Queue) GetCopyOfApps() map[string]*Application

GetCopyOfApps gets a shallow copy of all non-completed apps holding the lock

func (*Queue) GetCopyOfChildren

func (sq *Queue) GetCopyOfChildren() map[string]*Queue

GetCopyOfChildren return a shallow copy of the child queue map. This is used by the partition manager to find all queues to clean however we can not guarantee that there is no new child added while we clean up since there is no overall lock on the scheduler. We'll need to test just before to make sure the parent is empty

func (*Queue) GetCopyOfCompletedApps

func (sq *Queue) GetCopyOfCompletedApps() map[string]*Application

GetCopyOfCompletedApps returns a shallow copy of all completed apps holding the lock

func (*Queue) GetGuaranteedResource

func (sq *Queue) GetGuaranteedResource() *resources.Resource

GetGuaranteedResource returns a clone of the guaranteed resource for the queue.

func (*Queue) GetMaxQueueSet

func (sq *Queue) GetMaxQueueSet() *resources.Resource

GetMaxQueueSet returns the max resource for the queue. The max resource should never be larger than the max resource of the parent. The cluster size, which defines the root limit, is not relevant for this call. Contrary to the GetMaxResource call. This will return nil unless a limit is set. Used during scheduling in an auto-scaling cluster. NOTE: if a resource quantity is missing and a limit is defined the missing quantity will be seen as a limit of 0. When defining a limit you therefore should define all resource quantities.

func (*Queue) GetMaxResource

func (sq *Queue) GetMaxResource() *resources.Resource

GetMaxResource returns the max resource for the queue. The max resource should never be larger than the max resource of the parent. The root queue always has its limit set to the total cluster size (dynamic based on node registration) In case there are no nodes in a newly started cluster and no queues have a limit configured this call will return nil. NOTE: if a resource quantity is missing and a limit is defined the missing quantity will be seen as a limit of 0. When defining a limit you therefore should define all resource quantities.

func (*Queue) GetPartitionQueueDAOInfo

func (sq *Queue) GetPartitionQueueDAOInfo() dao.PartitionQueueDAOInfo

GetPartitionQueueDAOInfo returns the queue hierarchy as an object for a REST call.

func (*Queue) GetPendingResource

func (sq *Queue) GetPendingResource() *resources.Resource

GetPendingResource returns the pending resources for this queue.

func (*Queue) GetPreemptingResource

func (sq *Queue) GetPreemptingResource() *resources.Resource

GetPreemptingResource returns the resources marked for preemption in the queue

func (*Queue) GetQueueInfo

func (sq *Queue) GetQueueInfo() dao.QueueDAOInfo

GetQueueInfo returns the queue hierarchy as an object for a REST call. This object is used by the deprecated REST API and is succeeded by the GetPartitionQueueDAOInfo call.

func (*Queue) GetQueueOutstandingRequests

func (sq *Queue) GetQueueOutstandingRequests(total *[]*AllocationAsk)

GetQueueOutstandingRequests builds a slice of pending allocation asks that fits into the queue's headroom.

func (*Queue) GetQueuePath

func (sq *Queue) GetQueuePath() string

GetQueuePath returns the fully qualified path of this queue.

func (*Queue) IncAllocatedResource

func (sq *Queue) IncAllocatedResource(alloc *resources.Resource, nodeReported bool) error

IncAllocatedResource increments the allocated resources for this queue (recursively). Guard against going over max resources if set

func (*Queue) IncPreemptingResource

func (sq *Queue) IncPreemptingResource(newAlloc *resources.Resource)

IncPreemptingResource increments the number of resource marked for preemption in the queue.

func (*Queue) IsDraining

func (sq *Queue) IsDraining() bool

IsDraining returns true if the queue in Draining state. Existing applications will still be scheduled No new applications will be accepted.

func (*Queue) IsEmpty

func (sq *Queue) IsEmpty() bool

IsEmpty returns true if a queue is empty based on the following definition: A parent queue is empty when it has no children left A leaf queue is empty when there are no applications left

func (*Queue) IsLeafQueue

func (sq *Queue) IsLeafQueue() bool

IsLeafQueue returns true is the queue a leaf. Returns false for a parent queue.

func (*Queue) IsManaged

func (sq *Queue) IsManaged() bool

IsManaged returns true for a managed queue. Returns false for a dynamic queue.

func (*Queue) IsRunning

func (sq *Queue) IsRunning() bool

IsRunning returns true if the queue in Active state.

func (*Queue) IsStopped

func (sq *Queue) IsStopped() bool

IsStopped returns true if the queue in Stopped state. The queue is skipped for scheduling in this state.

func (*Queue) MarkQueueForRemoval

func (sq *Queue) MarkQueueForRemoval()

MarkQueueForRemoval marks the managed queue for removal from the system. This can be executed multiple times and is only effective the first time. This is a noop on an unmanaged queue.

func (*Queue) RemoveApplication

func (sq *Queue) RemoveApplication(app *Application)

RemoveApplication removes the app from the list of tracked applications. Make sure that the app is assigned to this queue and not removed yet. If not found this call is a noop

func (*Queue) RemoveQueue

func (sq *Queue) RemoveQueue() bool

RemoveQueue remove the queue from the structure. Since nothing is allocated there shouldn't be anything referencing this queue anymore. The real removal is the removal of the queue from the parent's child list. Use a read lock on this queue to prevent other changes but allow status checks etc.

func (*Queue) Reserve

func (sq *Queue) Reserve(appID string)

Reserve increments the number of reservations for the application adding it to the map if needed. No checks this is only called when a reservation is processed using the app stored in the queue.

func (*Queue) SetMaxResource

func (sq *Queue) SetMaxResource(max *resources.Resource)

SetMaxResource sets the max resource for root the queue. Called as part of adding or removing a node. Should only happen on the root, all other queues get it from the config via properties.

func (*Queue) String

func (sq *Queue) String() string

func (*Queue) SupportTaskGroup

func (sq *Queue) SupportTaskGroup() bool

SupportTaskGroup returns true if the queue supports task groups. FIFO and StateAware sorting policies can support this. NOTE: this call does not make sense for a parent queue, and always returns false

func (*Queue) TryAllocate

func (sq *Queue) TryAllocate(iterator func() NodeIterator) *Allocation

TryAllocate tries to allocate a pending requests. This only gets called if there is a pending request on this queue or its children. This is a depth first algorithm: descend into the depth of the queue tree first. Child queues are sorted based on the configured queue sortPolicy. Queues without pending resources are skipped. Applications are sorted based on the application sortPolicy. Applications without pending resources are skipped. Lock free call this all locks are taken when needed in called functions

func (*Queue) TryPlaceholderAllocate

func (sq *Queue) TryPlaceholderAllocate(iterator func() NodeIterator, getnode func(string) *Node) *Allocation

TryPlaceholderAllocate tries to replace a placeholders with a real allocation. This only gets called if there is a pending request on this queue or its children. This is a depth first algorithm: descend into the depth of the queue tree first. Child queues are sorted based on the configured queue sortPolicy. Queues without pending resources are skipped. Applications are sorted based on the application sortPolicy. Applications without pending resources are skipped. Lock free call this all locks are taken when needed in called functions

func (*Queue) TryReservedAllocate

func (sq *Queue) TryReservedAllocate(iterator func() NodeIterator) *Allocation

TryReservedAllocate tries to allocate a reservation. This only gets called if there is a pending request on this queue or its children. This is a depth first algorithm: descend into the depth of the queue tree first. Child queues are sorted based on the configured queue sortPolicy. Queues without pending resources are skipped. Applications are currently NOT sorted and are iterated over in a random order. Lock free call this all locks are taken when needed in called functions

func (*Queue) UnReserve

func (sq *Queue) UnReserve(appID string, releases int)

UnReserve decrements the number of reservations for the application removing it to the map if all reservations are removed. No checks this is only called when a reservation is processed using the app stored in the queue.

func (*Queue) UpdateSortType

func (sq *Queue) UpdateSortType()

UpdateSortType updates the sortType for the queue based on the current properties.

type StateLogEntry

type StateLogEntry struct {
	Time             time.Time
	ApplicationState string
}

Directories

Path Synopsis

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL