Skip to content
This repository has been archived by the owner on May 25, 2023. It is now read-only.

Commit

Permalink
Merge pull request #685 from asifdxtreme/automated-cherry-pick-of-#643-
Browse files Browse the repository at this point in the history
…#642-#638-#645-#647-#651-#652-#655-#658-#649-#660-#666-#671-#673-upstream-release-0.4

Automated cherry pick of #643: Return err in Allocate if any error occurs #642: Add event when task is scheduled #638: Take init containers into account when getting pod resource #645: Order task by CreationTimestamp first, then by UID #647: In allocate, skip adding Job if its queue is not found #651: Return err in functions of session.go if any error occurs #652: Change run option SchedulePeriod's type to make it clear #655: Do graceful eviction using default policy #658: Address helm install error in tutorial.md #649: Preempt lowest priority task first #660: Fix sub exception in reclaim and preempt #666: Fix wrong caculation for deserved in proportion plugin #671: Change base image to alphine to reduce image size #673: Do not create PodGroup and Job for task whose scheduler is
  • Loading branch information
k8s-ci-robot authored Mar 29, 2019
2 parents ddf0eef + 3af9100 commit bffcc78
Show file tree
Hide file tree
Showing 20 changed files with 460 additions and 69 deletions.
2 changes: 1 addition & 1 deletion Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ generate-code:

rel_bins:
go get github.com/mitchellh/gox
gox -osarch=${REL_OSARCH} -ldflags ${LD_FLAGS} \
CGO_ENABLED=0 gox -osarch=${REL_OSARCH} -ldflags ${LD_FLAGS} \
-output=${BIN_DIR}/{{.OS}}/{{.Arch}}/kube-batch ./cmd/kube-batch

images: rel_bins
Expand Down
20 changes: 12 additions & 8 deletions cmd/kube-batch/app/options/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,13 +23,20 @@ import (
"github.com/spf13/pflag"
)

const (
defaultSchedulerName = "kube-batch"
defaultSchedulerPeriod = time.Second
defaultQueue = "default"
defaultListenAddress = ":8080"
)

// ServerOption is the main context object for the controller manager.
type ServerOption struct {
Master string
Kubeconfig string
SchedulerName string
SchedulerConf string
SchedulePeriod string
SchedulePeriod time.Duration
EnableLeaderElection bool
LockObjectNamespace string
DefaultQueue string
Expand All @@ -48,25 +55,22 @@ func (s *ServerOption) AddFlags(fs *pflag.FlagSet) {
fs.StringVar(&s.Master, "master", s.Master, "The address of the Kubernetes API server (overrides any value in kubeconfig)")
fs.StringVar(&s.Kubeconfig, "kubeconfig", s.Kubeconfig, "Path to kubeconfig file with authorization and master location information")
// kube-batch will ignore pods with scheduler names other than specified with the option
fs.StringVar(&s.SchedulerName, "scheduler-name", "kube-batch", "kube-batch will handle pods with the scheduler-name")
fs.StringVar(&s.SchedulerName, "scheduler-name", defaultSchedulerName, "kube-batch will handle pods with the scheduler-name")
fs.StringVar(&s.SchedulerConf, "scheduler-conf", "", "The absolute path of scheduler configuration file")
fs.StringVar(&s.SchedulePeriod, "schedule-period", "1s", "The period between each scheduling cycle")
fs.StringVar(&s.DefaultQueue, "default-queue", "default", "The default queue name of the job")
fs.DurationVar(&s.SchedulePeriod, "schedule-period", defaultSchedulerPeriod, "The period between each scheduling cycle")
fs.StringVar(&s.DefaultQueue, "default-queue", defaultQueue, "The default queue name of the job")
fs.BoolVar(&s.EnableLeaderElection, "leader-elect", s.EnableLeaderElection,
"Start a leader election client and gain leadership before "+
"executing the main loop. Enable this when running replicated kube-batch for high availability")
fs.BoolVar(&s.PrintVersion, "version", false, "Show version and quit")
fs.StringVar(&s.LockObjectNamespace, "lock-object-namespace", s.LockObjectNamespace, "Define the namespace of the lock object")
fs.StringVar(&s.ListenAddress, "listen-address", ":8080", "The address to listen on for HTTP requests.")
fs.StringVar(&s.ListenAddress, "listen-address", defaultListenAddress, "The address to listen on for HTTP requests.")
}

func (s *ServerOption) CheckOptionOrDie() error {
if s.EnableLeaderElection && s.LockObjectNamespace == "" {
return fmt.Errorf("lock-object-namespace must not be nil when LeaderElection is enabled")
}
if _, err := time.ParseDuration(s.SchedulePeriod); err != nil {
return fmt.Errorf("failed to parse --schedule-period: %v", err)
}

return nil
}
48 changes: 48 additions & 0 deletions cmd/kube-batch/app/options/options_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
Copyright 2019 The Kubernetes Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/

package options

import (
"reflect"
"testing"
"time"

"github.com/spf13/pflag"
)

func TestAddFlags(t *testing.T) {
fs := pflag.NewFlagSet("addflagstest", pflag.ContinueOnError)
s := NewServerOption()
s.AddFlags(fs)

args := []string{
"--schedule-period=5m",
}
fs.Parse(args)

// This is a snapshot of expected options parsed by args.
expected := &ServerOption{
SchedulerName: defaultSchedulerName,
SchedulePeriod: 5 * time.Minute,
DefaultQueue: defaultQueue,
ListenAddress: defaultListenAddress,
}

if !reflect.DeepEqual(expected, s) {
t.Errorf("Got different run options than expected.\nGot: %+v\nExpected: %+v\n", s, expected)
}
}
2 changes: 1 addition & 1 deletion deployment/images/Dockerfile
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
From ubuntu:18.04
From alpine:3.9

ADD kube-batch /usr/local/bin

Expand Down
2 changes: 2 additions & 0 deletions doc/usage/tutorial.md
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,8 @@ Run the `kube-batch` as kubernetes scheduler
# helm install $GOPATH/src/github.com/kubernetes-sigs/kube-batch/deployment/kube-batch --namespace kube-system
```

Note: If there is an error `Error: apiVersion "scheduling.incubator.k8s.io/v1alpha1" in kube-batch/templates/default.yaml is not available`, please update your helm to latest version and try it again.

Verify the release

```bash
Expand Down
22 changes: 13 additions & 9 deletions pkg/scheduler/actions/allocate/allocate.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,12 +46,16 @@ func (alloc *allocateAction) Execute(ssn *framework.Session) {
jobsMap := map[api.QueueID]*util.PriorityQueue{}

for _, job := range ssn.Jobs {
if _, found := jobsMap[job.Queue]; !found {
jobsMap[job.Queue] = util.NewPriorityQueue(ssn.JobOrderFn)
}

if queue, found := ssn.Queues[job.Queue]; found {
queues.Push(queue)
} else {
glog.Warningf("Skip adding Job <%s/%s> because its queue %s is not found",
job.Namespace, job.Name, job.Queue)
continue
}

if _, found := jobsMap[job.Queue]; !found {
jobsMap[job.Queue] = util.NewPriorityQueue(ssn.JobOrderFn)
}

glog.V(4).Infof("Added Job <%s/%s> into Queue <%s>", job.Namespace, job.Name, job.Queue)
Expand Down Expand Up @@ -143,12 +147,12 @@ func (alloc *allocateAction) Execute(ssn *framework.Session) {
selectedNodes := util.SelectBestNode(nodeScores)
for _, node := range selectedNodes {
// Allocate idle resource to the task.
if task.Resreq.LessEqual(node.Idle) {
if task.InitResreq.LessEqual(node.Idle) {
glog.V(3).Infof("Binding Task <%v/%v> to node <%v>",
task.Namespace, task.Name, node.Name)
if err := ssn.Allocate(task, node.Name); err != nil {
glog.Errorf("Failed to bind Task %v on %v in Session %v",
task.UID, node.Name, ssn.UID)
glog.Errorf("Failed to bind Task %v on %v in Session %v, err: %v",
task.UID, node.Name, ssn.UID, err)
continue
}
assigned = true
Expand All @@ -162,9 +166,9 @@ func (alloc *allocateAction) Execute(ssn *framework.Session) {
}

// Allocate releasing resource to the task if any.
if task.Resreq.LessEqual(node.Releasing) {
if task.InitResreq.LessEqual(node.Releasing) {
glog.V(3).Infof("Pipelining Task <%v/%v> to node <%v> for <%v> on <%v>",
task.Namespace, task.Name, node.Name, task.Resreq, node.Releasing)
task.Namespace, task.Name, node.Name, task.InitResreq, node.Releasing)
if err := ssn.Pipeline(task, node.Name); err != nil {
glog.Errorf("Failed to pipeline Task %v on %v in Session %v",
task.UID, node.Name, ssn.UID)
Expand Down
2 changes: 1 addition & 1 deletion pkg/scheduler/actions/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ func (alloc *backfillAction) Execute(ssn *framework.Session) {
// TODO (k82cn): When backfill, it's also need to balance between Queues.
for _, job := range ssn.Jobs {
for _, task := range job.TaskStatusIndex[api.Pending] {
if task.Resreq.IsEmpty() {
if task.InitResreq.IsEmpty() {
// As task did not request resources, so it only need to meet predicates.
// TODO (k82cn): need to prioritize nodes to avoid pod hole.
for _, node := range ssn.Nodes {
Expand Down
20 changes: 13 additions & 7 deletions pkg/scheduler/actions/preempt/preempt.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,7 +203,7 @@ func preempt(

var preemptees []*api.TaskInfo
preempted := api.EmptyResource()
resreq := preemptor.Resreq.Clone()
resreq := preemptor.InitResreq.Clone()

for _, task := range node.Tasks {
if filter == nil {
Expand All @@ -220,8 +220,15 @@ func preempt(
continue
}

// Preempt victims for tasks.
for _, preemptee := range victims {
victimsQueue := util.NewPriorityQueue(func(l, r interface{}) bool {
return !ssn.TaskOrderFn(l, r)
})
for _, victim := range victims {
victimsQueue.Push(victim)
}
// Preempt victims for tasks, pick lowest priority task first.
for !victimsQueue.Empty() {
preemptee := victimsQueue.Pop().(*api.TaskInfo)
glog.Errorf("Try to preempt Task <%s/%s> for Tasks <%s/%s>",
preemptee.Namespace, preemptee.Name, preemptor.Namespace, preemptor.Name)
if err := stmt.Evict(preemptee, "preempt"); err != nil {
Expand All @@ -231,17 +238,16 @@ func preempt(
}
preempted.Add(preemptee.Resreq)
// If reclaimed enough resources, break loop to avoid Sub panic.
if resreq.LessEqual(preemptee.Resreq) {
if resreq.LessEqual(preempted) {
break
}
resreq.Sub(preemptee.Resreq)
}

metrics.RegisterPreemptionAttempts()
glog.V(3).Infof("Preempted <%v> for task <%s/%s> requested <%v>.",
preempted, preemptor.Namespace, preemptor.Name, preemptor.Resreq)
preempted, preemptor.Namespace, preemptor.Name, preemptor.InitResreq)

if preemptor.Resreq.LessEqual(preempted) {
if preemptor.InitResreq.LessEqual(preempted) {
if err := stmt.Pipeline(preemptor, node.Name); err != nil {
glog.Errorf("Failed to pipline Task <%s/%s> on Node <%s>",
preemptor.Namespace, preemptor.Name, node.Name)
Expand Down
10 changes: 4 additions & 6 deletions pkg/scheduler/actions/reclaim/reclaim.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,14 +110,13 @@ func (alloc *reclaimAction) Execute(ssn *framework.Session) {
}

assigned := false

for _, n := range ssn.Nodes {
// If predicates failed, next node.
if err := ssn.PredicateFn(task, n); err != nil {
continue
}

resreq := task.Resreq.Clone()
resreq := task.InitResreq.Clone()
reclaimed := api.EmptyResource()

glog.V(3).Infof("Considering Task <%s/%s> on Node <%s>.",
Expand Down Expand Up @@ -165,16 +164,15 @@ func (alloc *reclaimAction) Execute(ssn *framework.Session) {
}
reclaimed.Add(reclaimee.Resreq)
// If reclaimed enough resources, break loop to avoid Sub panic.
if resreq.LessEqual(reclaimee.Resreq) {
if resreq.LessEqual(reclaimed) {
break
}
resreq.Sub(reclaimee.Resreq)
}

glog.V(3).Infof("Reclaimed <%v> for task <%s/%s> requested <%v>.",
reclaimed, task.Namespace, task.Name, task.Resreq)
reclaimed, task.Namespace, task.Name, task.InitResreq)

if task.Resreq.LessEqual(reclaimed) {
if task.InitResreq.LessEqual(reclaimed) {
if err := ssn.Pipeline(task, n.Name); err != nil {
glog.Errorf("Failed to pipeline Task <%s/%s> on Node <%s>",
task.Namespace, task.Name, n.Name)
Expand Down
31 changes: 16 additions & 15 deletions pkg/scheduler/api/job_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,10 @@ type TaskInfo struct {
Name string
Namespace string

// Resreq is the resource that used when task running.
Resreq *Resource
// InitResreq is the resource that used to launch a task.
InitResreq *Resource

NodeName string
Status TaskStatus
Expand All @@ -61,25 +64,22 @@ func getJobID(pod *v1.Pod) JobID {
}

func NewTaskInfo(pod *v1.Pod) *TaskInfo {
req := EmptyResource()

// TODO(k82cn): also includes initContainers' resource.
for _, c := range pod.Spec.Containers {
req.Add(NewResource(c.Resources.Requests))
}
req := GetPodResourceWithoutInitContainers(pod)
initResreq := GetPodResourceRequest(pod)

jobID := getJobID(pod)

ti := &TaskInfo{
UID: TaskID(pod.UID),
Job: jobID,
Name: pod.Name,
Namespace: pod.Namespace,
NodeName: pod.Spec.NodeName,
Status: getTaskStatus(pod),
Priority: 1,
Pod: pod,
Resreq: req,
UID: TaskID(pod.UID),
Job: jobID,
Name: pod.Name,
Namespace: pod.Namespace,
NodeName: pod.Spec.NodeName,
Status: getTaskStatus(pod),
Priority: 1,
Pod: pod,
Resreq: req,
InitResreq: initResreq,
}

if pod.Spec.Priority != nil {
Expand All @@ -100,6 +100,7 @@ func (ti *TaskInfo) Clone() *TaskInfo {
Priority: ti.Priority,
Pod: ti.Pod,
Resreq: ti.Resreq.Clone(),
InitResreq: ti.InitResreq.Clone(),
VolumeReady: ti.VolumeReady,
}
}
Expand Down
71 changes: 71 additions & 0 deletions pkg/scheduler/api/pod_info.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
Copyright 2019 The Kubernetes Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/

package api

import (
"k8s.io/api/core/v1"
)

// Refer k8s.io/kubernetes/pkg/scheduler/algorithm/predicates/predicates.go#GetResourceRequest.
//
// GetResourceRequest returns a *Resource that covers the largest width in each resource dimension.
// Because init-containers run sequentially, we collect the max in each dimension iteratively.
// In contrast, we sum the resource vectors for regular containers since they run simultaneously.
//
// To be consistent with kubernetes default scheduler, it is only used for predicates of actions(e.g.
// allocate, backfill, preempt, reclaim), please use GetPodResourceWithoutInitContainers for other cases.
//
// Example:
//
// Pod:
// InitContainers
// IC1:
// CPU: 2
// Memory: 1G
// IC2:
// CPU: 2
// Memory: 3G
// Containers
// C1:
// CPU: 2
// Memory: 1G
// C2:
// CPU: 1
// Memory: 1G
//
// Result: CPU: 3, Memory: 3G
func GetPodResourceRequest(pod *v1.Pod) *Resource {
result := GetPodResourceWithoutInitContainers(pod)

// take max_resource(sum_pod, any_init_container)
for _, container := range pod.Spec.InitContainers {
result.SetMaxResource(NewResource(container.Resources.Requests))
}

return result
}

// GetPodResourceWithoutInitContainers returns Pod's resource request, it does not contain
// init containers' resource request.
func GetPodResourceWithoutInitContainers(pod *v1.Pod) *Resource {
result := EmptyResource()
for _, container := range pod.Spec.Containers {
result.Add(NewResource(container.Resources.Requests))
}

return result
}
Loading

0 comments on commit bffcc78

Please sign in to comment.