diff --git a/hack/integration-test.sh b/hack/integration-test.sh index f6e95e6555..e0ec88cfdd 100755 --- a/hack/integration-test.sh +++ b/hack/integration-test.sh @@ -48,7 +48,7 @@ runTests() { kube::log::status "Running integration test cases" # TODO: make args customizable. - go test -mod=vendor sigs.k8s.io/scheduler-plugins/test/integration/... + go test -count=1 -mod=vendor sigs.k8s.io/scheduler-plugins/test/integration/... cleanup } diff --git a/kep/42-podgroup-coscheduling/README.md b/kep/42-podgroup-coscheduling/README.md index d6f9466ac5..34147723cd 100644 --- a/kep/42-podgroup-coscheduling/README.md +++ b/kep/42-podgroup-coscheduling/README.md @@ -59,8 +59,8 @@ type PodGroupStatus struct { // Current phase of PodGroup. Phase PodGroupPhase `json:"phase"` - // OccupiedBy marks the workload (e.g., deployment, statefulset) UID that occupy the podgroup. - // It is empty if not initialized. + // OccupiedBy marks the workload (e.g., deployment, statefulset) UID that occupy the podgroup. + //It is empty if not initialized. OccupiedBy string `json:"occupiedBy,omitempty"` // The number of actively running pods. diff --git a/manifests/coscheduling/crd.yaml b/manifests/coscheduling/crd.yaml new file mode 100644 index 0000000000..e65a696cf9 --- /dev/null +++ b/manifests/coscheduling/crd.yaml @@ -0,0 +1,15 @@ +apiVersion: apiextensions.k8s.io/v1beta1 +kind: CustomResourceDefinition +metadata: + name: podgroups.scheduling.sigs.k8s.io +spec: + group: scheduling.sigs.k8s.io + names: + kind: PodGroup + plural: podgroups + singular: podgroup + shortNames: + - pg + - pgs + scope: Namespaced + version: v1 \ No newline at end of file diff --git a/manifests/coscheduling/scheduler-config.yaml b/manifests/coscheduling/scheduler-config.yaml index 6b445d24e2..34885dae49 100644 --- a/manifests/coscheduling/scheduler-config.yaml +++ b/manifests/coscheduling/scheduler-config.yaml @@ -18,11 +18,11 @@ profiles: permit: enabled: - name: Coscheduling - reserve: + postbind: enabled: - name: Coscheduling # optional plugin configs - pluginConfig: + pluginConfig: - name: Coscheduling args: permitWaitingTimeSeconds: 10 diff --git a/pkg/apis/config/types.go b/pkg/apis/config/types.go index 45dc7cd55b..bea7b06464 100644 --- a/pkg/apis/config/types.go +++ b/pkg/apis/config/types.go @@ -32,4 +32,8 @@ type CoschedulingArgs struct { // If the deleted PodGroup stays longer than the PodGroupExpirationTime, // the PodGroup will be deleted from PodGroupInfos. PodGroupExpirationTimeSeconds *int64 + // KubeMaster is the url of api-server + KubeMaster string + // KubeConfig for scheduler + KubeConfig string } diff --git a/pkg/apis/config/v1beta1/types.go b/pkg/apis/config/v1beta1/types.go index 30ab4a6796..55917ca6d5 100644 --- a/pkg/apis/config/v1beta1/types.go +++ b/pkg/apis/config/v1beta1/types.go @@ -33,4 +33,8 @@ type CoschedulingArgs struct { // If the deleted PodGroup stays longer than the PodGroupExpirationTime, // the PodGroup will be deleted from PodGroupInfos. PodGroupExpirationTimeSeconds *int64 `json:"podGroupExpirationTimeSeconds,omitempty"` + // KubeMaster is the url of api-server + KubeMaster string `json:"kubeMaster,omitempty"` + // KubeConfig for scheduler + KubeConfig string `json:"kubeConfig,omitempty"` } diff --git a/pkg/apis/config/v1beta1/zz_generated.conversion.go b/pkg/apis/config/v1beta1/zz_generated.conversion.go index 4f74c86c00..c369aef7c2 100644 --- a/pkg/apis/config/v1beta1/zz_generated.conversion.go +++ b/pkg/apis/config/v1beta1/zz_generated.conversion.go @@ -25,10 +25,13 @@ import ( conversion "k8s.io/apimachinery/pkg/conversion" runtime "k8s.io/apimachinery/pkg/runtime" - config "sigs.k8s.io/scheduler-plugins/pkg/apis/config" ) +func init() { + localSchemeBuilder.Register(RegisterConversions) +} + // RegisterConversions adds conversion functions to the given scheme. // Public to allow building arbitrary schemes. func RegisterConversions(s *runtime.Scheme) error { @@ -49,6 +52,8 @@ func autoConvert_v1beta1_CoschedulingArgs_To_config_CoschedulingArgs(in *Cosched out.PermitWaitingTimeSeconds = (*int64)(unsafe.Pointer(in.PermitWaitingTimeSeconds)) out.PodGroupGCIntervalSeconds = (*int64)(unsafe.Pointer(in.PodGroupGCIntervalSeconds)) out.PodGroupExpirationTimeSeconds = (*int64)(unsafe.Pointer(in.PodGroupExpirationTimeSeconds)) + out.KubeMaster = in.KubeMaster + out.KubeConfig = in.KubeConfig return nil } @@ -61,6 +66,8 @@ func autoConvert_config_CoschedulingArgs_To_v1beta1_CoschedulingArgs(in *config. out.PermitWaitingTimeSeconds = (*int64)(unsafe.Pointer(in.PermitWaitingTimeSeconds)) out.PodGroupGCIntervalSeconds = (*int64)(unsafe.Pointer(in.PodGroupGCIntervalSeconds)) out.PodGroupExpirationTimeSeconds = (*int64)(unsafe.Pointer(in.PodGroupExpirationTimeSeconds)) + out.KubeMaster = in.KubeMaster + out.KubeConfig = in.KubeConfig return nil } diff --git a/pkg/coscheduling/README.md b/pkg/coscheduling/README.md index 5e7baaa8e3..1b6d083494 100644 --- a/pkg/coscheduling/README.md +++ b/pkg/coscheduling/README.md @@ -1,7 +1,6 @@ # Overview -This folder holds the coscheduling plugin implementations based on [Lightweight coscheduling based on back-to-back queue -sorting](https://github.com/kubernetes-sigs/scheduler-plugins/tree/master/kep/2-lightweight-coscheduling). +This folder holds the coscheduling plugin implementations based on [Coscheduling based on PodGroup CRD](https://github.com/kubernetes-sigs/scheduler-plugins/tree/master/kep/42-podgroup-coscheduling). ## Maturity Level @@ -14,11 +13,19 @@ sorting](https://github.com/kubernetes-sigs/scheduler-plugins/tree/master/kep/2- ## Tutorial ### PodGroup -We use a special label named pod-group.scheduling.sigs.k8s.io/name to define a PodGroup. Pods that set this label and use the same value belong to the same PodGroup. +We use a special label named podgroup.scheduling.sigs.k8s.io to define a PodGroup. Pods that set this label and use the same value belong to the same PodGroup. + ``` +apiVersion: scheduling.sigs.k8s.io/v1alpha1 +kind: PodGroup +metadata: + name: nginx +spec: + maxScheduleTime: 10s + minMember: 3 +--- labels: - pod-group.scheduling.sigs.k8s.io/name: nginx - pod-group.scheduling.sigs.k8s.io/min-available: "2" + podgroups.scheduling.sigs.k8s.io: nginx ``` We will calculate the sum of the Running pods and the Waiting pods (assumed but not bind) in scheduler, if the sum is greater than or equal to the minAvailable, the Waiting pods will be created. @@ -53,7 +60,7 @@ profiles: permit: enabled: - name: Coscheduling - reserve: + postbind: enabled: - name: Coscheduling ``` @@ -61,6 +68,14 @@ profiles: ### Demo Suppose we have a cluster which can only afford 3 nginx pods. We create a ReplicaSet with replicas=6, and set the value of minAvailable to 3. ```yaml +apiVersion: scheduling.sigs.k8s.io/v1alpha1 +kind: PodGroup +metadata: + name: nginx +spec: + maxScheduleTime: 10s + minMember: 3 +--- apiVersion: apps/v1 kind: ReplicaSet metadata: @@ -77,8 +92,7 @@ spec: name: nginx labels: app: nginx - pod-group.scheduling.sigs.k8s.io/name: nginx - pod-group.scheduling.sigs.k8s.io/min-available: "3" + podgroup.scheduling.sigs.k8s.io: nginx spec: containers: - name: nginx @@ -113,4 +127,4 @@ nginx-gnjsv 0/1 Pending 0 3s nginx-hqhhz 0/1 Pending 0 3s nginx-n47r7 0/1 Pending 0 3s nginx-n7vtq 0/1 Pending 0 3s -``` +``` \ No newline at end of file diff --git a/pkg/coscheduling/apis/podgroup/v1alpha1/register.go b/pkg/coscheduling/apis/podgroup/v1alpha1/register.go index b35f120ec2..74bb879e2c 100644 --- a/pkg/coscheduling/apis/podgroup/v1alpha1/register.go +++ b/pkg/coscheduling/apis/podgroup/v1alpha1/register.go @@ -52,4 +52,4 @@ func addKnownTypes(scheme *runtime.Scheme) error { ) metav1.AddToGroupVersion(scheme, SchemeGroupVersion) return nil -} \ No newline at end of file +} diff --git a/pkg/coscheduling/coscheduler.go b/pkg/coscheduling/coscheduler.go new file mode 100644 index 0000000000..460c325792 --- /dev/null +++ b/pkg/coscheduling/coscheduler.go @@ -0,0 +1,459 @@ +/* +Copyright 2020 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 coscheduling + +import ( + "context" + "fmt" + v1 "k8s.io/apiserver/pkg/apis/example/v1" + "time" + + gochache "github.com/patrickmn/go-cache" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime" + "k8s.io/apimachinery/pkg/types" + "k8s.io/client-go/tools/cache" + "k8s.io/client-go/tools/clientcmd" + "k8s.io/klog/v2" + framework "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1" + + "sigs.k8s.io/scheduler-plugins/pkg/apis/config" + "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/apis/podgroup/v1alpha1" + pgclientset "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/generated/clientset/versioned" + pgformers "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/generated/informers/externalversions" + schecache "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/scheduler/cache" + "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/scheduler/core" + "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/util" +) + +// coScheduler support schedule batch pods +type coScheduler interface { + // UpdateBatchCache update the pod cache which should be scheduled as a group + UpdateBatchCache() + // StartBatchSchedule receives the group name of pod to start scheduling + StartBatchSchedule(string) +} + +type coSchedulingPlugin struct { + frameworkHandler framework.FrameworkHandle + operation *core.ScheduleOperation + startChan chan string + maxScheduleTime *time.Duration + client pgclientset.Interface + extension coSchedulingPluginExtension + cache schecache.Cache +} + +var _ coScheduler = &coSchedulingPlugin{} +var _ framework.PreFilterPlugin = &coSchedulingPlugin{} +var _ framework.PermitPlugin = &coSchedulingPlugin{} +var _ framework.PostBindPlugin = &coSchedulingPlugin{} +var _ framework.QueueSortPlugin = &coSchedulingPlugin{} + +// Name is the name of the plug used in Registry and configurations. +const ( + Name = "Coscheduling" + getWaitPodLimit = 3 +) + +// Name returns name of the plugin. It is used in logs, etc. +func (cs *coSchedulingPlugin) Name() string { + return Name +} + +type coSchedulingPluginExtension struct { + operation *core.ScheduleOperation +} + +// PreFilter is called at the beginning of the scheduling cycle. All PreFilter +// plugins must return success or the pod will be rejected. +func (cs *coSchedulingPlugin) PreFilter(ctx context.Context, _ *framework.CycleState, + p *corev1.Pod) *framework.Status { + if err := cs.operation.PreFilter(ctx, p); err != nil { + klog.Error(err) + return framework.NewStatus(framework.Unschedulable, err.Error()) + } + return framework.NewStatus(framework.Success, "") +} + +// PreFilterExtensions returns a PreFilterExtensions interface if the plugin implements one, +// or nil if it does not. A Pre-filter plugin can provide extensions to incrementally +// modify its pre-processed info. The framework guarantees that the extensions +// AddPod/RemovePod will only be called after PreFilter, possibly on a cloned +// CycleState, and may call those functions more than once before calling +// Filter again on a specific node. +func (cs *coSchedulingPlugin) PreFilterExtensions() framework.PreFilterExtensions { + return &cs.extension +} + +// AddPod is called by the framework while trying to evaluate the impact +// of adding podToAdd to the node while scheduling podToSchedule. +func (cs *coSchedulingPluginExtension) AddPod(ctx context.Context, _ *framework.CycleState, + _ *corev1.Pod, podToAdd *corev1.Pod, nodeInfo *framework.NodeInfo) *framework.Status { + if err := cs.operation.PreemptAddPod(ctx, podToAdd, nodeInfo.Node().Name); err != nil { + return framework.NewStatus(framework.Unschedulable, err.Error()) + } + return framework.NewStatus(framework.Success, "") +} + +// RemovePod is called by the framework while trying to evaluate the impact +// of removing podToRemove from the node while scheduling podToSchedule. +func (cs *coSchedulingPluginExtension) RemovePod(ctx context.Context, _ *framework.CycleState, + podToSchedule *corev1.Pod, podToRemove *corev1.Pod, _ *framework.NodeInfo) *framework.Status { + //klog.V(5).Infof("Batch scheduler try to remove pod %s/%s add pod %s/%s", + // podToRemove.Namespace, podToRemove.Name, podToSchedule.Namespace, podToSchedule.Name) + if err := cs.operation.PreemptRemovePod(ctx, podToSchedule, podToRemove); err != nil { + if klog.V(5).Enabled() { + klog.Error(err) + } + return framework.NewStatus(framework.Unschedulable, err.Error()) + } + klog.V(5).Infof("coSchedulingPluginExtension pass remove pod %v", podToRemove.Name) + return framework.NewStatus(framework.Success, "") +} + +// Permit is called before binding a pod (and before prebind plugins). Permit +// plugins are used to prevent or delay the binding of a Pod. A permit plugin +// must return success or wait with timeout duration, or the pod will be rejected. +// The pod will also be rejected if the wait timeout or the pod is rejected while +// waiting. Note that if the plugin returns "wait", the framework will wait only +// after running the remaining plugins given that no other plugin rejects the pod. +func (cs *coSchedulingPlugin) Permit(ctx context.Context, _ *framework.CycleState, p *corev1.Pod, nodeName string) (*framework.Status, time.Duration) { + var pgs *schecache.PodGroupSchedulingStatus + fullName := "" + ready, fullName, err := cs.operation.Permit(ctx, p, nodeName) + if fullName != "" { + pgObj := cs.operation.PodGroupStatusCache().Get(fullName) + if pgObj != nil { + pgs = pgObj.(*schecache.PodGroupSchedulingStatus) + } + } + waitTime := util.DefaultWaitTime + if pgs != nil { + if wait := util.GetWaitTimeDuration(pgs.PodGroup, cs.maxScheduleTime); wait != 0 { + waitTime = wait + } + } + // add 1 second to keep the ttl cache would + // expired before the the waiting deadline + if err != nil { + if err == util.ErrorWaiting { + klog.Infof("Pod: %v/%v is waiting to be scheduled to node: %v", p.Namespace, p.Name, nodeName) + return framework.NewStatus(framework.Wait, ""), waitTime + } + // For pod not belongs to any groups + if err == util.ErrorNotMatched { + return framework.NewStatus(framework.Success, ""), 0 + + } + klog.Infof("cs.operation.Permit error %v", err) + return framework.NewStatus(framework.Unschedulable, err.Error()), waitTime + } + klog.V(5).Infof("Pod requires pgName %v", fullName) + if ready { + go cs.sendStartScheduleSignal(fullName) + } + + return framework.NewStatus(framework.Wait, ""), waitTime +} + +func (cs *coSchedulingPlugin) Reserve(ctx context.Context, state *framework.CycleState, p *v1.Pod, nodeName string) *framework.Status { + return nil +} + +// Unreserve rejects all other Pods in the PodGroup when one of the pods in the group times out. +func (cs *coSchedulingPlugin) Unreserve(ctx context.Context, state *framework.CycleState, p *corev1.Pod, nodeName string) { + pgName, satisfied := util.VerifyPodLabelSatisfied(p) + if !satisfied { + return + } + if satisfied && len(pgName) != 0 { + fullName := fmt.Sprintf("%v/%v", p.Namespace, pgName) + pgsObj := cs.operation.PodGroupStatusCache().Get(fullName) + if pgsObj == nil { + return + } + pgs := pgsObj.(*schecache.PodGroupSchedulingStatus) + pgs.MatchedPodNodes.Delete(string(p.UID)) + } +} + +// PostBind is called after a pod is successfully bound. These plugins are +// informational. A common application of this extension point is for cleaning +// up. If a plugin needs to clean-up its state after a pod is scheduled and +// bound, Postbind is the extension point that it should register. +func (cs *coSchedulingPlugin) PostBind(ctx context.Context, _ *framework.CycleState, p *corev1.Pod, nodeName string) { + klog.V(5).Infof("PostBind pod: %v/%v", p.Namespace, p.Name) + cs.operation.PostBind(ctx, p, nodeName) +} + +// Less are used to sort pods in the scheduling queue. +func (cs *coSchedulingPlugin) Less(pi1 *framework.QueuedPodInfo, pi2 *framework.QueuedPodInfo) bool { + return cs.operation.Less(pi1, pi2) +} + +// UpdateLocalCache upadte pgstatus cache +func (cs *coSchedulingPlugin) UpdateBatchCache() { + // update pods cache + cs.frameworkHandler.IterateOverWaitingPods(func(waitingPod framework.WaitingPod) { + klog.V(5).Info("Start IterateOverWaitingPods") + pod := waitingPod.GetPod() + pgName, satisfied := util.VerifyPodLabelSatisfied(pod) + klog.V(5).Infof("Start walking through pod %v/%v uid: %v", pod.Namespace, pod.Name, pod.UID) + if satisfied && len(pgName) != 0 { + fullName := fmt.Sprintf("%v/%v", pod.Namespace, pgName) + pgsObj := cs.operation.PodGroupStatusCache().Get(fullName) + if pgsObj == nil { + return + } + pgs := pgsObj.(*schecache.PodGroupSchedulingStatus) + klog.V(5).Infof("Wanted cache pod %v/%v", pod.Namespace, pod.Name) + oldUID, found := pgs.PodNameUIDs.Get(fmt.Sprintf("%v/%v", pod.Namespace, pod.Name)) + // pod has been scheduled ever + if found { + if oldUID.(string) != string(pod.UID) { + // delete the expired one + pgs.MatchedPodNodes.Delete(oldUID.(string)) + pgs.PodNameUIDs.Delete(fmt.Sprintf("%v/%v", pod.Namespace, pod.Name)) + klog.V(3).Infof("Delete old cache data %v", oldUID) + } + } else { + // newly add one + klog.V(3).Infof("Add new cache data %v", pod.UID) + } + klog.V(6).Infof("Current pod group state %+v", pgs.PodNameUIDs.Items()) + cs.operation.PodGroupStatusCache().Set(util.GetPodGroupFullName(pgs.PodGroup), pgs) + } + }) +} + +// StartBatchSchedule receives the group name of pod to start scheduling +func (cs *coSchedulingPlugin) StartBatchSchedule(fullName string) { + pgs := cs.cache.Get(fullName).(*schecache.PodGroupSchedulingStatus) + if pgs.PodGroup.Status.Phase != v1alpha1.PodGroupPreScheduling && pgs.PodGroup.Status.Phase != v1alpha1. + PodGroupScheduling { + return + } + + // record time to avoid abnormal exit when bind + if pgs.PodGroup.Status.Scheduled >= pgs.PodGroup.Spec.MinMember { + pg, err := cs.client.SchedulingV1alpha1().PodGroups(pgs.PodGroup.Namespace).Get(context.TODO(), pgs.PodGroup.Name, + metav1.GetOptions{}) + if err != nil { + klog.Error(err) + return + } + pgCopy := pg.DeepCopy() + pgCopy.Status.ScheduleStartTime = metav1.Now() + + var patch []byte + patch, err = util.CreateMergePatch(pg, pgCopy) + if err != nil { + return + } + + pg, err = cs.client.SchedulingV1alpha1().PodGroups(pg.Namespace).Patch(context.TODO(), pg.Name, + types.MergePatchType, patch, metav1.PatchOptions{}) + if err != nil { + return + } + } + + // Start coscheduling + klog.V(4).Infof("Start scheduling %v", fullName) + pendingPods := cs.operation.GetPodNodePairs(fullName) + if pendingPods == nil { + klog.V(4).Infof("Can not found pending pods for %v", fullName) + return + } + pendingPodNameIDs := cs.operation.GetPodNameUIDs(fullName) + if pendingPodNameIDs == nil { + klog.V(4).Infof("Can not found pending IDs for %v", fullName) + return + } + pendingPodsMap := pendingPods.Items() + if uint32(len(pendingPodsMap)) < pgs.PodGroup.Spec.MinMember-pgs.PodGroup.Status.Scheduled { + return + } + + klog.V(5).Infof("Current pod group: %v state %+v, count: %d", fullName, pendingPodNameIDs, + len(pendingPodsMap)) + + for uid, pair := range pendingPodsMap { + // double check + var waitingPod framework.WaitingPod + for i := 0; i < getWaitPodLimit; i++ { + waitingPod = cs.frameworkHandler.GetWaitingPod(types.UID(uid)) + if waitingPod == nil { + if i == 2 { + // to avoid sig send, but scheduler cache have not been flushed + klog.V(4).Infof("Remove pod uid %v, pair %v", uid, pair.Object) + pendingPods.Delete(uid) + pnPair := pair.Object.(*schecache.PodNodePair) + pendingPodNameIDs.Delete(pnPair.PodName) + return + } + time.Sleep(10 * time.Millisecond) + continue + + } + } + if waitingPod != nil { + waitingPod.Allow(Name) + klog.V(0).Infof("Allowed pod %+v", pair.Object) + pendingPods.Delete(uid) + pendingPodNameIDs.Delete(uid) + } else { + klog.V(5).Infof("Group %v get waitingPod nil", fullName) + } + } +} + +// rejectPod rejects pod in cache +func (cs *coSchedulingPlugin) rejectPod(uid types.UID) { + waitingPod := cs.frameworkHandler.GetWaitingPod(uid) + if waitingPod == nil { + return + } + waitingPod.Reject("Group failed") +} + +// ReconcileStatus reconcile pod cache states and decide when to schedule +func (cs *coSchedulingPlugin) ReconcileStatus(stopChan <-chan struct{}) { + for { + select { + case pgName := <-cs.startChan: + cs.UpdateBatchCache() + cs.StartBatchSchedule(pgName) + case <-stopChan: + klog.Info("Reconcile exit") + return + } + } +} + +// sendStartScheduleSignal send stat scheduling signal to scheduler +func (cs *coSchedulingPlugin) sendStartScheduleSignal(pgName string) { + cs.startChan <- pgName + klog.Infof("Send StartScheduleSignal for %v success", pgName) +} + +// pgAdded reacts to a PG creation +func (cs *coSchedulingPlugin) pgAdded(obj interface{}) { + key, err := cache.MetaNamespaceKeyFunc(obj) + if err != nil { + klog.Error(err) + return + } + pg := obj.(*v1alpha1.PodGroup) + if pg.Status.Phase == v1alpha1.PodGroupFinished || pg.Status.Phase == v1alpha1.PodGroupFailed { + return + } + pgsObj := cs.cache.Get(key) + if pgsObj == nil { + klog.Infof("Add pg %v", key) + pgsObj = cs.initPodGroupSchedulingStatus(pg, key) + cs.cache.Set(key, pgsObj) + } +} + +// pgUpdated reacts to a PG update +func (cs *coSchedulingPlugin) pgUpdated(_, new interface{}) { + cs.pgAdded(new) +} + +// pgDelete reacts to a PG update +func (cs *coSchedulingPlugin) pgDelete(new interface{}) { + key, err := cache.MetaNamespaceKeyFunc(new) + if err != nil { + klog.Error(err) + return + } + cs.cache.Delete(key) +} + +// initPodGroupSchedulingStatus init pod groups +func (cs *coSchedulingPlugin) initPodGroupSchedulingStatus(pg *v1alpha1.PodGroup, key string) *schecache.PodGroupSchedulingStatus { + pgs := &schecache.PodGroupSchedulingStatus{ + PodGroup: pg, + MatchedPodNodes: gochache.New(1*time.Minute, 2*time.Minute), + PodNameUIDs: gochache.New(1*time.Minute, 2*time.Minute), + Failed: make(map[string]string), + Succeed: make(map[string]string), + } + pgs.PodNameUIDs.OnEvicted(func(s string, i interface{}) { + klog.V(4).Infof("Evict triggered group %v", key) + for podID := range pgs.MatchedPodNodes.Items() { + klog.Infof("Foreach %v", podID) + cs.rejectPod(types.UID(podID)) + if pgs.MatchedPodNodes != nil { + pgs.MatchedPodNodes.Delete(podID) + } + } + pgs.PodNameUIDs.Flush() + cs.operation.AddToDenyCache(key) + }) + return pgs +} + +// New initializes a new plugin and returns it. +func New(obj runtime.Object, f framework.FrameworkHandle) (framework.Plugin, error) { + args, ok := obj.(*config.CoschedulingArgs) + if !ok { + return nil, fmt.Errorf("want args to be of type CoschedulingArgs, got %T", obj) + } + conf, err := clientcmd.BuildConfigFromFlags(args.KubeMaster, args.KubeConfig) + if err != nil { + return nil, fmt.Errorf("failed to init rest.Config: %v", err) + } + pgClient := pgclientset.NewForConfigOrDie(conf) + scheduleInformer := pgformers.NewSharedInformerFactory(pgClient, 0) + pgInformer := scheduleInformer.Scheduling().V1alpha1().PodGroups() + + pgCache := schecache.NewPGSchedulingCache() + startChan := make(chan string) + + scheduleTimeDuration := 10 * time.Second + if args.PermitWaitingTimeSeconds != nil { + scheduleTimeDuration = time.Duration(*args.PermitWaitingTimeSeconds) + } + pgOperation := core.NewScheduleOperation(pgClient, pgCache, f, &scheduleTimeDuration, pgInformer) + plugin := &coSchedulingPlugin{ + frameworkHandler: f, + operation: pgOperation, + startChan: startChan, + maxScheduleTime: &scheduleTimeDuration, + client: pgClient, + cache: pgOperation.PodGroupStatusCache(), + extension: coSchedulingPluginExtension{pgOperation}, + } + + pgInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: plugin.pgAdded, + UpdateFunc: plugin.pgUpdated, + DeleteFunc: plugin.pgDelete, + }) + ctx := context.TODO() + go plugin.ReconcileStatus(ctx.Done()) + scheduleInformer.Start(ctx.Done()) + if !cache.WaitForCacheSync(ctx.Done(), pgInformer.Informer().HasSynced) { + klog.Errorf("Cannot sync caches") + return nil, fmt.Errorf("WaitForCacheSync failed") + } + return plugin, nil +} diff --git a/pkg/coscheduling/coscheduling.go b/pkg/coscheduling/coscheduling.go deleted file mode 100644 index 9ae65bf5ff..0000000000 --- a/pkg/coscheduling/coscheduling.go +++ /dev/null @@ -1,387 +0,0 @@ -/* -Copyright 2020 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 coscheduling - -import ( - "context" - "fmt" - "strconv" - "sync" - "time" - - v1 "k8s.io/api/core/v1" - "k8s.io/apimachinery/pkg/labels" - "k8s.io/apimachinery/pkg/runtime" - "k8s.io/apimachinery/pkg/util/wait" - corelisters "k8s.io/client-go/listers/core/v1" - "k8s.io/client-go/tools/cache" - "k8s.io/klog/v2" - podutil "k8s.io/kubernetes/pkg/api/v1/pod" - framework "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1" - "k8s.io/kubernetes/pkg/scheduler/util" - config "sigs.k8s.io/scheduler-plugins/pkg/apis/config" -) - -// Coscheduling is a plugin that implements the mechanism of gang scheduling. -type Coscheduling struct { - frameworkHandle framework.FrameworkHandle - podLister corelisters.PodLister - // key is / and value is *PodGroupInfo. - podGroupInfos sync.Map - // clock is used to get the current time. - clock util.Clock - // args is coscheduling parameters - args config.CoschedulingArgs -} - -// PodGroupInfo is a wrapper to a PodGroup with additional information. -// A PodGroup's priority, timestamp and minAvailable are set according to -// the values of the PodGroup's first pod that is added to the scheduling queue. -type PodGroupInfo struct { - // key is a unique PodGroup ID and currently implemented as /. - key string - // name is the PodGroup name and defined through a Pod label. - // The PodGroup name of a regular pod is empty. - name string - // priority is the priority of pods in a PodGroup. - // All pods in a PodGroup should have the same priority. - priority int32 - // timestamp stores the initialization timestamp of a PodGroup. - timestamp time.Time - // minAvailable is the minimum number of pods to be co-scheduled in a PodGroup. - // All pods in a PodGroup should have the same minAvailable. - minAvailable int - // deletionTimestamp stores the timestamp when the PodGroup marked as expired. - deletionTimestamp *time.Time -} - -var _ framework.QueueSortPlugin = &Coscheduling{} -var _ framework.PreFilterPlugin = &Coscheduling{} -var _ framework.PermitPlugin = &Coscheduling{} -var _ framework.ReservePlugin = &Coscheduling{} - -const ( - // Name is the name of the plugin used in Registry and configurations. - Name = "Coscheduling" - // PodGroupName is the name of a pod group that defines a coscheduling pod group. - PodGroupName = "pod-group.scheduling.sigs.k8s.io/name" - // PodGroupMinAvailable specifies the minimum number of pods to be scheduled together in a pod group. - PodGroupMinAvailable = "pod-group.scheduling.sigs.k8s.io/min-available" -) - -// Name returns name of the plugin. It is used in logs, etc. -func (cs *Coscheduling) Name() string { - return Name -} - -// New initializes a new plugin and returns it. -func New(obj runtime.Object, handle framework.FrameworkHandle) (framework.Plugin, error) { - args, ok := obj.(*config.CoschedulingArgs) - if !ok { - return nil, fmt.Errorf("want args to be of type CoschedulingArgs, got %T", obj) - } - - podLister := handle.SharedInformerFactory().Core().V1().Pods().Lister() - cs := &Coscheduling{frameworkHandle: handle, - podLister: podLister, - clock: util.RealClock{}, - args: *args, - } - podInformer := handle.SharedInformerFactory().Core().V1().Pods().Informer() - podInformer.AddEventHandler( - cache.FilteringResourceEventHandler{ - FilterFunc: func(obj interface{}) bool { - switch t := obj.(type) { - case *v1.Pod: - return responsibleForPod(t) - case cache.DeletedFinalStateUnknown: - if pod, ok := t.Obj.(*v1.Pod); ok { - return responsibleForPod(pod) - } - return false - default: - return false - } - }, - Handler: cache.ResourceEventHandlerFuncs{ - DeleteFunc: cs.markPodGroupAsExpired, - }, - }, - ) - go wait.Until(cs.podGroupInfoGC, time.Duration(*cs.args.PodGroupGCIntervalSeconds)*time.Second, nil) - - return cs, nil -} - -// Less is used to sort pods in the scheduling queue. -// 1. Compare the priorities of Pods. -// 2. Compare the initialization timestamps of PodGroups/Pods. -// 3. Compare the keys of PodGroups/Pods, i.e., if two pods are tied at priority and creation time, the one without podGroup will go ahead of the one with podGroup. -func (cs *Coscheduling) Less(podInfo1, podInfo2 *framework.QueuedPodInfo) bool { - pgInfo1, _ := cs.getOrCreatePodGroupInfo(podInfo1.Pod, podInfo1.InitialAttemptTimestamp) - pgInfo2, _ := cs.getOrCreatePodGroupInfo(podInfo2.Pod, podInfo2.InitialAttemptTimestamp) - - priority1 := pgInfo1.priority - priority2 := pgInfo2.priority - - if priority1 != priority2 { - return priority1 > priority2 - } - - time1 := pgInfo1.timestamp - time2 := pgInfo2.timestamp - - if !time1.Equal(time2) { - return time1.Before(time2) - } - - return pgInfo1.key < pgInfo2.key -} - -// getOrCreatePodGroupInfo returns the existing PodGroup in PodGroupInfos if present. -// Otherwise, it creates a PodGroup and returns the value, It stores -// the created PodGroup in PodGroupInfo if the pod defines a PodGroup and its -// PodGroupMinAvailable is greater than one. It also returns the pod's -// PodGroupMinAvailable (0 if not specified). -func (cs *Coscheduling) getOrCreatePodGroupInfo(pod *v1.Pod, ts time.Time) (*PodGroupInfo, int) { - podGroupName, podMinAvailable, _ := GetPodGroupLabels(pod) - - var pgKey string - if len(podGroupName) > 0 && podMinAvailable > 0 { - pgKey = fmt.Sprintf("%v/%v", pod.Namespace, podGroupName) - } - - // If it is a PodGroup and present in PodGroupInfos, return it. - if len(pgKey) != 0 { - value, exist := cs.podGroupInfos.Load(pgKey) - if exist { - pgInfo := value.(*PodGroupInfo) - // If the deleteTimestamp isn't nil, it means that the PodGroup is marked as expired before. - // So we need to set the deleteTimestamp as nil again to mark the PodGroup active. - if pgInfo.deletionTimestamp != nil { - pgInfo.deletionTimestamp = nil - cs.podGroupInfos.Store(pgKey, pgInfo) - } - return pgInfo, podMinAvailable - } - } - - // If the PodGroup is not present in PodGroupInfos or the pod is a regular pod, - // create a PodGroup for the Pod and store it in PodGroupInfos if it's not a regular pod. - pgInfo := &PodGroupInfo{ - name: podGroupName, - key: pgKey, - priority: podutil.GetPodPriority(pod), - timestamp: ts, - minAvailable: podMinAvailable, - } - - // If it's not a regular Pod, store the PodGroup in PodGroupInfos - if len(pgKey) > 0 { - cs.podGroupInfos.Store(pgKey, pgInfo) - } - return pgInfo, podMinAvailable -} - -// PreFilter performs the following validations. -// 1. Validate if minAvailables and priorities of all the pods in a PodGroup are the same. -// 2. Validate if the total number of pods belonging to the same `PodGroup` is less than `minAvailable`. -// If so, the scheduling process will be interrupted directly to avoid the partial Pods and hold the system resources -// until a timeout. It will reduce the overall scheduling time for the whole group. -func (cs *Coscheduling) PreFilter(ctx context.Context, state *framework.CycleState, pod *v1.Pod) *framework.Status { - pgInfo, podMinAvailable := cs.getOrCreatePodGroupInfo(pod, time.Now()) - pgKey := pgInfo.key - if len(pgKey) == 0 { - return framework.NewStatus(framework.Success, "") - } - pgMinAvailable := pgInfo.minAvailable - - // Check if the values of minAvailable are the same. - if podMinAvailable != pgMinAvailable { - klog.V(3).Infof("Pod %v has a different minAvailable (%v) as the PodGroup %v (%v)", pod.Name, podMinAvailable, pgKey, pgMinAvailable) - return framework.NewStatus(framework.Unschedulable, "PodGroupMinAvailables do not match") - } - // Check if the priorities are the same. - pgPriority := pgInfo.priority - podPriority := podutil.GetPodPriority(pod) - if pgPriority != podPriority { - klog.V(3).Infof("Pod %v has a different priority (%v) as the PodGroup %v (%v)", pod.Name, podPriority, pgKey, pgPriority) - return framework.NewStatus(framework.Unschedulable, "Priorities do not match") - } - - total := cs.calculateTotalPods(pgInfo.name, pod.Namespace) - if total < pgMinAvailable { - klog.V(3).Infof("The count of PodGroup %v (%v) is less than minAvailable(%d) in PreFilter: %d", - pgKey, pod.Name, pgMinAvailable, total) - return framework.NewStatus(framework.Unschedulable, "less than pgMinAvailable") - } - - return framework.NewStatus(framework.Success, "") -} - -// PreFilterExtensions returns nil. -func (cs *Coscheduling) PreFilterExtensions() framework.PreFilterExtensions { - return nil -} - -// Permit is the functions invoked by the framework at "Permit" extension point. -func (cs *Coscheduling) Permit(ctx context.Context, state *framework.CycleState, pod *v1.Pod, nodeName string) (*framework.Status, time.Duration) { - pgInfo, _ := cs.getOrCreatePodGroupInfo(pod, time.Now()) - if len(pgInfo.key) == 0 { - return framework.NewStatus(framework.Success, ""), 0 - } - - namespace := pod.Namespace - podGroupName := pgInfo.name - minAvailable := pgInfo.minAvailable - // bound includes both assigned and assumed Pods. - bound := cs.calculateBoundPods(podGroupName, namespace) - // The bound is calculated from the snapshot. The current pod does not exist in the snapshot during this scheduling cycle. - current := bound + 1 - - if current < minAvailable { - klog.V(3).Infof("The count of podGroup %v/%v/%v is not up to minAvailable(%d) in Permit: current(%d)", - pod.Namespace, podGroupName, pod.Name, minAvailable, current) - // TODO Change the timeout to a dynamic value depending on the size of the `PodGroup` - return framework.NewStatus(framework.Wait, ""), time.Duration(*cs.args.PermitWaitingTimeSeconds) * time.Second - } - - klog.V(3).Infof("The count of PodGroup %v/%v/%v is up to minAvailable(%d) in Permit: current(%d)", - pod.Namespace, podGroupName, pod.Name, minAvailable, current) - cs.frameworkHandle.IterateOverWaitingPods(func(waitingPod framework.WaitingPod) { - if waitingPod.GetPod().Namespace == namespace && waitingPod.GetPod().Labels[PodGroupName] == podGroupName { - klog.V(3).Infof("Permit allows the pod: %v/%v", podGroupName, waitingPod.GetPod().Name) - waitingPod.Allow(cs.Name()) - } - }) - - return framework.NewStatus(framework.Success, ""), 0 -} - -func (cs *Coscheduling) Reserve(ctx context.Context, state *framework.CycleState, p *v1.Pod, nodeName string) *framework.Status { - return nil -} - -// Unreserve rejects all other Pods in the PodGroup when one of the pods in the group times out. -func (cs *Coscheduling) Unreserve(ctx context.Context, state *framework.CycleState, pod *v1.Pod, nodeName string) { - pgInfo, _ := cs.getOrCreatePodGroupInfo(pod, time.Now()) - if len(pgInfo.key) == 0 { - return - } - podGroupName := pgInfo.name - cs.frameworkHandle.IterateOverWaitingPods(func(waitingPod framework.WaitingPod) { - if waitingPod.GetPod().Namespace == pod.Namespace && waitingPod.GetPod().Labels[PodGroupName] == podGroupName { - klog.V(3).Infof("Unreserve rejects the pod: %v/%v", podGroupName, waitingPod.GetPod().Name) - waitingPod.Reject(cs.Name()) - } - }) -} - -// GetPodGroupLabels checks if the pod belongs to a PodGroup. If so, it will return the -// podGroupName, minAvailable of the PodGroup. If not, it will return "" and 0. -func GetPodGroupLabels(pod *v1.Pod) (string, int, error) { - podGroupName, exist := pod.Labels[PodGroupName] - if !exist || len(podGroupName) == 0 { - return "", 0, nil - } - minAvailable, exist := pod.Labels[PodGroupMinAvailable] - if !exist || len(minAvailable) == 0 { - return "", 0, nil - } - minNum, err := strconv.Atoi(minAvailable) - if err != nil { - klog.Errorf("PodGroup %v/%v : PodGroupMinAvailable %v is invalid", pod.Namespace, pod.Name, minAvailable) - return "", 0, err - } - if minNum < 1 { - klog.Errorf("PodGroup %v/%v : PodGroupMinAvailable %v is less than 1", pod.Namespace, pod.Name, minAvailable) - return "", 0, err - } - return podGroupName, minNum, nil -} - -func (cs *Coscheduling) calculateTotalPods(podGroupName, namespace string) int { - // TODO get the total pods from the scheduler cache and queue instead of the hack manner. - selector := labels.Set{PodGroupName: podGroupName}.AsSelector() - pods, err := cs.podLister.Pods(namespace).List(selector) - if err != nil { - klog.Error(err) - return 0 - } - return len(pods) -} - -func (cs *Coscheduling) calculateBoundPods(podGroupName, namespace string) int { - nodeInfos, err := cs.frameworkHandle.SnapshotSharedLister().NodeInfos().List() - if err != nil { - klog.Errorf("Cannot get nodeInfos from frameworkHandle: %v", err) - return 0 - } - var count int - for _, nodeInfo := range nodeInfos { - for _, podInfo := range nodeInfo.Pods { - pod := podInfo.Pod - if pod.Labels[PodGroupName] == podGroupName && pod.Namespace == namespace && pod.Spec.NodeName != "" { - count++ - } - } - } - - return count -} - -// markPodGroupAsExpired set the deletionTimestamp of PodGroup to mark PodGroup as expired. -func (cs *Coscheduling) markPodGroupAsExpired(obj interface{}) { - pod := obj.(*v1.Pod) - podGroupName, podMinAvailable, _ := GetPodGroupLabels(pod) - if len(podGroupName) == 0 || podMinAvailable == 0 { - return - } - - pgKey := fmt.Sprintf("%v/%v", pod.Namespace, podGroupName) - // If it's a PodGroup and present in PodGroupInfos, set its deletionTimestamp. - value, exist := cs.podGroupInfos.Load(pgKey) - if !exist { - return - } - pgInfo := value.(*PodGroupInfo) - if pgInfo.deletionTimestamp == nil { - now := cs.clock.Now() - pgInfo.deletionTimestamp = &now - cs.podGroupInfos.Store(pgKey, pgInfo) - } -} - -// responsibleForPod selects pod that belongs to a PodGroup. -func responsibleForPod(pod *v1.Pod) bool { - podGroupName, podMinAvailable, _ := GetPodGroupLabels(pod) - if len(podGroupName) == 0 || podMinAvailable == 0 { - return false - } - return true -} - -func (cs *Coscheduling) podGroupInfoGC() { - cs.podGroupInfos.Range(func(key, value interface{}) bool { - pgInfo := value.(*PodGroupInfo) - if pgInfo.deletionTimestamp != nil && pgInfo.deletionTimestamp.Add(time.Duration(*cs.args.PodGroupExpirationTimeSeconds)*time.Second).Before(cs.clock.Now()) { - klog.V(3).Infof("%v is out of date and has been deleted in PodGroup GC", key) - cs.podGroupInfos.Delete(key) - } - return true - }) -} diff --git a/pkg/coscheduling/coscheduling_test.go b/pkg/coscheduling/coscheduling_test.go deleted file mode 100644 index e36b2f75c5..0000000000 --- a/pkg/coscheduling/coscheduling_test.go +++ /dev/null @@ -1,626 +0,0 @@ -/* -Copyright 2020 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 coscheduling - -import ( - "context" - "fmt" - "testing" - "time" - - v1 "k8s.io/api/core/v1" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - "k8s.io/apimachinery/pkg/util/clock" - "k8s.io/apimachinery/pkg/util/wait" - "k8s.io/client-go/informers" - clientsetfake "k8s.io/client-go/kubernetes/fake" - "k8s.io/kubernetes/pkg/scheduler/framework/plugins/defaultbinder" - fwkruntime "k8s.io/kubernetes/pkg/scheduler/framework/runtime" - framework "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1" - st "k8s.io/kubernetes/pkg/scheduler/testing" - "k8s.io/kubernetes/pkg/scheduler/util" - "sigs.k8s.io/scheduler-plugins/pkg/apis/config" - - // Ensure scheme package is initialized. - _ "sigs.k8s.io/scheduler-plugins/pkg/apis/config/scheme" -) - -func newInt64(i int64) *int64 { - val := i - return &val -} - -// FakeNew is used for test. -func FakeNew(clock util.Clock, stop chan struct{}) (*Coscheduling, error) { - cs := &Coscheduling{ - clock: clock, - args: config.CoschedulingArgs{ - PermitWaitingTimeSeconds: newInt64(10), - PodGroupGCIntervalSeconds: newInt64(30), - PodGroupExpirationTimeSeconds: newInt64(600), - }, - } - go wait.Until(cs.podGroupInfoGC, time.Duration(*cs.args.PodGroupGCIntervalSeconds)*time.Second, stop) - return cs, nil -} - -func TestLess(t *testing.T) { - labels1 := map[string]string{ - PodGroupName: "pg1", - PodGroupMinAvailable: "3", - } - labels2 := map[string]string{ - PodGroupName: "pg2", - PodGroupMinAvailable: "5", - } - - var lowPriority, highPriority = int32(10), int32(100) - t1 := time.Now() - t2 := t1.Add(time.Second) - for _, tt := range []struct { - name string - p1 *framework.QueuedPodInfo - p2 *framework.QueuedPodInfo - expected bool - }{ - { - name: "p1.priority less than p2.priority", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1"}, - Spec: v1.PodSpec{ - Priority: &lowPriority, - }, - }, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2"}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - }, - expected: false, // p2 should be ahead of p1 in the queue - }, - { - name: "p1.priority greater than p2.priority", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1"}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2"}, - Spec: v1.PodSpec{ - Priority: &lowPriority, - }, - }, - }, - expected: true, // p1 should be ahead of p2 in the queue - }, - { - name: "equal priority. p1 is added to schedulingQ earlier than p2", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1"}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t1, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2"}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t2, - }, - expected: true, // p1 should be ahead of p2 in the queue - }, - { - name: "equal priority. p2 is added to schedulingQ earlier than p1", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1"}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t2, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2"}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t1, - }, - expected: false, // p2 should be ahead of p1 in the queue - }, - { - name: "p1.priority less than p2.priority, p1 belongs to podGroup1", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1", Labels: labels1}, - Spec: v1.PodSpec{ - Priority: &lowPriority, - }, - }, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2"}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - }, - expected: false, // p2 should be ahead of p1 in the queue - }, - { - name: "p1.priority greater than p2.priority, p1 belongs to podGroup1", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1", Labels: labels1}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2"}, - Spec: v1.PodSpec{ - Priority: &lowPriority, - }, - }, - }, - expected: true, // p1 should be ahead of p2 in the queue - }, - { - name: "equal priority. p1 is added to schedulingQ earlier than p2, p1 belongs to podGroup1", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1", Labels: labels1}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t1, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2"}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t2, - }, - expected: true, // p1 should be ahead of p2 in the queue - }, - { - name: "equal priority. p2 is added to schedulingQ earlier than p1, p1 belongs to podGroup1", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1", Labels: labels1}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t2, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2"}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t1, - }, - expected: false, // p2 should be ahead of p1 in the queue - }, - - { - name: "p1.priority less than p2.priority, p1 belongs to podGroup1 and p2 belongs to podGroup2", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1", Labels: labels1}, - Spec: v1.PodSpec{ - Priority: &lowPriority, - }, - }, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2", Labels: labels2}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - }, - expected: false, // p2 should be ahead of p1 in the queue - }, - { - name: "p1.priority greater than p2.priority, p1 belongs to podGroup1 and p2 belongs to podGroup2", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1", Labels: labels1}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2", Labels: labels2}, - Spec: v1.PodSpec{ - Priority: &lowPriority, - }, - }, - }, - expected: true, // p1 should be ahead of p2 in the queue - }, - { - name: "equal priority. p1 is added to schedulingQ earlier than p2, p1 belongs to podGroup1 and p2 belongs to podGroup2", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1", Labels: labels1}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t1, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2", Labels: labels2}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t2, - }, - expected: true, // p1 should be ahead of p2 in the queue - }, - { - name: "equal priority. p2 is added to schedulingQ earlier than p1, p1 belongs to podGroup1 and p2 belongs to podGroup2", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1", Labels: labels1}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t2, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2", Labels: labels2}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t1, - }, - expected: false, // p2 should be ahead of p1 in the queue - }, - { - name: "equal priority and creation time, p1 belongs to podGroup1 and p2 belongs to podGroup2", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1", Labels: labels1}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t1, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2", Labels: labels2}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t1, - }, - expected: true, // p1 should be ahead of p2 in the queue - }, - { - name: "equal priority and creation time, p2 belong to podGroup2", - p1: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod1", Namespace: "namespace1"}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t1, - }, - p2: &framework.QueuedPodInfo{ - Pod: &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{Name: "pod2", Namespace: "namespace2", Labels: labels2}, - Spec: v1.PodSpec{ - Priority: &highPriority, - }, - }, - InitialAttemptTimestamp: t1, - }, - expected: true, // p1 should be ahead of p2 in the queue - }, - } { - t.Run(tt.name, func(t *testing.T) { - coscheduling := &Coscheduling{} - if got := coscheduling.Less(tt.p1, tt.p2); got != tt.expected { - t.Errorf("expected %v, got %v", tt.expected, got) - } - }) - } -} - -func TestPreFilter(t *testing.T) { - tests := []struct { - name string - pod *v1.Pod - pods []*v1.Pod - expected framework.Code - }{ - { - name: "pod does not belong to any podGroup", - pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Obj(), - pods: []*v1.Pod{ - st.MakePod().Name("pg1-1").UID("pg1-1").Namespace("ns1").Label(PodGroupName, "pg1").Obj(), - st.MakePod().Name("pg2-1").UID("pg2-1").Namespace("ns1").Label(PodGroupName, "pg2").Obj(), - }, - expected: framework.Success, - }, - { - name: "pod belongs to podGroup1 and its PodGroupMinAvailable does not match the group's", - pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Label(PodGroupName, "pg1").Label(PodGroupMinAvailable, "2").Obj(), - pods: []*v1.Pod{ - st.MakePod().Name("pg1-1").UID("pg1-1").Namespace("ns1").Label(PodGroupName, "pg1").Label(PodGroupMinAvailable, "3").Obj(), - }, - expected: framework.Unschedulable, - }, - { - name: "pod belongs to podGroup1 and its priority does not match the group's", - pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Priority(20).Label(PodGroupName, "pg1").Label(PodGroupMinAvailable, "2").Obj(), - pods: []*v1.Pod{ - st.MakePod().Name("pg1-1").UID("pg1-1").Namespace("ns1").Priority(10).Label(PodGroupName, "pg1").Label(PodGroupMinAvailable, "2").Obj(), - }, - expected: framework.Unschedulable, - }, - { - name: "pod belongs to podGroup1, the number of total pods is less than minAvailable", - pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Label(PodGroupName, "pg1").Label(PodGroupMinAvailable, "3").Obj(), - pods: []*v1.Pod{ - st.MakePod().Name("pg1-1").UID("pg1-1").Namespace("ns1").Label(PodGroupName, "pg1").Label(PodGroupMinAvailable, "3").Obj(), - st.MakePod().Name("pg2-1").UID("pg2-1").Namespace("ns1").Label(PodGroupName, "pg2").Label(PodGroupMinAvailable, "1").Obj(), - }, - expected: framework.Unschedulable, - }, - { - name: "pod belongs to podGroup2, the number of total pods is not less than minAvailable", - pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Label(PodGroupName, "pg2").Label(PodGroupMinAvailable, "3").Obj(), - pods: []*v1.Pod{ - st.MakePod().Name("pg2-1").UID("pg2-1").Namespace("ns1").Label(PodGroupName, "pg2").Label(PodGroupMinAvailable, "3").Obj(), - st.MakePod().Name("pg1-1").UID("pg1-1").Namespace("ns1").Label(PodGroupName, "pg1").Label(PodGroupMinAvailable, "1").Obj(), - st.MakePod().Name("pg2-2").UID("pg2-2").Namespace("ns1").Label(PodGroupName, "pg2").Label(PodGroupMinAvailable, "3").Obj(), - }, - expected: framework.Success, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - cs := clientsetfake.NewSimpleClientset() - informerFactory := informers.NewSharedInformerFactory(cs, 0) - podInformer := informerFactory.Core().V1().Pods() - coscheduling := &Coscheduling{podLister: podInformer.Lister()} - for _, p := range tt.pods { - coscheduling.getOrCreatePodGroupInfo(p, time.Now()) - podInformer.Informer().GetStore().Add(p) - } - - podInformer.Informer().GetStore().Add(tt.pod) - if got := coscheduling.PreFilter(nil, nil, tt.pod); got.Code() != tt.expected { - t.Errorf("expected %v, got %v", tt.expected, got.Code()) - } - }) - } -} - -func TestPermit(t *testing.T) { - tests := []struct { - name string - pods []*v1.Pod - expected []framework.Code - }{ - // { - // name: "pods do not belong to any podGroup", - // pods: []*v1.Pod{ - // st.MakePod().Name("pod1").UID("pod1").Obj(), - // st.MakePod().Name("pod2").UID("pod2").Obj(), - // st.MakePod().Name("pod3").UID("pod3").Obj(), - // }, - // expected: []framework.Code{framework.Success, framework.Success, framework.Success}, - // }, - { - name: "pods belong to a podGroup", - pods: []*v1.Pod{ - st.MakePod().Name("pod1").UID("pod1").Label(PodGroupName, "permit").Label(PodGroupMinAvailable, "3").Obj(), - st.MakePod().Name("pod2").UID("pod2").Label(PodGroupName, "permit").Label(PodGroupMinAvailable, "3").Obj(), - st.MakePod().Name("pod3").UID("pod3").Label(PodGroupName, "permit").Label(PodGroupMinAvailable, "3").Obj(), - }, - expected: []framework.Code{framework.Wait, framework.Wait, framework.Success}, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - cs := clientsetfake.NewSimpleClientset() - informerFactory := informers.NewSharedInformerFactory(cs, 0) - podInformer := informerFactory.Core().V1().Pods().Informer() - for _, p := range tt.pods { - podInformer.GetStore().Add(p) - } - - registeredPlugins := []st.RegisterPluginFunc{ - st.RegisterBindPlugin(defaultbinder.Name, defaultbinder.New), - st.RegisterPluginAsExtensions(Name, New, "QueueSort", "Permit"), - } - fakeNode := st.MakeNode().Name("fakeNode").Obj() - snapshot := newFakeSharedLister(nil, []*v1.Node{fakeNode}) - f, err := st.NewFramework( - registeredPlugins, - fwkruntime.WithClientSet(cs), - fwkruntime.WithInformerFactory(informerFactory), - fwkruntime.WithSnapshotSharedLister(snapshot), - ) - if err != nil { - t.Fatalf("fail to create framework: %s", err) - } - - for i := range tt.pods { - if got := f.RunPermitPlugins(context.TODO(), nil, tt.pods[i], "fakeNode"); got.Code() != tt.expected[i] { - t.Errorf("[%v] want %v, but got %v", i, tt.expected[i], got.Code()) - } - - // This operation simulates the operation of AssumePod in scheduling cycle. - // The current pod does not exist in the snapshot during this scheduling cycle. - tt.pods[i].Spec.NodeName = "fakeNode" - snapshot.nodeInfoMap["fakeNode"].AddPod(tt.pods[i]) - } - }) - } -} - -func TestPodGroupClean(t *testing.T) { - tests := []struct { - name string - pod *v1.Pod - podGroupName string - }{ - { - name: "pod belongs to a podGroup", - pod: st.MakePod().Name("pod1").UID("pod1").Label(PodGroupName, "gc").Label(PodGroupMinAvailable, "3").Obj(), - podGroupName: "gc", - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - stop := make(chan struct{}) - defer close(stop) - - c := clock.NewFakeClock(time.Now()) - cs, err := FakeNew(c, stop) - if err != nil { - t.Fatalf("fail to init coscheduling: %s", err) - } - - cs.getOrCreatePodGroupInfo(tt.pod, time.Now()) - _, ok := cs.podGroupInfos.Load(fmt.Sprintf("%v/%v", tt.pod.Namespace, tt.podGroupName)) - if !ok { - t.Fatalf("fail to create PodGroup in coscheduling: %s", tt.pod.Name) - } - - cs.markPodGroupAsExpired(tt.pod) - pg, ok := cs.podGroupInfos.Load(fmt.Sprintf("%v/%v", tt.pod.Namespace, tt.podGroupName)) - if ok && pg.(*PodGroupInfo).deletionTimestamp == nil { - t.Fatalf("fail to clean up PodGroup : %s", tt.pod.Name) - } - - c.Step(time.Duration(*cs.args.PodGroupExpirationTimeSeconds)*time.Second + time.Second) - // Wait for asynchronous deletion. - err = wait.Poll(time.Millisecond*200, 1*time.Second, func() (bool, error) { - _, ok = cs.podGroupInfos.Load(fmt.Sprintf("%v/%v", tt.pod.Namespace, tt.podGroupName)) - return !ok, nil - }) - - if err != nil { - t.Fatalf("fail to gc PodGroup in coscheduling: %s", tt.pod.Name) - } - }) - } -} - -var _ framework.SharedLister = &fakeSharedLister{} - -type fakeSharedLister struct { - nodeInfos []*framework.NodeInfo - nodeInfoMap map[string]*framework.NodeInfo -} - -func newFakeSharedLister(pods []*v1.Pod, nodes []*v1.Node) *fakeSharedLister { - nodeInfoMap := createNodeInfoMap(pods, nodes) - nodeInfos := make([]*framework.NodeInfo, 0) - for _, v := range nodeInfoMap { - nodeInfos = append(nodeInfos, v) - } - return &fakeSharedLister{ - nodeInfos: nodeInfos, - nodeInfoMap: nodeInfoMap, - } -} - -func createNodeInfoMap(pods []*v1.Pod, nodes []*v1.Node) map[string]*framework.NodeInfo { - nodeNameToInfo := make(map[string]*framework.NodeInfo) - for _, pod := range pods { - nodeName := pod.Spec.NodeName - if _, ok := nodeNameToInfo[nodeName]; !ok { - nodeNameToInfo[nodeName] = framework.NewNodeInfo() - } - nodeNameToInfo[nodeName].AddPod(pod) - } - - for _, node := range nodes { - if _, ok := nodeNameToInfo[node.Name]; !ok { - nodeNameToInfo[node.Name] = framework.NewNodeInfo() - } - nodeInfo := nodeNameToInfo[node.Name] - nodeInfo.SetNode(node) - } - return nodeNameToInfo -} - -func (f *fakeSharedLister) NodeInfos() framework.NodeInfoLister { - return f -} - -func (f *fakeSharedLister) List() ([]*framework.NodeInfo, error) { - return f.nodeInfos, nil -} - -func (f *fakeSharedLister) HavePodsWithAffinityList() ([]*framework.NodeInfo, error) { - return nil, nil -} - -func (f *fakeSharedLister) Get(nodeName string) (*framework.NodeInfo, error) { - return f.nodeInfoMap[nodeName], nil -} diff --git a/pkg/coscheduling/scheduler/cache/cache.go b/pkg/coscheduling/scheduler/cache/cache.go new file mode 100644 index 0000000000..8f1f2d8e5a --- /dev/null +++ b/pkg/coscheduling/scheduler/cache/cache.go @@ -0,0 +1,114 @@ +/* +Copyright 2020 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 cache + +import ( + "sync" + + gochache "github.com/patrickmn/go-cache" + v1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/types" + + "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/apis/podgroup/v1alpha1" +) + +// Cache provides func Get, Set, Delete to operation cache +type Cache interface { + Get(string) Object + Set(string, Object) + Delete(string) +} + +// Object is A abstract interface +type Object interface { +} + +var _ Cache = &PGSchedulingCache{} + +var _ Object = &PodGroupSchedulingStatus{} + +// PGSchedulingCache provide safe function Get, Set, Delete. +type PGSchedulingCache struct { + sync.RWMutex + // NodeDiskStatus maps podGroupName to its DiskStatus + PGStatusMap map[string]*PodGroupSchedulingStatus +} + +// PodGroupSchedulingStatus helps record pod group status +type PodGroupSchedulingStatus struct { + PodGroup *v1alpha1.PodGroup + // store permitted pod node map with ttl + MatchedPodNodes *gochache.Cache + // store permitted pod name and uuid map with ttl + PodNameUIDs *gochache.Cache + // failed pods + Failed map[string]string + // succeed pods + Succeed map[string]string + Pod *v1.Pod + // Scheduled marks if has scheduled + Scheduled bool +} + +// PodNodePair is a pair contains pod name and node +type PodNodePair struct { + PodName string + Node string +} + +// NewPGSchedulingCache initializers pod status cache +func NewPGSchedulingCache() *PGSchedulingCache { + return &PGSchedulingCache{ + PGStatusMap: map[string]*PodGroupSchedulingStatus{}, + } +} + +// DeepCopy copies the pod node pair +func DeepCopy(pairs map[types.UID]*PodNodePair) map[types.UID]*PodNodePair { + newPair := make(map[types.UID]*PodNodePair) + for id, pair := range pairs { + pairObj := *pair + pairObjCopy := pairObj + newPair[id] = &pairObjCopy + } + return newPair +} + +// Get cache according to podGroupName +func (psc *PGSchedulingCache) Get(podGroupName string) Object { + psc.RLock() + defer psc.RUnlock() + pg, found := psc.PGStatusMap[podGroupName] + if found == false { + return nil + } + return pg +} + +//Set node status to the cache +func (psc *PGSchedulingCache) Set(podGroupName string, pg Object) { + psc.Lock() + psc.PGStatusMap[podGroupName] = pg.(*PodGroupSchedulingStatus) + psc.Unlock() +} + +// Delete Node status from cache +func (psc *PGSchedulingCache) Delete(podGroupName string) { + psc.Lock() + delete(psc.PGStatusMap, podGroupName) + psc.Unlock() +} diff --git a/pkg/coscheduling/scheduler/core/core.go b/pkg/coscheduling/scheduler/core/core.go new file mode 100644 index 0000000000..72ec6b3c01 --- /dev/null +++ b/pkg/coscheduling/scheduler/core/core.go @@ -0,0 +1,600 @@ +/* +Copyright 2020 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 core + +import ( + "context" + "fmt" + "sort" + "strings" + "sync" + "time" + + "github.com/gogo/protobuf/sortkeys" + gochache "github.com/patrickmn/go-cache" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/klog/v2" + podutil "k8s.io/kubernetes/pkg/api/v1/pod" + framework "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1" + + "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/apis/podgroup/v1alpha1" + pgclientset "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/generated/clientset/versioned" + pginformer "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/generated/informers/externalversions/podgroup/v1alpha1" + pglister "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/generated/listers/podgroup/v1alpha1" + "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/scheduler/cache" + errors "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/util" +) + +// ScheduleOperation defines the scheduling operation called +type ScheduleOperation struct { + pgClient *pgclientset.Clientset + frameworkHandler framework.FrameworkHandle + podGroupStatusCache cache.Cache + maxScheduleTime *time.Duration + lastDeniedPG *gochache.Cache + lastPermittedPod *gochache.Cache + pgLister pglister.PodGroupLister + maxFinishedPG string + maxSchedulingPGStatus *cache.PodGroupSchedulingStatus + sync.RWMutex +} + +// NewScheduleOperation create a new operation object +func NewScheduleOperation(pgClient *pgclientset.Clientset, cache cache.Cache, + frameworkHandler framework.FrameworkHandle, maxScheduleTime *time.Duration, pgInformer pginformer.PodGroupInformer) *ScheduleOperation { + so := &ScheduleOperation{ + pgClient: pgClient, + podGroupStatusCache: cache, + frameworkHandler: frameworkHandler, + maxScheduleTime: maxScheduleTime, + pgLister: pgInformer.Lister(), + lastDeniedPG: gochache.New(30*time.Second, 3*time.Second), + lastPermittedPod: gochache.New(3*time.Second, 3*time.Second), + } + return so +} + +// PodGroupStatusCache returns the podGroupStatusCache +func (sop *ScheduleOperation) PodGroupStatusCache() cache.Cache { + return sop.podGroupStatusCache +} + +// PreFilter pre-filter pod include: +// 1. cluster resource +// 2. last denied pod to increase schedule qps +// 3. check resource for max finished group, evict others if it can not run +// 4. check pod priority, if it can not run, evict other groups +func (sop *ScheduleOperation) PreFilter(ctx context.Context, pod *corev1.Pod) error { + klog.V(10).Infof("Pre-filter %v", pod.Name) + pgFullName, pgsObj, err := sop.getPodGroup(pod) + if err != nil { + return err + } + if pgsObj == nil { + return nil + } + _, ok := sop.lastDeniedPG.Get(pgFullName) + if ok { + err := fmt.Errorf("pod with pgName: %v last failed in 20s, deny", pgFullName) + klog.V(6).Info(err) + return err + } + + pgs := pgsObj.(*cache.PodGroupSchedulingStatus) + if err := sop.fillOccupiedObj(pgs, pod); err != nil { + return err + } + + // compute resource to check if sum of cluster resource can meet required resource + pgCache := sop.podGroupStatusCache.(*cache.PGSchedulingCache) + pgCache.RLock() + maxFinishedPG, maxSchedulingPGStatus := searchMaxPG(pgCache) + sop.maxFinishedPG = maxFinishedPG + sop.maxSchedulingPGStatus = maxSchedulingPGStatus + pgCache.RUnlock() + if maxFinishedPG == "" || maxSchedulingPGStatus == nil || maxSchedulingPGStatus.PodGroup == nil { + klog.V(5).Infof("Pod %v/%v has passed pre-filter check", pod.Namespace, pod.Name) + return nil + } + klog.Infof("max group %+v", maxSchedulingPGStatus.PodNameUIDs.Items()) + // First scheduled pod group, set it to the current + matched := len(maxSchedulingPGStatus.MatchedPodNodes.Items()) + if matched == 0 { + maxSchedulingPGStatus, maxFinishedPG = pgs, pgFullName + if !sop.compareClusterResourceAndRequire(maxSchedulingPGStatus.Pod, framework.NewResource(*pgs.PodGroup.Spec.MinResources), 1) { + errMsg := fmt.Sprintf("LeftResource can not satisfy PodGroupRequirement: %+v", *pgs.PodGroup.Spec.MinResources) + klog.V(10).Infof(errMsg) + sop.AddToDenyCache(pgFullName) + return fmt.Errorf(errMsg) + } + return nil + } + preAllocatedResource := getPreAllocatedResource(maxSchedulingPGStatus, matched) + if !sop.compareClusterResourceAndRequire(maxSchedulingPGStatus.Pod, &preAllocatedResource, 1) { + clusterResource := sop.computeClusterResource(maxSchedulingPGStatus.Pod) + evictList, shouldEvict := sop.simulateEvictGroup(clusterResource, preAllocatedResource.ResourceList(), maxFinishedPG) + if shouldEvict && len(evictList) > 0 { + sop.evictGroup(evictList) + sop.lastDeniedPG.Delete(pgFullName) + } + } + + // If the pod group is the max finished one, pass it + if sop.maxFinishedPG == pgFullName { + klog.V(5).Infof("Current pod group %v is the currently max one, pass it", pgFullName) + return nil + } + + // PodGroup is not the max finished one, we should check if the max one still can schedule when this group add + preAllocatedResourceClone := preAllocatedResource.Clone() + preAllocatedResource.Add(getPodResourceRequire(pod).ResourceList()) + klog.Infof("%+v", preAllocatedResource) + if !sop.compareClusterResourceAndRequire(maxSchedulingPGStatus.Pod, &preAllocatedResource, 0.95) { + errMsg := fmt.Sprintf("cluster resource not enough, preAllocated: %v, current requeired: %v", + preAllocatedResourceClone, *pgs.PodGroup.Spec.MinResources) + // Check priority + if podutil.GetPodPriority(pod) <= podutil.GetPodPriority(maxSchedulingPGStatus.Pod) { + sop.AddToDenyCache(pgFullName) + return fmt.Errorf(errMsg) + } + clusterResource := sop.computeClusterResource(pod) + evictList, _ := sop.simulateEvictGroup(clusterResource, *pgs.PodGroup.Spec.MinResources, pgFullName) + if len(evictList) > 0 { + sop.evictGroup(evictList) + sop.lastDeniedPG.Delete(pgFullName) + } + } + return nil +} + +// PreemptAddPod currently always return nil +func (sop *ScheduleOperation) PreemptAddPod(ctx context.Context, podToAdd *corev1.Pod, nodeName string) error { + return nil +} + +// PreemptRemovePod requires: +// 1. online resource could preempt online offline +// 2. offline could preempt low process and not running offline +// 3. online preempt offline +// 4. offline preempt offline +func (sop *ScheduleOperation) PreemptRemovePod(ctx context.Context, podToSchedule, podToRemove *corev1.Pod) error { + pgNameToRemove, offlinePodToRemove := errors.VerifyPodLabelSatisfied(podToRemove) + pgNameToSchedule, offlinePodToSchedule := errors.VerifyPodLabelSatisfied(podToSchedule) + + // 1. online resource could preempt online offline + // 2. offline could preempt low process and not running offline + + // online preempt online + if !offlinePodToSchedule && !offlinePodToRemove { + return nil + } + + // offline preempt online: forbid + if offlinePodToSchedule && !offlinePodToRemove { + return fmt.Errorf("offline pods %v are forbidden to preempt online %v", podToSchedule.Name, podToRemove.Name) + } + + checkPreemption := func() (string, error) { + pgFullNameToRemove := fmt.Sprintf("%v/%v", podToRemove.Namespace, pgNameToRemove) + pgsObj := sop.podGroupStatusCache.Get(pgFullNameToRemove) + if pgsObj == nil { + return "", fmt.Errorf("can not found pod group: %v", pgFullNameToRemove) + } + pgs := pgsObj.(*cache.PodGroupSchedulingStatus) + + if pgs.PodGroup.Status.Phase == v1alpha1.PodGroupScheduled || + pgs.PodGroup.Status.Phase == v1alpha1.PodGroupRunning || pgs.PodGroup.Status.Scheduled == pgs.PodGroup.Spec.MinMember { + return "", fmt.Errorf("pod %v belongs to Scheduled or Running pod group can not be scheduled", podToRemove.Name) + } + return pgFullNameToRemove, nil + } + + pgFullNameToRemove, err := checkPreemption() + + // online preempt offline + if !offlinePodToSchedule && offlinePodToRemove { + return err + } + + // offline preempt offline + pgFullNameToSchedule := fmt.Sprintf("%v/%v", podToSchedule.Namespace, pgNameToSchedule) + if pgFullNameToRemove == pgFullNameToSchedule { + return fmt.Errorf("podToSchedule and podToRemove belong to same pod group, do not preempt") + } + if err != nil { + return err + } + klog.V(5).Infof("Try to preempt pod %v/%v, schedule pod %v/%v", podToRemove.Namespace, podToRemove.Name, + podToSchedule.Namespace, podToSchedule.Name) + return nil +} + +// Permit permits a pod to run, if the minMember match, it would send a signal to chan. +func (sop *ScheduleOperation) Permit(ctx context.Context, pod *corev1.Pod, nodeName string) (bool, string, error) { + pgFullName, pgsObj, err := sop.getPodGroup(pod) + if pgFullName == "" { + return true, pgFullName, errors.ErrorNotMatched + } + if err != nil { + return false, pgFullName, err + } + pgs := pgsObj.(*cache.PodGroupSchedulingStatus) + if pgs.PodGroup.Status.Phase == "" || pgs.PodGroup.Status.Phase == v1alpha1.PodGroupPending { + pgs.PodGroup.Status.Phase = v1alpha1.PodGroupPreScheduling + } + klog.V(5).Infof("Desired group %v phase: %v", pgs.PodGroup.Name, pgs.PodGroup.Status.Phase) + + pair := cache.PodNodePair{ + PodName: fmt.Sprintf("%v/%v", pod.Namespace, pod.Name), + Node: nodeName, + } + + waitTime := errors.GetWaitTimeDuration(pgs.PodGroup, sop.maxScheduleTime) + pgs.MatchedPodNodes.Set(string(pod.UID), &pair, waitTime) + oldUID, found := pgs.PodNameUIDs.Get(fmt.Sprintf("%v/%v", pod.Namespace, pod.Name)) + // pod has been scheduled ever + if found { + // delete the expired one + pgs.MatchedPodNodes.Delete(oldUID.(string)) + } else { + // newly add one + klog.V(4).Infof("Add new cache data %v", pod.UID) + } + pgs.PodNameUIDs.Set(fmt.Sprintf("%v/%v", pod.Namespace, pod.Name), string(pod.UID), waitTime) + + klog.V(5).Infof("Current pod group state: %+v", pgs.PodGroup) + ready := uint32(len(pgs.MatchedPodNodes.Items())) >= pgs.PodGroup.Spec.MinMember-pgs.PodGroup.Status.Scheduled + if ready { + pgs.Scheduled = true + return true, pgFullName, nil + } + return false, pgFullName, errors.ErrorWaiting +} + +// PostBind is used for send metrics and update to api if all finish +func (sop *ScheduleOperation) PostBind(ctx context.Context, pod *corev1.Pod, nodeName string) { + sop.Lock() + defer sop.Unlock() + pgFullName, pgsObj, _ := sop.getPodGroup(pod) + if pgFullName == "" || pgsObj == nil { + return + } + pgs := pgsObj.(*cache.PodGroupSchedulingStatus) + pgCopy := pgs.PodGroup.DeepCopy() + pgCopy.Status.Scheduled++ + + if pgCopy.Status.Scheduled >= pgs.PodGroup.Spec.MinMember { + pgCopy.Status.Phase = v1alpha1.PodGroupScheduled + } else { + pgCopy.Status.Phase = v1alpha1.PodGroupScheduling + if pgCopy.Status.ScheduleStartTime.IsZero() { + pgCopy.Status.ScheduleStartTime = metav1.Time{Time: time.Now()} + } + } + pgs.PodGroup.Status = pgCopy.Status + if pgCopy.Status.Phase != pgs.PodGroup.Status.Phase { + + pg, err := sop.pgLister.PodGroups(pgCopy.Namespace).Get(pgCopy.Name) + if err != nil { + klog.Error(err) + return + } + pg.Status = pgCopy.Status + pg, err = sop.pgClient.SchedulingV1alpha1().PodGroups(pg.Namespace).UpdateStatus(ctx, pgCopy, metav1.UpdateOptions{}) + if err != nil { + klog.Error(err) + } + + } + pgs.PodGroup.Status.Phase = pgCopy.Status.Phase + pgs.PodGroup.Status.Scheduled = pgCopy.Status.Scheduled + return +} + +// Compare returns the true or false, base on +// 1. priority +// 2. podGroup creation time +// 3. pod creation time +func (sop *ScheduleOperation) Less(podInfo1, podInfo2 interface{}) bool { + // Sort pod also record to progress + pInfo1 := podInfo1.(*framework.QueuedPodInfo) + pInfo2 := podInfo2.(*framework.QueuedPodInfo) + prio1 := podutil.GetPodPriority(pInfo1.Pod) + prio2 := podutil.GetPodPriority(pInfo2.Pod) + pgName1, _ := errors.VerifyPodLabelSatisfied(pInfo1.Pod) + pgName2, _ := errors.VerifyPodLabelSatisfied(pInfo2.Pod) + compare := func() bool { + if prio1 != prio2 { + return prio1 > prio2 + } + if pgName1 == "" && pgName2 == "" { + return pInfo1.Pod.CreationTimestamp.Before(&pInfo2.Pod.CreationTimestamp) + } + + if pgName1 == "" { + return true + } + if pgName2 == "" { + return false + } + pg1, err1 := sop.pgLister.PodGroups(pInfo1.Pod.Namespace).Get(pgName1) + pg2, err2 := sop.pgLister.PodGroups(pInfo2.Pod.Namespace).Get(pgName2) + if err1 != nil || err2 != nil { + return false + } + + if !pg1.CreationTimestamp.Equal(&pg2.CreationTimestamp) { + return pg1.CreationTimestamp.Before(&pg2.CreationTimestamp) + } + if !pInfo1.Pod.CreationTimestamp.Equal(&pInfo2.Pod.CreationTimestamp) { + return pInfo1.Pod.CreationTimestamp.Before(&pInfo2.Pod.CreationTimestamp) + } + return pInfo1.Pod.Name < pInfo2.Pod.Name + } + r := compare() + klog.V(7).Infof("Pod1 %v Pod2 %v, result:%v", pInfo1.Pod.Name, pInfo2.Pod.Name, r) + return r +} + +// GetPodNodePairs returns the pod-node paired to be scheduled. +func (sop *ScheduleOperation) GetPodNodePairs(pgFullName string) *gochache.Cache { + pgs := sop.podGroupStatusCache.Get(pgFullName) + if pgs == nil { + return nil + } + return pgs.(*cache.PodGroupSchedulingStatus).MatchedPodNodes +} + +// AddToDenyCache add podGroup to the back list cache +func (sop *ScheduleOperation) AddToDenyCache(pgFullName string) { + sop.lastDeniedPG.Add(pgFullName, "", 20*time.Second) +} + +// GetPodNameUIDs returns podName and uid pairs. +func (sop *ScheduleOperation) GetPodNameUIDs(pgFullName string) *gochache.Cache { + pgs := sop.podGroupStatusCache.Get(pgFullName) + if pgs == nil { + return nil + } + return pgs.(*cache.PodGroupSchedulingStatus).PodNameUIDs +} + +// getPodGroup get PodGroup from cache +func (sop *ScheduleOperation) getPodGroup(pod *corev1.Pod) (string, cache.Object, error) { + pgName, satisfied := errors.VerifyPodLabelSatisfied(pod) + if pgName == "" && !satisfied { + return "", nil, nil + } + pgFullName := fmt.Sprintf("%v/%v", pod.Namespace, pgName) + pgsObj := sop.podGroupStatusCache.Get(pgFullName) + if pgsObj == nil { + return pgFullName, pgsObj, fmt.Errorf("can not found pod group: %v", pgFullName) + } + return pgFullName, pgsObj, nil +} + +// simulateEvictGroup simulate eviction pod groups. We try to evict groups as fewer as possible. +func (sop *ScheduleOperation) simulateEvictGroup(clusterLeft *framework.Resource, podGroupRequired corev1.ResourceList, pgFullName string) ([]ResourcePair, bool) { + // TODO(cwdsuzhou): cluster Fit again here, pod in max finished PodGroup may be conflict with the current pod + sum := &framework.Resource{ScalarResources: map[corev1.ResourceName]int64{}} + sumInfo := make(map[string]*framework.Resource, 0) + pgCache := sop.podGroupStatusCache.(*cache.PGSchedulingCache) + pgCache.Lock() + defer pgCache.Unlock() + for pgName, pgs := range pgCache.PGStatusMap { + if pgName == pgFullName { + continue + } + r := &framework.Resource{ScalarResources: map[corev1.ResourceName]int64{}} + matched := len(pgs.MatchedPodNodes.Items()) + if uint32(matched) == pgs.PodGroup.Spec.MinMember { + continue + } + if pgs.Pod == nil { + continue + } + rl := getPodResourceRequire(pgs.Pod).ResourceList() + if rl == nil { + continue + } + for i := 0; i < matched; i++ { + r.Add(rl) + } + sum.Add(r.ResourceList()) + sumInfo[pgName] = r + } + if len(sumInfo) == 0 { + return nil, false + } + + sum.Add(clusterLeft.ResourceList()) + if !checkLeftResourceSatisfy(framework.NewResource(sum.ResourceList()), framework.NewResource(podGroupRequired)) { + return nil, false + } + simuList := make([]ResourcePair, 0) + list := sortMapByValue(sumInfo) + for _, resourcePair := range list { + if resourcePair.resource == nil { + continue + } + clusterLeft.Add(resourcePair.resource.ResourceList()) + simuList = append(simuList, resourcePair) + if checkLeftResourceSatisfy(clusterLeft, framework.NewResource(podGroupRequired)) { + return simuList, true + } + } + return nil, false +} + +// evictGroup do eviction of pod group +func (sop *ScheduleOperation) evictGroup(evictList []ResourcePair) { + for _, rp := range evictList { + klog.Infof("Evict group %v", rp.pgName) + pgsObj := sop.podGroupStatusCache.Get(rp.pgName) + pgs := pgsObj.(*cache.PodGroupSchedulingStatus) + pgs.MatchedPodNodes.Delete(string(pgs.Pod.UID)) + pgs.PodNameUIDs.Delete(fmt.Sprintf("%v/%v", pgs.Pod.Namespace, pgs.Pod.Name)) + sop.AddToDenyCache(rp.pgName) + } +} + +// fillOccupiedObj fill some infomation to pod group cache +func (sop *ScheduleOperation) fillOccupiedObj(pgs *cache.PodGroupSchedulingStatus, pod *corev1.Pod) error { + if pgs == nil || pgs.PodGroup == nil { + return fmt.Errorf("PodGroupSchedulingStatus is nil") + } + pgFullName := fmt.Sprintf("%v/%v", pod.Namespace, pgs.PodGroup.Name) + refs := make([]string, 0) + for _, ownerRef := range pod.OwnerReferences { + refs = append(refs, string(ownerRef.UID)) + } + if pgs.Pod == nil { + pgs.Pod = pod + } + if pgs.PodGroup.Spec.MinResources == nil { + // initialize pod group resource + req := getPodGroupMinResourceFromPod(pod, int(pgs.PodGroup.Spec.MinMember)).ResourceList() + pgs.PodGroup.Spec.MinResources = &req + } + if pgs.PodGroup.Status.OccupiedBy == "" { + // if do not have refs or initialize, return nil + if len(refs) != 0 { + // initialize pod group + sortkeys.Strings(refs) + pgs.PodGroup.Status.OccupiedBy = strings.Join(refs, ",") + } + return nil + } + // if refs not match, return error + if len(refs) == 0 { + return fmt.Errorf("pod group %s has been occupied by %v", pgFullName, pgs.PodGroup.Status.OccupiedBy) + } + sortkeys.Strings(refs) + if strings.Join(refs, ",") != pgs.PodGroup.Status.OccupiedBy { + return fmt.Errorf("pod group has been occupied by %v", pgs.PodGroup.Status.OccupiedBy) + } + return nil +} + +// computeClusterResource summary the resource for a pod. We also check affinity and nodeSelector. +func (sop *ScheduleOperation) computeClusterResource(pod *corev1.Pod) *framework.Resource { + snapShotList, err := sop.frameworkHandler.SnapshotSharedLister().NodeInfos().List() + if err != nil || snapShotList == nil { + return nil + } + klog.V(6).Infof("NodeInfoSnapshot: %+v", snapShotList) + var leftResources framework.Resource + for k, info := range snapShotList { + if info == nil { + continue + } + + if info.Node() == nil { + klog.Warningf("Node: %v can not found", k) + continue + } + + if info.Node().Spec.Unschedulable { + continue + } + + leftResource := singleNodeResource(info, pod, 1) + + leftResources.Add(leftResource.ResourceList()) + + } + return &leftResources +} + +// compareClusterResourceAndRequire check if the cluster resource satisfy the requirements. +// percent is the optional parameter to reserve resources when check. +func (sop *ScheduleOperation) compareClusterResourceAndRequire(pod *corev1.Pod, + reqResource *framework.Resource, percent float32) bool { + snapShotList, err := sop.frameworkHandler.SnapshotSharedLister().NodeInfos().List() + if err != nil || snapShotList == nil { + return false + } + klog.V(10).Infof("NodeInfoSnapshot %+v", snapShotList) + var leftResources framework.Resource + count := 0 + for k, info := range snapShotList { + count++ + if info == nil { + continue + } + + if info.Node() == nil { + klog.Warningf("Node: %v can not found", k) + continue + } + + if info.Node().Spec.Unschedulable { + continue + } + + leftResources.Add(singleNodeResource(info, pod, percent).ResourceList()) + + if checkLeftResourceSatisfy(&leftResources, reqResource) { + klog.V(6).Infof("Current left %v nodes resources %+v \nRequired resource %+v", count, + leftResources, *reqResource) + return true + } + klog.V(6).Infof("Current left %v nodes resources %+v \nRequired resource %+v", count, + leftResources, *reqResource) + } + return false +} + +// ResourcePair is the pair of pogGroup and allocated resource +type ResourcePair struct { + pgName string + resource *framework.Resource +} + +// ResourcePairList is the list of ResourcePair +type ResourcePairList []ResourcePair + +// Swap exchange resource pair +func (p ResourcePairList) Swap(i, j int) { p[i], p[j] = p[j], p[i] } + +// Len return the len of the list +func (p ResourcePairList) Len() int { return len(p) } + +// Less compare resources +func (p ResourcePairList) Less(i, j int) bool { + if p[i].resource == nil { + return true + } + if p[j].resource == nil { + return false + } + if p[i].resource.MilliCPU == p[j].resource.MilliCPU { + return p[i].resource.Memory < p[j].resource.Memory + } + return p[i].resource.MilliCPU < p[j].resource.MilliCPU +} + +func sortMapByValue(m map[string]*framework.Resource) ResourcePairList { + p := make(ResourcePairList, len(m)) + i := 0 + for k, v := range m { + p[i] = ResourcePair{k, v} + } + sort.Sort(p) + return p +} diff --git a/pkg/coscheduling/scheduler/core/core_test.go b/pkg/coscheduling/scheduler/core/core_test.go new file mode 100644 index 0000000000..313feb9d76 --- /dev/null +++ b/pkg/coscheduling/scheduler/core/core_test.go @@ -0,0 +1,474 @@ +/* +Copyright 2020 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 core + +import ( + "context" + "fmt" + "strconv" + "testing" + "time" + + gochache "github.com/patrickmn/go-cache" + corev1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/resource" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + fwkruntime "k8s.io/kubernetes/pkg/scheduler/framework/runtime" + framework "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1" + st "k8s.io/kubernetes/pkg/scheduler/testing" + + "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/apis/podgroup/v1alpha1" + "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/generated/clientset/versioned/fake" + pgformers "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/generated/informers/externalversions" + pglister "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/generated/listers/podgroup/v1alpha1" + batchcache "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/scheduler/cache" + "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/util" +) + +type resourceFunc func(nodes []*corev1.Node, coreNum int) + +func TestPreFilter(t *testing.T) { + + c := batchcache.NewPGSchedulingCache() + c.Set("pg1", &batchcache.PodGroupSchedulingStatus{}) + c1 := batchcache.NewPGSchedulingCache() + c1.Set("ns1/pg1", &batchcache.PodGroupSchedulingStatus{ + PodGroup: pg(2), + MatchedPodNodes: newCache(), + PodNameUIDs: newCache()}, + ) + + c2 := batchcache.NewPGSchedulingCache() + matchedTerms := newCache() + matchedTerms.SetDefault("p2", "node2") + c2.Set("ns1/pg1", &batchcache.PodGroupSchedulingStatus{ + PodGroup: pg(2), + MatchedPodNodes: matchedTerms, + PodNameUIDs: newCache(), + Pod: &st.MakePod().Name("p").UID("p1").Pod, + }) + + denyCache := newCache() + denyCache.SetDefault("pg1", "pg1") + + tests := []struct { + name string + pod *corev1.Pod + cache batchcache.Cache + lastDeniedPG *gochache.Cache + expectedSuccess bool + rf resourceFunc + }{ + { + name: "pod does not belong to any pg", + pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Obj(), + lastDeniedPG: newCache(), + expectedSuccess: true, + }, + { + name: "pod belongs to pg, pg in blacklist", + pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Label(util.PodGroupLabel, "pg1").Obj(), + cache: c, + lastDeniedPG: denyCache, + expectedSuccess: false, + }, + { + name: "pod belongs to pg, pg nil", + pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Label(util.PodGroupLabel, "pg1").Obj(), + cache: c, + lastDeniedPG: newCache(), + expectedSuccess: false, + }, + { + name: "pod belongs to pg, pg not nil, resource not enough", + pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Label(util.PodGroupLabel, "pg1").Obj(), + cache: c1, + lastDeniedPG: newCache(), + expectedSuccess: false, + }, + { + name: "pod belongs to pg, pg not nil, resource enough, the first group", + pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Label(util.PodGroupLabel, "pg1").Obj(), + cache: c1, + lastDeniedPG: newCache(), + expectedSuccess: true, + rf: resources, + }, + { + name: "pod belongs to pg, pg not nil, resource enough, another group", + pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Label(util.PodGroupLabel, "pg1").Obj(), + cache: c2, + lastDeniedPG: newCache(), + expectedSuccess: true, + rf: resources, + }, + } + for _, tt := range tests { + existingPods, allNodes := makeNodesAndPods(map[string]string{"test": "a"}, 60, 30, tt.rf) + snapshot := NewSnapshot(existingPods, allNodes) + fh, _ := fwkruntime.NewFramework(nil, nil, nil, fwkruntime.WithSnapshotSharedLister(snapshot)) + + so := &ScheduleOperation{podGroupStatusCache: tt.cache, lastDeniedPG: tt.lastDeniedPG, frameworkHandler: fh} + err := so.PreFilter(context.TODO(), tt.pod) + if err == nil { + if tt.expectedSuccess == false { + t.Fatalf("desire %v, err is %v", tt.expectedSuccess, err) + } + return + } + if tt.expectedSuccess == true { + t.Fatalf("desire %v, err is %v", tt.expectedSuccess, err) + } + } +} + +func TestPrePermit(t *testing.T) { + + c := batchcache.NewPGSchedulingCache() + c.Set("pg1", &batchcache.PodGroupSchedulingStatus{}) + + c1 := batchcache.NewPGSchedulingCache() + matchedTerms := newCache() + matchedTerms.SetDefault("p2", "node2") + td := time.Duration(10 * time.Second) + c1.Set("ns1/pg1", &batchcache.PodGroupSchedulingStatus{ + PodGroup: pg(3), + MatchedPodNodes: matchedTerms, + PodNameUIDs: newCache(), + Pod: &st.MakePod().Name("p").UID("p1").Pod, + }) + + c2 := batchcache.NewPGSchedulingCache() + c2.Set("ns1/pg1", &batchcache.PodGroupSchedulingStatus{ + PodGroup: pg(2), + MatchedPodNodes: matchedTerms, + PodNameUIDs: newCache(), + Pod: &st.MakePod().Name("p").UID("p1").Pod, + }) + + tests := []struct { + name string + pod *corev1.Pod + cache batchcache.Cache + allow bool + }{ + { + name: "pod does not belong to any pg, allow", + pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Obj(), + allow: true, + }, + { + name: "pod belongs to pg, pg nil", + pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Label(util.PodGroupLabel, "pg1").Obj(), + cache: c, + allow: false, + }, + { + name: "pod belongs to pg, not enough pod", + pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Label(util.PodGroupLabel, "pg1").Obj(), + cache: c1, + allow: false, + }, + { + name: "pod belongs to pg, enough pod", + pod: st.MakePod().Name("p").UID("p").Namespace("ns1").Label(util.PodGroupLabel, "pg1").Obj(), + cache: c2, + allow: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + so := &ScheduleOperation{podGroupStatusCache: tt.cache, maxScheduleTime: &td} + allow, _, err := so.Permit(context.TODO(), tt.pod, "test") + if allow != tt.allow { + t.Fatal(tt.name, err) + } + }) + } +} + +func TestCompare(t *testing.T) { + + ti := time.Now() + p1 := corev1.Pod{ + ObjectMeta: v1.ObjectMeta{Namespace: "ns1", CreationTimestamp: v1.Time{Time: ti}}, + } + + p4 := p1.DeepCopy() + + p2 := corev1.Pod{ + ObjectMeta: v1.ObjectMeta{Namespace: "ns1", CreationTimestamp: v1.Time{Time: ti.Add(10 * time.Second)}}, + } + p3 := corev1.Pod{ + ObjectMeta: v1.ObjectMeta{Namespace: "ns1", CreationTimestamp: v1.Time{Time: ti}}, + } + + pg1 := pgWithName("pg1", 1) + pg1.CreationTimestamp = v1.Time{Time: ti} + pg4 := pg1.DeepCopy() + pg4.Name = "pg4" + + pg3 := pgWithName("pg3", 1) + pg3.CreationTimestamp = v1.Time{Time: ti.Add(10 * time.Second)} + + cs := fake.NewSimpleClientset(pg1, pg3, pg4) + scheduleInformer := pgformers.NewSharedInformerFactory(cs, 0) + pgInformer := scheduleInformer.Scheduling().V1alpha1().PodGroups() + pgInformer.Informer().GetStore().Add(pg1) + pgInformer.Informer().GetStore().Add(pg3) + pgInformer.Informer().GetStore().Add(pg4) + pgLister := pgInformer.Lister() + + tests := []struct { + name string + pod1 *corev1.Pod + pod2 *corev1.Pod + pgLister pglister.PodGroupLister + expected bool + }{ + { + name: "p1 < p2", + pod1: &st.MakePod().Priority(1).Pod, + pod2: &st.MakePod().Priority(2).Pod, + expected: false, + }, + { + name: "p1 > p2", + pod1: &st.MakePod().Priority(3).Pod, + pod2: &st.MakePod().Priority(2).Pod, + expected: true, + }, + { + name: "p1 = p2, time1 < time2", + pod1: podWithCreateTime(&makePod(p1).Priority(1).Pod, time.Now().Add(10*time.Second)), + pod2: podWithCreateTime(&makePod(p2).Priority(1).Pod, time.Now().Add(11*time.Second)), + expected: true, + }, + { + name: "p1 = p2, time1 = time2, pg1 nil", + pod1: &makePod(p1).Priority(1).Pod, + pod2: &makePod(p3).Priority(1).Label(util.PodGroupLabel, "pg2").Pod, + expected: true, + }, + { + name: "p1 = p2, time1 = time2, pg2 nil", + pod1: &makePod(p1).Priority(1).Label(util.PodGroupLabel, "pg1").Pod, + pod2: &makePod(p3).Priority(1).Pod, + expected: false, + }, + { + name: "p1 = p2, time1 = time2, pg11 pg33 not found", + pod1: &makePod(p1).Priority(1).Label(util.PodGroupLabel, "pg11").Pod, + pod2: &makePod(p3).Priority(1).Label(util.PodGroupLabel, "pg33").Pod, + pgLister: pgLister, + expected: false, + }, + { + name: "p1 = p2, time1 = time2, pg found, pg1< pg3", + pod1: &makePod(p1).Priority(1).Label(util.PodGroupLabel, "pg1").Pod, + pod2: &makePod(p3).Priority(1).Label(util.PodGroupLabel, "pg3").Pod, + pgLister: pgLister, + expected: true, + }, + { + name: "p1 = p2, time1 = time2, pg found, pg1 > pg3", + pod2: &makePod(p1).Priority(1).Label(util.PodGroupLabel, "pg1").Pod, + pod1: &makePod(p3).Priority(1).Label(util.PodGroupLabel, "pg3").Pod, + pgLister: pgLister, + expected: false, + }, + { + name: "p1 = p2, time1 = time2, pg found, pg1 = pg4, compare name", + pod1: &makePod(p1).Priority(1).Label(util.PodGroupLabel, "pg1").Pod, + pod2: &makePod(*p4).Priority(1).Label(util.PodGroupLabel, "pg4").Pod, + pgLister: pgLister, + expected: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + so := &ScheduleOperation{pgLister: tt.pgLister} + if tt.expected != so.Less(&framework.QueuedPodInfo{Pod: tt.pod1, Timestamp: tt.pod1.CreationTimestamp.Time}, + &framework.QueuedPodInfo{Pod: tt.pod2, Timestamp: tt.pod2.CreationTimestamp.Time}) { + t.Log(tt.pod1) + t.Log(tt.pod2) + t.Fatalf("Expect %v, get not", tt.expected) + } + }) + } +} + +func TestSingleNodeResource(t *testing.T) { + pod := corev1.Pod{ + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Resources: corev1.ResourceRequirements{ + Limits: corev1.ResourceList{ + corev1.ResourceCPU: resource.MustParse("1"), + corev1.ResourceName("alpha.kubernetes.io/nvidia-gpu"): resource.MustParse("1"), + corev1.ResourceName("test.com/op"): resource.MustParse("1"), + }, + Requests: corev1.ResourceList{ + corev1.ResourceCPU: resource.MustParse("1"), + corev1.ResourceName("alpha.kubernetes.io/nvidia-gpu"): resource.MustParse("1"), + + corev1.ResourceName("test.com/op"): resource.MustParse("1"), + }, + }, + }, + }, + }, + } + + node := corev1.Node{ + Spec: corev1.NodeSpec{}, + Status: corev1.NodeStatus{ + Capacity: corev1.ResourceList{ + corev1.ResourceCPU: resource.MustParse("10"), + corev1.ResourceName("alpha.kubernetes.io/nvidia-gpu"): resource.MustParse("10"), + + corev1.ResourcePods: resource.MustParse("100"), + corev1.ResourceName("test.com/op"): resource.MustParse("20"), + }, + Allocatable: corev1.ResourceList{ + corev1.ResourceCPU: resource.MustParse("10"), + corev1.ResourceName("alpha.kubernetes.io/nvidia-gpu"): resource.MustParse("10"), + + corev1.ResourcePods: resource.MustParse("100"), + corev1.ResourceName("test.com/op"): resource.MustParse("20"), + }, + }, + } + + nodeIf := framework.NewNodeInfo() + nodeIf.SetNode(&node) + nodeIf.AddPod(&pod) + podCopy := pod.DeepCopy() + podCopy.Spec.Containers[0].Resources.Requests[corev1.ResourceName("alpha.kubernetes.io/nvidia-gpu")] = resource.MustParse("101") + podCopy.Spec.Containers[0].Resources.Limits[corev1.ResourceName("alpha.kubernetes.io/nvidia-gpu")] = resource.MustParse("101") + podCopy1 := pod.DeepCopy() + + podCopy1.Spec.Containers[0].Resources.Requests[corev1.ResourceName("test.com/op")] = resource.MustParse( + "101") + podCopy1.Spec.Containers[0].Resources.Limits[corev1.ResourceName("test.com/op")] = resource.MustParse( + "101") + cases := []struct { + name string + node *framework.NodeInfo + pod *corev1.Pod + desire bool + }{ + { + node: nodeIf, + pod: &pod, + desire: true, + }, + { + node: nodeIf, + pod: podCopy, + desire: false, + }, + { + node: nodeIf, + pod: podCopy1, + desire: false, + }, + } + + for _, c := range cases { + re := singleNodeResource(c.node, c.pod, 1) + req := getPodResourceRequire(c.pod) + if checkLeftResourceSatisfy(re, req) != c.desire { + t.Logf("%+v", c.node.Allocatable) + t.Logf("%+v", c.node.Requested) + t.Error("not desire") + } + } +} + +func newCache() *gochache.Cache { + return gochache.New(10*time.Second, 10*time.Second) +} + +func resources(nodes []*corev1.Node, coreNum int) { + rl := corev1.ResourceList{} + rl[corev1.ResourceCPU] = resource.MustParse(strconv.Itoa(coreNum)) + rl[corev1.ResourceMemory] = resource.MustParse("1000Gi") + rl[corev1.ResourcePods] = resource.MustParse("110") + for _, node := range nodes { + node.Status.Capacity = rl + node.Status.Allocatable = rl + } +} + +func makeNodesAndPods(labels map[string]string, existingPodsNum, allNodesNum int, + funcs ...resourceFunc) (existingPods []*corev1.Pod, + allNodes []*corev1.Node) { + type keyVal struct { + k string + v string + } + var labelPairs []keyVal + for k, v := range labels { + labelPairs = append(labelPairs, keyVal{k: k, v: v}) + } + // build nodes + for i := 0; i < allNodesNum; i++ { + node := st.MakeNode().Name(fmt.Sprintf("node%d", i)) + allNodes = append(allNodes, &node.Node) + } + // build pods + for i := 0; i < existingPodsNum; i++ { + podWrapper := st.MakePod().Name(fmt.Sprintf("pod%d", i)).Node(fmt.Sprintf("node%d", i%allNodesNum)) + // apply labels[0], labels[0,1], ..., labels[all] to each pod in turn + for _, p := range labelPairs[:i%len(labelPairs)+1] { + podWrapper = podWrapper.Label(p.k, p.v) + } + existingPods = append(existingPods, podWrapper.Obj()) + } + if funcs != nil { + for _, f := range funcs { + if f == nil { + continue + } + f(allNodes, 100) + } + } + return +} + +func pg(min uint32) *v1alpha1.PodGroup { + return pgWithName("pg1", min) +} + +func pgWithName(name string, min uint32) *v1alpha1.PodGroup { + ti := v1.Duration{Duration: 10 * time.Second} + return &v1alpha1.PodGroup{ + ObjectMeta: v1.ObjectMeta{Name: name, Namespace: "ns1"}, + Spec: v1alpha1.PodGroupSpec{MinMember: min, MaxScheduleTime: &ti}, + } +} + +func makePod(pod corev1.Pod) *st.PodWrapper { + return &st.PodWrapper{Pod: pod} +} + +func podWithCreateTime(p *corev1.Pod, t time.Time) *corev1.Pod { + p.SetCreationTimestamp(v1.Time{Time: t}) + return p +} diff --git a/pkg/coscheduling/scheduler/core/fakeSnapShot.go b/pkg/coscheduling/scheduler/core/fakeSnapShot.go new file mode 100644 index 0000000000..bfe602623b --- /dev/null +++ b/pkg/coscheduling/scheduler/core/fakeSnapShot.go @@ -0,0 +1,174 @@ +/* +Copyright 2020 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 core + +import ( + "fmt" + + v1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/labels" + "k8s.io/apimachinery/pkg/util/sets" + fwk "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1" +) + +// Snapshot is a snapshot of cache NodeInfo and NodeTree order. The scheduler takes a +// snapshot at the beginning of each scheduling cycle and uses it for its operations in that cycle. +type Snapshot struct { + // nodeInfoMap a map of node name to a snapshot of its NodeInfo. + nodeInfoMap map[string]*fwk.NodeInfo + // nodeInfoList is the list of nodes as ordered in the cache's nodeTree. + nodeInfoList []*fwk.NodeInfo + // havePodsWithAffinityNodeInfoList is the list of nodes with at least one pod declaring affinity terms. + havePodsWithAffinityNodeInfoList []*fwk.NodeInfo + generation int64 +} + +var _ fwk.SharedLister = &Snapshot{} + +// NewEmptySnapshot initializes a Snapshot struct and returns it. +func NewEmptySnapshot() *Snapshot { + return &Snapshot{ + nodeInfoMap: make(map[string]*fwk.NodeInfo), + } +} + +// NewSnapshot initializes a Snapshot struct and returns it. +func NewSnapshot(pods []*v1.Pod, nodes []*v1.Node) *Snapshot { + nodeInfoMap := createNodeInfoMap(pods, nodes) + nodeInfoList := make([]*fwk.NodeInfo, 0, len(nodeInfoMap)) + havePodsWithAffinityNodeInfoList := make([]*fwk.NodeInfo, 0, len(nodeInfoMap)) + for _, v := range nodeInfoMap { + nodeInfoList = append(nodeInfoList, v) + if len(v.PodsWithAffinity) > 0 { + havePodsWithAffinityNodeInfoList = append(havePodsWithAffinityNodeInfoList, v) + } + } + + s := NewEmptySnapshot() + s.nodeInfoMap = nodeInfoMap + s.nodeInfoList = nodeInfoList + s.havePodsWithAffinityNodeInfoList = havePodsWithAffinityNodeInfoList + + return s +} + +// createNodeInfoMap obtains a list of pods and pivots that list into a map +// where the keys are node names and the values are the aggregated information +// for that node. +func createNodeInfoMap(pods []*v1.Pod, nodes []*v1.Node) map[string]*fwk.NodeInfo { + nodeNameToInfo := make(map[string]*fwk.NodeInfo) + for _, pod := range pods { + nodeName := pod.Spec.NodeName + if _, ok := nodeNameToInfo[nodeName]; !ok { + nodeNameToInfo[nodeName] = fwk.NewNodeInfo() + } + nodeNameToInfo[nodeName].AddPod(pod) + } + imageExistenceMap := createImageExistenceMap(nodes) + + for _, node := range nodes { + if _, ok := nodeNameToInfo[node.Name]; !ok { + nodeNameToInfo[node.Name] = fwk.NewNodeInfo() + } + nodeInfo := nodeNameToInfo[node.Name] + nodeInfo.SetNode(node) + nodeInfo.ImageStates = getNodeImageStates(node, imageExistenceMap) + } + return nodeNameToInfo +} + +// getNodeImageStates returns the given node's image states based on the given imageExistence map. +func getNodeImageStates(node *v1.Node, imageExistenceMap map[string]sets.String) map[string]*fwk.ImageStateSummary { + imageStates := make(map[string]*fwk.ImageStateSummary) + + for _, image := range node.Status.Images { + for _, name := range image.Names { + imageStates[name] = &fwk.ImageStateSummary{ + Size: image.SizeBytes, + NumNodes: len(imageExistenceMap[name]), + } + } + } + return imageStates +} + +// createImageExistenceMap returns a map recording on which nodes the images exist, keyed by the images' names. +func createImageExistenceMap(nodes []*v1.Node) map[string]sets.String { + imageExistenceMap := make(map[string]sets.String) + for _, node := range nodes { + for _, image := range node.Status.Images { + for _, name := range image.Names { + if _, ok := imageExistenceMap[name]; !ok { + imageExistenceMap[name] = sets.NewString(node.Name) + } else { + imageExistenceMap[name].Insert(node.Name) + } + } + } + } + return imageExistenceMap +} + +// NodeInfos returns a NodeInfoLister. +func (s *Snapshot) NodeInfos() fwk.NodeInfoLister { + return s +} + +// NumNodes returns the number of nodes in the snapshot. +func (s *Snapshot) NumNodes() int { + return len(s.nodeInfoList) +} + +type podLister []*fwk.NodeInfo + +// List returns the list of pods in the snapshot. +func (p podLister) List(selector labels.Selector) ([]*v1.Pod, error) { + // podFilter is expected to return true for most or all of the pods. We + // can avoid expensive array growth without wasting too much memory by + // pre-allocating capacity. + maxSize := 0 + for _, n := range p { + maxSize += len(n.Pods) + } + pods := make([]*v1.Pod, 0, maxSize) + for _, n := range p { + for _, pod := range n.Pods { + if selector.Matches(labels.Set(pod.Pod.Labels)) { + pods = append(pods, pod.Pod) + } + } + } + return pods, nil +} + +// List returns the list of nodes in the snapshot. +func (s *Snapshot) List() ([]*fwk.NodeInfo, error) { + return s.nodeInfoList, nil +} + +// HavePodsWithAffinityList returns the list of nodes with at least one pods with inter-pod affinity +func (s *Snapshot) HavePodsWithAffinityList() ([]*fwk.NodeInfo, error) { + return s.havePodsWithAffinityNodeInfoList, nil +} + +// Get returns the NodeInfo of the given node name. +func (s *Snapshot) Get(nodeName string) (*fwk.NodeInfo, error) { + if v, ok := s.nodeInfoMap[nodeName]; ok && v.Node() != nil { + return v, nil + } + return nil, fmt.Errorf("nodeinfo not found for node name %q", nodeName) +} diff --git a/pkg/coscheduling/scheduler/core/helpers.go b/pkg/coscheduling/scheduler/core/helpers.go new file mode 100644 index 0000000000..d03817fb4d --- /dev/null +++ b/pkg/coscheduling/scheduler/core/helpers.go @@ -0,0 +1,197 @@ +/* +Copyright 2020 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 core + +import ( + corev1 "k8s.io/api/core/v1" + "k8s.io/klog/v2" + podutil "k8s.io/kubernetes/pkg/api/v1/pod" + v1helper "k8s.io/kubernetes/pkg/apis/core/v1/helper" + "k8s.io/kubernetes/pkg/scheduler/framework/plugins/helper" + framework "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1" + + "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/scheduler/cache" +) + +func getPodGroupMinResourceFromPod(pod *corev1.Pod, count int) *framework.Resource { + var requirements framework.Resource + for i := 0; i < count; i++ { + requirements.Add(getPodResourceRequire(pod).ResourceList()) + } + if requirements.AllowedPodNumber == 0 { + requirements.AllowedPodNumber = count + } + return &requirements +} + +func singleNodeResource(info *framework.NodeInfo, pod *corev1.Pod, percent float32) *framework.Resource { + leftResource := framework.Resource{ + ScalarResources: make(map[corev1.ResourceName]int64), + } + satisfied := false + if pod != nil { + satisfied = checkFit(pod, info) + if !satisfied { + return &leftResource + } + } + + allocatable := info.Allocatable + requested := info.Requested + + podCount := requested.AllowedPodNumber + if podCount == 0 { + podCount = len(info.Pods) + } + + // reserve `percent` resource + leftResource.AllowedPodNumber = int(float32(allocatable.AllowedPodNumber)*percent) - podCount + leftResource.MilliCPU = int64(float32(allocatable.MilliCPU)*percent) - requested.MilliCPU + leftResource.Memory = int64(float32(allocatable.Memory)*percent) - requested.Memory + leftResource.EphemeralStorage = int64(float32(allocatable.EphemeralStorage)*percent) - requested.EphemeralStorage + + // calculate extend resources + for k, allocatableEx := range allocatable.ScalarResources { + requestEx, _ := requested.ScalarResources[k] + leftResource.ScalarResources[k] = int64(float32(allocatableEx)*percent) - requestEx + } + return &leftResource +} + +func checkLeftResourceSatisfy(leftResource, req *framework.Resource) bool { + if leftResource.Memory < req.Memory { + return false + } + if leftResource.MilliCPU < req.MilliCPU { + return false + } + if leftResource.EphemeralStorage < req.EphemeralStorage { + return false + } + // AllowedPodNumber seems not correct, so we need check again + if leftResource.AllowedPodNumber < req.AllowedPodNumber { + return false + } + for k, v1 := range req.ScalarResources { + v2, ok := leftResource.ScalarResources[k] + if !ok { + if v1 != 0 { + return false + } + continue + } + if v1 >= v2 { + return false + } + } + return true +} + +func searchMaxPG(pgCache *cache.PGSchedulingCache) (maxFinishedPG string, + maxSchedulingPGStatus *cache.PodGroupSchedulingStatus) { + var maxFinished uint32 = 0 + for pgName, pgs := range pgCache.PGStatusMap { + // if we have meet min request, make other pods priority + var finished uint32 = 0 + if pgs.Scheduled { + continue + } + if pgs.Pod == nil { + continue + } + if pgs.PodGroup.Spec.MinMember-pgs.PodGroup. + Status.Scheduled <= 0 { + finished = 0 + } else { + finished = (uint32(len(pgs.MatchedPodNodes.Items())) + pgs. + PodGroup.Status.Scheduled) * 1000 / (pgs.PodGroup.Spec.MinMember) + klog.V(0).Infof("group %v schedule progress %d/100", pgName, finished/10) + } + + if finished > maxFinished { + maxFinished = finished + maxFinishedPG = pgName + maxSchedulingPGStatus = pgs + continue + } + if finished == maxFinished { + // avoid maxPG is always the finished one + // 1. if max is nil, set the value + // 2. if the max is scheduled and running, make the not scheduled first + // 3. according to priority + if maxSchedulingPGStatus == nil || + maxSchedulingPGStatus.PodGroup.Status.Scheduled >= maxSchedulingPGStatus.PodGroup.Spec.MinMember && + pgs.PodGroup.Status.Scheduled == 0 || + podutil.GetPodPriority(pgs.Pod) > podutil.GetPodPriority(maxSchedulingPGStatus.Pod) { + maxFinished = finished + maxFinishedPG = pgName + maxSchedulingPGStatus = pgs + } + } + } + return +} + +func checkFit(pod *corev1.Pod, info *framework.NodeInfo) bool { + fit := helper.PodMatchesNodeSelectorAndAffinityTerms(pod, info.Node()) + if !fit { + return fit + } + + taints := info.Node().Spec.Taints + filterPredicate := func(t *corev1.Taint) bool { + return t.Effect == corev1.TaintEffectNoSchedule || t.Effect == corev1.TaintEffectNoExecute + } + + _, isUntolerated := v1helper.FindMatchingUntoleratedTaint(taints, pod.Spec.Tolerations, filterPredicate) + if !isUntolerated { + return true + } + return false +} + +func getPodResourceRequire(pod *corev1.Pod) *framework.Resource { + var currentPodRequired framework.Resource + // Compute current pod required + for _, c := range pod.Spec.Containers { + if c.Resources.Requests != nil { + currentPodRequired.Add(c.Resources.Requests) + } else { + currentPodRequired.Add(c.Resources.Limits) + } + } + return ¤tPodRequired +} + +func getPreAllocatedResource(maxSchedulingPGStatus *cache.PodGroupSchedulingStatus, matched int) (preAllocatedResource framework.Resource) { + notFinished := 0 + scheduled := int(maxSchedulingPGStatus.PodGroup.Status.Scheduled) + if matched != 0 { + notFinished = int(maxSchedulingPGStatus.PodGroup.Spec.MinMember) - matched + } else { + // recover from stop + notFinished = int(maxSchedulingPGStatus.PodGroup.Spec.MinMember) - scheduled + } + for i := 0; i < notFinished; i++ { + resourceRequirePerPod := getPodResourceRequire(maxSchedulingPGStatus.Pod) + preAllocatedResource.Add(resourceRequirePerPod.ResourceList()) + } + if preAllocatedResource.AllowedPodNumber == 0 { + preAllocatedResource.AllowedPodNumber = int(maxSchedulingPGStatus.PodGroup.Spec.MinMember) - notFinished + } + return +} diff --git a/test/integration/base.go b/test/integration/base.go new file mode 100644 index 0000000000..847469ecf0 --- /dev/null +++ b/test/integration/base.go @@ -0,0 +1,42 @@ +/* +Copyright 2020 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 integration + +import ( + "context" + "k8s.io/klog/v2" + + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + clientset "k8s.io/client-go/kubernetes" +) + +var lowPriority, midPriority, highPriority = int32(0), int32(100), int32(1000) + +// podScheduled returns true if a node is assigned to the given pod. +func podScheduled(c clientset.Interface, podNamespace, podName string) bool { + pod, err := c.CoreV1().Pods(podNamespace).Get(context.TODO(), podName, metav1.GetOptions{}) + if err != nil { + // This could be a connection error so we want to retry. + klog.Errorf("klog error %v", err) + return false + } + if pod.Spec.NodeName == "" { + klog.Infof("node name %v", pod.Spec.NodeName) + return false + } + return true +} diff --git a/test/integration/coscheduling_test.go b/test/integration/coscheduling_test.go index e41a541510..5ddf71b089 100644 --- a/test/integration/coscheduling_test.go +++ b/test/integration/coscheduling_test.go @@ -18,14 +18,25 @@ package integration import ( "context" + "fmt" + "io/ioutil" + "os" + "strconv" "testing" "time" v1 "k8s.io/api/core/v1" + apiextensionsv1beta1 "k8s.io/apiextensions-apiserver/pkg/apis/apiextensions/v1beta1" + apiextensionsclient "k8s.io/apiextensions-apiserver/pkg/client/clientset/clientset" + "k8s.io/apimachinery/pkg/api/errors" "k8s.io/apimachinery/pkg/api/resource" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/util/uuid" "k8s.io/apimachinery/pkg/util/wait" - clientset "k8s.io/client-go/kubernetes" + "k8s.io/client-go/kubernetes" + restclient "k8s.io/client-go/rest" + "k8s.io/client-go/tools/clientcmd" + clientcmdapi "k8s.io/client-go/tools/clientcmd/api" "k8s.io/klog/v2" "k8s.io/kubernetes/pkg/scheduler" schedapi "k8s.io/kubernetes/pkg/scheduler/apis/config" @@ -34,16 +45,69 @@ import ( testutils "k8s.io/kubernetes/test/integration/util" imageutils "k8s.io/kubernetes/test/utils/image" + scheconfig "sigs.k8s.io/scheduler-plugins/pkg/apis/config" "sigs.k8s.io/scheduler-plugins/pkg/coscheduling" + "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/apis/podgroup/v1alpha1" + pgclientset "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/generated/clientset/versioned" + coschedulingutil "sigs.k8s.io/scheduler-plugins/pkg/coscheduling/util" "sigs.k8s.io/scheduler-plugins/test/util" - // Ensure scheme package is initialized. - _ "sigs.k8s.io/scheduler-plugins/pkg/apis/config/scheme" ) -var lowPriority, midPriority, highPriority = int32(0), int32(100), int32(1000) - func TestCoschedulingPlugin(t *testing.T) { + todo := context.TODO() + ctx, cancelFunc := context.WithCancel(todo) + testCtx := &testutils.TestContext{ + Ctx: ctx, + CancelFn: cancelFunc, + CloseFn: func() {}, + } registry := fwkruntime.Registry{coscheduling.Name: coscheduling.New} + t.Log("create apiserver") + _, config := util.StartApi(t, todo.Done()) + + config.ContentType = "application/json" + + apiExtensionClient, err := apiextensionsclient.NewForConfig(config) + if err != nil { + t.Fatal(err) + } + + kubeConfigPath := "" + if tempFile, err := ioutil.TempFile(os.TempDir(), "kubeconfig-"); err == nil { + kubeConfig := createKubeConfig(config) + clientcmd.WriteToFile(*kubeConfig, tempFile.Name()) + kubeConfigPath = tempFile.Name() + } + defer os.RemoveAll(kubeConfigPath) + + t.Log("create crd") + crd := &apiextensionsv1beta1.CustomResourceDefinition{ + ObjectMeta: metav1.ObjectMeta{ + Name: "podgroups.scheduling.sigs.k8s.io", + }, + Spec: apiextensionsv1beta1.CustomResourceDefinitionSpec{ + Group: "scheduling.sigs.k8s.io", + Version: "v1alpha1", + Scope: apiextensionsv1beta1.NamespaceScoped, + Names: apiextensionsv1beta1.CustomResourceDefinitionNames{ + Kind: "PodGroup", + Plural: "podgroups", + ShortNames: []string{"pg", "pgs"}, + }, + }, + } + + _, err = apiExtensionClient.ApiextensionsV1beta1().CustomResourceDefinitions().Create(ctx, crd, metav1.CreateOptions{}) + + if err != nil { + t.Fatal(err) + } + + time.Sleep(1 * time.Second) + cfg := &scheconfig.CoschedulingArgs{ + KubeConfig: kubeConfigPath, + } + profile := schedapi.KubeSchedulerProfile{ SchedulerName: v1.DefaultSchedulerName, Plugins: &schedapi.Plugins{ @@ -65,45 +129,74 @@ func TestCoschedulingPlugin(t *testing.T) { {Name: coscheduling.Name}, }, }, - Reserve: &schedapi.PluginSet{ + PostBind: &schedapi.PluginSet{ Enabled: []schedapi.Plugin{ {Name: coscheduling.Name}, }, }, }, + PluginConfig: []schedapi.PluginConfig{ + { + Name: coscheduling.Name, + Args: cfg, + }, + }, } - testCtx := util.InitTestSchedulerWithOptions( + extClient := pgclientset.NewForConfigOrDie(config) + + cs := kubernetes.NewForConfigOrDie(config) + ns, err := cs.CoreV1().Namespaces().Create(ctx, &v1.Namespace{ + ObjectMeta: metav1.ObjectMeta{Name: fmt.Sprintf("integration-test-%v", string(uuid.NewUUID()))}}, metav1.CreateOptions{}) + if err != nil && !errors.IsAlreadyExists(err) { + t.Fatalf("Failed to create ns default: %v", err) + } + + autoCreate := false + t.Logf("namespaces %+v", ns.Name) + _, err = cs.CoreV1().ServiceAccounts(ns.Name).Create(ctx, &v1.ServiceAccount{ + ObjectMeta: metav1.ObjectMeta{Name: "default", Namespace: ns.Name}, AutomountServiceAccountToken: &autoCreate}, metav1.CreateOptions{}) + if err != nil && !errors.IsAlreadyExists(err) { + t.Fatalf("Failed to create ns default: %v", err) + } + + testCtx.NS = ns + testCtx.ClientSet = cs + + testCtx = util.InitTestSchedulerWithOptions( t, - testutils.InitTestMaster(t, "sched-coscheduling", nil), + testCtx, true, scheduler.WithProfiles(profile), scheduler.WithFrameworkOutOfTreeRegistry(registry), ) - + t.Log("init scheduler success") defer testutils.CleanupTest(t, testCtx) - cs, ns := testCtx.ClientSet, testCtx.NS.Name // Create a Node. nodeName := "fake-node" node := st.MakeNode().Name("fake-node").Label("node", nodeName).Obj() node.Status.Allocatable = v1.ResourceList{ v1.ResourcePods: *resource.NewQuantity(32, resource.DecimalSI), - v1.ResourceMemory: *resource.NewQuantity(300, resource.DecimalSI), + v1.ResourceCPU: resource.MustParse("40"), + v1.ResourceMemory: resource.MustParse("300"), } node.Status.Capacity = v1.ResourceList{ v1.ResourcePods: *resource.NewQuantity(32, resource.DecimalSI), - v1.ResourceMemory: *resource.NewQuantity(300, resource.DecimalSI), + v1.ResourceCPU: resource.MustParse("40"), + v1.ResourceMemory: resource.MustParse("300"), + } + node.Status.Addresses = []v1.NodeAddress{ + {Type: v1.NodeInternalIP, Address: "127.0.0.1"}, + {Type: v1.NodeHostName, Address: "localhost"}, } - node, err := cs.CoreV1().Nodes().Create(context.TODO(), node, metav1.CreateOptions{}) + node, err = cs.CoreV1().Nodes().Create(ctx, node, metav1.CreateOptions{}) if err != nil { t.Fatalf("Failed to create Node %q: %v", nodeName, err) } - // Create Pods belongs to two podGroup. var pods []*v1.Pod pause := imageutils.GetPauseImageName() - type podInfo struct { podName string podGroupName string @@ -113,9 +206,10 @@ func TestCoschedulingPlugin(t *testing.T) { } for _, tt := range []struct { - name string - pods []podInfo - expectedPods []string + name string + pods []podInfo + expectedPods []string + allowPodsNotIn bool }{ { name: "equal priority, sequentially pg1 meet min and pg2 not meet min", @@ -164,7 +258,10 @@ func TestCoschedulingPlugin(t *testing.T) { {podName: "t4-p2-2", podGroupName: "pg4-2", minAvailable: "3", priority: highPriority, memReq: 50}, {podName: "t4-p2-3", podGroupName: "pg4-2", minAvailable: "3", priority: highPriority, memReq: 50}, }, - expectedPods: []string{"t4-p2-1", "t4-p2-2", "t4-p2-3"}, + // this case, we can not ensure which one would schedule, e.g. we always have one pod in queue, + // then the SortPlugin would not work + expectedPods: []string{"t4-p2-1", "t4-p2-2", "t4-p2-3"}, + allowPodsNotIn: true, }, { name: "different priority, not sequentially pg1 meet min and pg2 meet min", @@ -207,28 +304,65 @@ func TestCoschedulingPlugin(t *testing.T) { }, expectedPods: []string{"t7-p1-1", "t7-p1-2", "t7-p1-3"}, }, + { + name: "not equal priority, not sequentially pg1,pg3 meet min and p2 not meet min", + pods: []podInfo{ + {podName: "t8-p1-1", podGroupName: "pg8-1", minAvailable: "3", priority: midPriority, memReq: 50}, + {podName: "t8-p2-1", podGroupName: "pg8-2", minAvailable: "4", priority: midPriority, memReq: 30}, + {podName: "t8-p3-1", podGroupName: "pg8-3", minAvailable: "4", priority: highPriority, memReq: 30}, + {podName: "t8-p1-2", podGroupName: "pg8-1", minAvailable: "3", priority: midPriority, memReq: 50}, + {podName: "t8-p2-2", podGroupName: "pg8-2", minAvailable: "4", priority: midPriority, memReq: 30}, + {podName: "t8-p3-2", podGroupName: "pg8-3", minAvailable: "4", priority: highPriority, memReq: 30}, + {podName: "t8-p1-3", podGroupName: "pg8-1", minAvailable: "3", priority: midPriority, memReq: 50}, + {podName: "t8-p2-3", podGroupName: "pg8-2", minAvailable: "4", priority: midPriority, memReq: 30}, + {podName: "t8-p3-3", podGroupName: "pg8-3", minAvailable: "4", priority: highPriority, memReq: 30}, + {podName: "t8-p2-4", podGroupName: "pg8-2", minAvailable: "4", priority: midPriority, memReq: 30}, + {podName: "t8-p3-4", podGroupName: "pg8-3", minAvailable: "4", priority: highPriority, memReq: 30}, + }, + expectedPods: []string{"t8-p1-1", "t8-p1-2", "t8-p1-3", "t8-p3-1", "t8-p3-2", "t8-p3-3", "t8-p3-4"}, + }, } { t.Run(tt.name, func(t *testing.T) { t.Logf("Start-coscheduling-test %v", tt.name) testutils.CleanupPods(cs, t, pods) pods = make([]*v1.Pod, 0) + // create pod group for _, podInfo := range tt.pods { - pod := st.MakePod().Namespace(ns).Name(podInfo.podName).Container(pause). + if podInfo.minAvailable == "" { + continue + } + min, err := strconv.Atoi(podInfo.minAvailable) + if err != nil { + t.Fatal(err) + } + pg := &v1alpha1.PodGroup{ + ObjectMeta: metav1.ObjectMeta{Name: podInfo.podGroupName, Namespace: ns.Name}, + Spec: v1alpha1.PodGroupSpec{MinMember: uint32(min)}, + } + pg, err = extClient.SchedulingV1alpha1().PodGroups(ns.Name).Create(ctx, pg, metav1.CreateOptions{}) + if err != nil && !errors.IsAlreadyExists(err) { + t.Fatal(err) + } + } + for _, podInfo := range tt.pods { + pod := st.MakePod().Namespace(ns.Name).Name(podInfo.podName).Container(pause). Priority(podInfo.priority).ZeroTerminationGracePeriod().Obj() pod.Spec.Containers[0].Resources = v1.ResourceRequirements{ Requests: v1.ResourceList{ v1.ResourceMemory: *resource.NewQuantity(podInfo.memReq, resource.DecimalSI), }, } - pod.Labels = map[string]string{ - coscheduling.PodGroupName: podInfo.podGroupName, - coscheduling.PodGroupMinAvailable: podInfo.minAvailable, + if podInfo.podGroupName != "" { + pod.Labels = map[string]string{ + coschedulingutil.PodGroupLabel: podInfo.podGroupName, + } } pods = append(pods, pod) } // Create Pods, We will expect them to be scheduled in a reversed order. for i := range pods { + klog.Infof("Creating pod %v", pods[i].Name) _, err := cs.CoreV1().Pods(pods[i].Namespace).Create(testCtx.Ctx, pods[i], metav1.CreateOptions{}) if err != nil { t.Fatalf("Failed to create Pod %q: %v", pods[i].Name, err) @@ -248,9 +382,19 @@ func TestCoschedulingPlugin(t *testing.T) { testCtx.Scheduler.SchedulingQueue.MoveAllToActiveOrBackoffQueue("fake event") err = wait.Poll(1*time.Second, 120*time.Second, func() (bool, error) { + notFound := true for _, v := range tt.expectedPods { - if !podScheduled(cs, ns, v) { + if !podScheduled(cs, ns.Name, v) { t.Logf("waiting pod failed %v", v) + if !tt.allowPodsNotIn { + return false, nil + } + } else { + notFound = false + } + } + if notFound { + if !tt.allowPodsNotIn { return false, nil } } @@ -259,19 +403,60 @@ func TestCoschedulingPlugin(t *testing.T) { if err != nil { t.Fatalf("%v Waiting expectedPods error: %v", tt.name, err.Error()) } + // delete pod group + for _, podInfo := range tt.pods { + if podInfo.minAvailable == "" { + continue + } + min, err := strconv.Atoi(podInfo.minAvailable) + if err != nil { + t.Fatal(err) + } + pg := &v1alpha1.PodGroup{ + ObjectMeta: metav1.ObjectMeta{Name: podInfo.podGroupName, Namespace: ns.Name}, + Spec: v1alpha1.PodGroupSpec{MinMember: uint32(min)}, + } + klog.Infof("Delete pod group %v", pg.Name) + extClient.SchedulingV1alpha1().PodGroups(ns.Name).Delete(ctx, pg.Name, metav1.DeleteOptions{}) + } }) } } -// podScheduled returns true if a node is assigned to the given pod. -func podScheduled(c clientset.Interface, podNamespace, podName string) bool { - pod, err := c.CoreV1().Pods(podNamespace).Get(context.TODO(), podName, metav1.GetOptions{}) - if err != nil { - // This could be a connection error so we want to retry. - return false +func createKubeConfig(clientCfg *restclient.Config) *clientcmdapi.Config { + clusterNick := "cluster" + userNick := "user" + contextNick := "context" + + configCmd := clientcmdapi.NewConfig() + + credentials := clientcmdapi.NewAuthInfo() + credentials.Token = clientCfg.BearerToken + credentials.TokenFile = clientCfg.BearerTokenFile + credentials.ClientCertificate = clientCfg.TLSClientConfig.CertFile + if len(credentials.ClientCertificate) == 0 { + credentials.ClientCertificateData = clientCfg.TLSClientConfig.CertData } - if pod.Spec.NodeName == "" { - return false + credentials.ClientKey = clientCfg.TLSClientConfig.KeyFile + if len(credentials.ClientKey) == 0 { + credentials.ClientKeyData = clientCfg.TLSClientConfig.KeyData } - return true + configCmd.AuthInfos[userNick] = credentials + + cluster := clientcmdapi.NewCluster() + cluster.Server = clientCfg.Host + cluster.CertificateAuthority = clientCfg.CAFile + if len(cluster.CertificateAuthority) == 0 { + cluster.CertificateAuthorityData = clientCfg.CAData + } + cluster.InsecureSkipTLSVerify = clientCfg.Insecure + configCmd.Clusters[clusterNick] = cluster + + context := clientcmdapi.NewContext() + context.Cluster = clusterNick + context.AuthInfo = userNick + configCmd.Contexts[contextNick] = context + configCmd.CurrentContext = contextNick + + return configCmd } diff --git a/test/util/aggregator.go b/test/util/aggregator.go new file mode 100644 index 0000000000..953b57ab1c --- /dev/null +++ b/test/util/aggregator.go @@ -0,0 +1,309 @@ +/* +Copyright 2020 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 util + +import ( + "fmt" + "net/http" + "strings" + "sync" + + apiextensionsinformers "k8s.io/apiextensions-apiserver/pkg/client/informers/externalversions" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime" + "k8s.io/apimachinery/pkg/runtime/schema" + "k8s.io/apimachinery/pkg/util/sets" + "k8s.io/apiserver/pkg/admission" + "k8s.io/apiserver/pkg/features" + genericapiserver "k8s.io/apiserver/pkg/server" + "k8s.io/apiserver/pkg/server/healthz" + genericoptions "k8s.io/apiserver/pkg/server/options" + "k8s.io/apiserver/pkg/util/feature" + utilfeature "k8s.io/apiserver/pkg/util/feature" + kubeexternalinformers "k8s.io/client-go/informers" + "k8s.io/client-go/tools/cache" + "k8s.io/klog/v2" + v1 "k8s.io/kube-aggregator/pkg/apis/apiregistration/v1" + v1helper "k8s.io/kube-aggregator/pkg/apis/apiregistration/v1/helper" + "k8s.io/kube-aggregator/pkg/apis/apiregistration/v1beta1" + aggregatorapiserver "k8s.io/kube-aggregator/pkg/apiserver" + aggregatorscheme "k8s.io/kube-aggregator/pkg/apiserver/scheme" + apiregistrationclient "k8s.io/kube-aggregator/pkg/client/clientset_generated/clientset/typed/apiregistration/v1" + informers "k8s.io/kube-aggregator/pkg/client/informers/externalversions/apiregistration/v1" + "k8s.io/kube-aggregator/pkg/controllers/autoregister" + "k8s.io/kubernetes/cmd/kube-apiserver/app/options" + "k8s.io/kubernetes/pkg/master/controller/crdregistration" +) + +func createAggregatorConfig( + kubeAPIServerConfig genericapiserver.Config, + commandOptions *options.ServerRunOptions, + externalInformers kubeexternalinformers.SharedInformerFactory, + serviceResolver aggregatorapiserver.ServiceResolver, + proxyTransport *http.Transport, + pluginInitializers []admission.PluginInitializer, +) (*aggregatorapiserver.Config, error) { + // make a shallow copy to let us twiddle a few things + // most of the config actually remains the same. We only need to mess with a couple items related to the particulars of the aggregator + genericConfig := kubeAPIServerConfig + genericConfig.PostStartHooks = map[string]genericapiserver.PostStartHookConfigEntry{} + genericConfig.RESTOptionsGetter = nil + + // override genericConfig.AdmissionControl with kube-aggregator's scheme, + // because aggregator apiserver should use its own scheme to convert its own resources. + err := commandOptions.Admission.ApplyTo( + &genericConfig, + externalInformers, + genericConfig.LoopbackClientConfig, + feature.DefaultFeatureGate, + pluginInitializers...) + if err != nil { + return nil, err + } + + // copy the etcd options so we don't mutate originals. + etcdOptions := *commandOptions.Etcd + etcdOptions.StorageConfig.Paging = utilfeature.DefaultFeatureGate.Enabled(features.APIListChunking) + etcdOptions.StorageConfig.Codec = aggregatorscheme.Codecs.LegacyCodec(v1beta1.SchemeGroupVersion, v1.SchemeGroupVersion) + etcdOptions.StorageConfig.EncodeVersioner = runtime.NewMultiGroupVersioner(v1beta1.SchemeGroupVersion, schema.GroupKind{Group: v1beta1.GroupName}) + genericConfig.RESTOptionsGetter = &genericoptions.SimpleRestOptionsFactory{Options: etcdOptions} + + // override MergedResourceConfig with aggregator defaults and registry + if err := commandOptions.APIEnablement.ApplyTo( + &genericConfig, + aggregatorapiserver.DefaultAPIResourceConfigSource(), + aggregatorscheme.Scheme); err != nil { + return nil, err + } + + aggregatorConfig := &aggregatorapiserver.Config{ + GenericConfig: &genericapiserver.RecommendedConfig{ + Config: genericConfig, + SharedInformerFactory: externalInformers, + }, + ExtraConfig: aggregatorapiserver.ExtraConfig{ + ProxyClientCertFile: commandOptions.ProxyClientCertFile, + ProxyClientKeyFile: commandOptions.ProxyClientKeyFile, + ServiceResolver: serviceResolver, + ProxyTransport: proxyTransport, + }, + } + + // we need to clear the poststarthooks so we don't add them multiple times to all the servers (that fails) + aggregatorConfig.GenericConfig.PostStartHooks = map[string]genericapiserver.PostStartHookConfigEntry{} + + return aggregatorConfig, nil +} + +func createAggregatorServer(aggregatorConfig *aggregatorapiserver.Config, delegateAPIServer genericapiserver.DelegationTarget, apiExtensionInformers apiextensionsinformers.SharedInformerFactory) (*aggregatorapiserver.APIAggregator, error) { + aggregatorServer, err := aggregatorConfig.Complete().NewWithDelegate(delegateAPIServer) + if err != nil { + return nil, err + } + + // create controllers for auto-registration + apiRegistrationClient, err := apiregistrationclient.NewForConfig(aggregatorConfig.GenericConfig.LoopbackClientConfig) + if err != nil { + return nil, err + } + autoRegistrationController := autoregister.NewAutoRegisterController(aggregatorServer.APIRegistrationInformers.Apiregistration().V1().APIServices(), apiRegistrationClient) + apiServices := apiServicesToRegister(delegateAPIServer, autoRegistrationController) + crdRegistrationController := crdregistration.NewCRDRegistrationController( + apiExtensionInformers.Apiextensions().V1().CustomResourceDefinitions(), + autoRegistrationController) + + err = aggregatorServer.GenericAPIServer.AddPostStartHook("kube-apiserver-autoregistration", func(context genericapiserver.PostStartHookContext) error { + go crdRegistrationController.Run(5, context.StopCh) + go func() { + // let the CRD controller process the initial set of CRDs before starting the autoregistration controller. + // this prevents the autoregistration controller's initial sync from deleting APIServices for CRDs that still exist. + // we only need to do this if CRDs are enabled on this server. We can't use discovery because we are the source for discovery. + if aggregatorConfig.GenericConfig.MergedResourceConfig.AnyVersionForGroupEnabled("apiextensions.k8s.io") { + crdRegistrationController.WaitForInitialSync() + } + autoRegistrationController.Run(5, context.StopCh) + }() + return nil + }) + if err != nil { + return nil, err + } + + err = aggregatorServer.GenericAPIServer.AddBootSequenceHealthChecks( + makeAPIServiceAvailableHealthCheck( + "autoregister-completion", + apiServices, + aggregatorServer.APIRegistrationInformers.Apiregistration().V1().APIServices(), + ), + ) + if err != nil { + return nil, err + } + + return aggregatorServer, nil +} + +func makeAPIService(gv schema.GroupVersion) *v1.APIService { + apiServicePriority, ok := apiVersionPriorities[gv] + if !ok { + // if we aren't found, then we shouldn't register ourselves because it could result in a CRD group version + // being permanently stuck in the APIServices list. + klog.Infof("Skipping APIService creation for %v", gv) + return nil + } + return &v1.APIService{ + ObjectMeta: metav1.ObjectMeta{Name: gv.Version + "." + gv.Group}, + Spec: v1.APIServiceSpec{ + Group: gv.Group, + Version: gv.Version, + GroupPriorityMinimum: apiServicePriority.group, + VersionPriority: apiServicePriority.version, + }, + } +} + +// makeAPIServiceAvailableHealthCheck returns a healthz check that returns healthy +// once all of the specified services have been observed to be available at least once. +func makeAPIServiceAvailableHealthCheck(name string, apiServices []*v1.APIService, apiServiceInformer informers.APIServiceInformer) healthz.HealthChecker { + // Track the auto-registered API services that have not been observed to be available yet + pendingServiceNamesLock := &sync.RWMutex{} + pendingServiceNames := sets.NewString() + for _, service := range apiServices { + pendingServiceNames.Insert(service.Name) + } + + // When an APIService in the list is seen as available, remove it from the pending list + handleAPIServiceChange := func(service *v1.APIService) { + pendingServiceNamesLock.Lock() + defer pendingServiceNamesLock.Unlock() + if !pendingServiceNames.Has(service.Name) { + return + } + if v1helper.IsAPIServiceConditionTrue(service, v1.Available) { + pendingServiceNames.Delete(service.Name) + } + } + + // Watch add/update events for APIServices + apiServiceInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: func(obj interface{}) { handleAPIServiceChange(obj.(*v1.APIService)) }, + UpdateFunc: func(old, new interface{}) { handleAPIServiceChange(new.(*v1.APIService)) }, + }) + + // Don't return healthy until the pending list is empty + return healthz.NamedCheck(name, func(r *http.Request) error { + pendingServiceNamesLock.RLock() + defer pendingServiceNamesLock.RUnlock() + if pendingServiceNames.Len() > 0 { + return fmt.Errorf("missing APIService: %v", pendingServiceNames.List()) + } + return nil + }) +} + +// priority defines group priority that is used in discovery. This controls +// group position in the kubectl output. +type priority struct { + // group indicates the order of the group relative to other groups. + group int32 + // version indicates the relative order of the version inside of its group. + version int32 +} + +// The proper way to resolve this letting the aggregator know the desired group and version-within-group order of the underlying servers +// is to refactor the genericapiserver.DelegationTarget to include a list of priorities based on which APIs were installed. +// This requires the APIGroupInfo struct to evolve and include the concept of priorities and to avoid mistakes, the core storage map there needs to be updated. +// That ripples out every bit as far as you'd expect, so for 1.7 we'll include the list here instead of being built up during storage. +var apiVersionPriorities = map[schema.GroupVersion]priority{ + {Group: "", Version: "v1"}: {group: 18000, version: 1}, + // extensions is above the rest for CLI compatibility, though the level of unqualified resource compatibility we + // can reasonably expect seems questionable. + {Group: "extensions", Version: "v1beta1"}: {group: 17900, version: 1}, + // to my knowledge, nothing below here collides + {Group: "apps", Version: "v1"}: {group: 17800, version: 15}, + {Group: "events.k8s.io", Version: "v1beta1"}: {group: 17750, version: 5}, + {Group: "authentication.k8s.io", Version: "v1"}: {group: 17700, version: 15}, + {Group: "authentication.k8s.io", Version: "v1beta1"}: {group: 17700, version: 9}, + {Group: "authorization.k8s.io", Version: "v1"}: {group: 17600, version: 15}, + {Group: "authorization.k8s.io", Version: "v1beta1"}: {group: 17600, version: 9}, + {Group: "autoscaling", Version: "v1"}: {group: 17500, version: 15}, + {Group: "autoscaling", Version: "v2beta1"}: {group: 17500, version: 9}, + {Group: "autoscaling", Version: "v2beta2"}: {group: 17500, version: 1}, + {Group: "batch", Version: "v1"}: {group: 17400, version: 15}, + {Group: "batch", Version: "v1beta1"}: {group: 17400, version: 9}, + {Group: "batch", Version: "v2alpha1"}: {group: 17400, version: 9}, + {Group: "certificates.k8s.io", Version: "v1beta1"}: {group: 17300, version: 9}, + {Group: "networking.k8s.io", Version: "v1"}: {group: 17200, version: 15}, + {Group: "networking.k8s.io", Version: "v1beta1"}: {group: 17200, version: 9}, + {Group: "policy", Version: "v1beta1"}: {group: 17100, version: 9}, + {Group: "rbac.authorization.k8s.io", Version: "v1"}: {group: 17000, version: 15}, + {Group: "rbac.authorization.k8s.io", Version: "v1beta1"}: {group: 17000, version: 12}, + {Group: "rbac.authorization.k8s.io", Version: "v1alpha1"}: {group: 17000, version: 9}, + {Group: "settings.k8s.io", Version: "v1alpha1"}: {group: 16900, version: 9}, + {Group: "storage.k8s.io", Version: "v1"}: {group: 16800, version: 15}, + {Group: "storage.k8s.io", Version: "v1beta1"}: {group: 16800, version: 9}, + {Group: "storage.k8s.io", Version: "v1alpha1"}: {group: 16800, version: 1}, + {Group: "apiextensions.k8s.io", Version: "v1"}: {group: 16700, version: 15}, + {Group: "apiextensions.k8s.io", Version: "v1beta1"}: {group: 16700, version: 9}, + {Group: "admissionregistration.k8s.io", Version: "v1"}: {group: 16700, version: 15}, + {Group: "admissionregistration.k8s.io", Version: "v1beta1"}: {group: 16700, version: 12}, + {Group: "scheduling.k8s.io", Version: "v1"}: {group: 16600, version: 15}, + {Group: "scheduling.k8s.io", Version: "v1beta1"}: {group: 16600, version: 12}, + {Group: "scheduling.k8s.io", Version: "v1alpha1"}: {group: 16600, version: 9}, + {Group: "coordination.k8s.io", Version: "v1"}: {group: 16500, version: 15}, + {Group: "coordination.k8s.io", Version: "v1beta1"}: {group: 16500, version: 9}, + {Group: "auditregistration.k8s.io", Version: "v1alpha1"}: {group: 16400, version: 1}, + {Group: "node.k8s.io", Version: "v1alpha1"}: {group: 16300, version: 1}, + {Group: "node.k8s.io", Version: "v1beta1"}: {group: 16300, version: 9}, + {Group: "discovery.k8s.io", Version: "v1beta1"}: {group: 16200, version: 12}, + {Group: "discovery.k8s.io", Version: "v1alpha1"}: {group: 16200, version: 9}, + {Group: "flowcontrol.apiserver.k8s.io", Version: "v1alpha1"}: {group: 16100, version: 9}, + // Append a new group to the end of the list if unsure. + // You can use min(existing group)-100 as the initial value for a group. + // Version can be set to 9 (to have space around) for a new group. +} + +func apiServicesToRegister(delegateAPIServer genericapiserver.DelegationTarget, registration autoregister.AutoAPIServiceRegistration) []*v1.APIService { + apiServices := []*v1.APIService{} + + for _, curr := range delegateAPIServer.ListedPaths() { + if curr == "/api/v1" { + apiService := makeAPIService(schema.GroupVersion{Group: "", Version: "v1"}) + registration.AddAPIServiceToSyncOnStart(apiService) + apiServices = append(apiServices, apiService) + continue + } + + if !strings.HasPrefix(curr, "/apis/") { + continue + } + // this comes back in a list that looks like /apis/rbac.authorization.k8s.io/v1alpha1 + tokens := strings.Split(curr, "/") + if len(tokens) != 4 { + continue + } + + apiService := makeAPIService(schema.GroupVersion{Group: tokens[2], Version: tokens[3]}) + if apiService == nil { + continue + } + registration.AddAPIServiceToSyncOnStart(apiService) + apiServices = append(apiServices, apiService) + } + + return apiServices +} diff --git a/test/util/apiserver.go b/test/util/apiserver.go new file mode 100644 index 0000000000..4de78366a3 --- /dev/null +++ b/test/util/apiserver.go @@ -0,0 +1,205 @@ +/* +Copyright 2020 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 util + +import ( + "context" + "crypto/x509" + "encoding/pem" + "io/ioutil" + "net" + "net/http" + "os" + "path" + "testing" + "time" + + "github.com/google/uuid" + + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/util/wait" + genericapiserver "k8s.io/apiserver/pkg/server" + genericapiserveroptions "k8s.io/apiserver/pkg/server/options" + "k8s.io/apiserver/pkg/util/webhook" + client "k8s.io/client-go/kubernetes" + "k8s.io/client-go/rest" + "k8s.io/client-go/util/cert" + "k8s.io/kubernetes/cmd/kube-apiserver/app" + "k8s.io/kubernetes/cmd/kube-apiserver/app/options" + kubeserver "k8s.io/kubernetes/pkg/kubeapiserver/server" + "k8s.io/kubernetes/pkg/master" + "k8s.io/kubernetes/test/utils" +) + +// TestServerSetup holds configuration information for a kube-apiserver test server. +type TestServerSetup struct { + ModifyServerRunOptions func(*options.ServerRunOptions) + ModifyServerConfig func(*master.Config) +} + +// StartTestServer runs a kube-apiserver, optionally calling out to the setup.ModifyServerRunOptions and setup.ModifyServerConfig functions +func StartApi(t *testing.T, stopCh <-chan struct{}) (client.Interface, *rest.Config) { + certDir, _ := ioutil.TempDir("", "test-integration") + go func() { + <-stopCh + os.RemoveAll(certDir) + }() + + _, defaultServiceClusterIPRange, _ := net.ParseCIDR("10.0.0.0/24") + clientSigningKey, err := utils.NewPrivateKey() + if err != nil { + t.Fatalf("%+v", err) + } + clientSigningCert, err := cert.NewSelfSignedCACert(cert.Config{CommonName: "client-ca"}, clientSigningKey) + if err != nil { + t.Fatalf("%+v", err) + } + clientCACertFile, _ := ioutil.TempFile(certDir, "client-ca.crt") + if err := ioutil.WriteFile(clientCACertFile.Name(), utils.EncodeCertPEM(clientSigningCert), 0644); err != nil { + t.Fatalf("%+v", err) + } + + listener, _, err := genericapiserveroptions.CreateListener("tcp", "127.0.0.1:0", net.ListenConfig{}) + if err != nil { + t.Fatalf("%+v", err) + } + + privBytes, err := x509.MarshalPKCS8PrivateKey(clientSigningKey) + if err != nil { + t.Fatal(err) + } + + keyPath := path.Join(certDir, "client.key") + + if err := ioutil.WriteFile(keyPath, pem.EncodeToMemory(&pem.Block{Type: "PRIVATE KEY", Bytes: privBytes}), 0666); err != nil { + t.Fatal(err) + } + + kubeAPIServerOptions := options.NewServerRunOptions() + kubeAPIServerOptions.SecureServing.Listener = listener + kubeAPIServerOptions.SecureServing.BindAddress = net.ParseIP("127.0.0.1") + kubeAPIServerOptions.SecureServing.ServerCert.CertDirectory = certDir + kubeAPIServerOptions.InsecureServing.BindPort = 0 + kubeAPIServerOptions.Etcd.StorageConfig.Prefix = path.Join("/", uuid.New().String(), "registry") + kubeAPIServerOptions.Etcd.StorageConfig.Transport.ServerList = IntegrationEtcdServers() + kubeAPIServerOptions.ServiceClusterIPRanges = defaultServiceClusterIPRange.String() + kubeAPIServerOptions.Authentication.ClientCert.ClientCA = clientCACertFile.Name() + kubeAPIServerOptions.Authorization.Modes = []string{"Node", "RBAC"} + kubeAPIServerOptions.ServiceAccountSigningKeyFile = keyPath + kubeAPIServerOptions.Admission.GenericAdmission.DisablePlugins = []string{"Priority", "TaintNodesByCondition"} + + completedOptions, err := app.Complete(kubeAPIServerOptions) + if err != nil { + t.Fatalf("%+v", err) + } + tunneler, proxyTransport, err := app.CreateNodeDialer(completedOptions) + if err != nil { + t.Fatalf("%+v", err) + } + kubeAPIServerConfig, insecureServingInfo, serviceResolver, pluginInitializer, err := app.CreateKubeAPIServerConfig(completedOptions, tunneler, proxyTransport) + if err != nil { + t.Fatalf("%+v", err) + } + + // If additional API servers are added, they should be gated. + apiExtensionsConfig, err := createAPIExtensionsConfig(*kubeAPIServerConfig.GenericConfig, kubeAPIServerConfig.ExtraConfig.VersionedInformers, pluginInitializer, completedOptions.ServerRunOptions, completedOptions.MasterCount, + serviceResolver, webhook.NewDefaultAuthenticationInfoResolverWrapper(proxyTransport, kubeAPIServerConfig.GenericConfig.EgressSelector, kubeAPIServerConfig.GenericConfig.LoopbackClientConfig)) + if err != nil { + t.Fatalf("%+v", err) + } + apiExtensionsServer, err := createAPIExtensionsServer(apiExtensionsConfig, genericapiserver.NewEmptyDelegate()) + if err != nil { + t.Fatalf("%+v", err) + } + + kubeAPIServer, err := app.CreateKubeAPIServer(kubeAPIServerConfig, apiExtensionsServer.GenericAPIServer) + if err != nil { + t.Fatalf("%+v", err) + } + + // aggregator comes last in the chain + aggregatorConfig, err := createAggregatorConfig(*kubeAPIServerConfig.GenericConfig, completedOptions.ServerRunOptions, kubeAPIServerConfig.ExtraConfig.VersionedInformers, serviceResolver, proxyTransport, pluginInitializer) + if err != nil { + t.Fatalf("%+v", err) + } + aggregatorServer, err := createAggregatorServer(aggregatorConfig, kubeAPIServer.GenericAPIServer, apiExtensionsServer.Informers) + if err != nil { + // we don't need special handling for innerStopCh because the aggregator server doesn't create any go routines + t.Fatalf("%+v", err) + } + + if insecureServingInfo != nil { + insecureHandlerChain := kubeserver.BuildInsecureHandlerChain(aggregatorServer.GenericAPIServer.UnprotectedHandler(), kubeAPIServerConfig.GenericConfig) + if err := insecureServingInfo.Serve(insecureHandlerChain, kubeAPIServerConfig.GenericConfig.RequestTimeout, stopCh); err != nil { + t.Fatalf("%+v", err) + + } + } + + go func() { + prepared, err := aggregatorServer.PrepareRun() + if err != nil { + t.Fatalf("%+v", err) + } + + prepared.Run(stopCh) + }() + + // Adjust the loopback config for external use (external server name and CA) + kubeAPIServerClientConfig := rest.CopyConfig(kubeAPIServerConfig.GenericConfig.LoopbackClientConfig) + kubeAPIServerClientConfig.CAFile = path.Join(certDir, "apiserver.crt") + kubeAPIServerClientConfig.CAData = nil + kubeAPIServerClientConfig.ServerName = "" + + // wait for health + err = wait.PollImmediate(100*time.Millisecond, 10*time.Second, func() (done bool, err error) { + healthzConfig := rest.CopyConfig(kubeAPIServerClientConfig) + healthzConfig.ContentType = "" + healthzConfig.AcceptContentTypes = "" + kubeClient, err := client.NewForConfig(healthzConfig) + if err != nil { + // this happens because we race the API server start + t.Logf("%+v", err) + return false, nil + } + + healthStatus := 0 + kubeClient.Discovery().RESTClient().Get().AbsPath("/healthz").Do(context.TODO()).StatusCode(&healthStatus) + if healthStatus != http.StatusOK { + return false, nil + } + + if _, err := kubeClient.CoreV1().Namespaces().Get(context.TODO(), "default", metav1.GetOptions{}); err != nil { + return false, nil + } + if _, err := kubeClient.CoreV1().Namespaces().Get(context.TODO(), "kube-system", metav1.GetOptions{}); err != nil { + return false, nil + } + + return true, nil + }) + if err != nil { + t.Fatalf("%+v", err) + } + + kubeAPIServerClient, err := client.NewForConfig(kubeAPIServerClientConfig) + if err != nil { + t.Fatalf("%+v", err) + } + + return kubeAPIServerClient, kubeAPIServerClientConfig +} diff --git a/test/util/extension.go b/test/util/extension.go new file mode 100644 index 0000000000..e85da50063 --- /dev/null +++ b/test/util/extension.go @@ -0,0 +1,100 @@ +/* +Copyright 2020 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 util + +import ( + v1 "k8s.io/apiextensions-apiserver/pkg/apis/apiextensions/v1" + "k8s.io/apiextensions-apiserver/pkg/apis/apiextensions/v1beta1" + apiextensionsapiserver "k8s.io/apiextensions-apiserver/pkg/apiserver" + apiextensionsoptions "k8s.io/apiextensions-apiserver/pkg/cmd/server/options" + "k8s.io/apimachinery/pkg/runtime" + "k8s.io/apimachinery/pkg/runtime/schema" + "k8s.io/apiserver/pkg/admission" + "k8s.io/apiserver/pkg/features" + genericapiserver "k8s.io/apiserver/pkg/server" + genericoptions "k8s.io/apiserver/pkg/server/options" + "k8s.io/apiserver/pkg/util/feature" + utilfeature "k8s.io/apiserver/pkg/util/feature" + "k8s.io/apiserver/pkg/util/webhook" + kubeexternalinformers "k8s.io/client-go/informers" + "k8s.io/kubernetes/cmd/kube-apiserver/app/options" +) + +func createAPIExtensionsConfig( + kubeAPIServerConfig genericapiserver.Config, + externalInformers kubeexternalinformers.SharedInformerFactory, + pluginInitializers []admission.PluginInitializer, + commandOptions *options.ServerRunOptions, + masterCount int, + serviceResolver webhook.ServiceResolver, + authResolverWrapper webhook.AuthenticationInfoResolverWrapper, +) (*apiextensionsapiserver.Config, error) { + // make a shallow copy to let us twiddle a few things + // most of the config actually remains the same. We only need to mess with a couple items related to the particulars of the apiextensions + genericConfig := kubeAPIServerConfig + genericConfig.PostStartHooks = map[string]genericapiserver.PostStartHookConfigEntry{} + genericConfig.RESTOptionsGetter = nil + + // override genericConfig.AdmissionControl with apiextensions' scheme, + // because apiextentions apiserver should use its own scheme to convert resources. + err := commandOptions.Admission.ApplyTo( + &genericConfig, + externalInformers, + genericConfig.LoopbackClientConfig, + feature.DefaultFeatureGate, + pluginInitializers...) + if err != nil { + return nil, err + } + + // copy the etcd options so we don't mutate originals. + etcdOptions := *commandOptions.Etcd + etcdOptions.StorageConfig.Paging = utilfeature.DefaultFeatureGate.Enabled(features.APIListChunking) + etcdOptions.StorageConfig.Codec = apiextensionsapiserver.Codecs.LegacyCodec(v1beta1.SchemeGroupVersion, v1.SchemeGroupVersion) + etcdOptions.StorageConfig.EncodeVersioner = runtime.NewMultiGroupVersioner(v1beta1.SchemeGroupVersion, schema.GroupKind{Group: v1beta1.GroupName}) + genericConfig.RESTOptionsGetter = &genericoptions.SimpleRestOptionsFactory{Options: etcdOptions} + + // override MergedResourceConfig with apiextensions defaults and registry + if err := commandOptions.APIEnablement.ApplyTo( + &genericConfig, + apiextensionsapiserver.DefaultAPIResourceConfigSource(), + apiextensionsapiserver.Scheme); err != nil { + return nil, err + } + + apiextensionsConfig := &apiextensionsapiserver.Config{ + GenericConfig: &genericapiserver.RecommendedConfig{ + Config: genericConfig, + SharedInformerFactory: externalInformers, + }, + ExtraConfig: apiextensionsapiserver.ExtraConfig{ + CRDRESTOptionsGetter: apiextensionsoptions.NewCRDRESTOptionsGetter(etcdOptions), + MasterCount: masterCount, + AuthResolverWrapper: authResolverWrapper, + ServiceResolver: serviceResolver, + }, + } + + // we need to clear the poststarthooks so we don't add them multiple times to all the servers (that fails) + apiextensionsConfig.GenericConfig.PostStartHooks = map[string]genericapiserver.PostStartHookConfigEntry{} + + return apiextensionsConfig, nil +} + +func createAPIExtensionsServer(apiextensionsConfig *apiextensionsapiserver.Config, delegateAPIServer genericapiserver.DelegationTarget) (*apiextensionsapiserver.CustomResourceDefinitions, error) { + return apiextensionsConfig.Complete().New(delegateAPIServer) +} diff --git a/test/util/fixtures.go b/test/util/fixtures.go new file mode 100644 index 0000000000..e82363cff9 --- /dev/null +++ b/test/util/fixtures.go @@ -0,0 +1,26 @@ +/* +Copyright 2020 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 util + +import "os" + +func IntegrationEtcdServers() []string { + if etcdURL, ok := os.LookupEnv("KUBE_INTEGRATION_ETCD_URL"); ok { + return []string{etcdURL} + } + return []string{"http://127.0.0.1:2379"} +}