From de3bcf813b02a4776f7cf8b1e452ce649df214e4 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Sun, 5 May 2019 18:44:17 +0800 Subject: [PATCH 01/68] Add queue controller --- cmd/controllers/app/server.go | 15 +- pkg/controllers/job/job_controller.go | 19 +- pkg/controllers/queue/queue_controller.go | 267 ++++++++++++++++++++++ 3 files changed, 289 insertions(+), 12 deletions(-) create mode 100644 pkg/controllers/queue/queue_controller.go diff --git a/cmd/controllers/app/server.go b/cmd/controllers/app/server.go index a0e1f1e9b0..8d0e93a736 100644 --- a/cmd/controllers/app/server.go +++ b/cmd/controllers/app/server.go @@ -36,8 +36,12 @@ import ( "k8s.io/client-go/tools/leaderelection/resourcelock" "k8s.io/client-go/tools/record" + kbver "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned" + "volcano.sh/volcano/cmd/controllers/app/options" + vkclient "volcano.sh/volcano/pkg/client/clientset/versioned" "volcano.sh/volcano/pkg/controllers/job" + "volcano.sh/volcano/pkg/controllers/queue" ) const ( @@ -72,10 +76,17 @@ func Run(opt *options.ServerOption) error { return err } - jobController := job.NewJobController(config) + // TODO: add user agent for different controllers + kubeClient := clientset.NewForConfigOrDie(config) + kbClient := kbver.NewForConfigOrDie(config) + vkClient := vkclient.NewForConfigOrDie(config) + + jobController := job.NewJobController(kubeClient, kbClient, vkClient) + queueController := queue.NewQueueController(kubeClient, kbClient) run := func(ctx context.Context) { - jobController.Run(ctx.Done()) + go jobController.Run(ctx.Done()) + go queueController.Run(ctx.Done()) <-ctx.Done() } diff --git a/pkg/controllers/job/job_controller.go b/pkg/controllers/job/job_controller.go index b38518f2f0..13c0719ef8 100644 --- a/pkg/controllers/job/job_controller.go +++ b/pkg/controllers/job/job_controller.go @@ -29,7 +29,6 @@ import ( corev1 "k8s.io/client-go/kubernetes/typed/core/v1" corelisters "k8s.io/client-go/listers/core/v1" pclister "k8s.io/client-go/listers/scheduling/v1beta1" - "k8s.io/client-go/rest" "k8s.io/client-go/tools/cache" "k8s.io/client-go/tools/record" "k8s.io/client-go/util/workqueue" @@ -53,7 +52,6 @@ import ( // Controller the Job Controller type type Controller struct { - config *rest.Config kubeClients *kubernetes.Clientset vkClients *vkver.Clientset kbClients *kbver.Clientset @@ -102,21 +100,22 @@ type Controller struct { } // NewJobController create new Job Controller -func NewJobController(config *rest.Config) *Controller { - - kubeClients := kubernetes.NewForConfigOrDie(config) +func NewJobController( + kubeClient *kubernetes.Clientset, + kbClient *kbver.Clientset, + vkClient *vkver.Clientset, +) *Controller { //Initialize event client eventBroadcaster := record.NewBroadcaster() eventBroadcaster.StartLogging(glog.Infof) - eventBroadcaster.StartRecordingToSink(&corev1.EventSinkImpl{Interface: kubeClients.CoreV1().Events("")}) + eventBroadcaster.StartRecordingToSink(&corev1.EventSinkImpl{Interface: kubeClient.CoreV1().Events("")}) recorder := eventBroadcaster.NewRecorder(vkscheme.Scheme, v1.EventSource{Component: "vk-controller"}) cc := &Controller{ - config: config, - kubeClients: kubeClients, - vkClients: vkver.NewForConfigOrDie(config), - kbClients: kbver.NewForConfigOrDie(config), + kubeClients: kubeClient, + vkClients: vkClient, + kbClients: kbClient, queue: workqueue.NewRateLimitingQueue(workqueue.DefaultControllerRateLimiter()), commandQueue: workqueue.NewRateLimitingQueue(workqueue.DefaultControllerRateLimiter()), cache: jobcache.New(), diff --git a/pkg/controllers/queue/queue_controller.go b/pkg/controllers/queue/queue_controller.go new file mode 100644 index 0000000000..2f054c6590 --- /dev/null +++ b/pkg/controllers/queue/queue_controller.go @@ -0,0 +1,267 @@ +/* +Copyright 2019 The Volcano 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 queue + +import ( + "sync" + + "github.com/golang/glog" + "k8s.io/apimachinery/pkg/api/errors" + "k8s.io/apimachinery/pkg/util/wait" + "k8s.io/client-go/kubernetes" + "k8s.io/client-go/tools/cache" + "k8s.io/client-go/util/workqueue" + + kbv1alpha1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1" + kbclientset "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned" + kbinformerfactory "github.com/kubernetes-sigs/kube-batch/pkg/client/informers/externalversions" + kblister "github.com/kubernetes-sigs/kube-batch/pkg/client/listers/scheduling/v1alpha1" +) + +// Controller manages queue status. +type Controller struct { + kubeClient kubernetes.Interface + kbClient kbclientset.Interface + + // queueLister + queueLister kblister.QueueLister + queueSynced cache.InformerSynced + + // podGroup lister + pgLister kblister.PodGroupLister + pgSynced cache.InformerSynced + + // queues that need to be updated. + queue workqueue.RateLimitingInterface + + pgMutex sync.RWMutex + podGroups map[string]map[string]struct{} +} + +// NewQueueController creates a QueueController +func NewQueueController( + kubeClient kubernetes.Interface, + kbClient kbclientset.Interface, +) *Controller { + factory := kbinformerfactory.NewSharedInformerFactory(kbClient, 0) + queueInformer := factory.Scheduling().V1alpha1().Queues() + pgInformer := factory.Scheduling().V1alpha1().PodGroups() + + c := &Controller{ + kubeClient: kubeClient, + kbClient: kbClient, + + queueLister: queueInformer.Lister(), + queueSynced: queueInformer.Informer().HasSynced, + + pgLister: pgInformer.Lister(), + pgSynced: pgInformer.Informer().HasSynced, + + queue: workqueue.NewRateLimitingQueue(workqueue.DefaultControllerRateLimiter()), + podGroups: make(map[string]map[string]struct{}), + } + + queueInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: c.addQueue, + DeleteFunc: c.deleteQueue, + }) + + pgInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: c.addPodGroup, + UpdateFunc: c.updatePodGroup, + DeleteFunc: c.deletePodGroup, + }) + + return c +} + +// Run starts QueueController +func (c *Controller) Run(stopCh <-chan struct{}) { + + if !cache.WaitForCacheSync(stopCh, c.queueSynced, c.pgSynced) { + glog.Errorf("unable to sync caches for queue controller") + return + } + + go wait.Until(c.worker, 0, stopCh) + glog.Infof("QueueController is running ...... ") +} + +// worker runs a worker thread that just dequeues items, processes them, and +// marks them done. You may run as many of these in parallel as you wish; the +// workqueue guarantees that they will not end up processing the same `queue` +// at the same time. +func (c *Controller) worker() { + for c.processNextWorkItem() { + } +} + +func (c *Controller) processNextWorkItem() bool { + eKey, quit := c.queue.Get() + if quit { + return false + } + defer c.queue.Done(eKey) + + if err := c.syncQueue(eKey.(string)); err != nil { + glog.V(2).Infof("Error syncing queues %q, retrying. Error: %v", eKey, err) + c.queue.AddRateLimited(eKey) + return true + } + + c.queue.Forget(eKey) + return true +} + +func (c *Controller) syncQueue(key string) error { + glog.V(5).Infoln("Begin sync queue") + + var pending, running, unknown int32 + c.pgMutex.RLock() + podGroups := make([]string, len(c.podGroups[key])) + if c.podGroups[key] == nil { + glog.V(2).Infoln("queue %s has not been seen or deleted") + return nil + } + for pgKey := range c.podGroups[key] { + podGroups = append(podGroups, pgKey) + } + c.pgMutex.RUnlock() + + for _, pgKey := range podGroups { + // Ignore error here, tt can not occur. + ns, name, _ := cache.SplitMetaNamespaceKey(pgKey) + + pg, err := c.pgLister.PodGroups(ns).Get(name) + if err != nil { + return err + } + + switch pg.Status.Phase { + case kbv1alpha1.PodGroupPending: + pending++ + case kbv1alpha1.PodGroupRunning: + running++ + case kbv1alpha1.PodGroupUnknown: + unknown++ + } + } + + queue, err := c.queueLister.Get(key) + if err != nil { + if errors.IsNotFound(err) { + glog.V(2).Infof("queue %s has been deleted", queue) + return nil + } + return err + } + + // ignore update when status doesnot change + if pending == queue.Status.Pending && running == queue.Status.Running && unknown == queue.Status.Unknown { + return nil + } + + newQueue := queue.DeepCopy() + newQueue.Status.Pending = pending + newQueue.Status.Running = running + newQueue.Status.Unknown = unknown + + if _, err := c.kbClient.SchedulingV1alpha1().Queues().UpdateStatus(newQueue); err != nil { + glog.Errorf("Failed to update status of Queue %s: %v", newQueue.Name, err) + return err + } + + return nil +} + +func (c *Controller) addQueue(obj interface{}) { + queue := obj.(*kbv1alpha1.Queue) + c.queue.Add(queue.Name) +} + +func (c *Controller) updateQueue(old, new interface{}) { + +} + +func (c *Controller) deleteQueue(obj interface{}) { + queue, ok := obj.(*kbv1alpha1.Queue) + if !ok { + tombstone, ok := obj.(cache.DeletedFinalStateUnknown) + if !ok { + glog.Errorf("Couldn't get object from tombstone %#v", obj) + return + } + queue, ok = tombstone.Obj.(*kbv1alpha1.Queue) + if !ok { + glog.Errorf("Tombstone contained object that is not a Queue: %#v", obj) + return + } + } + + c.pgMutex.Lock() + delete(c.podGroups, queue.Name) + c.pgMutex.Unlock() +} + +func (c *Controller) addPodGroup(obj interface{}) { + pg := obj.(*kbv1alpha1.PodGroup) + key, _ := cache.MetaNamespaceKeyFunc(obj) + + c.pgMutex.Lock() + if c.podGroups[pg.Spec.Queue] == nil { + c.podGroups[pg.Spec.Queue] = make(map[string]struct{}) + } + c.podGroups[pg.Spec.Queue][key] = struct{}{} + c.pgMutex.Unlock() + + // enqueue + c.queue.Add(pg.Spec.Queue) +} + +func (c *Controller) updatePodGroup(old, new interface{}) { + oldPG := old.(*kbv1alpha1.PodGroup) + newPG := new.(*kbv1alpha1.PodGroup) + + // Note: we have no use case update PodGroup.Spec.Queue + // So do not consider it here. + if oldPG.Status.Phase != newPG.Status.Phase { + // enqueue + c.queue.Add(newPG.Spec.Queue) + } + +} + +func (c *Controller) deletePodGroup(obj interface{}) { + pg, ok := obj.(*kbv1alpha1.PodGroup) + if !ok { + tombstone, ok := obj.(cache.DeletedFinalStateUnknown) + if !ok { + glog.Errorf("Couldn't get object from tombstone %#v", obj) + return + } + pg, ok = tombstone.Obj.(*kbv1alpha1.PodGroup) + if !ok { + glog.Errorf("Tombstone contained object that is not a PodGroup: %#v", obj) + return + } + } + + key, _ := cache.MetaNamespaceKeyFunc(obj) + delete(c.podGroups[pg.Spec.Queue], key) + + c.queue.Add(pg.Spec.Queue) +} From 90d545b21cdfbe7842ddc66f0526962fac15d0b4 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Mon, 6 May 2019 09:43:37 +0800 Subject: [PATCH 02/68] update: add informer run --- pkg/controllers/queue/queue_controller.go | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/pkg/controllers/queue/queue_controller.go b/pkg/controllers/queue/queue_controller.go index 2f054c6590..56b87ad4f3 100644 --- a/pkg/controllers/queue/queue_controller.go +++ b/pkg/controllers/queue/queue_controller.go @@ -29,6 +29,7 @@ import ( kbv1alpha1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1" kbclientset "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned" kbinformerfactory "github.com/kubernetes-sigs/kube-batch/pkg/client/informers/externalversions" + kbinformer "github.com/kubernetes-sigs/kube-batch/pkg/client/informers/externalversions/scheduling/v1alpha1" kblister "github.com/kubernetes-sigs/kube-batch/pkg/client/listers/scheduling/v1alpha1" ) @@ -37,6 +38,10 @@ type Controller struct { kubeClient kubernetes.Interface kbClient kbclientset.Interface + // informer + queueInformer kbinformer.QueueInformer + pgInformer kbinformer.PodGroupInformer + // queueLister queueLister kblister.QueueLister queueSynced cache.InformerSynced @@ -60,11 +65,13 @@ func NewQueueController( factory := kbinformerfactory.NewSharedInformerFactory(kbClient, 0) queueInformer := factory.Scheduling().V1alpha1().Queues() pgInformer := factory.Scheduling().V1alpha1().PodGroups() - c := &Controller{ kubeClient: kubeClient, kbClient: kbClient, + queueInformer: queueInformer, + pgInformer: pgInformer, + queueLister: queueInformer.Lister(), queueSynced: queueInformer.Informer().HasSynced, @@ -92,6 +99,9 @@ func NewQueueController( // Run starts QueueController func (c *Controller) Run(stopCh <-chan struct{}) { + go c.queueInformer.Informer().Run(stopCh) + go c.pgInformer.Informer().Run(stopCh) + if !cache.WaitForCacheSync(stopCh, c.queueSynced, c.pgSynced) { glog.Errorf("unable to sync caches for queue controller") return From 1c0e3f903a1a179d998a45521daa4a4804380f49 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Sun, 5 May 2019 18:44:17 +0800 Subject: [PATCH 03/68] Add queue controller --- cmd/controllers/app/server.go | 15 +- pkg/controllers/job/job_controller.go | 19 +- pkg/controllers/queue/queue_controller.go | 267 ++++++++++++++++++++++ 3 files changed, 289 insertions(+), 12 deletions(-) create mode 100644 pkg/controllers/queue/queue_controller.go diff --git a/cmd/controllers/app/server.go b/cmd/controllers/app/server.go index a0e1f1e9b0..8d0e93a736 100644 --- a/cmd/controllers/app/server.go +++ b/cmd/controllers/app/server.go @@ -36,8 +36,12 @@ import ( "k8s.io/client-go/tools/leaderelection/resourcelock" "k8s.io/client-go/tools/record" + kbver "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned" + "volcano.sh/volcano/cmd/controllers/app/options" + vkclient "volcano.sh/volcano/pkg/client/clientset/versioned" "volcano.sh/volcano/pkg/controllers/job" + "volcano.sh/volcano/pkg/controllers/queue" ) const ( @@ -72,10 +76,17 @@ func Run(opt *options.ServerOption) error { return err } - jobController := job.NewJobController(config) + // TODO: add user agent for different controllers + kubeClient := clientset.NewForConfigOrDie(config) + kbClient := kbver.NewForConfigOrDie(config) + vkClient := vkclient.NewForConfigOrDie(config) + + jobController := job.NewJobController(kubeClient, kbClient, vkClient) + queueController := queue.NewQueueController(kubeClient, kbClient) run := func(ctx context.Context) { - jobController.Run(ctx.Done()) + go jobController.Run(ctx.Done()) + go queueController.Run(ctx.Done()) <-ctx.Done() } diff --git a/pkg/controllers/job/job_controller.go b/pkg/controllers/job/job_controller.go index 7ecdd999f2..76f73f09bb 100644 --- a/pkg/controllers/job/job_controller.go +++ b/pkg/controllers/job/job_controller.go @@ -29,7 +29,6 @@ import ( corev1 "k8s.io/client-go/kubernetes/typed/core/v1" corelisters "k8s.io/client-go/listers/core/v1" pclister "k8s.io/client-go/listers/scheduling/v1beta1" - "k8s.io/client-go/rest" "k8s.io/client-go/tools/cache" "k8s.io/client-go/tools/record" "k8s.io/client-go/util/workqueue" @@ -53,7 +52,6 @@ import ( // Controller the Job Controller type type Controller struct { - config *rest.Config kubeClients *kubernetes.Clientset vkClients *vkver.Clientset kbClients *kbver.Clientset @@ -102,21 +100,22 @@ type Controller struct { } // NewJobController create new Job Controller -func NewJobController(config *rest.Config) *Controller { - - kubeClients := kubernetes.NewForConfigOrDie(config) +func NewJobController( + kubeClient *kubernetes.Clientset, + kbClient *kbver.Clientset, + vkClient *vkver.Clientset, +) *Controller { //Initialize event client eventBroadcaster := record.NewBroadcaster() eventBroadcaster.StartLogging(glog.Infof) - eventBroadcaster.StartRecordingToSink(&corev1.EventSinkImpl{Interface: kubeClients.CoreV1().Events("")}) + eventBroadcaster.StartRecordingToSink(&corev1.EventSinkImpl{Interface: kubeClient.CoreV1().Events("")}) recorder := eventBroadcaster.NewRecorder(vkscheme.Scheme, v1.EventSource{Component: "vk-controller"}) cc := &Controller{ - config: config, - kubeClients: kubeClients, - vkClients: vkver.NewForConfigOrDie(config), - kbClients: kbver.NewForConfigOrDie(config), + kubeClients: kubeClient, + vkClients: vkClient, + kbClients: kbClient, queue: workqueue.NewRateLimitingQueue(workqueue.DefaultControllerRateLimiter()), commandQueue: workqueue.NewRateLimitingQueue(workqueue.DefaultControllerRateLimiter()), cache: jobcache.New(), diff --git a/pkg/controllers/queue/queue_controller.go b/pkg/controllers/queue/queue_controller.go new file mode 100644 index 0000000000..2f054c6590 --- /dev/null +++ b/pkg/controllers/queue/queue_controller.go @@ -0,0 +1,267 @@ +/* +Copyright 2019 The Volcano 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 queue + +import ( + "sync" + + "github.com/golang/glog" + "k8s.io/apimachinery/pkg/api/errors" + "k8s.io/apimachinery/pkg/util/wait" + "k8s.io/client-go/kubernetes" + "k8s.io/client-go/tools/cache" + "k8s.io/client-go/util/workqueue" + + kbv1alpha1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1" + kbclientset "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned" + kbinformerfactory "github.com/kubernetes-sigs/kube-batch/pkg/client/informers/externalversions" + kblister "github.com/kubernetes-sigs/kube-batch/pkg/client/listers/scheduling/v1alpha1" +) + +// Controller manages queue status. +type Controller struct { + kubeClient kubernetes.Interface + kbClient kbclientset.Interface + + // queueLister + queueLister kblister.QueueLister + queueSynced cache.InformerSynced + + // podGroup lister + pgLister kblister.PodGroupLister + pgSynced cache.InformerSynced + + // queues that need to be updated. + queue workqueue.RateLimitingInterface + + pgMutex sync.RWMutex + podGroups map[string]map[string]struct{} +} + +// NewQueueController creates a QueueController +func NewQueueController( + kubeClient kubernetes.Interface, + kbClient kbclientset.Interface, +) *Controller { + factory := kbinformerfactory.NewSharedInformerFactory(kbClient, 0) + queueInformer := factory.Scheduling().V1alpha1().Queues() + pgInformer := factory.Scheduling().V1alpha1().PodGroups() + + c := &Controller{ + kubeClient: kubeClient, + kbClient: kbClient, + + queueLister: queueInformer.Lister(), + queueSynced: queueInformer.Informer().HasSynced, + + pgLister: pgInformer.Lister(), + pgSynced: pgInformer.Informer().HasSynced, + + queue: workqueue.NewRateLimitingQueue(workqueue.DefaultControllerRateLimiter()), + podGroups: make(map[string]map[string]struct{}), + } + + queueInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: c.addQueue, + DeleteFunc: c.deleteQueue, + }) + + pgInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: c.addPodGroup, + UpdateFunc: c.updatePodGroup, + DeleteFunc: c.deletePodGroup, + }) + + return c +} + +// Run starts QueueController +func (c *Controller) Run(stopCh <-chan struct{}) { + + if !cache.WaitForCacheSync(stopCh, c.queueSynced, c.pgSynced) { + glog.Errorf("unable to sync caches for queue controller") + return + } + + go wait.Until(c.worker, 0, stopCh) + glog.Infof("QueueController is running ...... ") +} + +// worker runs a worker thread that just dequeues items, processes them, and +// marks them done. You may run as many of these in parallel as you wish; the +// workqueue guarantees that they will not end up processing the same `queue` +// at the same time. +func (c *Controller) worker() { + for c.processNextWorkItem() { + } +} + +func (c *Controller) processNextWorkItem() bool { + eKey, quit := c.queue.Get() + if quit { + return false + } + defer c.queue.Done(eKey) + + if err := c.syncQueue(eKey.(string)); err != nil { + glog.V(2).Infof("Error syncing queues %q, retrying. Error: %v", eKey, err) + c.queue.AddRateLimited(eKey) + return true + } + + c.queue.Forget(eKey) + return true +} + +func (c *Controller) syncQueue(key string) error { + glog.V(5).Infoln("Begin sync queue") + + var pending, running, unknown int32 + c.pgMutex.RLock() + podGroups := make([]string, len(c.podGroups[key])) + if c.podGroups[key] == nil { + glog.V(2).Infoln("queue %s has not been seen or deleted") + return nil + } + for pgKey := range c.podGroups[key] { + podGroups = append(podGroups, pgKey) + } + c.pgMutex.RUnlock() + + for _, pgKey := range podGroups { + // Ignore error here, tt can not occur. + ns, name, _ := cache.SplitMetaNamespaceKey(pgKey) + + pg, err := c.pgLister.PodGroups(ns).Get(name) + if err != nil { + return err + } + + switch pg.Status.Phase { + case kbv1alpha1.PodGroupPending: + pending++ + case kbv1alpha1.PodGroupRunning: + running++ + case kbv1alpha1.PodGroupUnknown: + unknown++ + } + } + + queue, err := c.queueLister.Get(key) + if err != nil { + if errors.IsNotFound(err) { + glog.V(2).Infof("queue %s has been deleted", queue) + return nil + } + return err + } + + // ignore update when status doesnot change + if pending == queue.Status.Pending && running == queue.Status.Running && unknown == queue.Status.Unknown { + return nil + } + + newQueue := queue.DeepCopy() + newQueue.Status.Pending = pending + newQueue.Status.Running = running + newQueue.Status.Unknown = unknown + + if _, err := c.kbClient.SchedulingV1alpha1().Queues().UpdateStatus(newQueue); err != nil { + glog.Errorf("Failed to update status of Queue %s: %v", newQueue.Name, err) + return err + } + + return nil +} + +func (c *Controller) addQueue(obj interface{}) { + queue := obj.(*kbv1alpha1.Queue) + c.queue.Add(queue.Name) +} + +func (c *Controller) updateQueue(old, new interface{}) { + +} + +func (c *Controller) deleteQueue(obj interface{}) { + queue, ok := obj.(*kbv1alpha1.Queue) + if !ok { + tombstone, ok := obj.(cache.DeletedFinalStateUnknown) + if !ok { + glog.Errorf("Couldn't get object from tombstone %#v", obj) + return + } + queue, ok = tombstone.Obj.(*kbv1alpha1.Queue) + if !ok { + glog.Errorf("Tombstone contained object that is not a Queue: %#v", obj) + return + } + } + + c.pgMutex.Lock() + delete(c.podGroups, queue.Name) + c.pgMutex.Unlock() +} + +func (c *Controller) addPodGroup(obj interface{}) { + pg := obj.(*kbv1alpha1.PodGroup) + key, _ := cache.MetaNamespaceKeyFunc(obj) + + c.pgMutex.Lock() + if c.podGroups[pg.Spec.Queue] == nil { + c.podGroups[pg.Spec.Queue] = make(map[string]struct{}) + } + c.podGroups[pg.Spec.Queue][key] = struct{}{} + c.pgMutex.Unlock() + + // enqueue + c.queue.Add(pg.Spec.Queue) +} + +func (c *Controller) updatePodGroup(old, new interface{}) { + oldPG := old.(*kbv1alpha1.PodGroup) + newPG := new.(*kbv1alpha1.PodGroup) + + // Note: we have no use case update PodGroup.Spec.Queue + // So do not consider it here. + if oldPG.Status.Phase != newPG.Status.Phase { + // enqueue + c.queue.Add(newPG.Spec.Queue) + } + +} + +func (c *Controller) deletePodGroup(obj interface{}) { + pg, ok := obj.(*kbv1alpha1.PodGroup) + if !ok { + tombstone, ok := obj.(cache.DeletedFinalStateUnknown) + if !ok { + glog.Errorf("Couldn't get object from tombstone %#v", obj) + return + } + pg, ok = tombstone.Obj.(*kbv1alpha1.PodGroup) + if !ok { + glog.Errorf("Tombstone contained object that is not a PodGroup: %#v", obj) + return + } + } + + key, _ := cache.MetaNamespaceKeyFunc(obj) + delete(c.podGroups[pg.Spec.Queue], key) + + c.queue.Add(pg.Spec.Queue) +} From 231a103089049a1c141ff65aa36c9fe7f179a4b3 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Mon, 6 May 2019 09:43:37 +0800 Subject: [PATCH 04/68] update: add informer run --- pkg/controllers/queue/queue_controller.go | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/pkg/controllers/queue/queue_controller.go b/pkg/controllers/queue/queue_controller.go index 2f054c6590..56b87ad4f3 100644 --- a/pkg/controllers/queue/queue_controller.go +++ b/pkg/controllers/queue/queue_controller.go @@ -29,6 +29,7 @@ import ( kbv1alpha1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1" kbclientset "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned" kbinformerfactory "github.com/kubernetes-sigs/kube-batch/pkg/client/informers/externalversions" + kbinformer "github.com/kubernetes-sigs/kube-batch/pkg/client/informers/externalversions/scheduling/v1alpha1" kblister "github.com/kubernetes-sigs/kube-batch/pkg/client/listers/scheduling/v1alpha1" ) @@ -37,6 +38,10 @@ type Controller struct { kubeClient kubernetes.Interface kbClient kbclientset.Interface + // informer + queueInformer kbinformer.QueueInformer + pgInformer kbinformer.PodGroupInformer + // queueLister queueLister kblister.QueueLister queueSynced cache.InformerSynced @@ -60,11 +65,13 @@ func NewQueueController( factory := kbinformerfactory.NewSharedInformerFactory(kbClient, 0) queueInformer := factory.Scheduling().V1alpha1().Queues() pgInformer := factory.Scheduling().V1alpha1().PodGroups() - c := &Controller{ kubeClient: kubeClient, kbClient: kbClient, + queueInformer: queueInformer, + pgInformer: pgInformer, + queueLister: queueInformer.Lister(), queueSynced: queueInformer.Informer().HasSynced, @@ -92,6 +99,9 @@ func NewQueueController( // Run starts QueueController func (c *Controller) Run(stopCh <-chan struct{}) { + go c.queueInformer.Informer().Run(stopCh) + go c.pgInformer.Informer().Run(stopCh) + if !cache.WaitForCacheSync(stopCh, c.queueSynced, c.pgSynced) { glog.Errorf("unable to sync caches for queue controller") return From 77e1c8df801bf4df24d026f30b6d0c6658f43f4e Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Wed, 8 May 2019 19:02:41 +0800 Subject: [PATCH 05/68] add e2e --- test/e2e/queue.go | 23 +++++++++++++++++++++++ test/e2e/util.go | 4 ++++ 2 files changed, 27 insertions(+) diff --git a/test/e2e/queue.go b/test/e2e/queue.go index 9921592968..6183638293 100644 --- a/test/e2e/queue.go +++ b/test/e2e/queue.go @@ -19,6 +19,8 @@ package e2e import ( "fmt" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" ) @@ -65,6 +67,27 @@ var _ = Describe("Queue E2E Test", func() { err = waitTasksReady(context, job1, expected) Expect(err).NotTo(HaveOccurred()) + + // Test Queue status + spec = &jobSpec{ + name: "", + queue: defaultQueue1, + tasks: []taskSpec{ + { + img: defaultNginxImage, + req: slot, + min: rep * 2, + rep: rep * 2, + }, + }, + } + + err = waitQueueStatus(func() (bool, error) { + queue, err := context.kbclient.SchedulingV1alpha1().Queues().Get(defaultQueue1, metav1.GetOptions{}) + Expect(err).NotTo(HaveOccurred()) + return queue.Status.Running == 1 && queue.Status.Pending == 1, nil + }) + Expect(err).NotTo(HaveOccurred()) }) }) diff --git a/test/e2e/util.go b/test/e2e/util.go index cd9149889d..f95586ab2c 100644 --- a/test/e2e/util.go +++ b/test/e2e/util.go @@ -543,6 +543,10 @@ func waitJobUnschedulable(ctx *context, job *vkv1.Job) error { return wait.Poll(10*time.Second, oneMinute, jobUnschedulable(ctx, job, now)) } +func waitQueueStatus(condition func() (bool, error)) error { + return wait.Poll(100*time.Millisecond, oneMinute, condition) +} + func createContainers(img, command, workingDir string, req v1.ResourceList, hostport int32) []v1.Container { var imageRepo []string container := v1.Container{ From 5c47c194e00a0737a07002e542f6bb1035578357 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Wed, 8 May 2019 19:05:23 +0800 Subject: [PATCH 06/68] rm updateQueue --- pkg/controllers/queue/queue_controller.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/pkg/controllers/queue/queue_controller.go b/pkg/controllers/queue/queue_controller.go index 56b87ad4f3..dc3d62cab1 100644 --- a/pkg/controllers/queue/queue_controller.go +++ b/pkg/controllers/queue/queue_controller.go @@ -203,10 +203,6 @@ func (c *Controller) addQueue(obj interface{}) { c.queue.Add(queue.Name) } -func (c *Controller) updateQueue(old, new interface{}) { - -} - func (c *Controller) deleteQueue(obj interface{}) { queue, ok := obj.(*kbv1alpha1.Queue) if !ok { From 167159aa3259a402198553e18ba1793ece14f8ee Mon Sep 17 00:00:00 2001 From: TommyLike Date: Thu, 9 May 2019 11:16:00 +0800 Subject: [PATCH 07/68] Print detail when timeout waiting --- test/e2e/util.go | 99 +++++++++++++++++++++++++++++++++++------------- 1 file changed, 73 insertions(+), 26 deletions(-) diff --git a/test/e2e/util.go b/test/e2e/util.go index cd9149889d..0bc54e4c26 100644 --- a/test/e2e/util.go +++ b/test/e2e/util.go @@ -55,6 +55,7 @@ var ( ) const ( + timeOutMessage = "timed out waiting for the condition" workerPriority = "worker-pri" masterPriority = "master-pri" defaultNginxImage = "nginx:1.14" @@ -364,8 +365,9 @@ func createJobInner(context *context, jobSpec *jobSpec) (*vkv1.Job, error) { return context.vkclient.BatchV1alpha1().Jobs(job.Namespace).Create(job) } -func taskPhase(ctx *context, job *vkv1.Job, phase []v1.PodPhase, taskNum int) wait.ConditionFunc { - return func() (bool, error) { +func waitTaskPhase(ctx *context, job *vkv1.Job, phase []v1.PodPhase, taskNum int) error { + var additionalError error + err := wait.Poll(100*time.Millisecond, oneMinute, func() (bool, error) { pods, err := ctx.kubeclient.CoreV1().Pods(job.Namespace).List(metav1.ListOptions{}) Expect(err).NotTo(HaveOccurred()) @@ -383,34 +385,53 @@ func taskPhase(ctx *context, job *vkv1.Job, phase []v1.PodPhase, taskNum int) wa } } - return taskNum <= readyTaskNum, nil + ready := taskNum <= readyTaskNum + if !ready { + additionalError = fmt.Errorf("expected job '%s' to have %d ready pods, actual got %d", job.Name, + taskNum, + readyTaskNum) + } + return ready, nil + }) + if err != nil && strings.Contains(err.Error(), timeOutMessage) { + return fmt.Errorf("[Wait time out]: %s", additionalError) } + return err } -func jobUnschedulable(ctx *context, job *vkv1.Job, time time.Time) wait.ConditionFunc { +func jobUnschedulable(ctx *context, job *vkv1.Job, now time.Time) error { + var additionalError error // TODO(k82cn): check Job's Condition instead of PodGroup's event. - return func() (bool, error) { + err := wait.Poll(10*time.Second, oneMinute, func() (bool, error) { pg, err := ctx.kbclient.SchedulingV1alpha1().PodGroups(job.Namespace).Get(job.Name, metav1.GetOptions{}) if err != nil { + additionalError = fmt.Errorf("expected to have job's podgroup %s created, actual got error %s", + job.Name, err.Error()) return false, nil } events, err := ctx.kubeclient.CoreV1().Events(pg.Namespace).List(metav1.ListOptions{}) if err != nil { + additionalError = fmt.Errorf("expected to have events for job %s, actual got error %s", + job.Name, err.Error()) return false, nil } - for _, event := range events.Items { target := event.InvolvedObject if strings.HasPrefix(target.Name, pg.Name) && target.Namespace == pg.Namespace { - if event.Reason == string("Unschedulable") && event.LastTimestamp.After(time) { + if event.Reason == string("Unschedulable") && event.LastTimestamp.After(now) { return true, nil } } } - + additionalError = fmt.Errorf( + "expected to have 'Unschedulable' events for podgroup %s, actual got nothing", job.Name) return false, nil + }) + if err != nil && strings.Contains(err.Error(), timeOutMessage) { + return fmt.Errorf("[Wait time out]: %s", additionalError) } + return err } func jobEvicted(ctx *context, job *vkv1.Job, time time.Time) wait.ConditionFunc { @@ -447,7 +468,7 @@ func waitJobPhases(ctx *context, job *vkv1.Job, phases []vkv1.JobPhase) error { func waitJobStates(ctx *context, job *vkv1.Job, phases []vkv1.JobPhase) error { for _, phase := range phases { - err := wait.Poll(100*time.Millisecond, oneMinute, jobPhaseExpect(ctx, job, phase)) + err := waitJobPhaseExpect(ctx, job, phase) if err != nil { return err } @@ -456,19 +477,21 @@ func waitJobStates(ctx *context, job *vkv1.Job, phases []vkv1.JobPhase) error { } func waitJobPhase(ctx *context, job *vkv1.Job, phase vkv1.JobPhase) error { + var additionalError error total := int32(0) for _, task := range job.Spec.Tasks { total += task.Replicas } - - return wait.Poll(100*time.Millisecond, twoMinute, func() (bool, error) { + err := wait.Poll(100*time.Millisecond, oneMinute, func() (bool, error) { newJob, err := ctx.vkclient.BatchV1alpha1().Jobs(job.Namespace).Get(job.Name, metav1.GetOptions{}) Expect(err).NotTo(HaveOccurred()) if newJob.Status.State.Phase != phase { + additionalError = fmt.Errorf( + "expected job '%s' to be in status %s, actual get %s", + job.Name, phase, newJob.Status.State.Phase) return false, nil } - var flag = false switch phase { case vkv1.Pending: @@ -491,8 +514,25 @@ func waitJobPhase(ctx *context, job *vkv1.Job, phase vkv1.JobPhase) error { return false, fmt.Errorf("unknown phase %s", phase) } + if !flag { + additionalError = fmt.Errorf( + "expected job '%s' to be in status %s, actual detail status %s", + job.Name, phase, getJobStatusDetail(job)) + } + return flag, nil }) + if err != nil && strings.Contains(err.Error(), timeOutMessage) { + return fmt.Errorf("[Wait time out]: %s", additionalError) + } + return err +} + +func getJobStatusDetail(job *vkv1.Job) string { + return fmt.Sprintf("\nName: %s\n Phase: %s\nPending: %d"+ + "\nRunning: %d\nSucceeded: %d\nTerminating: %d\nFailed: %d\n ", + job.Name, job.Status.State.Phase, job.Status.Pending, job.Status.Running, + job.Status.Succeeded, job.Status.Terminating, job.Status.Failed) } func waitJobReady(ctx *context, job *vkv1.Job) error { @@ -500,47 +540,54 @@ func waitJobReady(ctx *context, job *vkv1.Job) error { } func waitJobPending(ctx *context, job *vkv1.Job) error { - return wait.Poll(100*time.Millisecond, oneMinute, taskPhase(ctx, job, - []v1.PodPhase{v1.PodPending}, int(job.Spec.MinAvailable))) + return waitTaskPhase(ctx, job, []v1.PodPhase{v1.PodPending}, int(job.Spec.MinAvailable)) } func waitTasksReady(ctx *context, job *vkv1.Job, taskNum int) error { - return wait.Poll(100*time.Millisecond, oneMinute, taskPhase(ctx, job, - []v1.PodPhase{v1.PodRunning, v1.PodSucceeded}, taskNum)) + return waitTaskPhase(ctx, job, []v1.PodPhase{v1.PodRunning, v1.PodSucceeded}, taskNum) } func waitTasksPending(ctx *context, job *vkv1.Job, taskNum int) error { - return wait.Poll(100*time.Millisecond, oneMinute, taskPhase(ctx, job, - []v1.PodPhase{v1.PodPending}, taskNum)) + return waitTaskPhase(ctx, job, []v1.PodPhase{v1.PodPending}, taskNum) } func waitJobStateReady(ctx *context, job *vkv1.Job) error { - return wait.Poll(100*time.Millisecond, oneMinute, jobPhaseExpect(ctx, job, vkv1.Running)) + return waitJobPhaseExpect(ctx, job, vkv1.Running) } func waitJobStatePending(ctx *context, job *vkv1.Job) error { - return wait.Poll(100*time.Millisecond, oneMinute, jobPhaseExpect(ctx, job, vkv1.Pending)) + return waitJobPhaseExpect(ctx, job, vkv1.Pending) } func waitJobStateInqueue(ctx *context, job *vkv1.Job) error { - return wait.Poll(100*time.Millisecond, oneMinute, jobPhaseExpect(ctx, job, vkv1.Inqueue)) + return waitJobPhaseExpect(ctx, job, vkv1.Inqueue) } func waitJobStateAborted(ctx *context, job *vkv1.Job) error { - return wait.Poll(100*time.Millisecond, oneMinute, jobPhaseExpect(ctx, job, vkv1.Aborted)) + return waitJobPhaseExpect(ctx, job, vkv1.Aborted) } -func jobPhaseExpect(ctx *context, job *vkv1.Job, state vkv1.JobPhase) wait.ConditionFunc { - return func() (bool, error) { +func waitJobPhaseExpect(ctx *context, job *vkv1.Job, state vkv1.JobPhase) error { + var additionalError error + err := wait.Poll(100*time.Millisecond, oneMinute, func() (bool, error) { job, err := ctx.vkclient.BatchV1alpha1().Jobs(job.Namespace).Get(job.Name, metav1.GetOptions{}) Expect(err).NotTo(HaveOccurred()) - return job.Status.State.Phase == state, err + expected := job.Status.State.Phase == state + if !expected { + additionalError = fmt.Errorf("expected job '%s' phase in %s, actual got %s", job.Name, + state, job.Status.State.Phase) + } + return expected, nil + }) + if err != nil && strings.Contains(err.Error(), timeOutMessage) { + return fmt.Errorf("[Wait time out]: %s", additionalError) } + return err } func waitJobUnschedulable(ctx *context, job *vkv1.Job) error { now := time.Now() - return wait.Poll(10*time.Second, oneMinute, jobUnschedulable(ctx, job, now)) + return jobUnschedulable(ctx, job, now) } func createContainers(img, command, workingDir string, req v1.ResourceList, hostport int32) []v1.Container { From ce72d168054f0a3f6b2e162adf45f7904dbafecd Mon Sep 17 00:00:00 2001 From: TommyLike Date: Wed, 8 May 2019 15:06:30 +0800 Subject: [PATCH 08/68] Update volcano crds --- .../integrations/tensorflow/tf-example.yaml | 1 + .../duplicatedPolicyEvent-webhook-deny.yaml | 0 .../duplicatedTaskName-webhook-deny.yaml | 0 .../minAvailable-webhook-deny.yaml | 0 example/job.yaml | 56 +++++++++-------- example/kube-batch-conf.yaml | 11 ---- example/openmpi-hello.yaml | 56 ----------------- example/role.yaml | 12 ---- example/tensorflow-benchmark.yaml | 61 ------------------- .../volcano/templates/batch_v1alpha1_job.yaml | 21 +++++++ 10 files changed, 53 insertions(+), 165 deletions(-) rename example/{ => invalid_jobs}/duplicatedPolicyEvent-webhook-deny.yaml (100%) rename example/{ => invalid_jobs}/duplicatedTaskName-webhook-deny.yaml (100%) rename example/{ => invalid_jobs}/minAvailable-webhook-deny.yaml (100%) delete mode 100644 example/kube-batch-conf.yaml delete mode 100644 example/openmpi-hello.yaml delete mode 100644 example/role.yaml delete mode 100644 example/tensorflow-benchmark.yaml diff --git a/example/integrations/tensorflow/tf-example.yaml b/example/integrations/tensorflow/tf-example.yaml index d2090bbf68..05bc67d781 100644 --- a/example/integrations/tensorflow/tf-example.yaml +++ b/example/integrations/tensorflow/tf-example.yaml @@ -37,6 +37,7 @@ spec: minAvailable: 2 schedulerName: kube-batch plugins: + env: [] svc: [] policies: - event: PodEvicted diff --git a/example/duplicatedPolicyEvent-webhook-deny.yaml b/example/invalid_jobs/duplicatedPolicyEvent-webhook-deny.yaml similarity index 100% rename from example/duplicatedPolicyEvent-webhook-deny.yaml rename to example/invalid_jobs/duplicatedPolicyEvent-webhook-deny.yaml diff --git a/example/duplicatedTaskName-webhook-deny.yaml b/example/invalid_jobs/duplicatedTaskName-webhook-deny.yaml similarity index 100% rename from example/duplicatedTaskName-webhook-deny.yaml rename to example/invalid_jobs/duplicatedTaskName-webhook-deny.yaml diff --git a/example/minAvailable-webhook-deny.yaml b/example/invalid_jobs/minAvailable-webhook-deny.yaml similarity index 100% rename from example/minAvailable-webhook-deny.yaml rename to example/invalid_jobs/minAvailable-webhook-deny.yaml diff --git a/example/job.yaml b/example/job.yaml index ace1d986f6..c5247f8e7b 100644 --- a/example/job.yaml +++ b/example/job.yaml @@ -6,30 +6,36 @@ spec: minAvailable: 3 schedulerName: kube-batch policies: - - event: PodEvicted - action: RestartJob + - event: PodEvicted + action: RestartJob + plugins: + ssh: [] + env: [] + svc: [] + maxRetry: 5 + queue: default volumes: - - mountPath: "/myinput" - - mountPath: "/myoutput" - volumeClaimName: "testvolumeclaimname" - volumeClaim: - accessModes: [ "ReadWriteOnce" ] - storageClassName: "my-storage-class" - resources: - requests: - storage: 1Gi + - mountPath: "/myinput" + - mountPath: "/myoutput" + volumeClaimName: "testvolumeclaimname" + volumeClaim: + accessModes: [ "ReadWriteOnce" ] + storageClassName: "my-storage-class" + resources: + requests: + storage: 1Gi tasks: - - replicas: 6 - name: "default-nginx" - template: - metadata: - name: web - spec: - containers: - - image: nginx - imagePullPolicy: IfNotPresent - name: nginx - resources: - requests: - cpu: "1" - restartPolicy: OnFailure + - replicas: 6 + name: "default-nginx" + template: + metadata: + name: web + spec: + containers: + - image: nginx + imagePullPolicy: IfNotPresent + name: nginx + resources: + requests: + cpu: "1" + restartPolicy: OnFailure diff --git a/example/kube-batch-conf.yaml b/example/kube-batch-conf.yaml deleted file mode 100644 index add2e0b890..0000000000 --- a/example/kube-batch-conf.yaml +++ /dev/null @@ -1,11 +0,0 @@ -actions: "enqueue, reclaim, allocate, backfill, preempt" -tiers: - - plugins: - - name: priority - - name: gang - - name: conformance - - plugins: - - name: drf - - name: predicates - - name: proportion - - name: nodeorder diff --git a/example/openmpi-hello.yaml b/example/openmpi-hello.yaml deleted file mode 100644 index 5f136c736a..0000000000 --- a/example/openmpi-hello.yaml +++ /dev/null @@ -1,56 +0,0 @@ -apiVersion: batch.volcano.sh/v1alpha1 -kind: Job -metadata: - name: openmpi-hello -spec: - minAvailable: 3 - schedulerName: scheduler - plugins: - ssh: [] - env: [] - svc: [] - tasks: - - replicas: 1 - name: mpimaster - policies: - - event: TaskCompleted - action: CompleteJob - template: - spec: - imagePullSecrets: - - name: default-secret - containers: - - command: - - /bin/sh - - -c - - | - MPI_HOST=`cat /etc/volcano/mpiworker.host | tr "\n" ","`; - mkdir -p /var/run/sshd; /usr/sbin/sshd; - mpiexec --allow-run-as-root --host ${MPI_HOST} -np 2 mpi_hello_world > /home/re - image: 100.125.5.235:20202/l00427178/openmpi-hello:3.28 - name: mpimaster - ports: - - containerPort: 22 - name: mpijob-port - workingDir: /home - restartPolicy: OnFailure - - replicas: 2 - name: mpiworker - template: - spec: - imagePullSecrets: - - name: default-secret - containers: - - command: - - /bin/sh - - -c - - | - mkdir -p /var/run/sshd; /usr/sbin/sshd -D; - image: 100.125.5.235:20202/l00427178/openmpi-hello:3.28 - name: mpiworker - ports: - - containerPort: 22 - name: mpijob-port - workingDir: /home - restartPolicy: OnFailure - diff --git a/example/role.yaml b/example/role.yaml deleted file mode 100644 index fe6bdfacc0..0000000000 --- a/example/role.yaml +++ /dev/null @@ -1,12 +0,0 @@ -apiVersion: rbac.authorization.k8s.io/v1beta1 -kind: ClusterRoleBinding -metadata: - name: default-as-admin -subjects: - - kind: ServiceAccount - name: default - namespace: kube-system -roleRef: - kind: ClusterRole - name: cluster-admin - apiGroup: rbac.authorization.k8s.io diff --git a/example/tensorflow-benchmark.yaml b/example/tensorflow-benchmark.yaml deleted file mode 100644 index 49912a57e3..0000000000 --- a/example/tensorflow-benchmark.yaml +++ /dev/null @@ -1,61 +0,0 @@ -apiVersion: batch.volcano.sh/v1alpha1 -kind: Job -metadata: - name: tensorflow-benchmark -spec: - minAvailable: 5 - schedulerName: scheduler - plugins: - env: [] - svc: [] - policies: - - event: PodEvicted - action: RestartJob - tasks: - - replicas: 2 - name: ps - template: - spec: - imagePullSecrets: - - name: default-secret - containers: - - command: - - sh - - -c - - | - PS_HOST=`cat /etc/volcano/ps.host | sed 's/$/&:2222/g' | tr "\n" ","`; - WORKER_HOST=`cat /etc/volcano/worker.host | sed 's/$/&:2222/g' | tr "\n" ","`; - python tf_cnn_benchmarks.py --batch_size=32 --model=resnet50 --variable_update=parameter_server --flush_stdout=true --num_gpus=1 --local_parameter_device=cpu --device=cpu --data_format=NHWC --job_name=ps --task_index=${VK_TASK_INDEX} --ps_hosts=${PS_HOST} --worker_hosts=${WORKER_HOST} - image: 100.125.5.235:20202/l00427178/tf-benchmarks-cpu:v20171202-bdab599-dirty-284af3 - name: tensorflow - ports: - - containerPort: 2222 - name: tfjob-port - resources: {} - workingDir: /opt/tf-benchmarks/scripts/tf_cnn_benchmarks - restartPolicy: OnFailure - - replicas: 3 - name: worker - policies: - - event: TaskCompleted - action: CompleteJob - template: - spec: - imagePullSecrets: - - name: default-secret - containers: - - command: - - sh - - -c - - | - PS_HOST=`cat /etc/volcano/ps.host | sed 's/$/&:2222/g' | tr "\n" ","`; - WORKER_HOST=`cat /etc/volcano/worker.host | sed 's/$/&:2222/g' | tr "\n" ","`; - python tf_cnn_benchmarks.py --batch_size=32 --model=resnet50 --variable_update=parameter_server --flush_stdout=true --num_gpus=1 --local_parameter_device=cpu --device=cpu --data_format=NHWC --job_name=worker --task_index=${VK_TASK_INDEX} --ps_hosts=${PS_HOST} --worker_hosts=${WORKER_HOST} - image: 100.125.5.235:20202/l00427178/tf-benchmarks-cpu:v20171202-bdab599-dirty-284af3 - name: tensorflow - ports: - - containerPort: 2222 - name: tfjob-port - resources: {} - workingDir: /opt/tf-benchmarks/scripts/tf_cnn_benchmarks - restartPolicy: OnFailure diff --git a/installer/chart/volcano/templates/batch_v1alpha1_job.yaml b/installer/chart/volcano/templates/batch_v1alpha1_job.yaml index ad397c069e..eb54dca40b 100644 --- a/installer/chart/volcano/templates/batch_v1alpha1_job.yaml +++ b/installer/chart/volcano/templates/batch_v1alpha1_job.yaml @@ -74,6 +74,11 @@ spec: schedulerName: description: SchedulerName is the default value of `tasks.template.spec.schedulerName`. type: string + plugins: + description: Enabled task plugins when creating job. + type: object + additionalProperties: + type: array tasks: description: Tasks specifies the task specification of Job items: @@ -111,6 +116,13 @@ spec: type: object type: object type: array + queue: + description: The name of the queue on which job should been created + type: string + maxRetry: + description: The limit for retrying submiting job, default is 3 + format: int32 + type: integer type: object status: description: Current status of Job @@ -139,6 +151,15 @@ spec: description: Job's current version format: int32 type: integer + retryCount: + description: The number that volcano retried to submit the job. + format: int32 + type: integer + ControlledResources: + description: All of the resources that are controlled by this job. + type: object + additionalProperties: + type: string state: description: Current state of Job. properties: From 5b9c74e547c691a6b4f02cef6138ad10a4e2766d Mon Sep 17 00:00:00 2001 From: TommyLike Date: Wed, 8 May 2019 17:25:12 +0800 Subject: [PATCH 09/68] Fix gen admission secret issue --- hack/run-e2e-kind.sh | 12 +++++++++--- .../gen-admission-secret/gen-admission-secret.sh | 4 ++-- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/hack/run-e2e-kind.sh b/hack/run-e2e-kind.sh index a9e37d658d..796c885ae4 100755 --- a/hack/run-e2e-kind.sh +++ b/hack/run-e2e-kind.sh @@ -64,14 +64,20 @@ function install-volcano { echo "Install volcano plugin into cluster...." helm plugin install --kubeconfig ${KUBECONFIG} installer/chart/volcano/plugins/gen-admission-secret - helm gen-admission-secret --service integration-admission-service --namespace kube-system + + #If failed to generate secret for admission service, return immediately + helm gen-admission-secret --service ${CLUSTER_NAME}-admission-service --namespace kube-system + if [[ $? != 0 ]]; then + echo "Failed to install secret for admission service, usually we need a retry." + exit 1 + fi echo "Install volcano chart" - helm install installer/chart/volcano --namespace kube-system --name integration --kubeconfig ${KUBECONFIG} --set basic.image_tag_version=${TAG} + helm install installer/chart/volcano --namespace kube-system --name ${CLUSTER_NAME} --kubeconfig ${KUBECONFIG} --set basic.image_tag_version=${TAG} --wait } function uninstall-volcano { - helm delete integration --purge --kubeconfig ${KUBECONFIG} + helm delete ${CLUSTER_NAME} --purge --kubeconfig ${KUBECONFIG} } function generate-log { diff --git a/installer/chart/volcano/plugins/gen-admission-secret/gen-admission-secret.sh b/installer/chart/volcano/plugins/gen-admission-secret/gen-admission-secret.sh index da715a9dcf..84bb061fb1 100755 --- a/installer/chart/volcano/plugins/gen-admission-secret/gen-admission-secret.sh +++ b/installer/chart/volcano/plugins/gen-admission-secret/gen-admission-secret.sh @@ -108,7 +108,7 @@ done # approve and fetch the signed certificate kubectl certificate approve ${csrName} # verify certificate has been signed -for x in $(seq 15); do +for x in $(seq 20); do serverCert=$(kubectl get csr ${csrName} -o jsonpath='{.status.certificate}') if [[ ${serverCert} != '' ]]; then break @@ -116,7 +116,7 @@ for x in $(seq 15); do sleep 1 done if [[ ${serverCert} == '' ]]; then - echo "ERROR: After approving csr ${csrName}, the signed certificate did not appear on the resource. Giving up after 15 attempts." >&2 + echo "ERROR: After approving csr ${csrName}, the signed certificate did not appear on the resource. Giving up after 20 attempts." >&2 exit 1 fi echo ${serverCert} | openssl base64 -d -A -out ${tmpdir}/server-cert.pem From 43c1bd51cd5528935d26fe9fe509790723e90f02 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Thu, 9 May 2019 10:08:43 +0800 Subject: [PATCH 10/68] Add clusterrole for queues --- installer/chart/volcano/templates/controllers.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/installer/chart/volcano/templates/controllers.yaml b/installer/chart/volcano/templates/controllers.yaml index 846ac9bd49..8e6c25c4e9 100644 --- a/installer/chart/volcano/templates/controllers.yaml +++ b/installer/chart/volcano/templates/controllers.yaml @@ -39,7 +39,7 @@ rules: resources: ["services", "configmaps"] verbs: ["get", "list", "watch", "create", "delete"] - apiGroups: ["scheduling.incubator.k8s.io"] - resources: ["podgroups"] + resources: ["podgroups", "queues"] verbs: ["get", "list", "watch", "create", "delete"] - apiGroups: ["scheduling.k8s.io"] resources: ["priorityclasses"] From f2a91f7e54313f8eaac7905986ecdca07529b97c Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Thu, 9 May 2019 16:32:38 +0800 Subject: [PATCH 11/68] fix dead lock --- pkg/controllers/queue/queue_controller.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pkg/controllers/queue/queue_controller.go b/pkg/controllers/queue/queue_controller.go index dc3d62cab1..ecf926f045 100644 --- a/pkg/controllers/queue/queue_controller.go +++ b/pkg/controllers/queue/queue_controller.go @@ -142,11 +142,12 @@ func (c *Controller) syncQueue(key string) error { var pending, running, unknown int32 c.pgMutex.RLock() - podGroups := make([]string, len(c.podGroups[key])) if c.podGroups[key] == nil { - glog.V(2).Infoln("queue %s has not been seen or deleted") + c.pgMutex.RUnlock() + glog.V(2).Infoln("queue %s has not been seen or deleted", key) return nil } + podGroups := make([]string, len(c.podGroups[key])) for pgKey := range c.podGroups[key] { podGroups = append(podGroups, pgKey) } From 942f1983edbde7563d117e9d92baea2fef1e15b9 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Thu, 9 May 2019 18:03:38 +0800 Subject: [PATCH 12/68] fix --- installer/chart/volcano/templates/controllers.yaml | 4 ++-- .../volcano/templates/scheduling_v1alpha1_queue.yaml | 8 ++++++++ pkg/controllers/queue/queue_controller.go | 2 +- test/e2e/queue.go | 6 ++++-- 4 files changed, 15 insertions(+), 5 deletions(-) diff --git a/installer/chart/volcano/templates/controllers.yaml b/installer/chart/volcano/templates/controllers.yaml index 8e6c25c4e9..dcc6357597 100644 --- a/installer/chart/volcano/templates/controllers.yaml +++ b/installer/chart/volcano/templates/controllers.yaml @@ -39,8 +39,8 @@ rules: resources: ["services", "configmaps"] verbs: ["get", "list", "watch", "create", "delete"] - apiGroups: ["scheduling.incubator.k8s.io"] - resources: ["podgroups", "queues"] - verbs: ["get", "list", "watch", "create", "delete"] + resources: ["podgroups", "queues", "queues/status"] + verbs: ["get", "list", "watch", "create", "delete", "update"] - apiGroups: ["scheduling.k8s.io"] resources: ["priorityclasses"] verbs: ["get", "list", "watch", "create", "delete"] diff --git a/installer/chart/volcano/templates/scheduling_v1alpha1_queue.yaml b/installer/chart/volcano/templates/scheduling_v1alpha1_queue.yaml index ae80ccf2b3..46bc0ade69 100644 --- a/installer/chart/volcano/templates/scheduling_v1alpha1_queue.yaml +++ b/installer/chart/volcano/templates/scheduling_v1alpha1_queue.yaml @@ -27,3 +27,11 @@ spec: type: object type: object version: v1alpha1 + subresources: + status: {} +status: + acceptedNames: + kind: "" + plural: "" + conditions: [] + storedVersions: [] diff --git a/pkg/controllers/queue/queue_controller.go b/pkg/controllers/queue/queue_controller.go index ecf926f045..0bc2bce910 100644 --- a/pkg/controllers/queue/queue_controller.go +++ b/pkg/controllers/queue/queue_controller.go @@ -147,7 +147,7 @@ func (c *Controller) syncQueue(key string) error { glog.V(2).Infoln("queue %s has not been seen or deleted", key) return nil } - podGroups := make([]string, len(c.podGroups[key])) + podGroups := make([]string, 0, len(c.podGroups[key])) for pgKey := range c.podGroups[key] { podGroups = append(podGroups, pgKey) } diff --git a/test/e2e/queue.go b/test/e2e/queue.go index 6183638293..038a70b0e6 100644 --- a/test/e2e/queue.go +++ b/test/e2e/queue.go @@ -70,7 +70,7 @@ var _ = Describe("Queue E2E Test", func() { // Test Queue status spec = &jobSpec{ - name: "", + name: "q1-qj-2", queue: defaultQueue1, tasks: []taskSpec{ { @@ -81,7 +81,9 @@ var _ = Describe("Queue E2E Test", func() { }, }, } - + job3 := createJob(context, spec) + err = waitJobStatePending(context, job3) + Expect(err).NotTo(HaveOccurred()) err = waitQueueStatus(func() (bool, error) { queue, err := context.kbclient.SchedulingV1alpha1().Queues().Get(defaultQueue1, metav1.GetOptions{}) Expect(err).NotTo(HaveOccurred()) From 5a83d2852d9f8ce6fbce3f097892ea6bd143a7a4 Mon Sep 17 00:00:00 2001 From: TommyLike Date: Thu, 9 May 2019 19:46:23 +0800 Subject: [PATCH 13/68] Update tf related resource files --- example/integrations/tensorflow/Dockerfile | 8 ++++++-- example/integrations/tensorflow/tf-example.yaml | 7 ++++--- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/example/integrations/tensorflow/Dockerfile b/example/integrations/tensorflow/Dockerfile index 5821747c08..5d0d12ed91 100644 --- a/example/integrations/tensorflow/Dockerfile +++ b/example/integrations/tensorflow/Dockerfile @@ -1,6 +1,10 @@ -#NOTE: the build process would change during developing. +# NOTE: the build process would change during developing, +# the commit ID when first creating the image: 62c833f806db621943a6cf8195657b9d0fa67d93 (master) +# original image is: gcr.io/kubeflow/tf-benchmarks-cpu:v20171202-bdab599-dirty-284af3, +# the image needs an update to use the latest tf-benchmark logic +# ref => https://github.com/tensorflow/benchmarks/tree/master/scripts/tf_cnn_benchmarks. FROM python:2.7 -MAINTAINER volcano +MAINTAINER volcano RUN apt-get update --fix-missing \ && apt-get install -y git \ && apt-get clean \ diff --git a/example/integrations/tensorflow/tf-example.yaml b/example/integrations/tensorflow/tf-example.yaml index 05bc67d781..92d004e45b 100644 --- a/example/integrations/tensorflow/tf-example.yaml +++ b/example/integrations/tensorflow/tf-example.yaml @@ -27,14 +27,15 @@ # 2019-04-23 11:10:25.552861: I tensorflow/core/distributed_runtime/rpc/grpc_channel.cc:215] # Initialize GrpcChannelCache for job worker -> {0 -> tensorflow-benchmark-worker-0.tensorflow-benchmark:2222} # -# **NOTES**: This example may take about an hour to finish. +# **NOTES**: This example may take about an hour to finish. When running multiple jobs, please ensure enough resource +# is guaranteed for each of the worker pods. apiVersion: batch.volcano.sh/v1alpha1 kind: Job metadata: name: tensorflow-benchmark spec: - minAvailable: 2 + minAvailable: 3 schedulerName: kube-batch plugins: env: [] @@ -65,7 +66,7 @@ spec: resources: {} workingDir: /opt/tf-benchmarks/scripts/tf_cnn_benchmarks restartPolicy: OnFailure - - replicas: 1 + - replicas: 2 name: worker policies: - event: TaskCompleted From a70cf2c418fcdc6d5f76a958002582d4671b019b Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Fri, 10 May 2019 09:41:36 +0800 Subject: [PATCH 14/68] Fix print --- pkg/controllers/queue/queue_controller.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pkg/controllers/queue/queue_controller.go b/pkg/controllers/queue/queue_controller.go index 0bc2bce910..daf46b6e35 100644 --- a/pkg/controllers/queue/queue_controller.go +++ b/pkg/controllers/queue/queue_controller.go @@ -138,13 +138,13 @@ func (c *Controller) processNextWorkItem() bool { } func (c *Controller) syncQueue(key string) error { - glog.V(5).Infoln("Begin sync queue") + glog.V(4).Infof("Begin sync queue %s", key) var pending, running, unknown int32 c.pgMutex.RLock() if c.podGroups[key] == nil { c.pgMutex.RUnlock() - glog.V(2).Infoln("queue %s has not been seen or deleted", key) + glog.V(2).Infof("queue %s has not been seen or deleted", key) return nil } podGroups := make([]string, 0, len(c.podGroups[key])) @@ -181,6 +181,7 @@ func (c *Controller) syncQueue(key string) error { return err } + glog.V(4).Infof("queue %s jobs pending %d, running %d, unknown %d", key, pending, running, unknown) // ignore update when status doesnot change if pending == queue.Status.Pending && running == queue.Status.Running && unknown == queue.Status.Unknown { return nil From d3e719993c0af9aa0216a0fdddd7f35926f803d8 Mon Sep 17 00:00:00 2001 From: Zhang Jinghui Date: Fri, 10 May 2019 09:58:33 +0800 Subject: [PATCH 15/68] admission to get tls certificate from kubeconfig, if tls config not defined in command line --- cmd/admission/app/server.go | 52 +++++++++++++++++++++---------------- cmd/admission/main.go | 13 ++++++++-- 2 files changed, 41 insertions(+), 24 deletions(-) diff --git a/cmd/admission/app/server.go b/cmd/admission/app/server.go index 49ebf8b289..05e79f7b73 100644 --- a/cmd/admission/app/server.go +++ b/cmd/admission/app/server.go @@ -26,8 +26,7 @@ import ( "k8s.io/api/admission/v1beta1" "k8s.io/apimachinery/pkg/runtime" "k8s.io/client-go/kubernetes" - "k8s.io/client-go/rest" - "k8s.io/client-go/tools/clientcmd" + restclient "k8s.io/client-go/rest" appConf "volcano.sh/volcano/cmd/admission/app/configure" admissioncontroller "volcano.sh/volcano/pkg/admission" @@ -38,34 +37,43 @@ const ( APPLICATIONJSON = "application/json" ) -// Get a clientset with in-cluster config. -func GetClient(c *appConf.Config) *kubernetes.Clientset { - var config *rest.Config - var err error - if c.Master != "" || c.Kubeconfig != "" { - config, err = clientcmd.BuildConfigFromFlags(c.Master, c.Kubeconfig) - } else { - config, err = rest.InClusterConfig() - } - - if err != nil { - glog.Fatal(err) - } - clientset, err := kubernetes.NewForConfig(config) +// Get a clientset with restConfig. +func GetClient(restConfig *restclient.Config) *kubernetes.Clientset { + clientset, err := kubernetes.NewForConfig(restConfig) if err != nil { glog.Fatal(err) } return clientset } -func ConfigTLS(config *appConf.Config, clientset *kubernetes.Clientset) *tls.Config { - sCert, err := tls.LoadX509KeyPair(config.CertFile, config.KeyFile) - if err != nil { - glog.Fatal(err) +// ConfigTLS is a helper function that generate tls certificates from directly defined tls config or kubeconfig +// These are passed in as command line for cluster certification. If tls config is passed in, we use the directly +// defined tls config, else use that defined in kubeconfig +func ConfigTLS(config *appConf.Config, restConfig *restclient.Config) *tls.Config { + if len(config.CertFile) != 0 && len(config.KeyFile) != 0 { + sCert, err := tls.LoadX509KeyPair(config.CertFile, config.KeyFile) + if err != nil { + glog.Fatal(err) + } + + return &tls.Config{ + Certificates: []tls.Certificate{sCert}, + } } - return &tls.Config{ - Certificates: []tls.Certificate{sCert}, + + if len(restConfig.CertData) != 0 && len(restConfig.KeyData) != 0 { + sCert, err := tls.X509KeyPair(restConfig.CertData, restConfig.KeyData) + if err != nil { + glog.Fatal(err) + } + + return &tls.Config{ + Certificates: []tls.Certificate{sCert}, + } } + + glog.Fatal("tls: failed to find any tls config data") + return &tls.Config{} } func Serve(w http.ResponseWriter, r *http.Request, admit admissioncontroller.AdmitFunc) { diff --git a/cmd/admission/main.go b/cmd/admission/main.go index d784575358..e60a6ef50f 100644 --- a/cmd/admission/main.go +++ b/cmd/admission/main.go @@ -27,6 +27,8 @@ import ( appConf "volcano.sh/volcano/cmd/admission/app/configure" admissioncontroller "volcano.sh/volcano/pkg/admission" "volcano.sh/volcano/pkg/version" + + "k8s.io/client-go/tools/clientcmd" ) func serveJobs(w http.ResponseWriter, r *http.Request) { @@ -55,7 +57,13 @@ func main() { } addr := ":" + strconv.Itoa(config.Port) - clientset := app.GetClient(config) + restConfig, err := clientcmd.BuildConfigFromFlags(config.Master, config.Kubeconfig) + if err != nil { + fmt.Fprintf(os.Stderr, "%v\n", err) + os.Exit(1) + } + + clientset := app.GetClient(restConfig) caCertPem, err := ioutil.ReadFile(config.CaCertFile) if err != nil { @@ -71,9 +79,10 @@ func main() { fmt.Fprintf(os.Stderr, "%v\n", err) } } + server := &http.Server{ Addr: addr, - TLSConfig: app.ConfigTLS(config, clientset), + TLSConfig: app.ConfigTLS(config, restConfig), } server.ListenAndServeTLS("", "") } From 8c01bf2ea3ac2e70ff95af00bef67bd46935af99 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Fri, 10 May 2019 11:24:08 +0800 Subject: [PATCH 16/68] fix e2e --- pkg/controllers/queue/queue_controller.go | 3 +++ test/e2e/queue.go | 9 ++++++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/pkg/controllers/queue/queue_controller.go b/pkg/controllers/queue/queue_controller.go index daf46b6e35..544dcadbb3 100644 --- a/pkg/controllers/queue/queue_controller.go +++ b/pkg/controllers/queue/queue_controller.go @@ -269,7 +269,10 @@ func (c *Controller) deletePodGroup(obj interface{}) { } key, _ := cache.MetaNamespaceKeyFunc(obj) + + c.pgMutex.Lock() delete(c.podGroups[pg.Spec.Queue], key) + c.pgMutex.Unlock() c.queue.Add(pg.Spec.Queue) } diff --git a/test/e2e/queue.go b/test/e2e/queue.go index 038a70b0e6..e4416b84bc 100644 --- a/test/e2e/queue.go +++ b/test/e2e/queue.go @@ -50,6 +50,13 @@ var _ = Describe("Queue E2E Test", func() { err := waitJobReady(context, job1) Expect(err).NotTo(HaveOccurred()) + err = waitQueueStatus(func() (bool, error) { + queue, err := context.kbclient.SchedulingV1alpha1().Queues().Get(defaultQueue1, metav1.GetOptions{}) + Expect(err).NotTo(HaveOccurred()) + return queue.Status.Running == 1, nil + }) + Expect(err).NotTo(HaveOccurred()) + expected := int(rep) / 2 // Reduce one pod to tolerate decimal fraction. if expected > 1 { @@ -87,7 +94,7 @@ var _ = Describe("Queue E2E Test", func() { err = waitQueueStatus(func() (bool, error) { queue, err := context.kbclient.SchedulingV1alpha1().Queues().Get(defaultQueue1, metav1.GetOptions{}) Expect(err).NotTo(HaveOccurred()) - return queue.Status.Running == 1 && queue.Status.Pending == 1, nil + return queue.Status.Pending == 1, nil }) Expect(err).NotTo(HaveOccurred()) }) From a8d5b0b037286ff7fea8a72e665cb482cc43058a Mon Sep 17 00:00:00 2001 From: lminzhw Date: Thu, 9 May 2019 20:31:16 +0800 Subject: [PATCH 17/68] doc for drf fair share --- docs/design/drf - fairshare.md | 179 +++++++++++++++++++++++++++++++++ 1 file changed, 179 insertions(+) create mode 100644 docs/design/drf - fairshare.md diff --git a/docs/design/drf - fairshare.md b/docs/design/drf - fairshare.md new file mode 100644 index 0000000000..4b69448f97 --- /dev/null +++ b/docs/design/drf - fairshare.md @@ -0,0 +1,179 @@ +# Namespace fair share + +[@lminzhw](http://github.com/lminzhw); May 8, 2019 + +## Motivation + +`Queue` was introduced in [kube-batch](http://github.com/kubernetes-sigs/kube-batch) to share resources among users. + +But, the user in the same `Queue` are equivalent during scheduling. For example, we have a `Queue` contains a small amount of resources, and there are 10 pods belong to UserA and 1000 pods belong to UserB. In this case, pods of UserA would have less probability to bind with node. + +So, we need a more fine-grained strategy to balance resource usage among users in the same `Queue`. + +In consideration of multi-user model in kubernetes, we use namespace to distinguish different user. Each namespace would have its weight to control resources usage. + +## Function Specification + +Weight have these features: +> 1. `Queue` level +> 2. an `integer` with default value 1 +> 3. record in namespace `quota` +> 4. higher value means more resources after balancing + +### where is the weight + +```yaml +apiVersion: v1 +kind: ResourceQuota +metadata: + namespace: default +spec: + hard: + limits.memory: 2Gi + volcano.sh/namespace.weight: 1 <- this field represent the weight of this namespace +``` + +If many `ResourceQuota` in the same namespace have weight, the weight for this namespace is the highest one of them. + +This weight should be positive, any invalid value is treated as default value 1. + +### Scheduler Framework + +Introduce two new fields in SchedulerCache + +```go +type NamespaceInfo struct { + Weight int +} + +type SchedulerCache struct { + ... + quotaInformer infov1.ResourceQuotaInformer + ... + NamespaceInfo map[string]*kbapi.NamespaceInfo + ... +} +``` + +The Scheduler will watch the lifecycle of `ResourceQuota` by `quotaInformer`, and refresh the info in `NamespaceInfo`. + +In `openSession` function, we should pass the `NamespaceInfo` through function `cache.Snapshot` into `Session` by using a new filed in `Session`/`ClusterInfo` struct. + +```go +type Session struct { + ... + NamespaceInfo map[string]*kbapi.NamespaceInfo + ... +} +type ClusterInfo struct { + ... + NamespaceInfo map[string]*kbapi.NamespaceInfo + ... +} +``` + +### Allocate Action + +#### Scheduling loop + +The behavior of `allocate` action is scheduling job in `Queue` one by one. + +At the beginning of scheduling loop, it will take a job with highest priority from `Queue`. And try to schedule tasks that belong to it until job is ready (matches the minMember) then go to next round. + +The priority of job mentioned above is defined by `JobOrder` functions registered by plugins. Such as job ready order of Gang plugin, priority order of Priority plugin, and also the share order of DRF plugin. + +#### JobOrder + +Namespace weight `should not` implement with JobOrder func. Because the scheduling of job would affect priority of the others. + +> e.g. +> +> ns1 has job1, job2, ns2 has job3, job4. The original order is job1-job2-job3-job4. +> +> After the scheduling of job1, right order should be job3-job4-job2. But in priority queue, we have no chance to fix the priority for job2 + +#### Namespace Order + +To add namespace weight, we introduce a new order function named `NamespaceOrder` in `Session`. + +```go +type Session struct { + ... + NamespaceOrderFns map[string]api.CompareFn + ... +} +``` + +The scheduling loop in allocate should change as follows. + +In scheduling loop, firstly, choose a namespace having highest priority by calling `NamespaceOrderFn`, and then choose a job having highest priority using `JobOrderFn` in this namespace. + +After scheduling of job, push the namespace and job back to priority queue in order to refresh its priority. Because once a job is scheduled, assigned resource may decrease the priority of this namespace, the other jobs in the same namespace may be scheduled later. + +Always assigning resources to namespace with highest priority (lower resource usage) in every turn will make the resource balanced. + +### DRF plugin + +DRF plugin use preemption and order of job to balance resource among jobs. The `share` in this plugin is defined as resource usage, the higher `share` means this job occupies the more resource now. + +#### Namespace Compare + +To introduce namespace weight into this plugin, we should define how to compare namespace having weight firstly. + +For namespace n1 having weight w1 and namespace n2 having weight w2, we can compute the `share` of resource and recorded as u1 and u2. Now, the resource usage of n1 less than n2 can be defined as (u1 / w1 < u2 / w2) + +`e.g.` ns1 having weight w1=2 use 6cpu, ns2 having weight w2=1 use 2cpu. In the scope of cpu, the ns1 use less resource than ns2. (6 / 3 < 3 / 1) + +#### Namespace Order + +Register `NamespaceOrder` function using the strategy mentioned above. + +#### preemption + +> The `preempt` action is disabled now. Do this later. + +In the `preemption` function now, strategy is just simply comparing the resource share among jobs . + +After adding namespace weight, we should check namespace of preemptor and preemptee firstly. The job in namespace with less resources can preempt others, or when namespace resource usage are the same, compare share of job instead. + +### Feature Interaction + +#### preempt action + +Preempt is a strategy set to choose victims and finally evict it. + +The way to choose victims is a function set named `Preemptable` registered by plugins. Such as job ready protection of Gang plugin, special pod protection of Conformance plugin, job share balance strategy of DRF plugin. + +All these plugin would choose some victims respective, and the intersection of them would be the final victim set. So, the choice made by DRF plugin would never break the requirement of others. + +### short hand + +1. Preempt may cause killing of some running pod. + +### Cases: + +- cluster have __16 cpu__, queue and namespace have default weight. + + | queue | namespace | requested | queue assigned | namespace assigned | + | ----- | --------- | --------- | -------------- | ------------------ | + | q1 | ns1 | 5 cpu | 8 cpu | 4 cpu | + | | ns2 | 10 cpu | | 4 cpu | + | q2 | ns3 | 10 cpu | 8 cpu | 6 cpu | + | | ns4 | 2 cpu | | 2 cpu | + +- cluster have __16 cpu__, q1 with weight 1, q2 with weight 3. ns1 with weight 3, ns2 have weight 1, ns3 have weight 2, ns4 have weight 6. + + | queue | namespace | requested | queue assigned | namespace assigned | + | ----- | --------- | --------- | -------------- | ------------------ | + | q1 w1 | ns1 w3 | 5 cpu | 4 cpu | 3 cpu | + | | ns2 w1 | 10 cpu | | 1 cpu | + | q2 w3 | ns3 w2 | 10 cpu | 12 cpu | 10 cpu | + | | ns4 w6 | 2 cpu | | 2 cpu | + +- cluster have __16 cpu__, q1 with weight 1, q2 with weight 3. ns1 have weight 2, ns2 have weight 6. + + | queue | namespace | requested | queue assigned | namespace assigned | + | ----- | --------- | --------- | -------------- | ------------------ | + | q1 w1 | ns1 w2 | | 4 cpu | | + | q2 w3 | na1 w2 | 5 cpu | 12 cpu | 3 cpu | + | | ns2 w6 | 20 cpu | | 9 cpu | \ No newline at end of file From ef5142550531da066c71d34c4e2d3249a9ccbe63 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Thu, 9 May 2019 11:32:05 +0800 Subject: [PATCH 18/68] Add TTL for finished job --- pkg/apis/batch/v1alpha1/job.go | 9 +++++++++ pkg/apis/batch/v1alpha1/zz_generated.deepcopy.go | 5 +++++ 2 files changed, 14 insertions(+) diff --git a/pkg/apis/batch/v1alpha1/job.go b/pkg/apis/batch/v1alpha1/job.go index a9406dd365..fb6b8d0365 100644 --- a/pkg/apis/batch/v1alpha1/job.go +++ b/pkg/apis/batch/v1alpha1/job.go @@ -70,6 +70,15 @@ type JobSpec struct { // Defaults to 3. // +optional MaxRetry int32 `json:"maxRetry,omitempty" protobuf:"bytes,8,opt,name=maxRetry"` + + // ttlSecondsAfterFinished limits the lifetime of a Job that has finished + // execution (either Completed or Failed). If this field is set, + // ttlSecondsAfterFinished after the Job finishes, it is eligible to be + // automatically deleted. If this field is unset, + // the Job won't be automatically deleted. If this field is set to zero, + // the Job becomes eligible to be deleted immediately after it finishes. + // +optional + TTLSecondsAfterFinished *int32 `json:"ttlSecondsAfterFinished,omitempty" protobuf:"varint,8,opt,name=ttlSecondsAfterFinished"` } // VolumeSpec defines the specification of Volume, e.g. PVC diff --git a/pkg/apis/batch/v1alpha1/zz_generated.deepcopy.go b/pkg/apis/batch/v1alpha1/zz_generated.deepcopy.go index ab368e414e..2ed790063e 100644 --- a/pkg/apis/batch/v1alpha1/zz_generated.deepcopy.go +++ b/pkg/apis/batch/v1alpha1/zz_generated.deepcopy.go @@ -126,6 +126,11 @@ func (in *JobSpec) DeepCopyInto(out *JobSpec) { (*out)[key] = outVal } } + if in.TTLSecondsAfterFinished != nil { + in, out := &in.TTLSecondsAfterFinished, &out.TTLSecondsAfterFinished + *out = new(int32) + **out = **in + } return } From 21941696dda6fece764aab35d33e3c33253b28f3 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Fri, 10 May 2019 10:38:43 +0800 Subject: [PATCH 19/68] add LastTransitionTime --- pkg/apis/batch/v1alpha1/job.go | 4 ++++ pkg/apis/batch/v1alpha1/zz_generated.deepcopy.go | 3 ++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/pkg/apis/batch/v1alpha1/job.go b/pkg/apis/batch/v1alpha1/job.go index fb6b8d0365..88694906ec 100644 --- a/pkg/apis/batch/v1alpha1/job.go +++ b/pkg/apis/batch/v1alpha1/job.go @@ -227,6 +227,10 @@ type JobState struct { // +optional Phase JobPhase `json:"phase,omitempty" protobuf:"bytes,1,opt,name=phase"` + // Last time the condition transit from one phase to another. + // +optional + LastTransitionTime metav1.Time `json:"lastTransitionTime,omitempty" protobuf:"bytes,4,opt,name=lastTransitionTime"` + // Unique, one-word, CamelCase reason for the phase's last transition. // +optional Reason string `json:"reason,omitempty" protobuf:"bytes,2,opt,name=reason"` diff --git a/pkg/apis/batch/v1alpha1/zz_generated.deepcopy.go b/pkg/apis/batch/v1alpha1/zz_generated.deepcopy.go index 2ed790063e..7fd00a4301 100644 --- a/pkg/apis/batch/v1alpha1/zz_generated.deepcopy.go +++ b/pkg/apis/batch/v1alpha1/zz_generated.deepcopy.go @@ -147,6 +147,7 @@ func (in *JobSpec) DeepCopy() *JobSpec { // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *JobState) DeepCopyInto(out *JobState) { *out = *in + in.LastTransitionTime.DeepCopyInto(&out.LastTransitionTime) return } @@ -163,7 +164,7 @@ func (in *JobState) DeepCopy() *JobState { // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *JobStatus) DeepCopyInto(out *JobStatus) { *out = *in - out.State = in.State + in.State.DeepCopyInto(&out.State) if in.ControlledResources != nil { in, out := &in.ControlledResources, &out.ControlledResources *out = make(map[string]string, len(*in)) From e2b4a4354b3ee2108ef57adc3be7b794ed5c40bf Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Fri, 10 May 2019 10:15:53 +0800 Subject: [PATCH 20/68] add gc --- .../garbagecollector/garbagecollector.go | 283 ++++++++++++++++++ 1 file changed, 283 insertions(+) create mode 100644 pkg/controllers/garbagecollector/garbagecollector.go diff --git a/pkg/controllers/garbagecollector/garbagecollector.go b/pkg/controllers/garbagecollector/garbagecollector.go new file mode 100644 index 0000000000..25d3dc242f --- /dev/null +++ b/pkg/controllers/garbagecollector/garbagecollector.go @@ -0,0 +1,283 @@ +/* +Copyright 2019 The Volcano 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 garbagecollector + +import ( + "fmt" + "time" + + "github.com/golang/glog" + + "k8s.io/apimachinery/pkg/api/errors" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/util/wait" + "k8s.io/client-go/tools/cache" + "k8s.io/client-go/util/workqueue" + "k8s.io/kubernetes/pkg/controller" + + "volcano.sh/volcano/pkg/apis/batch/v1alpha1" + vkver "volcano.sh/volcano/pkg/client/clientset/versioned" + vkinfoext "volcano.sh/volcano/pkg/client/informers/externalversions" + vkbatchinfo "volcano.sh/volcano/pkg/client/informers/externalversions/batch/v1alpha1" + vkbatchlister "volcano.sh/volcano/pkg/client/listers/batch/v1alpha1" +) + +// GarbageCollector runs reflectors to watch for changes of managed API +// objects. Currently it only watches Jobs. Triggered by Job creation +// and updates, it enqueues Jobs that have non-nil `.spec.ttlSecondsAfterFinished` +// to the `queue`. The GarbageCollector has workers who consume `queue`, check whether +// the Job TTL has expired or not; if the Job TTL hasn't expired, it will add the +// Job to the queue after the TTL is expected to expire; if the TTL has expired, the +// worker will send requests to the API server to delete the Jobs accordingly. +// This is implemented outside of Job controller for separation of concerns, and +// because it will be extended to handle other finishable resource types. +type GarbageCollector struct { + vkClient vkver.Interface + + jobInformer vkbatchinfo.JobInformer + + // A store of jobs + jobLister vkbatchlister.JobLister + jobSynced func() bool + + // queues that need to be updated. + queue workqueue.RateLimitingInterface +} + +// New creates an instance of GarbageCollector +func New(vkClient vkver.Interface) *GarbageCollector { + jobInformer := vkinfoext.NewSharedInformerFactory(vkClient, 0).Batch().V1alpha1().Jobs() + + gb := &GarbageCollector{ + vkClient: vkClient, + jobInformer: jobInformer, + jobLister: jobInformer.Lister(), + jobSynced: jobInformer.Informer().HasSynced, + queue: workqueue.NewRateLimitingQueue(workqueue.DefaultControllerRateLimiter()), + } + jobInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: gb.addJob, + UpdateFunc: gb.updateJob, + }) + return gb +} + +// Run starts the worker to clean up Jobs. +func (gb *GarbageCollector) Run(stopCh <-chan struct{}) { + defer gb.queue.ShutDown() + + glog.Infof("Starting garbage collector") + defer glog.Infof("Shutting down garbage collector") + + go gb.jobInformer.Informer().Run(stopCh) + if !controller.WaitForCacheSync("garbage collector", stopCh, gb.jobSynced) { + return + } + + go wait.Until(gb.worker, time.Second, stopCh) + + <-stopCh +} + +func (gb *GarbageCollector) addJob(obj interface{}) { + job := obj.(*v1alpha1.Job) + glog.V(4).Infof("Adding job %s/%s", job.Namespace, job.Name) + + if job.DeletionTimestamp == nil && needsCleanup(job) { + gb.enqueue(job) + } +} + +func (gb *GarbageCollector) updateJob(old, cur interface{}) { + job := cur.(*v1alpha1.Job) + glog.V(4).Infof("Updating job %s/%s", job.Namespace, job.Name) + + if job.DeletionTimestamp == nil && needsCleanup(job) { + gb.enqueue(job) + } +} + +func (gb *GarbageCollector) enqueue(job *v1alpha1.Job) { + glog.V(4).Infof("Add job %s/%s to cleanup", job.Namespace, job.Name) + key, err := controller.KeyFunc(job) + if err != nil { + glog.Errorf("couldn't get key for object %#v: %v", job, err) + return + } + + gb.queue.Add(key) +} + +func (gb *GarbageCollector) enqueueAfter(job *v1alpha1.Job, after time.Duration) { + key, err := controller.KeyFunc(job) + if err != nil { + glog.Errorf("couldn't get key for object %#v: %v", job, err) + return + } + + gb.queue.AddAfter(key, after) +} + +func (gb *GarbageCollector) worker() { + for gb.processNextWorkItem() { + } +} + +func (gb *GarbageCollector) processNextWorkItem() bool { + key, quit := gb.queue.Get() + if quit { + return false + } + defer gb.queue.Done(key) + + err := gb.processJob(key.(string)) + gb.handleErr(err, key) + + return true +} + +func (gb *GarbageCollector) handleErr(err error, key interface{}) { + if err == nil { + gb.queue.Forget(key) + return + } + + glog.Errorf("error cleaning up Job %v, will retry: %v", key, err) + gb.queue.AddRateLimited(key) +} + +// processJob will check the Job's state and TTL and delete the Job when it +// finishes and its TTL after finished has expired. If the Job hasn't finished or +// its TTL hasn't expired, it will be added to the queue after the TTL is expected +// to expire. +// This function is not meant to be invoked concurrently with the same key. +func (gb *GarbageCollector) processJob(key string) error { + namespace, name, err := cache.SplitMetaNamespaceKey(key) + if err != nil { + return err + } + + glog.V(4).Infof("Checking if Job %s/%s is ready for cleanup", namespace, name) + // Ignore the Jobs that are already deleted or being deleted, or the ones that don't need clean up. + job, err := gb.jobLister.Jobs(namespace).Get(name) + if errors.IsNotFound(err) { + return nil + } + if err != nil { + return err + } + + if expired, err := gb.processTTL(job); err != nil { + return err + } else if !expired { + return nil + } + + // The Job's TTL is assumed to have expired, but the Job TTL might be stale. + // Before deleting the Job, do a final sanity check. + // If TTL is modified before we do this check, we cannot be sure if the TTL truly expires. + // The latest Job may have a different UID, but it's fine because the checks will be run again. + fresh, err := gb.vkClient.BatchV1alpha1().Jobs(namespace).Get(name, metav1.GetOptions{}) + if errors.IsNotFound(err) { + return nil + } + if err != nil { + return err + } + // Use the latest Job TTL to see if the TTL truly expires. + if expired, err := gb.processTTL(fresh); err != nil { + return err + } else if !expired { + return nil + } + // Cascade deletes the Jobs if TTL truly expires. + policy := metav1.DeletePropagationForeground + options := &metav1.DeleteOptions{ + PropagationPolicy: &policy, + Preconditions: &metav1.Preconditions{UID: &fresh.UID}, + } + glog.V(4).Infof("Cleaning up Job %s/%s", namespace, name) + return gb.vkClient.BatchV1alpha1().Jobs(fresh.Namespace).Delete(fresh.Name, options) +} + +// processTTL checks whether a given Job's TTL has expired, and add it to the queue after the TTL is expected to expire +// if the TTL will expire later. +func (gb *GarbageCollector) processTTL(job *v1alpha1.Job) (expired bool, err error) { + // We don't care about the Jobs that are going to be deleted, or the ones that don't need clean up. + if job.DeletionTimestamp != nil || !needsCleanup(job) { + return false, nil + } + + now := time.Now() + t, err := timeLeft(job, &now) + if err != nil { + return false, err + } + + // TTL has expired + if *t <= 0 { + return true, nil + } + + gb.enqueueAfter(job, *t) + return false, nil +} + +// needsCleanup checks whether a Job has finished and has a TTL set. +func needsCleanup(j *v1alpha1.Job) bool { + return j.Spec.TTLSecondsAfterFinished != nil && isJobFinished(j) +} + +func isJobFinished(job *v1alpha1.Job) bool { + return job.Status.State.Phase == v1alpha1.Completed || + job.Status.State.Phase == v1alpha1.Failed || + job.Status.State.Phase == v1alpha1.Terminated +} + +func getFinishAndExpireTime(j *v1alpha1.Job) (*time.Time, *time.Time, error) { + if !needsCleanup(j) { + return nil, nil, fmt.Errorf("Job %s/%s should not be cleaned up", j.Namespace, j.Name) + } + finishAt, err := jobFinishTime(j) + if err != nil { + return nil, nil, err + } + finishAtUTC := finishAt.UTC() + expireAtUTC := finishAtUTC.Add(time.Duration(*j.Spec.TTLSecondsAfterFinished) * time.Second) + return &finishAtUTC, &expireAtUTC, nil +} + +func timeLeft(j *v1alpha1.Job, since *time.Time) (*time.Duration, error) { + finishAt, expireAt, err := getFinishAndExpireTime(j) + if err != nil { + return nil, err + } + if finishAt.UTC().After(since.UTC()) { + glog.Warningf("Warning: Found Job %s/%s finished in the future. This is likely due to time skew in the cluster. Job cleanup will be deferred.", j.Namespace, j.Name) + } + remaining := expireAt.UTC().Sub(since.UTC()) + glog.V(4).Infof("Found Job %s/%s finished at %v, remaining TTL %v since %v, TTL will expire at %v", j.Namespace, j.Name, finishAt.UTC(), remaining, since.UTC(), expireAt.UTC()) + return &remaining, nil +} + +// jobFinishTime takes an already finished Job and returns the time it finishes. +func jobFinishTime(finishedJob *v1alpha1.Job) (metav1.Time, error) { + if finishedJob.Status.State.LastTransitionTime.IsZero() { + return metav1.Time{}, fmt.Errorf("unable to find the time when the Job %s/%s finished", finishedJob.Namespace, finishedJob.Name) + } + return finishedJob.Status.State.LastTransitionTime, nil +} From 8041ac285e2087a3de5ee1f4070a63d73bf68a17 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Fri, 10 May 2019 14:59:52 +0800 Subject: [PATCH 21/68] Set job state transition timestatmp --- pkg/controllers/job/state/aborted.go | 3 +++ pkg/controllers/job/state/aborting.go | 11 +++++++---- pkg/controllers/job/state/completing.go | 10 ++++++---- pkg/controllers/job/state/inqueue.go | 9 ++++++--- pkg/controllers/job/state/pending.go | 10 +++++++--- pkg/controllers/job/state/restarting.go | 4 ++++ pkg/controllers/job/state/running.go | 7 +++++++ pkg/controllers/job/state/terminating.go | 10 ++++++---- 8 files changed, 46 insertions(+), 18 deletions(-) diff --git a/pkg/controllers/job/state/aborted.go b/pkg/controllers/job/state/aborted.go index 104170615b..9952bff23c 100644 --- a/pkg/controllers/job/state/aborted.go +++ b/pkg/controllers/job/state/aborted.go @@ -17,6 +17,8 @@ limitations under the License. package state import ( + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -30,6 +32,7 @@ func (as *abortedState) Execute(action vkv1.Action) error { case vkv1.ResumeJobAction: return KillJob(as.job, func(status *vkv1.JobStatus) { status.State.Phase = vkv1.Restarting + status.State.LastTransitionTime = metav1.Now() status.RetryCount++ }) default: diff --git a/pkg/controllers/job/state/aborting.go b/pkg/controllers/job/state/aborting.go index cae21cb466..338a86533f 100644 --- a/pkg/controllers/job/state/aborting.go +++ b/pkg/controllers/job/state/aborting.go @@ -17,6 +17,8 @@ limitations under the License. package state import ( + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -31,17 +33,18 @@ func (ps *abortingState) Execute(action vkv1.Action) error { // Already in Restarting phase, just sync it return KillJob(ps.job, func(status *vkv1.JobStatus) { status.State.Phase = vkv1.Restarting + status.State.LastTransitionTime = metav1.Now() status.RetryCount++ }) default: return KillJob(ps.job, func(status *vkv1.JobStatus) { // If any "alive" pods, still in Aborting phase - phase := vkv1.Aborted if status.Terminating != 0 || status.Pending != 0 || status.Running != 0 { - phase = vkv1.Aborting + status.State.Phase = vkv1.Aborting + } else { + status.State.Phase = vkv1.Aborted + status.State.LastTransitionTime = metav1.Now() } - - status.State.Phase = phase }) } } diff --git a/pkg/controllers/job/state/completing.go b/pkg/controllers/job/state/completing.go index 1fb48ab040..27ce1a6f23 100644 --- a/pkg/controllers/job/state/completing.go +++ b/pkg/controllers/job/state/completing.go @@ -17,6 +17,8 @@ limitations under the License. package state import ( + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -28,11 +30,11 @@ type completingState struct { func (ps *completingState) Execute(action vkv1.Action) error { return KillJob(ps.job, func(status *vkv1.JobStatus) { // If any "alive" pods, still in Completing phase - phase := vkv1.Completed if status.Terminating != 0 || status.Pending != 0 || status.Running != 0 { - phase = vkv1.Completing + status.State.Phase = vkv1.Completing + } else { + status.State.Phase = vkv1.Completed + status.State.LastTransitionTime = metav1.Now() } - - status.State.Phase = phase }) } diff --git a/pkg/controllers/job/state/inqueue.go b/pkg/controllers/job/state/inqueue.go index f15415a862..a7a917e2c9 100644 --- a/pkg/controllers/job/state/inqueue.go +++ b/pkg/controllers/job/state/inqueue.go @@ -17,6 +17,8 @@ limitations under the License. package state import ( + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -34,7 +36,7 @@ func (ps *inqueueState) Execute(action vkv1.Action) error { phase = vkv1.Restarting status.RetryCount++ } - + status.State.LastTransitionTime = metav1.Now() status.State.Phase = phase }) @@ -44,7 +46,7 @@ func (ps *inqueueState) Execute(action vkv1.Action) error { if status.Terminating != 0 { phase = vkv1.Aborting } - + status.State.LastTransitionTime = metav1.Now() status.State.Phase = phase }) case vkv1.CompleteJobAction: @@ -53,7 +55,7 @@ func (ps *inqueueState) Execute(action vkv1.Action) error { if status.Terminating != 0 { phase = vkv1.Completing } - + status.State.LastTransitionTime = metav1.Now() status.State.Phase = phase }) default: @@ -61,6 +63,7 @@ func (ps *inqueueState) Execute(action vkv1.Action) error { phase := vkv1.Inqueue if ps.job.Job.Spec.MinAvailable <= status.Running+status.Succeeded+status.Failed { + status.State.LastTransitionTime = metav1.Now() phase = vkv1.Running } diff --git a/pkg/controllers/job/state/pending.go b/pkg/controllers/job/state/pending.go index f3bb4b501f..0ff1a3a3c1 100644 --- a/pkg/controllers/job/state/pending.go +++ b/pkg/controllers/job/state/pending.go @@ -17,6 +17,8 @@ limitations under the License. package state import ( + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -34,8 +36,8 @@ func (ps *pendingState) Execute(action vkv1.Action) error { phase = vkv1.Restarting status.RetryCount++ } - status.State.Phase = phase + status.State.LastTransitionTime = metav1.Now() }) case vkv1.AbortJobAction: @@ -44,8 +46,8 @@ func (ps *pendingState) Execute(action vkv1.Action) error { if status.Terminating != 0 { phase = vkv1.Aborting } - status.State.Phase = phase + status.State.LastTransitionTime = metav1.Now() }) case vkv1.CompleteJobAction: return KillJob(ps.job, func(status *vkv1.JobStatus) { @@ -53,8 +55,8 @@ func (ps *pendingState) Execute(action vkv1.Action) error { if status.Terminating != 0 { phase = vkv1.Completing } - status.State.Phase = phase + status.State.LastTransitionTime = metav1.Now() }) case vkv1.EnqueueAction: return SyncJob(ps.job, func(status *vkv1.JobStatus) { @@ -65,10 +67,12 @@ func (ps *pendingState) Execute(action vkv1.Action) error { } status.State.Phase = phase + status.State.LastTransitionTime = metav1.Now() }) default: return CreateJob(ps.job, func(status *vkv1.JobStatus) { status.State.Phase = vkv1.Pending + status.State.LastTransitionTime = metav1.Now() }) } } diff --git a/pkg/controllers/job/state/restarting.go b/pkg/controllers/job/state/restarting.go index a58dbd7811..55ffd54019 100644 --- a/pkg/controllers/job/state/restarting.go +++ b/pkg/controllers/job/state/restarting.go @@ -17,6 +17,8 @@ limitations under the License. package state import ( + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -38,6 +40,7 @@ func (ps *restartingState) Execute(action vkv1.Action) error { if status.RetryCount >= maxRetry { // Failed is the phase that the job is restarted failed reached the maximum number of retries. phase = vkv1.Failed + status.State.LastTransitionTime = metav1.Now() } else { total := int32(0) for _, task := range ps.job.Job.Spec.Tasks { @@ -46,6 +49,7 @@ func (ps *restartingState) Execute(action vkv1.Action) error { if total-status.Terminating >= status.MinAvailable { phase = vkv1.Pending + status.State.LastTransitionTime = metav1.Now() } } diff --git a/pkg/controllers/job/state/running.go b/pkg/controllers/job/state/running.go index b2204ba727..e1c920bcf6 100644 --- a/pkg/controllers/job/state/running.go +++ b/pkg/controllers/job/state/running.go @@ -17,6 +17,8 @@ limitations under the License. package state import ( + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -32,6 +34,7 @@ func (ps *runningState) Execute(action vkv1.Action) error { phase := vkv1.Running if status.Terminating != 0 { phase = vkv1.Restarting + status.State.LastTransitionTime = metav1.Now() status.RetryCount++ } @@ -42,6 +45,7 @@ func (ps *runningState) Execute(action vkv1.Action) error { phase := vkv1.Running if status.Terminating != 0 { phase = vkv1.Aborting + status.State.LastTransitionTime = metav1.Now() } status.State.Phase = phase @@ -51,6 +55,7 @@ func (ps *runningState) Execute(action vkv1.Action) error { phase := vkv1.Running if status.Terminating != 0 { phase = vkv1.Terminating + status.State.LastTransitionTime = metav1.Now() } status.State.Phase = phase @@ -63,12 +68,14 @@ func (ps *runningState) Execute(action vkv1.Action) error { } status.State.Phase = phase + status.State.LastTransitionTime = metav1.Now() }) default: return SyncJob(ps.job, func(status *vkv1.JobStatus) { phase := vkv1.Running if status.Succeeded+status.Failed == TotalTasks(ps.job.Job) { phase = vkv1.Completed + status.State.LastTransitionTime = metav1.Now() } status.State.Phase = phase diff --git a/pkg/controllers/job/state/terminating.go b/pkg/controllers/job/state/terminating.go index a818988797..5372865196 100644 --- a/pkg/controllers/job/state/terminating.go +++ b/pkg/controllers/job/state/terminating.go @@ -17,6 +17,8 @@ limitations under the License. package state import ( + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -28,11 +30,11 @@ type terminatingState struct { func (ps *terminatingState) Execute(action vkv1.Action) error { return KillJob(ps.job, func(status *vkv1.JobStatus) { // If any "alive" pods, still in Terminating phase - phase := vkv1.Terminated if status.Terminating != 0 || status.Pending != 0 || status.Running != 0 { - phase = vkv1.Terminating + status.State.Phase = vkv1.Terminating + } else { + status.State.Phase = vkv1.Terminated + status.State.LastTransitionTime = metav1.Now() } - - status.State.Phase = phase }) } From 2e25e8348392b84a0514aab155e266be43cb47d0 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Mon, 13 May 2019 10:14:01 +0800 Subject: [PATCH 22/68] re order --- pkg/apis/batch/v1alpha1/job.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/apis/batch/v1alpha1/job.go b/pkg/apis/batch/v1alpha1/job.go index 88694906ec..8a62a7b222 100644 --- a/pkg/apis/batch/v1alpha1/job.go +++ b/pkg/apis/batch/v1alpha1/job.go @@ -227,10 +227,6 @@ type JobState struct { // +optional Phase JobPhase `json:"phase,omitempty" protobuf:"bytes,1,opt,name=phase"` - // Last time the condition transit from one phase to another. - // +optional - LastTransitionTime metav1.Time `json:"lastTransitionTime,omitempty" protobuf:"bytes,4,opt,name=lastTransitionTime"` - // Unique, one-word, CamelCase reason for the phase's last transition. // +optional Reason string `json:"reason,omitempty" protobuf:"bytes,2,opt,name=reason"` @@ -238,6 +234,10 @@ type JobState struct { // Human-readable message indicating details about last transition. // +optional Message string `json:"message,omitempty" protobuf:"bytes,3,opt,name=message"` + + // Last time the condition transit from one phase to another. + // +optional + LastTransitionTime metav1.Time `json:"lastTransitionTime,omitempty" protobuf:"bytes,4,opt,name=lastTransitionTime"` } // JobStatus represents the current status of a Job From b4c2a8501a7d97e1c9362b6ce60ab2817e030724 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Mon, 13 May 2019 11:03:20 +0800 Subject: [PATCH 23/68] fix comments --- pkg/controllers/job/job_controller_actions.go | 12 ++++-- pkg/controllers/job/state/aborted.go | 6 +-- pkg/controllers/job/state/aborting.go | 10 ++--- pkg/controllers/job/state/completing.go | 8 ++-- pkg/controllers/job/state/factory.go | 2 +- pkg/controllers/job/state/inqueue.go | 25 +++++------ pkg/controllers/job/state/pending.go | 22 +++++----- pkg/controllers/job/state/restarting.go | 16 +++---- pkg/controllers/job/state/running.go | 43 ++++++++----------- pkg/controllers/job/state/terminating.go | 8 ++-- 10 files changed, 67 insertions(+), 85 deletions(-) diff --git a/pkg/controllers/job/job_controller_actions.go b/pkg/controllers/job/job_controller_actions.go index 129524bc46..94017c11b6 100644 --- a/pkg/controllers/job/job_controller_actions.go +++ b/pkg/controllers/job/job_controller_actions.go @@ -104,7 +104,9 @@ func (cc *Controller) killJob(jobInfo *apis.JobInfo, updateStatus state.UpdateSt } if updateStatus != nil { - updateStatus(&job.Status) + if updateStatus(&job.Status) { + job.Status.State.LastTransitionTime = metav1.Now() + } } // Update Job status @@ -165,7 +167,9 @@ func (cc *Controller) createJob(jobInfo *apis.JobInfo, updateStatus state.Update } if updateStatus != nil { - updateStatus(&job.Status) + if updateStatus(&job.Status) { + job.Status.State.LastTransitionTime = metav1.Now() + } } if job, err := cc.vkClients.BatchV1alpha1().Jobs(job.Namespace).UpdateStatus(job); err != nil { @@ -316,7 +320,9 @@ func (cc *Controller) syncJob(jobInfo *apis.JobInfo, updateStatus state.UpdateSt } if updateStatus != nil { - updateStatus(&job.Status) + if updateStatus(&job.Status) { + job.Status.State.LastTransitionTime = metav1.Now() + } } if job, err := cc.vkClients.BatchV1alpha1().Jobs(job.Namespace).UpdateStatus(job); err != nil { diff --git a/pkg/controllers/job/state/aborted.go b/pkg/controllers/job/state/aborted.go index 9952bff23c..953ee392d5 100644 --- a/pkg/controllers/job/state/aborted.go +++ b/pkg/controllers/job/state/aborted.go @@ -17,8 +17,6 @@ limitations under the License. package state import ( - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -30,10 +28,10 @@ type abortedState struct { func (as *abortedState) Execute(action vkv1.Action) error { switch action { case vkv1.ResumeJobAction: - return KillJob(as.job, func(status *vkv1.JobStatus) { + return KillJob(as.job, func(status *vkv1.JobStatus) bool { status.State.Phase = vkv1.Restarting - status.State.LastTransitionTime = metav1.Now() status.RetryCount++ + return true }) default: return KillJob(as.job, nil) diff --git a/pkg/controllers/job/state/aborting.go b/pkg/controllers/job/state/aborting.go index 338a86533f..8d123b6ada 100644 --- a/pkg/controllers/job/state/aborting.go +++ b/pkg/controllers/job/state/aborting.go @@ -31,19 +31,19 @@ func (ps *abortingState) Execute(action vkv1.Action) error { switch action { case vkv1.ResumeJobAction: // Already in Restarting phase, just sync it - return KillJob(ps.job, func(status *vkv1.JobStatus) { - status.State.Phase = vkv1.Restarting - status.State.LastTransitionTime = metav1.Now() + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { status.RetryCount++ + return false }) default: - return KillJob(ps.job, func(status *vkv1.JobStatus) { + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { // If any "alive" pods, still in Aborting phase if status.Terminating != 0 || status.Pending != 0 || status.Running != 0 { - status.State.Phase = vkv1.Aborting + return false } else { status.State.Phase = vkv1.Aborted status.State.LastTransitionTime = metav1.Now() + return true } }) } diff --git a/pkg/controllers/job/state/completing.go b/pkg/controllers/job/state/completing.go index 27ce1a6f23..cb9f7074bd 100644 --- a/pkg/controllers/job/state/completing.go +++ b/pkg/controllers/job/state/completing.go @@ -17,8 +17,6 @@ limitations under the License. package state import ( - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -28,13 +26,13 @@ type completingState struct { } func (ps *completingState) Execute(action vkv1.Action) error { - return KillJob(ps.job, func(status *vkv1.JobStatus) { + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { // If any "alive" pods, still in Completing phase if status.Terminating != 0 || status.Pending != 0 || status.Running != 0 { - status.State.Phase = vkv1.Completing + return false } else { status.State.Phase = vkv1.Completed - status.State.LastTransitionTime = metav1.Now() + return true } }) } diff --git a/pkg/controllers/job/state/factory.go b/pkg/controllers/job/state/factory.go index a24f605543..f33cb753f7 100644 --- a/pkg/controllers/job/state/factory.go +++ b/pkg/controllers/job/state/factory.go @@ -21,7 +21,7 @@ import ( "volcano.sh/volcano/pkg/controllers/apis" ) -type UpdateStatusFn func(status *vkv1.JobStatus) +type UpdateStatusFn func(status *vkv1.JobStatus) (jobPhaseChanged bool) type ActionFn func(job *apis.JobInfo, fn UpdateStatusFn) error var ( diff --git a/pkg/controllers/job/state/inqueue.go b/pkg/controllers/job/state/inqueue.go index a7a917e2c9..44cbfa514f 100644 --- a/pkg/controllers/job/state/inqueue.go +++ b/pkg/controllers/job/state/inqueue.go @@ -17,8 +17,6 @@ limitations under the License. package state import ( - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -30,44 +28,41 @@ type inqueueState struct { func (ps *inqueueState) Execute(action vkv1.Action) error { switch action { case vkv1.RestartJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) { + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { phase := vkv1.Pending if status.Terminating != 0 { phase = vkv1.Restarting status.RetryCount++ } - status.State.LastTransitionTime = metav1.Now() status.State.Phase = phase + return true }) case vkv1.AbortJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) { + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { phase := vkv1.Pending if status.Terminating != 0 { phase = vkv1.Aborting } - status.State.LastTransitionTime = metav1.Now() status.State.Phase = phase + return true }) case vkv1.CompleteJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) { + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { phase := vkv1.Completed if status.Terminating != 0 { phase = vkv1.Completing } - status.State.LastTransitionTime = metav1.Now() status.State.Phase = phase + return true }) default: - return SyncJob(ps.job, func(status *vkv1.JobStatus) { - phase := vkv1.Inqueue - + return SyncJob(ps.job, func(status *vkv1.JobStatus) bool { if ps.job.Job.Spec.MinAvailable <= status.Running+status.Succeeded+status.Failed { - status.State.LastTransitionTime = metav1.Now() - phase = vkv1.Running + status.State.Phase = vkv1.Running + return true } - - status.State.Phase = phase + return false }) } return nil diff --git a/pkg/controllers/job/state/pending.go b/pkg/controllers/job/state/pending.go index 0ff1a3a3c1..9cead263a5 100644 --- a/pkg/controllers/job/state/pending.go +++ b/pkg/controllers/job/state/pending.go @@ -17,8 +17,6 @@ limitations under the License. package state import ( - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -30,36 +28,36 @@ type pendingState struct { func (ps *pendingState) Execute(action vkv1.Action) error { switch action { case vkv1.RestartJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) { + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { phase := vkv1.Pending if status.Terminating != 0 { phase = vkv1.Restarting status.RetryCount++ } status.State.Phase = phase - status.State.LastTransitionTime = metav1.Now() + return true }) case vkv1.AbortJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) { + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { phase := vkv1.Pending if status.Terminating != 0 { phase = vkv1.Aborting } status.State.Phase = phase - status.State.LastTransitionTime = metav1.Now() + return true }) case vkv1.CompleteJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) { + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { phase := vkv1.Completed if status.Terminating != 0 { phase = vkv1.Completing } status.State.Phase = phase - status.State.LastTransitionTime = metav1.Now() + return true }) case vkv1.EnqueueAction: - return SyncJob(ps.job, func(status *vkv1.JobStatus) { + return SyncJob(ps.job, func(status *vkv1.JobStatus) bool { phase := vkv1.Inqueue if ps.job.Job.Spec.MinAvailable <= status.Running+status.Succeeded+status.Failed { @@ -67,12 +65,12 @@ func (ps *pendingState) Execute(action vkv1.Action) error { } status.State.Phase = phase - status.State.LastTransitionTime = metav1.Now() + return true }) default: - return CreateJob(ps.job, func(status *vkv1.JobStatus) { + return CreateJob(ps.job, func(status *vkv1.JobStatus) bool { status.State.Phase = vkv1.Pending - status.State.LastTransitionTime = metav1.Now() + return true }) } } diff --git a/pkg/controllers/job/state/restarting.go b/pkg/controllers/job/state/restarting.go index 55ffd54019..b11fb460a2 100644 --- a/pkg/controllers/job/state/restarting.go +++ b/pkg/controllers/job/state/restarting.go @@ -17,8 +17,6 @@ limitations under the License. package state import ( - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -28,9 +26,7 @@ type restartingState struct { } func (ps *restartingState) Execute(action vkv1.Action) error { - return KillJob(ps.job, func(status *vkv1.JobStatus) { - phase := vkv1.Restarting - + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { // Get the maximum number of retries. maxRetry := DefaultMaxRetry if ps.job.Job.Spec.MaxRetry != 0 { @@ -39,8 +35,8 @@ func (ps *restartingState) Execute(action vkv1.Action) error { if status.RetryCount >= maxRetry { // Failed is the phase that the job is restarted failed reached the maximum number of retries. - phase = vkv1.Failed - status.State.LastTransitionTime = metav1.Now() + status.State.Phase = vkv1.Failed + return true } else { total := int32(0) for _, task := range ps.job.Job.Spec.Tasks { @@ -48,12 +44,12 @@ func (ps *restartingState) Execute(action vkv1.Action) error { } if total-status.Terminating >= status.MinAvailable { - phase = vkv1.Pending - status.State.LastTransitionTime = metav1.Now() + status.State.Phase = vkv1.Pending + return true } } - status.State.Phase = phase + return false }) } diff --git a/pkg/controllers/job/state/running.go b/pkg/controllers/job/state/running.go index e1c920bcf6..e19fecc151 100644 --- a/pkg/controllers/job/state/running.go +++ b/pkg/controllers/job/state/running.go @@ -17,8 +17,6 @@ limitations under the License. package state import ( - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -30,55 +28,50 @@ type runningState struct { func (ps *runningState) Execute(action vkv1.Action) error { switch action { case vkv1.RestartJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) { - phase := vkv1.Running + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { if status.Terminating != 0 { - phase = vkv1.Restarting - status.State.LastTransitionTime = metav1.Now() + status.State.Phase = vkv1.Restarting status.RetryCount++ + return true } - - status.State.Phase = phase + return false }) case vkv1.AbortJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) { - phase := vkv1.Running + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { if status.Terminating != 0 { - phase = vkv1.Aborting - status.State.LastTransitionTime = metav1.Now() + status.State.Phase = vkv1.Aborting + return true } - status.State.Phase = phase + return false }) case vkv1.TerminateJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) { - phase := vkv1.Running + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { if status.Terminating != 0 { - phase = vkv1.Terminating - status.State.LastTransitionTime = metav1.Now() + status.State.Phase = vkv1.Terminating + return true } - status.State.Phase = phase + return false }) case vkv1.CompleteJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) { + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { phase := vkv1.Completed if status.Terminating != 0 { phase = vkv1.Completing } status.State.Phase = phase - status.State.LastTransitionTime = metav1.Now() + return true }) default: - return SyncJob(ps.job, func(status *vkv1.JobStatus) { - phase := vkv1.Running + return SyncJob(ps.job, func(status *vkv1.JobStatus) bool { if status.Succeeded+status.Failed == TotalTasks(ps.job.Job) { - phase = vkv1.Completed - status.State.LastTransitionTime = metav1.Now() + status.State.Phase = vkv1.Completed + return true } - status.State.Phase = phase + return false }) } } diff --git a/pkg/controllers/job/state/terminating.go b/pkg/controllers/job/state/terminating.go index 5372865196..5ceddecb78 100644 --- a/pkg/controllers/job/state/terminating.go +++ b/pkg/controllers/job/state/terminating.go @@ -17,8 +17,6 @@ limitations under the License. package state import ( - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) @@ -28,13 +26,13 @@ type terminatingState struct { } func (ps *terminatingState) Execute(action vkv1.Action) error { - return KillJob(ps.job, func(status *vkv1.JobStatus) { + return KillJob(ps.job, func(status *vkv1.JobStatus) bool { // If any "alive" pods, still in Terminating phase if status.Terminating != 0 || status.Pending != 0 || status.Running != 0 { - status.State.Phase = vkv1.Terminating + return false } else { status.State.Phase = vkv1.Terminated - status.State.LastTransitionTime = metav1.Now() + return true } }) } From 0fea0f18e389c5ae621583e77caa85f7789e5275 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Mon, 13 May 2019 11:12:40 +0800 Subject: [PATCH 24/68] Enable garbage collector --- cmd/controllers/app/server.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cmd/controllers/app/server.go b/cmd/controllers/app/server.go index 8d0e93a736..57e85b1cf6 100644 --- a/cmd/controllers/app/server.go +++ b/cmd/controllers/app/server.go @@ -40,6 +40,7 @@ import ( "volcano.sh/volcano/cmd/controllers/app/options" vkclient "volcano.sh/volcano/pkg/client/clientset/versioned" + "volcano.sh/volcano/pkg/controllers/garbagecollector" "volcano.sh/volcano/pkg/controllers/job" "volcano.sh/volcano/pkg/controllers/queue" ) @@ -83,10 +84,12 @@ func Run(opt *options.ServerOption) error { jobController := job.NewJobController(kubeClient, kbClient, vkClient) queueController := queue.NewQueueController(kubeClient, kbClient) + garbageCollector := garbagecollector.New(vkClient) run := func(ctx context.Context) { go jobController.Run(ctx.Done()) go queueController.Run(ctx.Done()) + go garbageCollector.Run(ctx.Done()) <-ctx.Done() } From f6bd09143fc86b08e104138fb96b63a467f67237 Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Mon, 13 May 2019 11:22:06 +0800 Subject: [PATCH 25/68] update pb index --- pkg/apis/batch/v1alpha1/job.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/apis/batch/v1alpha1/job.go b/pkg/apis/batch/v1alpha1/job.go index 8a62a7b222..2c92264751 100644 --- a/pkg/apis/batch/v1alpha1/job.go +++ b/pkg/apis/batch/v1alpha1/job.go @@ -78,7 +78,7 @@ type JobSpec struct { // the Job won't be automatically deleted. If this field is set to zero, // the Job becomes eligible to be deleted immediately after it finishes. // +optional - TTLSecondsAfterFinished *int32 `json:"ttlSecondsAfterFinished,omitempty" protobuf:"varint,8,opt,name=ttlSecondsAfterFinished"` + TTLSecondsAfterFinished *int32 `json:"ttlSecondsAfterFinished,omitempty" protobuf:"varint,9,opt,name=ttlSecondsAfterFinished"` } // VolumeSpec defines the specification of Volume, e.g. PVC From 2237aba2b4c18de3ce0af65f8867b90d36c1d0c5 Mon Sep 17 00:00:00 2001 From: asifdxtreme Date: Mon, 13 May 2019 14:19:00 +0530 Subject: [PATCH 26/68] Add Changelog for v0.1 Release --- CHANGELOG.md | 128 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 128 insertions(+) create mode 100644 CHANGELOG.md diff --git a/CHANGELOG.md b/CHANGELOG.md new file mode 100644 index 0000000000..a813afb2b1 --- /dev/null +++ b/CHANGELOG.md @@ -0,0 +1,128 @@ +# Change Log + +## [v0.1](https://github.com/volcano-sh/volcano/tree/v0.1) (2019-05-11) +**Implemented enhancements:** + +- Enable robot for Volcano [\#15](https://github.com/volcano-sh/volcano/issues/15) + +**Closed issues:** + +- Propose moving volume claim validation&generate logic into admission hook [\#139](https://github.com/volcano-sh/volcano/issues/139) +- Sync codes from CodeClub [\#112](https://github.com/volcano-sh/volcano/issues/112) +- Sync kubemark patches into volcano [\#109](https://github.com/volcano-sh/volcano/issues/109) +- Makefile cleanup [\#105](https://github.com/volcano-sh/volcano/issues/105) +- Make OWNERS file for Volcano [\#102](https://github.com/volcano-sh/volcano/issues/102) +- Makefile cleanup [\#69](https://github.com/volcano-sh/volcano/issues/69) +- How do we support Job.Spec update [\#68](https://github.com/volcano-sh/volcano/issues/68) +- make mutating and validating admission controllers consistent [\#63](https://github.com/volcano-sh/volcano/issues/63) +- Labels key can not contain '/' [\#62](https://github.com/volcano-sh/volcano/issues/62) +- Job deletion handler should consider `DeletedFinalStateUnknown` [\#58](https://github.com/volcano-sh/volcano/issues/58) +- Unable get csr when building test cluster [\#54](https://github.com/volcano-sh/volcano/issues/54) +- Reclaim CI failed [\#49](https://github.com/volcano-sh/volcano/issues/49) +- Add example on MPI Job [\#42](https://github.com/volcano-sh/volcano/issues/42) +- Deleting helm chart exits with error [\#35](https://github.com/volcano-sh/volcano/issues/35) +- Update Imports [\#30](https://github.com/volcano-sh/volcano/issues/30) +- CI Build Failure [\#27](https://github.com/volcano-sh/volcano/issues/27) +- Queue controller and related cli [\#16](https://github.com/volcano-sh/volcano/issues/16) +- Update tensorflow example [\#146](https://github.com/volcano-sh/volcano/issues/146) +- Move vkctrl queue to Volcano from kube-batch [\#133](https://github.com/volcano-sh/volcano/issues/133) +- Cherry pick .spec.Capability of Queue [\#131](https://github.com/volcano-sh/volcano/issues/131) +- Sync combine input & output volumes into array logic. [\#122](https://github.com/volcano-sh/volcano/issues/122) +- Fix state machine issue [\#121](https://github.com/volcano-sh/volcano/issues/121) +- 11 tests are failed in CI [\#120](https://github.com/volcano-sh/volcano/issues/120) +- Set default value of PodGroup in admission controller [\#118](https://github.com/volcano-sh/volcano/issues/118) +- Update kube-batch dependency to master [\#114](https://github.com/volcano-sh/volcano/issues/114) +- Support printing version for binaries [\#77](https://github.com/volcano-sh/volcano/issues/77) +- The docker image name should align with binaries' [\#72](https://github.com/volcano-sh/volcano/issues/72) +- Update mpi example to use hostfile [\#71](https://github.com/volcano-sh/volcano/issues/71) +- Add error handling for exit code [\#55](https://github.com/volcano-sh/volcano/issues/55) +- Move job/cache job/apis to upper dir [\#43](https://github.com/volcano-sh/volcano/issues/43) +- Support Task/Job retry [\#40](https://github.com/volcano-sh/volcano/issues/40) +- Fix Failing testcases [\#32](https://github.com/volcano-sh/volcano/issues/32) +- Add e2e test for admission service [\#31](https://github.com/volcano-sh/volcano/issues/31) +- Support TaskSpec level error handling [\#26](https://github.com/volcano-sh/volcano/issues/26) +- Support Job plugins [\#14](https://github.com/volcano-sh/volcano/issues/14) +- Delay pod creation [\#12](https://github.com/volcano-sh/volcano/issues/12) +- Setup travis as CI env [\#7](https://github.com/volcano-sh/volcano/issues/7) + +**Merged pull requests:** + +- admission to get tls certificate from kubeconfig, if tls config not defined in command line [\#152](https://github.com/volcano-sh/volcano/pull/152) ([sivanzcw](https://github.com/sivanzcw)) +- Update tf related resource files [\#150](https://github.com/volcano-sh/volcano/pull/150) ([TommyLike](https://github.com/TommyLike)) +- Print detail message when wait timeout in tests [\#148](https://github.com/volcano-sh/volcano/pull/148) ([TommyLike](https://github.com/TommyLike)) +- Return immediately when failed to generate admission secret [\#145](https://github.com/volcano-sh/volcano/pull/145) ([TommyLike](https://github.com/TommyLike)) +- version related changes [\#144](https://github.com/volcano-sh/volcano/pull/144) ([SrinivasChilveri](https://github.com/SrinivasChilveri)) +- Update volcano crds & sample files [\#143](https://github.com/volcano-sh/volcano/pull/143) ([TommyLike](https://github.com/TommyLike)) +- Fix combine volume feature issues&Adding testcase [\#140](https://github.com/volcano-sh/volcano/pull/140) ([TommyLike](https://github.com/TommyLike)) +- Bump Kube-batch version [\#138](https://github.com/volcano-sh/volcano/pull/138) ([asifdxtreme](https://github.com/asifdxtreme)) +- Moving vkctrl queue to Volcano from kube-batch [\#136](https://github.com/volcano-sh/volcano/pull/136) ([Rajadeepan](https://github.com/Rajadeepan)) +- Fix log tailed issue [\#132](https://github.com/volcano-sh/volcano/pull/132) ([TommyLike](https://github.com/TommyLike)) +- remove enableNamespaceAsQueue in e2e [\#130](https://github.com/volcano-sh/volcano/pull/130) ([hzxuzhonghu](https://github.com/hzxuzhonghu)) +- Add queue controller [\#128](https://github.com/volcano-sh/volcano/pull/128) ([hzxuzhonghu](https://github.com/hzxuzhonghu)) +- Add default queue in admission hook [\#127](https://github.com/volcano-sh/volcano/pull/127) ([TommyLike](https://github.com/TommyLike)) +- \[Issue \#121\]fix state convert [\#126](https://github.com/volcano-sh/volcano/pull/126) ([wangyuqing4](https://github.com/wangyuqing4)) +- Combine input & output volumes [\#124](https://github.com/volcano-sh/volcano/pull/124) ([TommyLike](https://github.com/TommyLike)) +- Move mxnet examples into integration folder [\#123](https://github.com/volcano-sh/volcano/pull/123) ([TommyLike](https://github.com/TommyLike)) +- pkg improvement. [\#119](https://github.com/volcano-sh/volcano/pull/119) ([k82cn](https://github.com/k82cn)) +- Updated kube-batch to vk-kube-batch. [\#115](https://github.com/volcano-sh/volcano/pull/115) ([k82cn](https://github.com/k82cn)) +- Update kube batch source [\#111](https://github.com/volcano-sh/volcano/pull/111) ([TommyLike](https://github.com/TommyLike)) +- Makefile Cleanup. [\#106](https://github.com/volcano-sh/volcano/pull/106) ([k82cn](https://github.com/k82cn)) +- Updated README. [\#104](https://github.com/volcano-sh/volcano/pull/104) ([k82cn](https://github.com/k82cn)) +- Added OWNERS. [\#103](https://github.com/volcano-sh/volcano/pull/103) ([k82cn](https://github.com/k82cn)) +- Register global options when start [\#99](https://github.com/volcano-sh/volcano/pull/99) ([TommyLike](https://github.com/TommyLike)) +- Add TF example [\#98](https://github.com/volcano-sh/volcano/pull/98) ([TommyLike](https://github.com/TommyLike)) +- Updated kube-batch to release-0.4 [\#97](https://github.com/volcano-sh/volcano/pull/97) ([k82cn](https://github.com/k82cn)) +- Added Queue design doc. [\#95](https://github.com/volcano-sh/volcano/pull/95) ([k82cn](https://github.com/k82cn)) +- Clean code [\#92](https://github.com/volcano-sh/volcano/pull/92) ([hzxuzhonghu](https://github.com/hzxuzhonghu)) +- Refine Common Service intro in volcano intro image [\#91](https://github.com/volcano-sh/volcano/pull/91) ([TommyLike](https://github.com/TommyLike)) +- Update volcano intro image [\#90](https://github.com/volcano-sh/volcano/pull/90) ([TommyLike](https://github.com/TommyLike)) +- Abstract a common pod delete func [\#89](https://github.com/volcano-sh/volcano/pull/89) ([hzxuzhonghu](https://github.com/hzxuzhonghu)) +- Fix issue raised from previous MR [\#83](https://github.com/volcano-sh/volcano/pull/83) ([TommyLike](https://github.com/TommyLike)) +- implement error code handling [\#81](https://github.com/volcano-sh/volcano/pull/81) ([hzxuzhonghu](https://github.com/hzxuzhonghu)) +- Move apis&cache into pkg/controllers folder [\#80](https://github.com/volcano-sh/volcano/pull/80) ([TommyLike](https://github.com/TommyLike)) +- Exit code error handling [\#79](https://github.com/volcano-sh/volcano/pull/79) ([hzxuzhonghu](https://github.com/hzxuzhonghu)) +- Update scripts as per docker image [\#78](https://github.com/volcano-sh/volcano/pull/78) ([asifdxtreme](https://github.com/asifdxtreme)) +- Fix task name default and validate it [\#76](https://github.com/volcano-sh/volcano/pull/76) ([hzxuzhonghu](https://github.com/hzxuzhonghu)) +- Cleanup makefile [\#75](https://github.com/volcano-sh/volcano/pull/75) ([TommyLike](https://github.com/TommyLike)) +- Do not handle job update [\#74](https://github.com/volcano-sh/volcano/pull/74) ([hzxuzhonghu](https://github.com/hzxuzhonghu)) +- Use hostfile in MPI tests [\#73](https://github.com/volcano-sh/volcano/pull/73) ([TommyLike](https://github.com/TommyLike)) +- Improve Quick Start Guide & update docker image repo [\#70](https://github.com/volcano-sh/volcano/pull/70) ([asifdxtreme](https://github.com/asifdxtreme)) +- ignore already exist error [\#61](https://github.com/volcano-sh/volcano/pull/61) ([hzxuzhonghu](https://github.com/hzxuzhonghu)) +- fix build error [\#60](https://github.com/volcano-sh/volcano/pull/60) ([hzxuzhonghu](https://github.com/hzxuzhonghu)) +- Support displaying log files [\#59](https://github.com/volcano-sh/volcano/pull/59) ([TommyLike](https://github.com/TommyLike)) +- some minor cleanup and optmization [\#57](https://github.com/volcano-sh/volcano/pull/57) ([hzxuzhonghu](https://github.com/hzxuzhonghu)) +- Upgrade kube-batch to 0.4.2 [\#53](https://github.com/volcano-sh/volcano/pull/53) ([k82cn](https://github.com/k82cn)) +- Fix comment issue for MPI example MR [\#52](https://github.com/volcano-sh/volcano/pull/52) ([TommyLike](https://github.com/TommyLike)) +- Add MPI example and tests [\#51](https://github.com/volcano-sh/volcano/pull/51) ([TommyLike](https://github.com/TommyLike)) +- Support Plugins [\#50](https://github.com/volcano-sh/volcano/pull/50) ([TommyLike](https://github.com/TommyLike)) +- Refresh volcano intro image [\#46](https://github.com/volcano-sh/volcano/pull/46) ([TommyLike](https://github.com/TommyLike)) +- Add admission e2e test [\#45](https://github.com/volcano-sh/volcano/pull/45) ([TommyLike](https://github.com/TommyLike)) +- Support setting queue name in job [\#39](https://github.com/volcano-sh/volcano/pull/39) ([TommyLike](https://github.com/TommyLike)) +- Add ConfigMap for scheduler chart [\#38](https://github.com/volcano-sh/volcano/pull/38) ([TommyLike](https://github.com/TommyLike)) +- Fix slice out of range error [\#37](https://github.com/volcano-sh/volcano/pull/37) ([TommyLike](https://github.com/TommyLike)) +- Support TaskComplete in LifecyclePolicy [\#36](https://github.com/volcano-sh/volcano/pull/36) ([TommyLike](https://github.com/TommyLike)) +- Update Readme with verification details [\#34](https://github.com/volcano-sh/volcano/pull/34) ([asifdxtreme](https://github.com/asifdxtreme)) +- Fixed build error. [\#29](https://github.com/volcano-sh/volcano/pull/29) ([k82cn](https://github.com/k82cn)) +- Support Travis CI [\#28](https://github.com/volcano-sh/volcano/pull/28) ([TommyLike](https://github.com/TommyLike)) +- Fix some chart issues [\#24](https://github.com/volcano-sh/volcano/pull/24) ([TommyLike](https://github.com/TommyLike)) +- Fix volcano chart RBAC issue [\#23](https://github.com/volcano-sh/volcano/pull/23) ([TommyLike](https://github.com/TommyLike)) +- Create default queue for e2e test. [\#22](https://github.com/volcano-sh/volcano/pull/22) ([TommyLike](https://github.com/TommyLike)) +- Update Travis CI batch [\#21](https://github.com/volcano-sh/volcano/pull/21) ([asifdxtreme](https://github.com/asifdxtreme)) +- Add Slack batch [\#20](https://github.com/volcano-sh/volcano/pull/20) ([asifdxtreme](https://github.com/asifdxtreme)) +- Add issue templates [\#19](https://github.com/volcano-sh/volcano/pull/19) ([asifdxtreme](https://github.com/asifdxtreme)) +- Ensure golint binary exists before linting [\#18](https://github.com/volcano-sh/volcano/pull/18) ([TommyLike](https://github.com/TommyLike)) +- Code/add more tests [\#17](https://github.com/volcano-sh/volcano/pull/17) ([TommyLike](https://github.com/TommyLike)) +- Updated README. [\#11](https://github.com/volcano-sh/volcano/pull/11) ([k82cn](https://github.com/k82cn)) +- Added README about scheduler. [\#10](https://github.com/volcano-sh/volcano/pull/10) ([k82cn](https://github.com/k82cn)) +- Added actions/plugins. [\#9](https://github.com/volcano-sh/volcano/pull/9) ([k82cn](https://github.com/k82cn)) +- Fixed code fmt. [\#8](https://github.com/volcano-sh/volcano/pull/8) ([k82cn](https://github.com/k82cn)) +- Codes/refactor controllers [\#6](https://github.com/volcano-sh/volcano/pull/6) ([TommyLike](https://github.com/TommyLike)) +- Support helm chart [\#5](https://github.com/volcano-sh/volcano/pull/5) ([TommyLike](https://github.com/TommyLike)) +- Fix some typos in code and document [\#3](https://github.com/volcano-sh/volcano/pull/3) ([TommyLike](https://github.com/TommyLike)) +- Support Adm controllers [\#2](https://github.com/volcano-sh/volcano/pull/2) ([TommyLike](https://github.com/TommyLike)) +- Rename hpw.cloud keyword to volcano.sh [\#1](https://github.com/volcano-sh/volcano/pull/1) ([TommyLike](https://github.com/TommyLike)) +- Improve README.md documentation [\#48](https://github.com/volcano-sh/volcano/pull/48) ([quinton-hoole](https://github.com/quinton-hoole)) + + + +\* *This Change Log was automatically generated by [github_changelog_generator](https://github.com/skywinder/Github-Changelog-Generator)* \ No newline at end of file From 1da7f43219e409b936e6a8014c52a8d976aeab22 Mon Sep 17 00:00:00 2001 From: thandayuthapani Date: Mon, 13 May 2019 17:58:40 +0530 Subject: [PATCH 27/68] Add JobType in cli command --- pkg/apis/batch/v1alpha1/labels.go | 1 + pkg/cli/job/list.go | 13 +++++++++---- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/pkg/apis/batch/v1alpha1/labels.go b/pkg/apis/batch/v1alpha1/labels.go index 99e016f11e..a0fdfbd1f2 100644 --- a/pkg/apis/batch/v1alpha1/labels.go +++ b/pkg/apis/batch/v1alpha1/labels.go @@ -22,4 +22,5 @@ const ( JobNamespaceKey = "volcano.sh/job-namespace" DefaultTaskSpec = "default" JobVersion = "volcano.sh/job-version" + JobTypeKey = "volcano.sh/job-type" ) diff --git a/pkg/cli/job/list.go b/pkg/cli/job/list.go index 7ca63522d8..bf6afac074 100644 --- a/pkg/cli/job/list.go +++ b/pkg/cli/job/list.go @@ -45,6 +45,7 @@ const ( Succeeded string = "Succeeded" Failed string = "Failed" RetryCount string = "RetryCount" + JobType string = "JobType" ) var listJobFlags = &listFlags{} @@ -78,8 +79,8 @@ func ListJobs() error { func PrintJobs(jobs *v1alpha1.JobList, writer io.Writer) { maxNameLen := getMaxNameLen(jobs) - _, err := fmt.Fprintf(writer, fmt.Sprintf("%%-%ds%%-25s%%-12s%%-12s%%-6s%%-10s%%-10s%%-12s%%-10s%%-12s\n", maxNameLen), - Name, Creation, Phase, Replicas, Min, Pending, Running, Succeeded, Failed, RetryCount) + _, err := fmt.Fprintf(writer, fmt.Sprintf("%%-%ds%%-25s%%-12s%%-12s%%-12s%%-6s%%-10s%%-10s%%-12s%%-10s%%-12s\n", maxNameLen), + Name, Creation, Phase, JobType, Replicas, Min, Pending, Running, Succeeded, Failed, RetryCount) if err != nil { fmt.Printf("Failed to print list command result: %s.\n", err) } @@ -89,8 +90,12 @@ func PrintJobs(jobs *v1alpha1.JobList, writer io.Writer) { for _, ts := range job.Spec.Tasks { replicas += ts.Replicas } - _, err = fmt.Fprintf(writer, fmt.Sprintf("%%-%ds%%-25s%%-12s%%-12d%%-6d%%-10d%%-10d%%-12d%%-10d%%-12d\n", maxNameLen), - job.Name, job.CreationTimestamp.Format("2006-01-02 15:04:05"), job.Status.State.Phase, replicas, + jobType := job.ObjectMeta.Labels[v1alpha1.JobTypeKey] + if jobType == "" { + jobType = "Batch" + } + _, err = fmt.Fprintf(writer, fmt.Sprintf("%%-%ds%%-25s%%-12s%%-12s%%-12d%%-6d%%-10d%%-10d%%-12d%%-10d%%-12d\n", maxNameLen), + job.Name, job.CreationTimestamp.Format("2006-01-02 15:04:05"), job.Status.State.Phase, jobType, replicas, job.Status.MinAvailable, job.Status.Pending, job.Status.Running, job.Status.Succeeded, job.Status.Failed, job.Status.RetryCount) if err != nil { fmt.Printf("Failed to print list command result: %s.\n", err) From b4a75bdb40379307985082f96e60c925fcf1a878 Mon Sep 17 00:00:00 2001 From: Mohammad Asif Siddiqui Date: Tue, 14 May 2019 11:21:25 +0530 Subject: [PATCH 28/68] Update CHANGELOG.md --- CHANGELOG.md | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a813afb2b1..276ffaeab0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,9 +1,6 @@ # Change Log ## [v0.1](https://github.com/volcano-sh/volcano/tree/v0.1) (2019-05-11) -**Implemented enhancements:** - -- Enable robot for Volcano [\#15](https://github.com/volcano-sh/volcano/issues/15) **Closed issues:** @@ -125,4 +122,4 @@ -\* *This Change Log was automatically generated by [github_changelog_generator](https://github.com/skywinder/Github-Changelog-Generator)* \ No newline at end of file +\* *This Change Log was automatically generated by [github_changelog_generator](https://github.com/skywinder/Github-Changelog-Generator)* From f92ae58f69830c1070e1b5fa925a146e025120be Mon Sep 17 00:00:00 2001 From: xuzhonghu Date: Tue, 14 May 2019 17:26:09 +0800 Subject: [PATCH 29/68] add e2e --- test/e2e/util.go | 38 +++++++++++++++++++++++++++++++++++--- 1 file changed, 35 insertions(+), 3 deletions(-) diff --git a/test/e2e/util.go b/test/e2e/util.go index 0e4bbbe4ac..53e274a9f7 100644 --- a/test/e2e/util.go +++ b/test/e2e/util.go @@ -278,6 +278,8 @@ type jobSpec struct { min int32 plugins map[string][]string volumes []vkv1.VolumeSpec + // ttl seconds after job finished + ttl *int32 } func getNS(context *context, job *jobSpec) string { @@ -305,9 +307,10 @@ func createJobInner(context *context, jobSpec *jobSpec) (*vkv1.Job, error) { Namespace: ns, }, Spec: vkv1.JobSpec{ - Policies: jobSpec.policies, - Queue: jobSpec.queue, - Plugins: jobSpec.plugins, + Policies: jobSpec.policies, + Queue: jobSpec.queue, + Plugins: jobSpec.plugins, + TTLSecondsAfterFinished: jobSpec.ttl, }, } @@ -672,6 +675,35 @@ func createReplicaSet(context *context, name string, rep int32, img string, req return deployment } +func waitJobCleanedUp(ctx *context, job *vkv1.Job) error { + var additionalError error + err := wait.Poll(100*time.Millisecond, oneMinute, func() (bool, error) { + job, err := ctx.vkclient.BatchV1alpha1().Jobs(job.Namespace).Get(job.Name, metav1.GetOptions{}) + if err != nil && !errors.IsNotFound(err) { + return false, nil + } + if job != nil { + additionalError = fmt.Errorf("job %s/%s still exist", job.Namespace, job.Name) + return false, nil + } + + pg, err := ctx.kbclient.SchedulingV1alpha1().PodGroups(job.Namespace).Get(job.Name, metav1.GetOptions{}) + if err != nil && !errors.IsNotFound(err) { + return false, nil + } + if pg != nil { + additionalError = fmt.Errorf("pdgroup %s/%s still exist", job.Namespace, job.Name) + return false, nil + } + + return true, nil + }) + if err != nil && strings.Contains(err.Error(), timeOutMessage) { + return fmt.Errorf("[Wait time out]: %s", additionalError) + } + return err +} + func deleteReplicaSet(ctx *context, name string) error { foreground := metav1.DeletePropagationForeground return ctx.kubeclient.AppsV1().ReplicaSets(ctx.namespace).Delete(name, &metav1.DeleteOptions{ From 80d498e2c042ea4dc04c7353af07fce1d01b4af7 Mon Sep 17 00:00:00 2001 From: TommyLike Date: Tue, 14 May 2019 17:09:09 +0800 Subject: [PATCH 30/68] Fix resource calculation issue --- pkg/controllers/job/job_controller_actions.go | 7 +-- pkg/controllers/job/job_controller_util.go | 11 ++++ test/e2e/job_controlled_resource.go | 57 +++++++++++++++++++ test/e2e/util.go | 6 +- 4 files changed, 75 insertions(+), 6 deletions(-) diff --git a/pkg/controllers/job/job_controller_actions.go b/pkg/controllers/job/job_controller_actions.go index 129524bc46..37b390d48d 100644 --- a/pkg/controllers/job/job_controller_actions.go +++ b/pkg/controllers/job/job_controller_actions.go @@ -28,7 +28,6 @@ import ( metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" kbv1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1" - kbapi "github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api" vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/apis/helpers" "volcano.sh/volcano/pkg/controllers/apis" @@ -486,7 +485,7 @@ func (cc *Controller) calcPGMinResources(job *vkv1.Job) *v1.ResourceList { sort.Sort(tasksPriority) - minAvailableTasksRes := kbapi.EmptyResource() + minAvailableTasksRes := v1.ResourceList{} podCnt := int32(0) for _, task := range tasksPriority { for i := int32(0); i < task.Replicas; i++ { @@ -495,10 +494,10 @@ func (cc *Controller) calcPGMinResources(job *vkv1.Job) *v1.ResourceList { } podCnt++ for _, c := range task.Template.Spec.Containers { - minAvailableTasksRes.Add(kbapi.NewResource(c.Resources.Requests)) + addResourceList(minAvailableTasksRes, c.Resources.Requests) } } } - return minAvailableTasksRes.Convert2K8sResource() + return &minAvailableTasksRes } diff --git a/pkg/controllers/job/job_controller_util.go b/pkg/controllers/job/job_controller_util.go index 70f96616e0..22c7458923 100644 --- a/pkg/controllers/job/job_controller_util.go +++ b/pkg/controllers/job/job_controller_util.go @@ -187,6 +187,17 @@ func applyPolicies(job *vkv1.Job, req *apis.Request) vkv1.Action { return vkv1.SyncJobAction } +func addResourceList(list, new v1.ResourceList) { + for name, quantity := range new { + if value, ok := list[name]; !ok { + list[name] = *quantity.Copy() + } else { + value.Add(quantity) + list[name] = value + } + } +} + type TaskPriority struct { priority int32 diff --git a/test/e2e/job_controlled_resource.go b/test/e2e/job_controlled_resource.go index f6277f872d..b1f3ffb0fe 100644 --- a/test/e2e/job_controlled_resource.go +++ b/test/e2e/job_controlled_resource.go @@ -19,6 +19,8 @@ package e2e import ( . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" + v12 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/resource" "k8s.io/apimachinery/pkg/apis/meta/v1" "volcano.sh/volcano/pkg/apis/batch/v1alpha1" ) @@ -68,4 +70,59 @@ var _ = Describe("Job E2E Test: Test Job PVCs", func() { "PVC name should be generated for manually specified.") } }) + + It("Generate PodGroup and valid minResource when creating job", func() { + jobName := "job-name-podgroup" + namespace := "test" + context := initTestContext() + defer cleanupTestContext(context) + + resource := v12.ResourceList{ + "cpu": resource.MustParse("1000m"), + "memory": resource.MustParse("1000Mi"), + "nvidia.com/gpu": resource.MustParse("1"), + } + + job := createJob(context, &jobSpec{ + namespace: namespace, + name: jobName, + tasks: []taskSpec{ + { + img: defaultNginxImage, + min: 1, + rep: 1, + name: "task-1", + req: resource, + limit: resource, + }, + { + img: defaultNginxImage, + min: 1, + rep: 1, + name: "task-2", + req: resource, + limit: resource, + }, + }, + }) + + expected := map[string]int64{ + "cpu": 2, + "memory": 1024 * 1024 * 2000, + "nvidia.com/gpu": 2, + } + + err := waitJobStatePending(context, job) + Expect(err).NotTo(HaveOccurred()) + + pGroup, err := context.kbclient.SchedulingV1alpha1().PodGroups(namespace).Get(jobName, v1.GetOptions{}) + Expect(err).NotTo(HaveOccurred()) + + for name, q := range *pGroup.Spec.MinResources { + value, ok := expected[string(name)] + Expect(ok).To(Equal(true), "Resource %s should exists in PodGroup", name) + Expect(q.Value()).To(Equal(value), "Resource %s 's value should equal to %d", name, value) + } + + }) }) diff --git a/test/e2e/util.go b/test/e2e/util.go index 0e4bbbe4ac..2e75c1c3a0 100644 --- a/test/e2e/util.go +++ b/test/e2e/util.go @@ -262,6 +262,7 @@ type taskSpec struct { workingDir string hostport int32 req v1.ResourceList + limit v1.ResourceList affinity *v1.Affinity labels map[string]string policies []vkv1.LifecyclePolicy @@ -335,7 +336,7 @@ func createJobInner(context *context, jobSpec *jobSpec) (*vkv1.Job, error) { Spec: v1.PodSpec{ SchedulerName: "kube-batch", RestartPolicy: restartPolicy, - Containers: createContainers(task.img, task.command, task.workingDir, task.req, task.hostport), + Containers: createContainers(task.img, task.command, task.workingDir, task.req, task.limit, task.hostport), Affinity: task.affinity, }, }, @@ -594,13 +595,14 @@ func waitQueueStatus(condition func() (bool, error)) error { return wait.Poll(100*time.Millisecond, oneMinute, condition) } -func createContainers(img, command, workingDir string, req v1.ResourceList, hostport int32) []v1.Container { +func createContainers(img, command, workingDir string, req, limit v1.ResourceList, hostport int32) []v1.Container { var imageRepo []string container := v1.Container{ Image: img, ImagePullPolicy: v1.PullIfNotPresent, Resources: v1.ResourceRequirements{ Requests: req, + Limits: limit, }, } if strings.Index(img, ":") < 0 { From 7902db1e572a8c69b9cbc805c8bbb711b558334b Mon Sep 17 00:00:00 2001 From: asifdxtreme Date: Tue, 14 May 2019 14:47:44 +0530 Subject: [PATCH 31/68] Rename scheduler to kube-batch --- Makefile | 6 +++--- cmd/{scheduler => kube-batch}/main.go | 0 hack/run-e2e-kind.sh | 4 ++-- installer/chart/volcano/values.yaml | 2 +- installer/dockerfile/{scheduler => kube-batch}/Dockerfile | 4 ++-- 5 files changed, 8 insertions(+), 8 deletions(-) rename cmd/{scheduler => kube-batch}/main.go (100%) rename installer/dockerfile/{scheduler => kube-batch}/Dockerfile (90%) diff --git a/Makefile b/Makefile index 3477a5afd1..25bb4215fc 100644 --- a/Makefile +++ b/Makefile @@ -19,7 +19,7 @@ init: mkdir -p ${BIN_DIR} kube-batch: init - go build -ldflags ${LD_FLAGS} -o=${BIN_DIR}/kube-batch ./cmd/scheduler + go build -ldflags ${LD_FLAGS} -o=${BIN_DIR}/vk-kube-batch ./cmd/kube-batch vk-controllers: init go build -ldflags ${LD_FLAGS} -o=${BIN_DIR}/vk-controllers ./cmd/controllers @@ -33,12 +33,12 @@ vkctl: init image_bins: go get github.com/mitchellh/gox CGO_ENABLED=0 gox -osarch=${REL_OSARCH} -ldflags ${LD_FLAGS} -output ${BIN_DIR}/${REL_OSARCH}/vkctl ./cmd/cli - for name in controllers scheduler admission; do\ + for name in controllers kube-batch admission; do\ CGO_ENABLED=0 gox -osarch=${REL_OSARCH} -ldflags ${LD_FLAGS} -output ${BIN_DIR}/${REL_OSARCH}/vk-$$name ./cmd/$$name; \ done images: image_bins - for name in controllers scheduler admission; do\ + for name in controllers kube-batch admission; do\ cp ${BIN_DIR}/${REL_OSARCH}/vk-$$name ./installer/dockerfile/$$name/; \ docker build --no-cache -t $(IMAGE_PREFIX)-$$name:$(TAG) ./installer/dockerfile/$$name; \ rm installer/dockerfile/$$name/vk-$$name; \ diff --git a/cmd/scheduler/main.go b/cmd/kube-batch/main.go similarity index 100% rename from cmd/scheduler/main.go rename to cmd/kube-batch/main.go diff --git a/hack/run-e2e-kind.sh b/hack/run-e2e-kind.sh index 796c885ae4..341c8899de 100755 --- a/hack/run-e2e-kind.sh +++ b/hack/run-e2e-kind.sh @@ -58,7 +58,7 @@ function install-volcano { echo "Loading docker images into kind cluster" kind load docker-image ${IMAGE_PREFIX}-controllers:${TAG} ${CLUSTER_CONTEXT} - kind load docker-image ${IMAGE_PREFIX}-scheduler:${TAG} ${CLUSTER_CONTEXT} + kind load docker-image ${IMAGE_PREFIX}-kube-batch:${TAG} ${CLUSTER_CONTEXT} kind load docker-image ${IMAGE_PREFIX}-admission:${TAG} ${CLUSTER_CONTEXT} kind load docker-image ${MPI_EXAMPLE_IMAGE} ${CLUSTER_CONTEXT} @@ -84,7 +84,7 @@ function generate-log { echo "Generating volcano log files" kubectl logs deployment/${CLUSTER_NAME}-admission -n kube-system > volcano-admission.log kubectl logs deployment/${CLUSTER_NAME}-controllers -n kube-system > volcano-controller.log - kubectl logs deployment/${CLUSTER_NAME}-scheduler -n kube-system > volcano-scheduler.log + kubectl logs deployment/${CLUSTER_NAME}-kube-batch -n kube-system > volcano-kube-batch.log } # clean up diff --git a/installer/chart/volcano/values.yaml b/installer/chart/volcano/values.yaml index 3a2fcc61aa..41137671eb 100644 --- a/installer/chart/volcano/values.yaml +++ b/installer/chart/volcano/values.yaml @@ -1,7 +1,7 @@ basic: image_tag_version: "latest" controller_image_name: "volcanosh/vk-controllers" - scheduler_image_name: "volcanosh/vk-scheduler" + scheduler_image_name: "volcanosh/vk-kube-batch" admission_image_name: "volcanosh/vk-admission" admission_secret_name: "volcano-admission-secret" image_pull_secret: "" diff --git a/installer/dockerfile/scheduler/Dockerfile b/installer/dockerfile/kube-batch/Dockerfile similarity index 90% rename from installer/dockerfile/scheduler/Dockerfile rename to installer/dockerfile/kube-batch/Dockerfile index 8b8d6d640d..616c6e16fe 100644 --- a/installer/dockerfile/scheduler/Dockerfile +++ b/installer/dockerfile/kube-batch/Dockerfile @@ -15,5 +15,5 @@ FROM alpine:latest -ADD vk-scheduler /vk-scheduler -ENTRYPOINT ["/vk-scheduler"] +ADD vk-kube-batch /vk-kube-batch +ENTRYPOINT ["/vk-kube-batch"] From 3dad87de91535c6ddf99a6e873897855e9388d8f Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Wed, 15 May 2019 11:00:16 +0530 Subject: [PATCH 32/68] Added ttl validation in admission controller --- pkg/admission/admit_job.go | 5 +++++ test/e2e/admission.go | 30 ++++++++++++++++++++++++++++++ 2 files changed, 35 insertions(+) diff --git a/pkg/admission/admit_job.go b/pkg/admission/admit_job.go index 7698895484..45faf9dc2f 100644 --- a/pkg/admission/admit_job.go +++ b/pkg/admission/admit_job.go @@ -86,6 +86,11 @@ func validateJob(job v1alpha1.Job, reviewResponse *v1beta1.AdmissionResponse) st return fmt.Sprintf("'maxRetry' cannot be less than zero.") } + if job.Spec.TTLSecondsAfterFinished != nil && *job.Spec.TTLSecondsAfterFinished < 0 { + reviewResponse.Allowed = false + return fmt.Sprintf("'ttlSecondsAfterFinished' cannot be less than zero.") + } + if len(job.Spec.Tasks) == 0 { reviewResponse.Allowed = false return fmt.Sprintf("No task specified in job spec") diff --git a/test/e2e/admission.go b/test/e2e/admission.go index c7b25f4efe..60cd71a564 100644 --- a/test/e2e/admission.go +++ b/test/e2e/admission.go @@ -180,4 +180,34 @@ var _ = Describe("Job E2E Test: Test Admission service", func() { Expect(createdJob.Spec.Queue).Should(Equal("default"), "Job queue attribute would default to 'default' ") }) + + It("ttl illegal", func() { + jobName := "job-ttl-illegal" + namespace := "test" + var ttl int32 + ttl = -1 + context := initTestContext() + defer cleanupTestContext(context) + + _, err := createJobInner(context, &jobSpec{ + min: 1, + namespace: namespace, + name: jobName, + ttl: &ttl, + tasks: []taskSpec{ + { + img: defaultNginxImage, + req: oneCPU, + min: 1, + rep: 1, + name: "taskname", + }, + }, + }) + Expect(err).To(HaveOccurred()) + stError, ok := err.(*errors.StatusError) + Expect(ok).To(Equal(true)) + Expect(stError.ErrStatus.Code).To(Equal(int32(500))) + Expect(stError.ErrStatus.Message).To(ContainSubstring("'ttlSecondsAfterFinished' cannot be less than zero.")) + }) }) From fa65bcbe60e92c8549a755f85389dbb22a7f4d3a Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Wed, 15 May 2019 17:26:04 +0530 Subject: [PATCH 33/68] moved some of admission e2e test cases as ut test cases --- .travis.yml | 3 + pkg/admission/admit_job_test.go | 291 ++++++++++++++++++++++ pkg/controllers/queue/queue_controller.go | 2 +- test/e2e/admission.go | 159 ------------ 4 files changed, 295 insertions(+), 160 deletions(-) create mode 100644 pkg/admission/admit_job_test.go diff --git a/.travis.yml b/.travis.yml index 323510fd61..47a8a6a858 100644 --- a/.travis.yml +++ b/.travis.yml @@ -15,6 +15,9 @@ jobs: - go get -u golang.org/x/lint/golint script: - make verify + - stage: UT Tests + script: + - make unit-test - stage: E2E Tests before_script: # Download kubectl diff --git a/pkg/admission/admit_job_test.go b/pkg/admission/admit_job_test.go new file mode 100644 index 0000000000..35bb645182 --- /dev/null +++ b/pkg/admission/admit_job_test.go @@ -0,0 +1,291 @@ +package admission + +import ( + "strings" + "testing" + + "k8s.io/api/admission/v1beta1" + v1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + v1alpha1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" +) + +func TestValidateExecution(t *testing.T) { + + namespace := "test" + var invTtl int32 = -1 + + testCases := []struct { + Name string + Job v1alpha1.Job + ExpectErr bool + reviewResponse v1beta1.AdmissionResponse + ret string + }{ + { + Name: "validate valid-job", + Job: v1alpha1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "valid-Job", + Namespace: namespace, + }, + Spec: v1alpha1.JobSpec{ + MinAvailable: 1, + Tasks: []v1alpha1.TaskSpec{ + { + Name: "task-1", + Replicas: 1, + Template: v1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{"name": "test"}, + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "fake-name", + Image: "busybox:1.24", + }, + }, + }, + }, + }, + }, + }, + }, + reviewResponse: v1beta1.AdmissionResponse{Allowed: true}, + ret: "", + ExpectErr: false, + }, + // duplicate task name + { + Name: "duplicate-task-job", + Job: v1alpha1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "duplicate-task-job", + Namespace: namespace, + }, + Spec: v1alpha1.JobSpec{ + MinAvailable: 1, + Tasks: []v1alpha1.TaskSpec{ + { + Name: "duplicated-task-1", + Replicas: 1, + Template: v1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{"name": "test"}, + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "fake-name", + Image: "busybox:1.24", + }, + }, + }, + }, + }, + { + Name: "duplicated-task-1", + Replicas: 1, + Template: v1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{"name": "test"}, + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "fake-name", + Image: "busybox:1.24", + }, + }, + }, + }, + }, + }, + }, + }, + reviewResponse: v1beta1.AdmissionResponse{Allowed: true}, + ret: "duplicated task name duplicated-task-1", + ExpectErr: true, + }, + // Duplicated Policy Event + { + Name: "job-policy-duplicated", + Job: v1alpha1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job-policy-duplicated", + Namespace: namespace, + }, + Spec: v1alpha1.JobSpec{ + MinAvailable: 1, + Tasks: []v1alpha1.TaskSpec{ + { + Name: "task-1", + Replicas: 1, + Template: v1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{"name": "test"}, + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "fake-name", + Image: "busybox:1.24", + }, + }, + }, + }, + }, + }, + Policies: []v1alpha1.LifecyclePolicy{ + { + Event: v1alpha1.PodFailedEvent, + Action: v1alpha1.AbortJobAction, + }, + { + Event: v1alpha1.PodFailedEvent, + Action: v1alpha1.RestartJobAction, + }, + }, + }, + }, + reviewResponse: v1beta1.AdmissionResponse{Allowed: true}, + ret: "duplicate", + ExpectErr: true, + }, + // Min Available illegal + { + Name: "Min Available illegal", + Job: v1alpha1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job-min-illegal", + Namespace: namespace, + }, + Spec: v1alpha1.JobSpec{ + MinAvailable: 2, + Tasks: []v1alpha1.TaskSpec{ + { + Name: "task-1", + Replicas: 1, + Template: v1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{"name": "test"}, + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "fake-name", + Image: "busybox:1.24", + }, + }, + }, + }, + }, + }, + }, + }, + reviewResponse: v1beta1.AdmissionResponse{Allowed: true}, + ret: "'minAvailable' should not be greater than total replicas in tasks", + ExpectErr: true, + }, + // Job Plugin illegal + { + Name: "Job Plugin illegal", + Job: v1alpha1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job-plugin-illegal", + Namespace: namespace, + }, + Spec: v1alpha1.JobSpec{ + MinAvailable: 1, + Tasks: []v1alpha1.TaskSpec{ + { + Name: "task-1", + Replicas: 1, + Template: v1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{"name": "test"}, + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "fake-name", + Image: "busybox:1.24", + }, + }, + }, + }, + }, + }, + Plugins: map[string][]string{ + "big_plugin": {}, + }, + }, + }, + reviewResponse: v1beta1.AdmissionResponse{Allowed: true}, + ret: "unable to find job plugin: big_plugin", + ExpectErr: true, + }, + // ttl-illegal + { + Name: "job-ttl-illegal", + Job: v1alpha1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job-ttl-illegal", + Namespace: namespace, + }, + Spec: v1alpha1.JobSpec{ + MinAvailable: 1, + Tasks: []v1alpha1.TaskSpec{ + { + Name: "task-1", + Replicas: 1, + Template: v1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{"name": "test"}, + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "fake-name", + Image: "busybox:1.24", + }, + }, + }, + }, + }, + }, + TTLSecondsAfterFinished: &invTtl, + }, + }, + reviewResponse: v1beta1.AdmissionResponse{Allowed: true}, + ret: "'ttlSecondsAfterFinished' cannot be less than zero", + ExpectErr: true, + }, + } + + for _, testCase := range testCases { + + ret := validateJob(testCase.Job, &testCase.reviewResponse) + //fmt.Printf("test-case name:%s, ret:%v testCase.reviewResponse:%v \n", testCase.Name, ret,testCase.reviewResponse) + if testCase.ExpectErr == true && ret == "" { + t.Errorf("%s: test case Expect error msg :%s, but got nil.", testCase.Name, testCase.ret) + } + if testCase.ExpectErr == true && testCase.reviewResponse.Allowed != false { + t.Errorf("%s: test case Expect Allowed as false but got true.", testCase.Name) + } + if testCase.ExpectErr == true && !strings.Contains(ret, testCase.ret) { + t.Errorf("%s: test case Expect error msg :%s, but got diff error %v", testCase.Name, testCase.ret, ret) + } + + if testCase.ExpectErr == false && ret != "" { + t.Errorf("%s: test case Expect no error, but got error %v", testCase.Name, ret) + } + if testCase.ExpectErr == false && testCase.reviewResponse.Allowed != true { + t.Errorf("%s: test case Expect Allowed as true but got false. %v", testCase.Name, testCase.reviewResponse) + } + + } + +} diff --git a/pkg/controllers/queue/queue_controller.go b/pkg/controllers/queue/queue_controller.go index 544dcadbb3..2a2a828f82 100644 --- a/pkg/controllers/queue/queue_controller.go +++ b/pkg/controllers/queue/queue_controller.go @@ -175,7 +175,7 @@ func (c *Controller) syncQueue(key string) error { queue, err := c.queueLister.Get(key) if err != nil { if errors.IsNotFound(err) { - glog.V(2).Infof("queue %s has been deleted", queue) + glog.V(2).Infof("queue %s has been deleted", key) return nil } return err diff --git a/test/e2e/admission.go b/test/e2e/admission.go index 60cd71a564..968240ac82 100644 --- a/test/e2e/admission.go +++ b/test/e2e/admission.go @@ -19,140 +19,10 @@ package e2e import ( . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" - "k8s.io/apimachinery/pkg/api/errors" "k8s.io/apimachinery/pkg/apis/meta/v1" - "volcano.sh/volcano/pkg/apis/batch/v1alpha1" ) var _ = Describe("Job E2E Test: Test Admission service", func() { - It("Duplicated Task Name", func() { - jobName := "job-duplicated" - namespace := "test" - context := initTestContext() - defer cleanupTestContext(context) - rep := clusterSize(context, oneCPU) - - _, err := createJobInner(context, &jobSpec{ - namespace: namespace, - name: jobName, - tasks: []taskSpec{ - { - img: defaultNginxImage, - req: oneCPU, - min: rep, - rep: rep, - name: "duplicated", - }, - { - img: defaultNginxImage, - req: oneCPU, - min: rep, - rep: rep, - name: "duplicated", - }, - }, - }) - Expect(err).To(HaveOccurred()) - stError, ok := err.(*errors.StatusError) - Expect(ok).To(Equal(true)) - Expect(stError.ErrStatus.Code).To(Equal(int32(500))) - Expect(stError.ErrStatus.Message).To(ContainSubstring("duplicated task name")) - }) - - It("Duplicated Policy Event", func() { - jobName := "job-policy-duplicated" - namespace := "test" - context := initTestContext() - defer cleanupTestContext(context) - rep := clusterSize(context, oneCPU) - - _, err := createJobInner(context, &jobSpec{ - namespace: namespace, - name: jobName, - tasks: []taskSpec{ - { - img: defaultNginxImage, - req: oneCPU, - min: rep, - rep: rep, - name: "taskname", - }, - }, - policies: []v1alpha1.LifecyclePolicy{ - { - Event: v1alpha1.PodFailedEvent, - Action: v1alpha1.AbortJobAction, - }, - { - Event: v1alpha1.PodFailedEvent, - Action: v1alpha1.RestartJobAction, - }, - }, - }) - Expect(err).To(HaveOccurred()) - stError, ok := err.(*errors.StatusError) - Expect(ok).To(Equal(true)) - Expect(stError.ErrStatus.Code).To(Equal(int32(500))) - Expect(stError.ErrStatus.Message).To(ContainSubstring("duplicate event PodFailed")) - }) - - It("Min Available illegal", func() { - jobName := "job-min-illegal" - namespace := "test" - context := initTestContext() - defer cleanupTestContext(context) - rep := clusterSize(context, oneCPU) - - _, err := createJobInner(context, &jobSpec{ - min: rep * 2, - namespace: namespace, - name: jobName, - tasks: []taskSpec{ - { - img: defaultNginxImage, - req: oneCPU, - min: rep, - rep: rep, - name: "taskname", - }, - }, - }) - Expect(err).To(HaveOccurred()) - stError, ok := err.(*errors.StatusError) - Expect(ok).To(Equal(true)) - Expect(stError.ErrStatus.Code).To(Equal(int32(500))) - Expect(stError.ErrStatus.Message).To(ContainSubstring("'minAvailable' should not be greater than total replicas in tasks")) - }) - - It("Job Plugin illegal", func() { - jobName := "job-plugin-illegal" - namespace := "test" - context := initTestContext() - defer cleanupTestContext(context) - - _, err := createJobInner(context, &jobSpec{ - min: 1, - namespace: namespace, - name: jobName, - plugins: map[string][]string{ - "big_plugin": {}, - }, - tasks: []taskSpec{ - { - img: defaultNginxImage, - req: oneCPU, - min: 1, - rep: 1, - name: "taskname", - }, - }, - }) - Expect(err).To(HaveOccurred()) - stError, ok := err.(*errors.StatusError) - Expect(ok).To(Equal(true)) - Expect(stError.ErrStatus.Code).To(Equal(int32(500))) - Expect(stError.ErrStatus.Message).To(ContainSubstring("unable to find job plugin: big_plugin")) - }) It("Default queue would be added", func() { jobName := "job-default-queue" @@ -181,33 +51,4 @@ var _ = Describe("Job E2E Test: Test Admission service", func() { "Job queue attribute would default to 'default' ") }) - It("ttl illegal", func() { - jobName := "job-ttl-illegal" - namespace := "test" - var ttl int32 - ttl = -1 - context := initTestContext() - defer cleanupTestContext(context) - - _, err := createJobInner(context, &jobSpec{ - min: 1, - namespace: namespace, - name: jobName, - ttl: &ttl, - tasks: []taskSpec{ - { - img: defaultNginxImage, - req: oneCPU, - min: 1, - rep: 1, - name: "taskname", - }, - }, - }) - Expect(err).To(HaveOccurred()) - stError, ok := err.(*errors.StatusError) - Expect(ok).To(Equal(true)) - Expect(stError.ErrStatus.Code).To(Equal(int32(500))) - Expect(stError.ErrStatus.Message).To(ContainSubstring("'ttlSecondsAfterFinished' cannot be less than zero.")) - }) }) From e1edeeb71b57c52814fb9c054fc1cf7be7d40568 Mon Sep 17 00:00:00 2001 From: TommyLike Date: Thu, 16 May 2019 10:00:20 +0800 Subject: [PATCH 34/68] Support job level priorityClassName --- pkg/admission/admit_job_test.go | 16 ++ pkg/admission/mutate_job.go | 31 ++-- pkg/admission/mutate_job_test.go | 141 ++++++++++++++++++ pkg/apis/batch/v1alpha1/job.go | 4 + pkg/controllers/job/job_controller_actions.go | 7 +- 5 files changed, 187 insertions(+), 12 deletions(-) create mode 100644 pkg/admission/mutate_job_test.go diff --git a/pkg/admission/admit_job_test.go b/pkg/admission/admit_job_test.go index 35bb645182..b62b1a94a6 100644 --- a/pkg/admission/admit_job_test.go +++ b/pkg/admission/admit_job_test.go @@ -1,3 +1,19 @@ +/* +Copyright 2019 The Volcano 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 admission import ( diff --git a/pkg/admission/mutate_job.go b/pkg/admission/mutate_job.go index 6b39268586..637761f133 100644 --- a/pkg/admission/mutate_job.go +++ b/pkg/admission/mutate_job.go @@ -74,28 +74,41 @@ func MutateJobs(ar v1beta1.AdmissionReview) *v1beta1.AdmissionResponse { func createPatch(job v1alpha1.Job) ([]byte, error) { var patch []patchOperation - patch = append(patch, mutateSpec(job.Spec.Tasks, "/spec/tasks")...) + pathQueue := patchDefaultQueue(job) + if pathQueue != nil { + patch = append(patch, *pathQueue) + } + pathSpec := mutateSpec(job.Spec.Tasks, "/spec/tasks") + if pathSpec != nil { + patch = append(patch, *pathSpec) + } + return json.Marshal(patch) +} + +func patchDefaultQueue(job v1alpha1.Job) *patchOperation { //Add default queue if not specified. if job.Spec.Queue == "" { - patch = append(patch, patchOperation{Op: "add", Path: "/spec/queue", Value: DefaultQueue}) + return &patchOperation{Op: "add", Path: "/spec/queue", Value: DefaultQueue} } - - return json.Marshal(patch) + return nil } -func mutateSpec(tasks []v1alpha1.TaskSpec, basePath string) (patch []patchOperation) { +func mutateSpec(tasks []v1alpha1.TaskSpec, basePath string) *patchOperation { + patched := false for index := range tasks { // add default task name taskName := tasks[index].Name if len(taskName) == 0 { + patched = true tasks[index].Name = v1alpha1.DefaultTaskSpec + strconv.Itoa(index) } } - patch = append(patch, patchOperation{ + if !patched { + return nil + } + return &patchOperation{ Op: "replace", Path: basePath, Value: tasks, - }) - - return patch + } } diff --git a/pkg/admission/mutate_job_test.go b/pkg/admission/mutate_job_test.go new file mode 100644 index 0000000000..f786198b96 --- /dev/null +++ b/pkg/admission/mutate_job_test.go @@ -0,0 +1,141 @@ +/* +Copyright 2019 The Volcano 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 admission + +import ( + "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "testing" + "volcano.sh/volcano/pkg/apis/batch/v1alpha1" +) + +func TestCreatePatchExecution(t *testing.T) { + + namespace := "test" + + testCase := struct { + Name string + Job v1alpha1.Job + operation patchOperation + }{ + Name: "patch default task name", + Job: v1alpha1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "path-task-name", + Namespace: namespace, + }, + Spec: v1alpha1.JobSpec{ + MinAvailable: 1, + Tasks: []v1alpha1.TaskSpec{ + { + Replicas: 1, + Template: v1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{"name": "test"}, + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "fake-name", + Image: "busybox:1.24", + }, + }, + }, + }, + }, + { + Replicas: 1, + Template: v1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{"name": "test"}, + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "fake-name", + Image: "busybox:1.24", + }, + }, + }, + }, + }, + }, + }, + }, + operation: patchOperation{ + Op: "replace", + Path: "/spec/tasks", + Value: []v1alpha1.TaskSpec{ + { + Name: v1alpha1.DefaultTaskSpec + "0", + Replicas: 1, + Template: v1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{"name": "test"}, + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "fake-name", + Image: "busybox:1.24", + }, + }, + }, + }, + }, + { + Name: v1alpha1.DefaultTaskSpec + "1", + Replicas: 1, + Template: v1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{"name": "test"}, + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "fake-name", + Image: "busybox:1.24", + }, + }, + }, + }, + }, + }, + }, + } + + ret := mutateSpec(testCase.Job.Spec.Tasks, "/spec/tasks") + if ret.Path != testCase.operation.Path || ret.Op != testCase.operation.Op { + t.Errorf("testCase %s's expected patch operation %v, but got %v", + testCase.Name, testCase.operation, *ret) + } + + actualTasks, ok := ret.Value.([]v1alpha1.TaskSpec) + if !ok { + t.Errorf("testCase '%s' path value expected to be '[]v1alpha1.TaskSpec', but negative", + testCase.Name) + } + expectedTasks, _ := testCase.operation.Value.([]v1alpha1.TaskSpec) + for index, task := range expectedTasks { + aTask := actualTasks[index] + if aTask.Name != task.Name { + t.Errorf("testCase '%s's expected patch operation with value %v, but got %v", + testCase.Name, testCase.operation.Value, ret.Value) + } + } + +} diff --git a/pkg/apis/batch/v1alpha1/job.go b/pkg/apis/batch/v1alpha1/job.go index 2c92264751..d636eabd2c 100644 --- a/pkg/apis/batch/v1alpha1/job.go +++ b/pkg/apis/batch/v1alpha1/job.go @@ -79,6 +79,10 @@ type JobSpec struct { // the Job becomes eligible to be deleted immediately after it finishes. // +optional TTLSecondsAfterFinished *int32 `json:"ttlSecondsAfterFinished,omitempty" protobuf:"varint,9,opt,name=ttlSecondsAfterFinished"` + + // If specified, indicates the job's priority. + // +optional + PriorityClassName string `json:"priorityClassName,omitempty" protobuf:"bytes,10,opt,name=priorityClassName"` } // VolumeSpec defines the specification of Volume, e.g. PVC diff --git a/pkg/controllers/job/job_controller_actions.go b/pkg/controllers/job/job_controller_actions.go index f3d2f21e37..a88382761f 100644 --- a/pkg/controllers/job/job_controller_actions.go +++ b/pkg/controllers/job/job_controller_actions.go @@ -448,9 +448,10 @@ func (cc *Controller) createPodGroupIfNotExist(job *vkv1.Job) error { }, }, Spec: kbv1.PodGroupSpec{ - MinMember: job.Spec.MinAvailable, - Queue: job.Spec.Queue, - MinResources: cc.calcPGMinResources(job), + MinMember: job.Spec.MinAvailable, + Queue: job.Spec.Queue, + MinResources: cc.calcPGMinResources(job), + PriorityClassName: job.Spec.PriorityClassName, }, } From f278a836f67b3c94e59c36cf7f2f058931b424fb Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Thu, 16 May 2019 14:53:13 +0530 Subject: [PATCH 35/68] vkctl delete feature --- cmd/cli/job.go | 10 ++++++++ pkg/cli/job/delete.go | 58 +++++++++++++++++++++++++++++++++++++++++++ test/e2e/cli_util.go | 10 ++++++++ test/e2e/command.go | 39 +++++++++++++++++++++++++++++ 4 files changed, 117 insertions(+) create mode 100644 pkg/cli/job/delete.go diff --git a/cmd/cli/job.go b/cmd/cli/job.go index d858b5df4c..a8a49daf98 100644 --- a/cmd/cli/job.go +++ b/cmd/cli/job.go @@ -52,5 +52,15 @@ func buildJobCmd() *cobra.Command { job.InitResumeFlags(jobResumeCmd) jobCmd.AddCommand(jobResumeCmd) + jobDelCmd := &cobra.Command{ + Use: "delete", + Short: "delete a job ", + Run: func(cmd *cobra.Command, args []string) { + checkError(cmd, job.DeleteJob()) + }, + } + job.InitDeleteFlags(jobDelCmd) + jobCmd.AddCommand(jobDelCmd) + return jobCmd } diff --git a/pkg/cli/job/delete.go b/pkg/cli/job/delete.go new file mode 100644 index 0000000000..f60acd9760 --- /dev/null +++ b/pkg/cli/job/delete.go @@ -0,0 +1,58 @@ +/* +Copyright 2019 The Vulcan 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 job + +import ( + "fmt" + "github.com/spf13/cobra" + + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + "volcano.sh/volcano/pkg/client/clientset/versioned" +) + +type deleteFlags struct { + commonFlags + + Namespace string + JobName string +} + +var deleteJobFlags = &deleteFlags{} + +func InitDeleteFlags(cmd *cobra.Command) { + initFlags(cmd, &deleteJobFlags.commonFlags) + + cmd.Flags().StringVarP(&deleteJobFlags.Namespace, "namespace", "N", "default", "the namespace of job") + cmd.Flags().StringVarP(&deleteJobFlags.JobName, "name", "n", "", "the name of job") +} + +func DeleteJob() error { + config, err := buildConfig(deleteJobFlags.Master, deleteJobFlags.Kubeconfig) + if err != nil { + return err + } + + jobClient := versioned.NewForConfigOrDie(config) + err = jobClient.BatchV1alpha1().Jobs(deleteJobFlags.Namespace).Delete(deleteJobFlags.JobName, &metav1.DeleteOptions{}) + if err != nil { + return err + } + fmt.Printf("delete job %v successfully\n", deleteJobFlags.JobName) + return nil + +} diff --git a/test/e2e/cli_util.go b/test/e2e/cli_util.go index 7953f3e336..341ab6de93 100644 --- a/test/e2e/cli_util.go +++ b/test/e2e/cli_util.go @@ -52,6 +52,16 @@ func ListJobs(namespace string) string { return RunCliCommand(command) } +func DeleteJob(name string, namespace string) string { + command := []string{"job", "delete"} + Expect(name).NotTo(Equal(""), "Job name should not be empty in delete job command") + command = append(command, "--name", name) + if namespace != "" { + command = append(command, "--namespace", namespace) + } + return RunCliCommand(command) +} + func RunCliCommand(command []string) string { if masterURL() != "" { command = append(command, "--master", masterURL()) diff --git a/test/e2e/command.go b/test/e2e/command.go index 4300022172..9c1654a4f0 100644 --- a/test/e2e/command.go +++ b/test/e2e/command.go @@ -152,4 +152,43 @@ var _ = Describe("Job E2E Test: Test Job Command", func() { Expect(apierrors.IsNotFound(err)).To(BeTrue(), "Job related pod should be deleted when job aborted.") }) + + It("delete a job", func() { + + jobName := "test-del-job" + namespace := "test" + context := initTestContext() + defer cleanupTestContext(context) + rep := clusterSize(context, oneCPU) + + job := createJob(context, &jobSpec{ + namespace: namespace, + name: jobName, + tasks: []taskSpec{ + { + img: defaultNginxImage, + req: oneCPU, + min: rep, + rep: rep, + }, + }, + }) + // Pod is running + err := waitJobReady(context, job) + Expect(err).NotTo(HaveOccurred()) + // Job Status is running + err = waitJobStateReady(context, job) + Expect(err).NotTo(HaveOccurred()) + + _, err = context.vkclient.BatchV1alpha1().Jobs(namespace).Get(jobName, metav1.GetOptions{}) + Expect(err).NotTo(HaveOccurred()) + + // Delete job + DeleteJob(jobName, namespace) + + _, err = context.vkclient.BatchV1alpha1().Jobs(namespace).Get(jobName, metav1.GetOptions{}) + Expect(apierrors.IsNotFound(err)).To(BeTrue(), + "Job should be deleted on vkctl job delete.") + + }) }) From 11f5451d717ec2cf5f750fec814d90faa35e4549 Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Thu, 16 May 2019 15:52:50 +0530 Subject: [PATCH 36/68] Added queue get command & changed list command --- cmd/cli/queue.go | 10 ++++++ pkg/cli/queue/get.go | 82 +++++++++++++++++++++++++++++++++++++++++++ pkg/cli/queue/list.go | 17 ++++++--- 3 files changed, 105 insertions(+), 4 deletions(-) create mode 100644 pkg/cli/queue/get.go diff --git a/cmd/cli/queue.go b/cmd/cli/queue.go index c4b3795f00..039c8672b7 100644 --- a/cmd/cli/queue.go +++ b/cmd/cli/queue.go @@ -48,5 +48,15 @@ func buildQueueCmd() *cobra.Command { queue.InitListFlags(queueListCmd) jobCmd.AddCommand(queueListCmd) + queueGetCmd := &cobra.Command{ + Use: "get", + Short: "get a queue", + Run: func(cmd *cobra.Command, args []string) { + checkError(cmd, queue.GetQueue()) + }, + } + queue.InitGetFlags(queueGetCmd) + jobCmd.AddCommand(queueGetCmd) + return jobCmd } diff --git a/pkg/cli/queue/get.go b/pkg/cli/queue/get.go new file mode 100644 index 0000000000..af73b6cf7a --- /dev/null +++ b/pkg/cli/queue/get.go @@ -0,0 +1,82 @@ +/* +Copyright 2019 The Volcano 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 queue + +import ( + "fmt" + "io" + "os" + + "github.com/spf13/cobra" + + "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1" + "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" +) + +type getFlags struct { + commonFlags + + Name string +} + +var getQueueFlags = &getFlags{} + +// InitGetFlags is used to init all flags +func InitGetFlags(cmd *cobra.Command) { + initFlags(cmd, &getQueueFlags.commonFlags) + + cmd.Flags().StringVarP(&getQueueFlags.Name, "name", "n", "", "the name of queue") + +} + +// GetQueue gets a queue +func GetQueue() error { + config, err := buildConfig(getQueueFlags.Master, getQueueFlags.Kubeconfig) + if err != nil { + return err + } + + if getQueueFlags.Name == "" { + err := fmt.Errorf("name is mandaorty to get the partiular queue details") + return err + } + + queueClient := versioned.NewForConfigOrDie(config) + queue, err := queueClient.SchedulingV1alpha1().Queues().Get(getQueueFlags.Name, metav1.GetOptions{}) + if err != nil { + return err + } + + PrintQueue(queue, os.Stdout) + + return nil +} + +// PrintQueue prints queue information +func PrintQueue(queue *v1alpha1.Queue, writer io.Writer) { + _, err := fmt.Fprintf(writer, "%-25s%-8s%-8s%-8s%-8s\n", + Name, Weight, Pending, Running, Unknown) + if err != nil { + fmt.Printf("Failed to print queue command result: %s.\n", err) + } + _, err = fmt.Fprintf(writer, "%-25s%-8d%-8d%-8d%-8d\n", + queue.Name, queue.Spec.Weight, queue.Status.Pending, queue.Status.Running, queue.Status.Unknown) + if err != nil { + fmt.Printf("Failed to print queue command result: %s.\n", err) + } +} diff --git a/pkg/cli/queue/list.go b/pkg/cli/queue/list.go index 35e5a59346..5487ae8670 100644 --- a/pkg/cli/queue/list.go +++ b/pkg/cli/queue/list.go @@ -38,6 +38,15 @@ const ( // Name of queue Name string = "Name" + + // Pending status of the queue + Pending string = "Pending" + + // Running status of the queue + Running string = "Running" + + // Unknown status of the queue + Unknown string = "Unknown" ) var listQueueFlags = &listFlags{} @@ -71,14 +80,14 @@ func ListQueue() error { // PrintQueues prints queue information func PrintQueues(queues *v1alpha1.QueueList, writer io.Writer) { - _, err := fmt.Fprintf(writer, "%-25s%-8s\n", - Name, Weight) + _, err := fmt.Fprintf(writer, "%-25s%-8s%-8s%-8s%-8s\n", + Name, Weight, Pending, Running, Unknown) if err != nil { fmt.Printf("Failed to print queue command result: %s.\n", err) } for _, queue := range queues.Items { - _, err = fmt.Fprintf(writer, "%-25s%-8d\n", - queue.Name, queue.Spec.Weight) + _, err = fmt.Fprintf(writer, "%-25s%-8d%-8d%-8d%-8d\n", + queue.Name, queue.Spec.Weight, queue.Status.Pending, queue.Status.Running, queue.Status.Unknown) if err != nil { fmt.Printf("Failed to print queue command result: %s.\n", err) } From baf12fffc637589bde17130a2da2d41d0104711f Mon Sep 17 00:00:00 2001 From: lminzhw Date: Thu, 16 May 2019 16:51:47 +0800 Subject: [PATCH 37/68] retain pod with special phase --- pkg/controllers/job/job_controller_actions.go | 35 +++++++++++-------- pkg/controllers/job/state/aborted.go | 4 +-- pkg/controllers/job/state/aborting.go | 4 +-- pkg/controllers/job/state/completing.go | 2 +- pkg/controllers/job/state/factory.go | 14 ++++++-- pkg/controllers/job/state/finished.go | 2 +- pkg/controllers/job/state/inqueue.go | 6 ++-- pkg/controllers/job/state/pending.go | 6 ++-- pkg/controllers/job/state/restarting.go | 2 +- pkg/controllers/job/state/running.go | 8 ++--- pkg/controllers/job/state/terminating.go | 2 +- 11 files changed, 51 insertions(+), 34 deletions(-) diff --git a/pkg/controllers/job/job_controller_actions.go b/pkg/controllers/job/job_controller_actions.go index a88382761f..8d2bc0200a 100644 --- a/pkg/controllers/job/job_controller_actions.go +++ b/pkg/controllers/job/job_controller_actions.go @@ -35,7 +35,7 @@ import ( "volcano.sh/volcano/pkg/controllers/job/state" ) -func (cc *Controller) killJob(jobInfo *apis.JobInfo, updateStatus state.UpdateStatusFn) error { +func (cc *Controller) killJob(jobInfo *apis.JobInfo, podRetainPhase state.PhaseMap, updateStatus state.UpdateStatusFn) error { glog.V(3).Infof("Killing Job <%s/%s>", jobInfo.Job.Namespace, jobInfo.Job.Name) defer glog.V(3).Infof("Finished Job <%s/%s> killing", jobInfo.Job.Namespace, jobInfo.Job.Name) @@ -62,20 +62,27 @@ func (cc *Controller) killJob(jobInfo *apis.JobInfo, updateStatus state.UpdateSt continue } - if err := cc.deleteJobPod(job.Name, pod); err == nil { - terminating++ - } else { - errs = append(errs, err) - switch pod.Status.Phase { - case v1.PodRunning: - running++ - case v1.PodPending: - pending++ - case v1.PodSucceeded: - succeeded++ - case v1.PodFailed: - failed++ + _, retain := podRetainPhase[pod.Status.Phase] + + if !retain { + err := cc.deleteJobPod(job.Name, pod) + if err == nil { + terminating++ + continue } + // record the err, and then collect the pod info like retained pod + errs = append(errs, err) + } + + switch pod.Status.Phase { + case v1.PodRunning: + running++ + case v1.PodPending: + pending++ + case v1.PodSucceeded: + succeeded++ + case v1.PodFailed: + failed++ } } } diff --git a/pkg/controllers/job/state/aborted.go b/pkg/controllers/job/state/aborted.go index 953ee392d5..3fa6c5c8bd 100644 --- a/pkg/controllers/job/state/aborted.go +++ b/pkg/controllers/job/state/aborted.go @@ -28,12 +28,12 @@ type abortedState struct { func (as *abortedState) Execute(action vkv1.Action) error { switch action { case vkv1.ResumeJobAction: - return KillJob(as.job, func(status *vkv1.JobStatus) bool { + return KillJob(as.job, PodRetainPhaseSoft, func(status *vkv1.JobStatus) bool { status.State.Phase = vkv1.Restarting status.RetryCount++ return true }) default: - return KillJob(as.job, nil) + return KillJob(as.job, PodRetainPhaseSoft, nil) } } diff --git a/pkg/controllers/job/state/aborting.go b/pkg/controllers/job/state/aborting.go index 8d123b6ada..ef40326507 100644 --- a/pkg/controllers/job/state/aborting.go +++ b/pkg/controllers/job/state/aborting.go @@ -31,12 +31,12 @@ func (ps *abortingState) Execute(action vkv1.Action) error { switch action { case vkv1.ResumeJobAction: // Already in Restarting phase, just sync it - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseSoft, func(status *vkv1.JobStatus) bool { status.RetryCount++ return false }) default: - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseSoft, func(status *vkv1.JobStatus) bool { // If any "alive" pods, still in Aborting phase if status.Terminating != 0 || status.Pending != 0 || status.Running != 0 { return false diff --git a/pkg/controllers/job/state/completing.go b/pkg/controllers/job/state/completing.go index cb9f7074bd..acad88e88a 100644 --- a/pkg/controllers/job/state/completing.go +++ b/pkg/controllers/job/state/completing.go @@ -26,7 +26,7 @@ type completingState struct { } func (ps *completingState) Execute(action vkv1.Action) error { - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseSoft, func(status *vkv1.JobStatus) bool { // If any "alive" pods, still in Completing phase if status.Terminating != 0 || status.Pending != 0 || status.Running != 0 { return false diff --git a/pkg/controllers/job/state/factory.go b/pkg/controllers/job/state/factory.go index f33cb753f7..e3e30a582d 100644 --- a/pkg/controllers/job/state/factory.go +++ b/pkg/controllers/job/state/factory.go @@ -17,18 +17,28 @@ limitations under the License. package state import ( + "k8s.io/api/core/v1" + vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/apis" ) +type PhaseMap map[v1.PodPhase]struct{} type UpdateStatusFn func(status *vkv1.JobStatus) (jobPhaseChanged bool) type ActionFn func(job *apis.JobInfo, fn UpdateStatusFn) error +type KillActionFn func(job *apis.JobInfo, podRetainPhase PhaseMap, fn UpdateStatusFn) error + +var PodRetainPhaseNone = PhaseMap{} +var PodRetainPhaseSoft = PhaseMap{ + v1.PodSucceeded: {}, + v1.PodFailed: {}, +} var ( // SyncJob will create or delete Pods according to Job's spec. SyncJob ActionFn - // KillJob kill all Pods of Job. - KillJob ActionFn + // KillJob kill all Pods of Job with phase not in podRetainPhase. + KillJob KillActionFn // CreateJob will prepare to create Job. CreateJob ActionFn ) diff --git a/pkg/controllers/job/state/finished.go b/pkg/controllers/job/state/finished.go index 52480bd95d..d39671fe6b 100644 --- a/pkg/controllers/job/state/finished.go +++ b/pkg/controllers/job/state/finished.go @@ -27,5 +27,5 @@ type finishedState struct { func (ps *finishedState) Execute(action vkv1.Action) error { // In finished state, e.g. Completed, always kill the whole job. - return KillJob(ps.job, nil) + return KillJob(ps.job, PodRetainPhaseSoft, nil) } diff --git a/pkg/controllers/job/state/inqueue.go b/pkg/controllers/job/state/inqueue.go index 44cbfa514f..981aeb40e5 100644 --- a/pkg/controllers/job/state/inqueue.go +++ b/pkg/controllers/job/state/inqueue.go @@ -28,7 +28,7 @@ type inqueueState struct { func (ps *inqueueState) Execute(action vkv1.Action) error { switch action { case vkv1.RestartJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseNone, func(status *vkv1.JobStatus) bool { phase := vkv1.Pending if status.Terminating != 0 { phase = vkv1.Restarting @@ -39,7 +39,7 @@ func (ps *inqueueState) Execute(action vkv1.Action) error { }) case vkv1.AbortJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseSoft, func(status *vkv1.JobStatus) bool { phase := vkv1.Pending if status.Terminating != 0 { phase = vkv1.Aborting @@ -48,7 +48,7 @@ func (ps *inqueueState) Execute(action vkv1.Action) error { return true }) case vkv1.CompleteJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseSoft, func(status *vkv1.JobStatus) bool { phase := vkv1.Completed if status.Terminating != 0 { phase = vkv1.Completing diff --git a/pkg/controllers/job/state/pending.go b/pkg/controllers/job/state/pending.go index 9cead263a5..38fa2e08a4 100644 --- a/pkg/controllers/job/state/pending.go +++ b/pkg/controllers/job/state/pending.go @@ -28,7 +28,7 @@ type pendingState struct { func (ps *pendingState) Execute(action vkv1.Action) error { switch action { case vkv1.RestartJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseNone, func(status *vkv1.JobStatus) bool { phase := vkv1.Pending if status.Terminating != 0 { phase = vkv1.Restarting @@ -39,7 +39,7 @@ func (ps *pendingState) Execute(action vkv1.Action) error { }) case vkv1.AbortJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseSoft, func(status *vkv1.JobStatus) bool { phase := vkv1.Pending if status.Terminating != 0 { phase = vkv1.Aborting @@ -48,7 +48,7 @@ func (ps *pendingState) Execute(action vkv1.Action) error { return true }) case vkv1.CompleteJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseSoft, func(status *vkv1.JobStatus) bool { phase := vkv1.Completed if status.Terminating != 0 { phase = vkv1.Completing diff --git a/pkg/controllers/job/state/restarting.go b/pkg/controllers/job/state/restarting.go index b11fb460a2..df83503b83 100644 --- a/pkg/controllers/job/state/restarting.go +++ b/pkg/controllers/job/state/restarting.go @@ -26,7 +26,7 @@ type restartingState struct { } func (ps *restartingState) Execute(action vkv1.Action) error { - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseNone, func(status *vkv1.JobStatus) bool { // Get the maximum number of retries. maxRetry := DefaultMaxRetry if ps.job.Job.Spec.MaxRetry != 0 { diff --git a/pkg/controllers/job/state/running.go b/pkg/controllers/job/state/running.go index e19fecc151..e25b3a77af 100644 --- a/pkg/controllers/job/state/running.go +++ b/pkg/controllers/job/state/running.go @@ -28,7 +28,7 @@ type runningState struct { func (ps *runningState) Execute(action vkv1.Action) error { switch action { case vkv1.RestartJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseNone, func(status *vkv1.JobStatus) bool { if status.Terminating != 0 { status.State.Phase = vkv1.Restarting status.RetryCount++ @@ -37,7 +37,7 @@ func (ps *runningState) Execute(action vkv1.Action) error { return false }) case vkv1.AbortJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseSoft, func(status *vkv1.JobStatus) bool { if status.Terminating != 0 { status.State.Phase = vkv1.Aborting return true @@ -46,7 +46,7 @@ func (ps *runningState) Execute(action vkv1.Action) error { return false }) case vkv1.TerminateJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseSoft, func(status *vkv1.JobStatus) bool { if status.Terminating != 0 { status.State.Phase = vkv1.Terminating return true @@ -55,7 +55,7 @@ func (ps *runningState) Execute(action vkv1.Action) error { return false }) case vkv1.CompleteJobAction: - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseSoft, func(status *vkv1.JobStatus) bool { phase := vkv1.Completed if status.Terminating != 0 { phase = vkv1.Completing diff --git a/pkg/controllers/job/state/terminating.go b/pkg/controllers/job/state/terminating.go index 5ceddecb78..b46c2429d6 100644 --- a/pkg/controllers/job/state/terminating.go +++ b/pkg/controllers/job/state/terminating.go @@ -26,7 +26,7 @@ type terminatingState struct { } func (ps *terminatingState) Execute(action vkv1.Action) error { - return KillJob(ps.job, func(status *vkv1.JobStatus) bool { + return KillJob(ps.job, PodRetainPhaseSoft, func(status *vkv1.JobStatus) bool { // If any "alive" pods, still in Terminating phase if status.Terminating != 0 || status.Pending != 0 || status.Running != 0 { return false From 757886241dba57e16bd3a5e39bc963fd4cffca37 Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Fri, 17 May 2019 11:04:53 +0530 Subject: [PATCH 38/68] Simple Issue Fixes --- pkg/cli/job/common.go | 2 +- pkg/cli/job/delete.go | 5 +++++ pkg/cli/job/resume.go | 5 +++++ pkg/cli/job/suspend.go | 6 ++++++ pkg/cli/queue/get.go | 2 +- 5 files changed, 18 insertions(+), 2 deletions(-) diff --git a/pkg/cli/job/common.go b/pkg/cli/job/common.go index ed1ba0e32f..43fff20759 100644 --- a/pkg/cli/job/common.go +++ b/pkg/cli/job/common.go @@ -28,7 +28,7 @@ type commonFlags struct { } func initFlags(cmd *cobra.Command, cf *commonFlags) { - cmd.Flags().StringVarP(&cf.SchedulerName, "scheduler", "S", "vn-scheduler", "the scheduler for this job") + cmd.Flags().StringVarP(&cf.SchedulerName, "scheduler", "S", "kube-batch", "the scheduler for this job") cmd.Flags().StringVarP(&cf.Master, "master", "s", "", "the address of apiserver") if home := homeDir(); home != "" { diff --git a/pkg/cli/job/delete.go b/pkg/cli/job/delete.go index f60acd9760..e6f697b500 100644 --- a/pkg/cli/job/delete.go +++ b/pkg/cli/job/delete.go @@ -47,6 +47,11 @@ func DeleteJob() error { return err } + if deleteJobFlags.JobName == "" { + err := fmt.Errorf("job name is mandaorty to delete a particular job") + return err + } + jobClient := versioned.NewForConfigOrDie(config) err = jobClient.BatchV1alpha1().Jobs(deleteJobFlags.Namespace).Delete(deleteJobFlags.JobName, &metav1.DeleteOptions{}) if err != nil { diff --git a/pkg/cli/job/resume.go b/pkg/cli/job/resume.go index 54733bd223..d73c1d6dcd 100644 --- a/pkg/cli/job/resume.go +++ b/pkg/cli/job/resume.go @@ -16,6 +16,7 @@ limitations under the License. package job import ( + "fmt" "github.com/spf13/cobra" "volcano.sh/volcano/pkg/apis/batch/v1alpha1" @@ -42,6 +43,10 @@ func ResumeJob() error { if err != nil { return err } + if resumeJobFlags.JobName == "" { + err := fmt.Errorf("job name is mandaorty to resume a particular job") + return err + } return createJobCommand(config, resumeJobFlags.Namespace, resumeJobFlags.JobName, diff --git a/pkg/cli/job/suspend.go b/pkg/cli/job/suspend.go index 85f0d807b0..504cdd2381 100644 --- a/pkg/cli/job/suspend.go +++ b/pkg/cli/job/suspend.go @@ -16,6 +16,7 @@ limitations under the License. package job import ( + "fmt" "github.com/spf13/cobra" "volcano.sh/volcano/pkg/apis/batch/v1alpha1" @@ -43,6 +44,11 @@ func SuspendJob() error { return err } + if suspendJobFlags.JobName == "" { + err := fmt.Errorf("job name is mandaorty to suspend a particular job") + return err + } + return createJobCommand(config, suspendJobFlags.Namespace, suspendJobFlags.JobName, v1alpha1.AbortJobAction) diff --git a/pkg/cli/queue/get.go b/pkg/cli/queue/get.go index af73b6cf7a..bc5a256bcf 100644 --- a/pkg/cli/queue/get.go +++ b/pkg/cli/queue/get.go @@ -52,7 +52,7 @@ func GetQueue() error { } if getQueueFlags.Name == "" { - err := fmt.Errorf("name is mandaorty to get the partiular queue details") + err := fmt.Errorf("name is mandaorty to get the particular queue details") return err } From 721319584e47ad4c7c5418cac77b46222a6d9b43 Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Fri, 17 May 2019 11:48:14 +0530 Subject: [PATCH 39/68] Removed repeated validation and unused code --- pkg/admission/admission_controller.go | 68 --------------------------- pkg/admission/admit_job.go | 23 +-------- 2 files changed, 1 insertion(+), 90 deletions(-) diff --git a/pkg/admission/admission_controller.go b/pkg/admission/admission_controller.go index 1bb2569ca8..65c7c4aa40 100644 --- a/pkg/admission/admission_controller.go +++ b/pkg/admission/admission_controller.go @@ -84,74 +84,6 @@ func ToAdmissionResponse(err error) *v1beta1.AdmissionResponse { } } -func CheckPolicyDuplicate(policies []v1alpha1.LifecyclePolicy) (string, bool) { - policyEvents := map[v1alpha1.Event]v1alpha1.Event{} - hasDuplicate := false - var duplicateInfo string - - for _, policy := range policies { - if _, found := policyEvents[policy.Event]; found { - hasDuplicate = true - duplicateInfo = fmt.Sprintf("%v", policy.Event) - break - } else { - policyEvents[policy.Event] = policy.Event - } - } - - if _, found := policyEvents[v1alpha1.AnyEvent]; found && len(policyEvents) > 1 { - hasDuplicate = true - duplicateInfo = "if there's * here, no other policy should be here" - } - - return duplicateInfo, hasDuplicate -} - -func ValidatePolicies(policies []v1alpha1.LifecyclePolicy) error { - var err error - policyEvents := map[v1alpha1.Event]struct{}{} - exitCodes := map[int32]struct{}{} - - for _, policy := range policies { - if policy.Event != "" && policy.ExitCode != nil { - err = multierror.Append(err, fmt.Errorf("must not specify event and exitCode simultaneously")) - break - } - - if policy.Event == "" && policy.ExitCode == nil { - err = multierror.Append(err, fmt.Errorf("either event and exitCode should be specified")) - break - } - - if policy.Event != "" { - // TODO: check event is in supported Event - if _, found := policyEvents[policy.Event]; found { - err = multierror.Append(err, fmt.Errorf("duplicate event %v", policy.Event)) - break - } else { - policyEvents[policy.Event] = struct{}{} - } - } else { - if *policy.ExitCode == 0 { - err = multierror.Append(err, fmt.Errorf("0 is not a valid error code")) - break - } - if _, found := exitCodes[*policy.ExitCode]; found { - err = multierror.Append(err, fmt.Errorf("duplicate exitCode %v", *policy.ExitCode)) - break - } else { - exitCodes[*policy.ExitCode] = struct{}{} - } - } - } - - if _, found := policyEvents[v1alpha1.AnyEvent]; found && len(policyEvents) > 1 { - err = multierror.Append(err, fmt.Errorf("if there's * here, no other policy should be here")) - } - - return err -} - func DecodeJob(object runtime.RawExtension, resource metav1.GroupVersionResource) (v1alpha1.Job, error) { jobResource := metav1.GroupVersionResource{Group: v1alpha1.SchemeGroupVersion.Group, Version: v1alpha1.SchemeGroupVersion.Version, Resource: "jobs"} raw := object.Raw diff --git a/pkg/admission/admit_job.go b/pkg/admission/admit_job.go index 45faf9dc2f..586451d3e8 100644 --- a/pkg/admission/admit_job.go +++ b/pkg/admission/admit_job.go @@ -18,7 +18,6 @@ package admission import ( "fmt" - "reflect" "strings" "github.com/golang/glog" @@ -32,7 +31,7 @@ import ( k8scorev1 "k8s.io/kubernetes/pkg/apis/core/v1" k8scorevalid "k8s.io/kubernetes/pkg/apis/core/validation" - v1alpha1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" + "volcano.sh/volcano/pkg/apis/batch/v1alpha1" "volcano.sh/volcano/pkg/controllers/job/plugins" ) @@ -117,11 +116,6 @@ func validateJob(job v1alpha1.Job, reviewResponse *v1beta1.AdmissionResponse) st taskNames[task.Name] = task.Name } - //duplicate task event policies - if duplicateInfo, ok := CheckPolicyDuplicate(task.Policies); ok { - msg = msg + fmt.Sprintf(" duplicated task event policies: %s;", duplicateInfo) - } - if err := validatePolicies(task.Policies, field.NewPath("spec.tasks.policies")); err != nil { msg = msg + err.Error() + fmt.Sprintf(" valid events are %v, valid actions are %v", getValidEvents(), getValidActions()) @@ -134,11 +128,6 @@ func validateJob(job v1alpha1.Job, reviewResponse *v1beta1.AdmissionResponse) st msg = msg + " 'minAvailable' should not be greater than total replicas in tasks;" } - //duplicate job event policies - if duplicateInfo, ok := CheckPolicyDuplicate(job.Spec.Policies); ok { - msg = msg + fmt.Sprintf(" duplicated job event policies: %s;", duplicateInfo) - } - if err := validatePolicies(job.Spec.Policies, field.NewPath("spec.policies")); err != nil { msg = msg + err.Error() + fmt.Sprintf(" valid events are %v, valid actions are %v;", getValidEvents(), getValidActions()) @@ -164,16 +153,6 @@ func validateJob(job v1alpha1.Job, reviewResponse *v1beta1.AdmissionResponse) st return msg } -func specDeepEqual(newJob v1alpha1.Job, oldJob v1alpha1.Job, reviewResponse *v1beta1.AdmissionResponse) string { - var msg string - if !reflect.DeepEqual(newJob.Spec, oldJob.Spec) { - reviewResponse.Allowed = false - msg = "job.spec is not allowed to modify when update jobs;" - } - - return msg -} - func validateTaskTemplate(task v1alpha1.TaskSpec, job v1alpha1.Job, index int) string { var v1PodTemplate v1.PodTemplate v1PodTemplate.Template = *task.Template.DeepCopy() From b4886dce9d54925f9cc4af8277bd0fd1f287f3b8 Mon Sep 17 00:00:00 2001 From: lminzhw Date: Fri, 17 May 2019 14:59:05 +0800 Subject: [PATCH 40/68] support vkctl job run --limits --- pkg/cli/job/run.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pkg/cli/job/run.go b/pkg/cli/job/run.go index 277bb1889d..79372f5920 100644 --- a/pkg/cli/job/run.go +++ b/pkg/cli/job/run.go @@ -35,6 +35,7 @@ type runFlags struct { MinAvailable int Replicas int Requests string + Limits string } var launchJobFlags = &runFlags{} @@ -48,6 +49,7 @@ func InitRunFlags(cmd *cobra.Command) { cmd.Flags().IntVarP(&launchJobFlags.MinAvailable, "min", "m", 1, "the minimal available tasks of job") cmd.Flags().IntVarP(&launchJobFlags.Replicas, "replicas", "r", 1, "the total tasks of job") cmd.Flags().StringVarP(&launchJobFlags.Requests, "requests", "R", "cpu=1000m,memory=100Mi", "the resource request of the task") + cmd.Flags().StringVarP(&launchJobFlags.Limits, "limits", "L", "cpu=1000m,memory=100Mi", "the resource limit of the task") } var jobName = "job.volcano.sh" @@ -63,6 +65,11 @@ func RunJob() error { return err } + limit, err := populateResourceListV1(launchJobFlags.Limits) + if err != nil { + return err + } + job := &vkapi.Job{ ObjectMeta: metav1.ObjectMeta{ Name: launchJobFlags.Name, @@ -88,6 +95,7 @@ func RunJob() error { Name: launchJobFlags.Name, ImagePullPolicy: v1.PullIfNotPresent, Resources: v1.ResourceRequirements{ + Limits: limit, Requests: req, }, }, From ef9e41f88dbc8eeada0ec67fb8481296b31479ae Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Sat, 18 May 2019 19:16:26 +0530 Subject: [PATCH 41/68] Optimizing E2E test cases --- test/e2e/command.go | 23 ++++++- test/e2e/job_plugins.go | 80 +++++++++++++++++++---- test/e2e/predicates.go | 139 ---------------------------------------- 3 files changed, 89 insertions(+), 153 deletions(-) diff --git a/test/e2e/command.go b/test/e2e/command.go index 9c1654a4f0..8040a9691f 100644 --- a/test/e2e/command.go +++ b/test/e2e/command.go @@ -19,6 +19,7 @@ package e2e import ( "bytes" "fmt" + v1 "k8s.io/api/core/v1" . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" @@ -153,7 +154,7 @@ var _ = Describe("Job E2E Test: Test Job Command", func() { "Job related pod should be deleted when job aborted.") }) - It("delete a job", func() { + It("delete a job with all nodes taints", func() { jobName := "test-del-job" namespace := "test" @@ -161,6 +162,17 @@ var _ = Describe("Job E2E Test: Test Job Command", func() { defer cleanupTestContext(context) rep := clusterSize(context, oneCPU) + taints := []v1.Taint{ + { + Key: "test-taint-key", + Value: "test-taint-val", + Effect: v1.TaintEffectNoSchedule, + }, + } + + err := taintAllNodes(context, taints) + Expect(err).NotTo(HaveOccurred()) + job := createJob(context, &jobSpec{ namespace: namespace, name: jobName, @@ -173,8 +185,15 @@ var _ = Describe("Job E2E Test: Test Job Command", func() { }, }, }) + + err = waitJobPending(context, job) + Expect(err).NotTo(HaveOccurred()) + + err = removeTaintsFromAllNodes(context, taints) + Expect(err).NotTo(HaveOccurred()) + // Pod is running - err := waitJobReady(context, job) + err = waitJobReady(context, job) Expect(err).NotTo(HaveOccurred()) // Job Status is running err = waitJobStateReady(context, job) diff --git a/test/e2e/job_plugins.go b/test/e2e/job_plugins.go index aba5bef5e2..abad42df4e 100644 --- a/test/e2e/job_plugins.go +++ b/test/e2e/job_plugins.go @@ -20,12 +20,14 @@ import ( "fmt" . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" + cv1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/kubernetes/pkg/scheduler/api" "volcano.sh/volcano/pkg/controllers/job/helpers" ) var _ = Describe("Job E2E Test: Test Job Plugins", func() { - It("SVC Plugin", func() { + It("SVC Plugin with Node Affinity", func() { jobName := "job-with-svc-plugin" namespace := "test" taskName := "task" @@ -33,6 +35,27 @@ var _ = Describe("Job E2E Test: Test Job Plugins", func() { context := initTestContext() defer cleanupTestContext(context) + nodeName, rep := computeNode(context, oneCPU) + Expect(rep).NotTo(Equal(0)) + + affinity := &cv1.Affinity{ + NodeAffinity: &cv1.NodeAffinity{ + RequiredDuringSchedulingIgnoredDuringExecution: &cv1.NodeSelector{ + NodeSelectorTerms: []cv1.NodeSelectorTerm{ + { + MatchFields: []cv1.NodeSelectorRequirement{ + { + Key: api.NodeFieldSelectorKeyNodeName, + Operator: cv1.NodeSelectorOpIn, + Values: []string{nodeName}, + }, + }, + }, + }, + }, + }, + } + job := createJob(context, &jobSpec{ namespace: namespace, name: jobName, @@ -41,11 +64,12 @@ var _ = Describe("Job E2E Test: Test Job Plugins", func() { }, tasks: []taskSpec{ { - img: defaultNginxImage, - req: oneCPU, - min: 1, - rep: 1, - name: taskName, + img: defaultNginxImage, + req: oneCPU, + min: 1, + rep: 1, + name: taskName, + affinity: affinity, }, }, }) @@ -68,9 +92,14 @@ var _ = Describe("Job E2E Test: Test Job Plugins", func() { } } Expect(foundVolume).To(BeTrue()) + + pods := getTasksOfJob(context, job) + for _, pod := range pods { + Expect(pod.Spec.NodeName).To(Equal(nodeName)) + } }) - It("SSh Plugin", func() { + It("SSh Plugin with Pod Affinity", func() { jobName := "job-with-ssh-plugin" namespace := "test" taskName := "task" @@ -78,6 +107,24 @@ var _ = Describe("Job E2E Test: Test Job Plugins", func() { context := initTestContext() defer cleanupTestContext(context) + _, rep := computeNode(context, oneCPU) + Expect(rep).NotTo(Equal(0)) + + labels := map[string]string{"foo": "bar"} + + affinity := &cv1.Affinity{ + PodAffinity: &cv1.PodAffinity{ + RequiredDuringSchedulingIgnoredDuringExecution: []cv1.PodAffinityTerm{ + { + LabelSelector: &v1.LabelSelector{ + MatchLabels: labels, + }, + TopologyKey: "kubernetes.io/hostname", + }, + }, + }, + } + job := createJob(context, &jobSpec{ namespace: namespace, name: jobName, @@ -86,11 +133,13 @@ var _ = Describe("Job E2E Test: Test Job Plugins", func() { }, tasks: []taskSpec{ { - img: defaultNginxImage, - req: oneCPU, - min: 1, - rep: 1, - name: taskName, + img: defaultNginxImage, + req: oneCPU, + min: rep, + rep: rep, + affinity: affinity, + labels: labels, + name: taskName, }, }, }) @@ -113,5 +162,12 @@ var _ = Describe("Job E2E Test: Test Job Plugins", func() { } } Expect(foundVolume).To(BeTrue()) + + pods := getTasksOfJob(context, job) + // All pods should be scheduled to the same node. + nodeName := pods[0].Spec.NodeName + for _, pod := range pods { + Expect(pod.Spec.NodeName).To(Equal(nodeName)) + } }) }) diff --git a/test/e2e/predicates.go b/test/e2e/predicates.go index 59e76fff52..b94cab41c5 100644 --- a/test/e2e/predicates.go +++ b/test/e2e/predicates.go @@ -19,61 +19,9 @@ package e2e import ( . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" - - "k8s.io/api/core/v1" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - "k8s.io/kubernetes/pkg/scheduler/api" ) var _ = Describe("Predicates E2E Test", func() { - It("NodeAffinity", func() { - context := initTestContext() - defer cleanupTestContext(context) - - slot := oneCPU - nodeName, rep := computeNode(context, oneCPU) - Expect(rep).NotTo(Equal(0)) - - affinity := &v1.Affinity{ - NodeAffinity: &v1.NodeAffinity{ - RequiredDuringSchedulingIgnoredDuringExecution: &v1.NodeSelector{ - NodeSelectorTerms: []v1.NodeSelectorTerm{ - { - MatchFields: []v1.NodeSelectorRequirement{ - { - Key: api.NodeFieldSelectorKeyNodeName, - Operator: v1.NodeSelectorOpIn, - Values: []string{nodeName}, - }, - }, - }, - }, - }, - }, - } - - spec := &jobSpec{ - name: "na-spec", - tasks: []taskSpec{ - { - img: defaultNginxImage, - req: slot, - min: 1, - rep: 1, - affinity: affinity, - }, - }, - } - - job := createJob(context, spec) - err := waitJobReady(context, job) - Expect(err).NotTo(HaveOccurred()) - - pods := getTasksOfJob(context, job) - for _, pod := range pods { - Expect(pod.Spec.NodeName).To(Equal(nodeName)) - } - }) It("Hostport", func() { context := initTestContext() @@ -103,91 +51,4 @@ var _ = Describe("Predicates E2E Test", func() { Expect(err).NotTo(HaveOccurred()) }) - It("Pod Affinity", func() { - context := initTestContext() - defer cleanupTestContext(context) - - slot := oneCPU - _, rep := computeNode(context, oneCPU) - Expect(rep).NotTo(Equal(0)) - - labels := map[string]string{"foo": "bar"} - - affinity := &v1.Affinity{ - PodAffinity: &v1.PodAffinity{ - RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{ - { - LabelSelector: &metav1.LabelSelector{ - MatchLabels: labels, - }, - TopologyKey: "kubernetes.io/hostname", - }, - }, - }, - } - - spec := &jobSpec{ - name: "pa-spec", - tasks: []taskSpec{ - { - img: defaultNginxImage, - req: slot, - min: rep, - rep: rep, - affinity: affinity, - labels: labels, - }, - }, - } - - job := createJob(context, spec) - err := waitJobReady(context, job) - Expect(err).NotTo(HaveOccurred()) - - pods := getTasksOfJob(context, job) - // All pods should be scheduled to the same node. - nodeName := pods[0].Spec.NodeName - for _, pod := range pods { - Expect(pod.Spec.NodeName).To(Equal(nodeName)) - } - }) - - It("Taints/Tolerations", func() { - context := initTestContext() - defer cleanupTestContext(context) - - taints := []v1.Taint{ - { - Key: "test-taint-key", - Value: "test-taint-val", - Effect: v1.TaintEffectNoSchedule, - }, - } - - err := taintAllNodes(context, taints) - Expect(err).NotTo(HaveOccurred()) - - spec := &jobSpec{ - name: "tt-spec", - tasks: []taskSpec{ - { - img: defaultNginxImage, - req: oneCPU, - min: 1, - rep: 1, - }, - }, - } - - job := createJob(context, spec) - err = waitJobPending(context, job) - Expect(err).NotTo(HaveOccurred()) - - err = removeTaintsFromAllNodes(context, taints) - Expect(err).NotTo(HaveOccurred()) - - err = waitJobReady(context, job) - Expect(err).NotTo(HaveOccurred()) - }) - }) From bc7559d7a20cebf3a59339b279b6d45a397cf132 Mon Sep 17 00:00:00 2001 From: lminzhw Date: Sun, 19 May 2019 16:28:01 +0800 Subject: [PATCH 42/68] move scheduler name out of common args --- pkg/cli/job/common.go | 6 ++---- pkg/cli/job/list.go | 7 ++++++- pkg/cli/job/run.go | 10 ++++++---- 3 files changed, 14 insertions(+), 9 deletions(-) diff --git a/pkg/cli/job/common.go b/pkg/cli/job/common.go index 43fff20759..040b566112 100644 --- a/pkg/cli/job/common.go +++ b/pkg/cli/job/common.go @@ -22,13 +22,11 @@ import ( ) type commonFlags struct { - Master string - Kubeconfig string - SchedulerName string + Master string + Kubeconfig string } func initFlags(cmd *cobra.Command, cf *commonFlags) { - cmd.Flags().StringVarP(&cf.SchedulerName, "scheduler", "S", "kube-batch", "the scheduler for this job") cmd.Flags().StringVarP(&cf.Master, "master", "s", "", "the address of apiserver") if home := homeDir(); home != "" { diff --git a/pkg/cli/job/list.go b/pkg/cli/job/list.go index bf6afac074..0720726aa0 100644 --- a/pkg/cli/job/list.go +++ b/pkg/cli/job/list.go @@ -31,7 +31,8 @@ import ( type listFlags struct { commonFlags - Namespace string + Namespace string + SchedulerName string } const ( @@ -54,6 +55,7 @@ func InitListFlags(cmd *cobra.Command) { initFlags(cmd, &listJobFlags.commonFlags) cmd.Flags().StringVarP(&listJobFlags.Namespace, "namespace", "N", "default", "the namespace of job") + cmd.Flags().StringVarP(&listJobFlags.SchedulerName, "scheduler", "S", "", "list job with specified scheduler name") } func ListJobs() error { @@ -86,6 +88,9 @@ func PrintJobs(jobs *v1alpha1.JobList, writer io.Writer) { } for _, job := range jobs.Items { + if listJobFlags.SchedulerName != "" && listJobFlags.SchedulerName != job.Spec.SchedulerName { + continue + } replicas := int32(0) for _, ts := range job.Spec.Tasks { replicas += ts.Replicas diff --git a/pkg/cli/job/run.go b/pkg/cli/job/run.go index 79372f5920..4fc8c28afb 100644 --- a/pkg/cli/job/run.go +++ b/pkg/cli/job/run.go @@ -32,10 +32,11 @@ type runFlags struct { Namespace string Image string - MinAvailable int - Replicas int - Requests string - Limits string + MinAvailable int + Replicas int + Requests string + Limits string + SchedulerName string } var launchJobFlags = &runFlags{} @@ -50,6 +51,7 @@ func InitRunFlags(cmd *cobra.Command) { cmd.Flags().IntVarP(&launchJobFlags.Replicas, "replicas", "r", 1, "the total tasks of job") cmd.Flags().StringVarP(&launchJobFlags.Requests, "requests", "R", "cpu=1000m,memory=100Mi", "the resource request of the task") cmd.Flags().StringVarP(&launchJobFlags.Limits, "limits", "L", "cpu=1000m,memory=100Mi", "the resource limit of the task") + cmd.Flags().StringVarP(&listJobFlags.SchedulerName, "scheduler", "S", "kube-batch", "the scheduler for this job") } var jobName = "job.volcano.sh" From 6e0e9985396cdb071f8d67e687caccb4ee65916d Mon Sep 17 00:00:00 2001 From: lminzhw Date: Sun, 19 May 2019 17:17:31 +0800 Subject: [PATCH 43/68] support vkctl job view --- cmd/cli/job.go | 10 ++++++ pkg/cli/job/list.go | 25 ++++++++------ pkg/cli/job/view.go | 82 +++++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 106 insertions(+), 11 deletions(-) create mode 100644 pkg/cli/job/view.go diff --git a/cmd/cli/job.go b/cmd/cli/job.go index a8a49daf98..c6892af68a 100644 --- a/cmd/cli/job.go +++ b/cmd/cli/job.go @@ -32,6 +32,16 @@ func buildJobCmd() *cobra.Command { job.InitListFlags(jobListCmd) jobCmd.AddCommand(jobListCmd) + jobViewCmd := &cobra.Command{ + Use: "view", + Short: "show job information", + Run: func(cmd *cobra.Command, args []string) { + checkError(cmd, job.ViewJob()) + }, + } + job.InitViewFlags(jobViewCmd) + jobCmd.AddCommand(jobViewCmd) + jobSuspendCmd := &cobra.Command{ Use: "suspend", Short: "abort a job", diff --git a/pkg/cli/job/list.go b/pkg/cli/job/list.go index bf6afac074..ba14c07324 100644 --- a/pkg/cli/job/list.go +++ b/pkg/cli/job/list.go @@ -35,17 +35,20 @@ type listFlags struct { } const ( - Name string = "Name" - Creation string = "Creation" - Phase string = "Phase" - Replicas string = "Replicas" - Min string = "Min" - Pending string = "Pending" - Running string = "Running" - Succeeded string = "Succeeded" - Failed string = "Failed" - RetryCount string = "RetryCount" - JobType string = "JobType" + Name string = "Name" + Creation string = "Creation" + Phase string = "Phase" + Replicas string = "Replicas" + Min string = "Min" + Scheduler string = "Scheduler" + Pending string = "Pending" + Running string = "Running" + Succeeded string = "Succeeded" + Terminating string = "Terminating" + Version string = "Version" + Failed string = "Failed" + RetryCount string = "RetryCount" + JobType string = "JobType" ) var listJobFlags = &listFlags{} diff --git a/pkg/cli/job/view.go b/pkg/cli/job/view.go new file mode 100644 index 0000000000..65c11a37d5 --- /dev/null +++ b/pkg/cli/job/view.go @@ -0,0 +1,82 @@ +package job + +import ( + "fmt" + "io" + "os" + "strings" + + "github.com/spf13/cobra" + + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + "volcano.sh/volcano/pkg/apis/batch/v1alpha1" + "volcano.sh/volcano/pkg/client/clientset/versioned" +) + +type viewFlags struct { + commonFlags + + Namespace string + JobName string +} + +var viewJobFlags = &viewFlags{} + +func InitViewFlags(cmd *cobra.Command) { + initFlags(cmd, &viewJobFlags.commonFlags) + + cmd.Flags().StringVarP(&viewJobFlags.Namespace, "namespace", "N", "default", "the namespace of job") + cmd.Flags().StringVarP(&viewJobFlags.JobName, "name", "n", "", "the name of job") +} + +func ViewJob() error { + config, err := buildConfig(viewJobFlags.Master, viewJobFlags.Kubeconfig) + if err != nil { + return err + } + if viewJobFlags.JobName == "" { + err := fmt.Errorf("job name (specified by --name or -n) is mandaorty to view a particular job") + return err + } + + jobClient := versioned.NewForConfigOrDie(config) + job, err := jobClient.BatchV1alpha1().Jobs(viewJobFlags.Namespace).Get(viewJobFlags.JobName, metav1.GetOptions{}) + if err != nil { + return err + } + if job == nil { + fmt.Printf("No resources found\n") + return nil + } + PrintJob(job, os.Stdout) + + return nil +} + +func PrintJob(job *v1alpha1.Job, writer io.Writer) { + replicas := int32(0) + for _, ts := range job.Spec.Tasks { + replicas += ts.Replicas + } + lines := []string{ + fmt.Sprintf("%s:\t\t%s", Name, job.Name), + fmt.Sprintf("%s:\t%s", Creation, job.CreationTimestamp.Format("2006-01-02 15:04:05")), + fmt.Sprintf("%s:\t%d", Replicas, replicas), + fmt.Sprintf("%s:\t\t%d", Min, job.Status.MinAvailable), + fmt.Sprintf("%s:\t%s", Scheduler, job.Spec.SchedulerName), + "Status", + fmt.Sprintf(" %s:\t%s", Phase, job.Status.State.Phase), + fmt.Sprintf(" %s:\t%d", Version, job.Status.Version), + fmt.Sprintf(" %s:\t%d", RetryCount, job.Status.RetryCount), + fmt.Sprintf(" %s:\t%d", Pending, job.Status.Pending), + fmt.Sprintf(" %s:\t%d", Running, job.Status.Running), + fmt.Sprintf(" %s:\t%d", Succeeded, job.Status.Succeeded), + fmt.Sprintf(" %s:\t%d", Failed, job.Status.Failed), + fmt.Sprintf(" %s:\t%d", Terminating, job.Status.Terminating), + } + _, err := fmt.Fprint(writer, strings.Join(lines, "\n"), "\n") + if err != nil { + fmt.Printf("Failed to print view command result: %s.\n", err) + } +} From a7fca3d1584131198687d0fa976c7c81ebcef423 Mon Sep 17 00:00:00 2001 From: TommyLike Date: Mon, 20 May 2019 10:29:44 +0800 Subject: [PATCH 44/68] Remove unnecessary folder --- README.md | 12 ++++++------ hack/run-e2e-kind.sh | 4 ++-- installer/chart/{volcano => }/Chart.yaml | 0 installer/chart/{volcano => }/config/kube-batch.conf | 0 .../gen-admission-secret/gen-admission-secret.sh | 0 .../plugins/gen-admission-secret/plugin.yaml | 0 .../{volcano => }/templates/admission-config.yaml | 0 .../chart/{volcano => }/templates/admission.yaml | 0 .../{volcano => }/templates/batch_v1alpha1_job.yaml | 0 .../templates/bus_v1alpha1_command.yaml | 0 .../chart/{volcano => }/templates/controllers.yaml | 0 .../chart/{volcano => }/templates/default-queue.yaml | 0 .../chart/{volcano => }/templates/scheduler.yaml | 0 .../templates/scheduling_v1alpha1_podgroup.yaml | 0 .../templates/scheduling_v1alpha1_queue.yaml | 0 installer/chart/{volcano => }/values.yaml | 0 16 files changed, 8 insertions(+), 8 deletions(-) rename installer/chart/{volcano => }/Chart.yaml (100%) rename installer/chart/{volcano => }/config/kube-batch.conf (100%) rename installer/chart/{volcano => }/plugins/gen-admission-secret/gen-admission-secret.sh (100%) rename installer/chart/{volcano => }/plugins/gen-admission-secret/plugin.yaml (100%) rename installer/chart/{volcano => }/templates/admission-config.yaml (100%) rename installer/chart/{volcano => }/templates/admission.yaml (100%) rename installer/chart/{volcano => }/templates/batch_v1alpha1_job.yaml (100%) rename installer/chart/{volcano => }/templates/bus_v1alpha1_command.yaml (100%) rename installer/chart/{volcano => }/templates/controllers.yaml (100%) rename installer/chart/{volcano => }/templates/default-queue.yaml (100%) rename installer/chart/{volcano => }/templates/scheduler.yaml (100%) rename installer/chart/{volcano => }/templates/scheduling_v1alpha1_podgroup.yaml (100%) rename installer/chart/{volcano => }/templates/scheduling_v1alpha1_queue.yaml (100%) rename installer/chart/{volcano => }/values.yaml (100%) diff --git a/README.md b/README.md index 1b66ff4173..a3db80393c 100644 --- a/README.md +++ b/README.md @@ -79,9 +79,9 @@ make images ## Verify your images # docker images REPOSITORY TAG IMAGE ID CREATED SIZE -volcanosh/volcano-admission latest a83338506638 8 seconds ago 41.4MB -volcanosh/volcano-scheduler latest faa3c2a25ac3 9 seconds ago 49.6MB -volcanosh/volcano-controllers latest 7b11606ebfb8 10 seconds ago 44.2MB +volcanosh/vk-admission latest a83338506638 8 seconds ago 41.4MB +volcanosh/vk-kube-batch latest faa3c2a25ac3 9 seconds ago 49.6MB +volcanosh/vk-controllers latest 7b11606ebfb8 10 seconds ago 44.2MB ``` @@ -97,7 +97,7 @@ kubernetes API server. ``` #1. Install helm plugin -helm plugin install installer/chart/volcano/plugins/gen-admission-secret +helm plugin install installer/chart/plugins/gen-admission-secret #2. Generate secret within service name helm gen-admission-secret --service -admission-service --namespace @@ -112,10 +112,10 @@ helm gen-admission-secret --service volcano-trial-admission-service --namespace Finally, install helm chart. ``` -helm install installer/chart/volcano --namespace --name +helm install installer/chart --namespace --name For eg : -helm install installer/chart/volcano --namespace volcano-trial --name volcano-trial +helm install installer/chart --namespace volcano-trial --name volcano-trial ``` diff --git a/hack/run-e2e-kind.sh b/hack/run-e2e-kind.sh index 341c8899de..884b5002cd 100755 --- a/hack/run-e2e-kind.sh +++ b/hack/run-e2e-kind.sh @@ -63,7 +63,7 @@ function install-volcano { kind load docker-image ${MPI_EXAMPLE_IMAGE} ${CLUSTER_CONTEXT} echo "Install volcano plugin into cluster...." - helm plugin install --kubeconfig ${KUBECONFIG} installer/chart/volcano/plugins/gen-admission-secret + helm plugin install --kubeconfig ${KUBECONFIG} installer/chart/plugins/gen-admission-secret #If failed to generate secret for admission service, return immediately helm gen-admission-secret --service ${CLUSTER_NAME}-admission-service --namespace kube-system @@ -73,7 +73,7 @@ function install-volcano { fi echo "Install volcano chart" - helm install installer/chart/volcano --namespace kube-system --name ${CLUSTER_NAME} --kubeconfig ${KUBECONFIG} --set basic.image_tag_version=${TAG} --wait + helm install installer/chart --namespace kube-system --name ${CLUSTER_NAME} --kubeconfig ${KUBECONFIG} --set basic.image_tag_version=${TAG} --wait } function uninstall-volcano { diff --git a/installer/chart/volcano/Chart.yaml b/installer/chart/Chart.yaml similarity index 100% rename from installer/chart/volcano/Chart.yaml rename to installer/chart/Chart.yaml diff --git a/installer/chart/volcano/config/kube-batch.conf b/installer/chart/config/kube-batch.conf similarity index 100% rename from installer/chart/volcano/config/kube-batch.conf rename to installer/chart/config/kube-batch.conf diff --git a/installer/chart/volcano/plugins/gen-admission-secret/gen-admission-secret.sh b/installer/chart/plugins/gen-admission-secret/gen-admission-secret.sh similarity index 100% rename from installer/chart/volcano/plugins/gen-admission-secret/gen-admission-secret.sh rename to installer/chart/plugins/gen-admission-secret/gen-admission-secret.sh diff --git a/installer/chart/volcano/plugins/gen-admission-secret/plugin.yaml b/installer/chart/plugins/gen-admission-secret/plugin.yaml similarity index 100% rename from installer/chart/volcano/plugins/gen-admission-secret/plugin.yaml rename to installer/chart/plugins/gen-admission-secret/plugin.yaml diff --git a/installer/chart/volcano/templates/admission-config.yaml b/installer/chart/templates/admission-config.yaml similarity index 100% rename from installer/chart/volcano/templates/admission-config.yaml rename to installer/chart/templates/admission-config.yaml diff --git a/installer/chart/volcano/templates/admission.yaml b/installer/chart/templates/admission.yaml similarity index 100% rename from installer/chart/volcano/templates/admission.yaml rename to installer/chart/templates/admission.yaml diff --git a/installer/chart/volcano/templates/batch_v1alpha1_job.yaml b/installer/chart/templates/batch_v1alpha1_job.yaml similarity index 100% rename from installer/chart/volcano/templates/batch_v1alpha1_job.yaml rename to installer/chart/templates/batch_v1alpha1_job.yaml diff --git a/installer/chart/volcano/templates/bus_v1alpha1_command.yaml b/installer/chart/templates/bus_v1alpha1_command.yaml similarity index 100% rename from installer/chart/volcano/templates/bus_v1alpha1_command.yaml rename to installer/chart/templates/bus_v1alpha1_command.yaml diff --git a/installer/chart/volcano/templates/controllers.yaml b/installer/chart/templates/controllers.yaml similarity index 100% rename from installer/chart/volcano/templates/controllers.yaml rename to installer/chart/templates/controllers.yaml diff --git a/installer/chart/volcano/templates/default-queue.yaml b/installer/chart/templates/default-queue.yaml similarity index 100% rename from installer/chart/volcano/templates/default-queue.yaml rename to installer/chart/templates/default-queue.yaml diff --git a/installer/chart/volcano/templates/scheduler.yaml b/installer/chart/templates/scheduler.yaml similarity index 100% rename from installer/chart/volcano/templates/scheduler.yaml rename to installer/chart/templates/scheduler.yaml diff --git a/installer/chart/volcano/templates/scheduling_v1alpha1_podgroup.yaml b/installer/chart/templates/scheduling_v1alpha1_podgroup.yaml similarity index 100% rename from installer/chart/volcano/templates/scheduling_v1alpha1_podgroup.yaml rename to installer/chart/templates/scheduling_v1alpha1_podgroup.yaml diff --git a/installer/chart/volcano/templates/scheduling_v1alpha1_queue.yaml b/installer/chart/templates/scheduling_v1alpha1_queue.yaml similarity index 100% rename from installer/chart/volcano/templates/scheduling_v1alpha1_queue.yaml rename to installer/chart/templates/scheduling_v1alpha1_queue.yaml diff --git a/installer/chart/volcano/values.yaml b/installer/chart/values.yaml similarity index 100% rename from installer/chart/volcano/values.yaml rename to installer/chart/values.yaml From 77c296fb956b2e5497a96ee2d00943ea6ae59290 Mon Sep 17 00:00:00 2001 From: TommyLike Date: Mon, 20 May 2019 11:09:51 +0800 Subject: [PATCH 45/68] Support admission job --- README.md | 13 +------- hack/run-e2e-kind.sh | 10 ------ .../plugins/gen-admission-secret/plugin.yaml | 7 ----- installer/chart/templates/admission.yaml | 31 +++++++++++++++++++ installer/dockerfile/admission/Dockerfile | 11 +++++++ .../admission}/gen-admission-secret.sh | 13 +++++--- 6 files changed, 51 insertions(+), 34 deletions(-) delete mode 100644 installer/chart/plugins/gen-admission-secret/plugin.yaml rename installer/{chart/plugins/gen-admission-secret => dockerfile/admission}/gen-admission-secret.sh (94%) diff --git a/README.md b/README.md index a3db80393c..810a6373e9 100644 --- a/README.md +++ b/README.md @@ -99,17 +99,9 @@ kubernetes API server. #1. Install helm plugin helm plugin install installer/chart/plugins/gen-admission-secret -#2. Generate secret within service name -helm gen-admission-secret --service -admission-service --namespace - -## For eg: -kubectl create namespace volcano-trial - -helm gen-admission-secret --service volcano-trial-admission-service --namespace volcano-trial - ``` -Finally, install helm chart. +Secondly, install helm chart. ``` helm install installer/chart --namespace --name @@ -119,9 +111,6 @@ helm install installer/chart --namespace volcano-trial --name volcano-trial ``` -**NOTE**:The `````` used in the two commands above should be identical. - - To Verify your installation run the following commands: ``` diff --git a/hack/run-e2e-kind.sh b/hack/run-e2e-kind.sh index 884b5002cd..fe617a262f 100755 --- a/hack/run-e2e-kind.sh +++ b/hack/run-e2e-kind.sh @@ -62,16 +62,6 @@ function install-volcano { kind load docker-image ${IMAGE_PREFIX}-admission:${TAG} ${CLUSTER_CONTEXT} kind load docker-image ${MPI_EXAMPLE_IMAGE} ${CLUSTER_CONTEXT} - echo "Install volcano plugin into cluster...." - helm plugin install --kubeconfig ${KUBECONFIG} installer/chart/plugins/gen-admission-secret - - #If failed to generate secret for admission service, return immediately - helm gen-admission-secret --service ${CLUSTER_NAME}-admission-service --namespace kube-system - if [[ $? != 0 ]]; then - echo "Failed to install secret for admission service, usually we need a retry." - exit 1 - fi - echo "Install volcano chart" helm install installer/chart --namespace kube-system --name ${CLUSTER_NAME} --kubeconfig ${KUBECONFIG} --set basic.image_tag_version=${TAG} --wait } diff --git a/installer/chart/plugins/gen-admission-secret/plugin.yaml b/installer/chart/plugins/gen-admission-secret/plugin.yaml deleted file mode 100644 index 3159e1d93e..0000000000 --- a/installer/chart/plugins/gen-admission-secret/plugin.yaml +++ /dev/null @@ -1,7 +0,0 @@ -name: "gen-admission-secret" -version: "1.0.0" -usage: "Generate valid cert for admission server" -description: This plugin provides signed cert to admission server. -ignoreFlags: false -useTunnel: false -command: "$HELM_PLUGIN_DIR/gen-admission-secret.sh" \ No newline at end of file diff --git a/installer/chart/templates/admission.yaml b/installer/chart/templates/admission.yaml index 85693450ec..9b57314d0e 100644 --- a/installer/chart/templates/admission.yaml +++ b/installer/chart/templates/admission.yaml @@ -19,6 +19,16 @@ rules: - apiGroups: ["admissionregistration.k8s.io"] resources: ["validatingwebhookconfigurations"] verbs: ["get", "list", "watch", "patch"] + # Rules below is used generate admission service secret + - apiGroups: ["certificates.k8s.io"] + resources: ["certificatesigningrequests"] + verbs: ["get", "list", "create", "delete"] + - apiGroups: ["certificates.k8s.io"] + resources: ["certificatesigningrequests/approval"] + verbs: ["create", "update"] + - apiGroups: [""] + resources: ["secrets"] + verbs: ["create", "get", "patch"] --- kind: ClusterRoleBinding @@ -98,3 +108,24 @@ spec: selector: app: volcano-admission sessionAffinity: None + +--- +apiVersion: batch/v1 +kind: Job +metadata: + name: {{ .Release.Name }}-admission-init + namespace: {{ .Release.Namespace }} + labels: + app: volcano-admission-init +spec: + backoffLimit: 3 + template: + spec: + serviceAccountName: {{ .Release.Name }}-admission + restartPolicy: Never + containers: + - name: main + image: {{.Values.basic.admission_image_name}}:{{.Values.basic.image_tag_version}} + imagePullPolicy: IfNotPresent + command: ["./gen-admission-secret.sh", "--service", "{{ .Release.Name }}-admission-service", "--namespace", + "{{ .Release.Namespace }}", "--secret", "{{.Values.basic.admission_secret_name}}"] diff --git a/installer/dockerfile/admission/Dockerfile b/installer/dockerfile/admission/Dockerfile index 1827514ef0..c275ae5015 100644 --- a/installer/dockerfile/admission/Dockerfile +++ b/installer/dockerfile/admission/Dockerfile @@ -15,5 +15,16 @@ FROM alpine:latest +# Install requirements +ARG KUBE_VERSION="1.13.1" +RUN apk add --update ca-certificates && \ + apk add --update openssl && \ + apk add --update -t deps curl && \ + curl -L https://storage.googleapis.com/kubernetes-release/release/v$KUBE_VERSION/bin/linux/amd64/kubectl -o /usr/local/bin/kubectl && \ + chmod +x /usr/local/bin/kubectl && \ + apk del --purge deps && \ + rm /var/cache/apk/* + ADD vk-admission /vk-admission +ADD gen-admission-secret.sh /gen-admission-secret.sh ENTRYPOINT ["/vk-admission"] diff --git a/installer/chart/plugins/gen-admission-secret/gen-admission-secret.sh b/installer/dockerfile/admission/gen-admission-secret.sh similarity index 94% rename from installer/chart/plugins/gen-admission-secret/gen-admission-secret.sh rename to installer/dockerfile/admission/gen-admission-secret.sh index 84bb061fb1..3ad7e40abc 100755 --- a/installer/chart/plugins/gen-admission-secret/gen-admission-secret.sh +++ b/installer/dockerfile/admission/gen-admission-secret.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/bin/sh #TODO: this file is used for release, should not place it here set -e @@ -41,16 +41,19 @@ while [[ $# -gt 0 ]]; do shift done -if [ -z ${service} ]; then +if [[ -z ${service} ]]; then echo "'--service' must be specified" exit 1 fi +if [[ -z ${secret} ]]; then + echo "'--secret' must be specified" + exit 1 +fi -[ -z ${secret} ] && secret=volcano-admission-secret -[ -z ${namespace} ] && namespace=default +[[ -z ${namespace} ]] && namespace=default -if [ ! -x "$(command -v openssl)" ]; then +if [[ ! -x "$(command -v openssl)" ]]; then echo "openssl not found" exit 1 fi From a6694e4f2f1c3590a9203aa92ff0606d0d134d52 Mon Sep 17 00:00:00 2001 From: thandayuthapani Date: Thu, 30 May 2019 14:13:41 +0530 Subject: [PATCH 46/68] UT cases for files in Job Controller pkg --- .../job/job_controller_handler_test.go | 519 ++++++++++++++++++ 1 file changed, 519 insertions(+) create mode 100644 pkg/controllers/job/job_controller_handler_test.go diff --git a/pkg/controllers/job/job_controller_handler_test.go b/pkg/controllers/job/job_controller_handler_test.go new file mode 100644 index 0000000000..b96efcefef --- /dev/null +++ b/pkg/controllers/job/job_controller_handler_test.go @@ -0,0 +1,519 @@ +/* +Copyright 2019 The Volcano 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 job + +import ( + "fmt" + "k8s.io/apimachinery/pkg/types" + "k8s.io/client-go/rest" + "testing" + + kbv1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1" + kubebatchclient "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned" + "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + kubeclientset "k8s.io/client-go/kubernetes" + vkbatchv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" + vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" + vkbusv1 "volcano.sh/volcano/pkg/apis/bus/v1alpha1" + vkclientset "volcano.sh/volcano/pkg/client/clientset/versioned" + //"volcano.sh/volcano/pkg/controllers/job" +) + +func newController() *Controller { + kubeClientSet := kubeclientset.NewForConfigOrDie(&rest.Config{ + Host: "", + ContentConfig: rest.ContentConfig{ + GroupVersion: &v1.SchemeGroupVersion, + }, + }, + ) + + kubeBatchClientSet := kubebatchclient.NewForConfigOrDie(&rest.Config{ + Host: "", + ContentConfig: rest.ContentConfig{ + GroupVersion: &kbv1.SchemeGroupVersion, + }, + }, + ) + + config := &rest.Config{ + Host: "", + ContentConfig: rest.ContentConfig{ + GroupVersion: &vkv1.SchemeGroupVersion, + }, + } + + vkclient := vkclientset.NewForConfigOrDie(config) + controller := NewJobController(kubeClientSet, kubeBatchClientSet, vkclient) + + return controller +} + +func buildPod(namespace, name string, p v1.PodPhase, labels map[string]string) *v1.Pod { + return &v1.Pod{ + ObjectMeta: metav1.ObjectMeta{ + UID: types.UID(fmt.Sprintf("%v-%v", namespace, name)), + Name: name, + Namespace: namespace, + Labels: labels, + }, + Status: v1.PodStatus{ + Phase: p, + }, + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "nginx", + Image: "nginx:latest", + }, + }, + }, + } +} + +func addPodAnnotation(pod *v1.Pod, annotations map[string]string) *v1.Pod { + podWithAnnotation := pod + for key, value := range annotations { + if podWithAnnotation.Annotations == nil { + podWithAnnotation.Annotations = make(map[string]string) + } + podWithAnnotation.Annotations[key] = value + } + return podWithAnnotation +} + +func TestAddCommandFunc(t *testing.T) { + + namespace := "test" + + testCases := []struct { + Name string + command interface{} + ExpectValue int + }{ + { + Name: "AddCommand Sucess Case", + command: &vkbusv1.Command{ + ObjectMeta: metav1.ObjectMeta{ + Name: "Valid Command", + Namespace: namespace, + }, + }, + ExpectValue: 1, + }, + { + Name: "AddCommand Failure Case", + command: "Command", + ExpectValue: 0, + }, + } + + for i, testcase := range testCases { + controller := newController() + controller.addCommand(testcase.command) + len := controller.commandQueue.Len() + if testcase.ExpectValue != len { + t.Errorf("case %d (%s): expected: %v, got %v ", i, testcase.Name, testcase.ExpectValue, len) + } + } +} + +func TestJobAddFunc(t *testing.T) { + namespace := "test" + + testCases := []struct { + Name string + job *vkbatchv1.Job + ExpectValue int + }{ + { + Name: "AddJob Success", + job: &vkbatchv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "Job1", + Namespace: namespace, + }, + }, + ExpectValue: 1, + }, + } + for i, testcase := range testCases { + controller := newController() + controller.addJob(testcase.job) + key := fmt.Sprintf("%s/%s", testcase.job.Namespace, testcase.job.Name) + job, err := controller.cache.Get(key) + if job == nil || err != nil { + t.Errorf("Error while Adding Job in case %d with error %s", i, err) + } + len := controller.queue.Len() + if testcase.ExpectValue != len { + t.Errorf("case %d (%s): expected: %v, got %v ", i, testcase.Name, testcase.ExpectValue, len) + } + } +} + +func TestUpdateJobFunc(t *testing.T) { + namespace := "test" + + testcases := []struct { + Name string + oldJob *vkbatchv1.Job + newJob *vkbatchv1.Job + }{ + { + Name: "Job Update Success Case", + oldJob: &vkbatchv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job1", + Namespace: namespace, + }, + Spec: vkbatchv1.JobSpec{ + SchedulerName: "kube-batch", + MinAvailable: 5, + }, + Status: vkbatchv1.JobStatus{ + State: vkbatchv1.JobState{ + Phase: vkbatchv1.Pending, + }, + }, + }, + newJob: &vkbatchv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job1", + Namespace: namespace, + }, + Spec: vkbatchv1.JobSpec{ + SchedulerName: "kube-batch", + MinAvailable: 5, + }, + Status: vkbatchv1.JobStatus{ + State: vkbatchv1.JobState{ + Phase: vkbatchv1.Running, + }, + }, + }, + }, + { + Name: "Job Update Failure Case", + oldJob: &vkbatchv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job1", + Namespace: namespace, + }, + Spec: vkbatchv1.JobSpec{ + SchedulerName: "kube-batch", + MinAvailable: 5, + }, + Status: vkbatchv1.JobStatus{ + State: vkbatchv1.JobState{ + Phase: vkbatchv1.Pending, + }, + }, + }, + newJob: &vkbatchv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job1", + Namespace: namespace, + }, + Spec: vkbatchv1.JobSpec{ + SchedulerName: "kube-batch", + MinAvailable: 5, + }, + Status: vkbatchv1.JobStatus{ + State: vkbatchv1.JobState{ + Phase: vkbatchv1.Pending, + }, + }, + }, + }, + } + + for i, testcase := range testcases { + controller := newController() + controller.addJob(testcase.oldJob) + controller.updateJob(testcase.oldJob, testcase.newJob) + key := fmt.Sprintf("%s/%s", testcase.newJob.Namespace, testcase.newJob.Name) + job, err := controller.cache.Get(key) + + if job == nil || err != nil { + t.Errorf("Error while Updating Job in case %d with error %s", i, err) + } + + if job.Job.Status.State.Phase != testcase.newJob.Status.State.Phase { + t.Errorf("Error while Updating Job in case %d with error %s", i, err) + } + } +} + +func TestAddPodFunc(t *testing.T) { + namespace := "test" + + testcases := []struct { + Name string + Job *vkbatchv1.Job + pods []*v1.Pod + Annotation map[string]string + ExpectedValue int + }{ + { + Name: "AddPod Success case", + Job: &vkbatchv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job1", + Namespace: namespace, + }, + }, + pods: []*v1.Pod{ + buildPod(namespace, "pod1", v1.PodPending, nil), + }, + Annotation: map[string]string{ + vkbatchv1.JobNameKey: "job1", + vkbatchv1.JobVersion: "0", + vkbatchv1.TaskSpecKey: "task1", + }, + ExpectedValue: 1, + }, + { + Name: "AddPod Duplicate Pod case", + Job: &vkbatchv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job1", + Namespace: namespace, + }, + }, + pods: []*v1.Pod{ + buildPod(namespace, "pod1", v1.PodPending, nil), + buildPod(namespace, "pod1", v1.PodPending, nil), + }, + Annotation: map[string]string{ + vkbatchv1.JobNameKey: "job1", + vkbatchv1.JobVersion: "0", + vkbatchv1.TaskSpecKey: "task1", + }, + ExpectedValue: 1, + }, + } + + for i, testcase := range testcases { + controller := newController() + controller.addJob(testcase.Job) + for _, pod := range testcase.pods { + addPodAnnotation(pod, testcase.Annotation) + controller.addPod(pod) + } + + key := fmt.Sprintf("%s/%s", testcase.Job.Namespace, testcase.Job.Name) + job, err := controller.cache.Get(key) + + if job == nil || err != nil { + t.Errorf("Error while Getting Job in case %d with error %s", i, err) + } + + var totalPods int + for _, task := range job.Pods { + totalPods = len(task) + } + if totalPods != testcase.ExpectedValue { + t.Errorf("case %d (%s): expected: %v, got %v ", i, testcase.Name, testcase.ExpectedValue, totalPods) + } + } +} + +func TestUpdatePodFunc(t *testing.T) { + namespace := "test" + + testcases := []struct { + Name string + Job *vkbatchv1.Job + oldPod *v1.Pod + newPod *v1.Pod + Annotation map[string]string + ExpectedValue v1.PodPhase + }{ + { + Name: "UpdatePod Success case", + Job: &vkbatchv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job1", + Namespace: namespace, + }, + }, + oldPod: buildPod(namespace, "pod1", v1.PodPending, nil), + newPod: buildPod(namespace, "pod1", v1.PodRunning, nil), + Annotation: map[string]string{ + vkbatchv1.JobNameKey: "job1", + vkbatchv1.JobVersion: "0", + vkbatchv1.TaskSpecKey: "task1", + }, + ExpectedValue: v1.PodRunning, + }, + } + + for i, testcase := range testcases { + controller := newController() + controller.addJob(testcase.Job) + addPodAnnotation(testcase.oldPod, testcase.Annotation) + addPodAnnotation(testcase.newPod, testcase.Annotation) + controller.addPod(testcase.oldPod) + controller.updatePod(testcase.oldPod, testcase.newPod) + + key := fmt.Sprintf("%s/%s", testcase.Job.Namespace, testcase.Job.Name) + job, err := controller.cache.Get(key) + + if job == nil || err != nil { + t.Errorf("Error while Getting Job in case %d with error %s", i, err) + } + + pod := job.Pods[testcase.Annotation[vkbatchv1.TaskSpecKey]][testcase.oldPod.Name] + + if pod.Status.Phase != testcase.ExpectedValue { + t.Errorf("case %d (%s): expected: %v, got %v ", i, testcase.Name, testcase.ExpectedValue, pod.Status.Phase) + } + } +} + +func TestDeletePodFunc(t *testing.T) { + namespace := "test" + + testcases := []struct { + Name string + Job *vkbatchv1.Job + availablePods []*v1.Pod + deletePod *v1.Pod + Annotation map[string]string + ExpectedValue int + }{ + { + Name: "DeletePod success case", + Job: &vkbatchv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job1", + Namespace: namespace, + }, + }, + availablePods: []*v1.Pod{ + buildPod(namespace, "pod1", v1.PodRunning, nil), + buildPod(namespace, "pod2", v1.PodRunning, nil), + }, + deletePod: buildPod(namespace, "pod2", v1.PodRunning, nil), + Annotation: map[string]string{ + vkbatchv1.JobNameKey: "job1", + vkbatchv1.JobVersion: "0", + vkbatchv1.TaskSpecKey: "task1", + }, + ExpectedValue: 1, + }, + { + Name: "DeletePod Pod NotAvailable case", + Job: &vkbatchv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job1", + Namespace: namespace, + }, + }, + availablePods: []*v1.Pod{ + buildPod(namespace, "pod1", v1.PodRunning, nil), + buildPod(namespace, "pod2", v1.PodRunning, nil), + }, + deletePod: buildPod(namespace, "pod3", v1.PodRunning, nil), + Annotation: map[string]string{ + vkbatchv1.JobNameKey: "job1", + vkbatchv1.JobVersion: "0", + vkbatchv1.TaskSpecKey: "task1", + }, + ExpectedValue: 2, + }, + } + + for i, testcase := range testcases { + controller := newController() + controller.addJob(testcase.Job) + for _, pod := range testcase.availablePods { + addPodAnnotation(pod, testcase.Annotation) + controller.addPod(pod) + } + + addPodAnnotation(testcase.deletePod, testcase.Annotation) + controller.deletePod(testcase.deletePod) + key := fmt.Sprintf("%s/%s", testcase.Job.Namespace, testcase.Job.Name) + job, err := controller.cache.Get(key) + + if job == nil || err != nil { + t.Errorf("Error while Getting Job in case %d with error %s", i, err) + } + + var totalPods int + for _, task := range job.Pods { + totalPods = len(task) + } + + if totalPods != testcase.ExpectedValue { + t.Errorf("case %d (%s): expected: %v, got %v ", i, testcase.Name, testcase.ExpectedValue, totalPods) + } + } +} + +func TestUpdatePodGroupFunc(t *testing.T) { + + namespace := "test" + + testCases := []struct { + Name string + oldPodGroup *kbv1.PodGroup + newPodGroup *kbv1.PodGroup + ExpectValue int + }{ + { + Name: "AddCommand Sucess Case", + oldPodGroup: &kbv1.PodGroup{ + ObjectMeta: metav1.ObjectMeta{ + Name: "pg1", + Namespace: namespace, + }, + Spec: kbv1.PodGroupSpec{ + MinMember: 3, + }, + Status: kbv1.PodGroupStatus{ + Phase: kbv1.PodGroupPending, + }, + }, + newPodGroup: &kbv1.PodGroup{ + ObjectMeta: metav1.ObjectMeta{ + Name: "pg1", + Namespace: namespace, + }, + Spec: kbv1.PodGroupSpec{ + MinMember: 3, + }, + Status: kbv1.PodGroupStatus{ + Phase: kbv1.PodGroupRunning, + }, + }, + ExpectValue: 1, + }, + } + + for i, testcase := range testCases { + controller := newController() + controller.updatePodGroup(testcase.oldPodGroup, testcase.newPodGroup) + len := controller.queue.Len() + if testcase.ExpectValue != len { + t.Errorf("case %d (%s): expected: %v, got %v ", i, testcase.Name, testcase.ExpectValue, len) + } + } +} From e22bc356681fe3a38e40be578392b041c344366b Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Thu, 30 May 2019 15:57:22 +0530 Subject: [PATCH 47/68] allowing the controller to update the configmaps --- installer/chart/templates/controllers.yaml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/installer/chart/templates/controllers.yaml b/installer/chart/templates/controllers.yaml index dcc6357597..6ce0743edc 100644 --- a/installer/chart/templates/controllers.yaml +++ b/installer/chart/templates/controllers.yaml @@ -36,8 +36,11 @@ rules: resources: ["persistentvolumeclaims"] verbs: ["get", "list", "watch", "create"] - apiGroups: [""] - resources: ["services", "configmaps"] + resources: ["services"] verbs: ["get", "list", "watch", "create", "delete"] + - apiGroups: [""] + resources: ["configmaps"] + verbs: ["get", "list", "watch", "create", "delete", "update"] - apiGroups: ["scheduling.incubator.k8s.io"] resources: ["podgroups", "queues", "queues/status"] verbs: ["get", "list", "watch", "create", "delete", "update"] From d724454d45603e56e0ea7284653aa4552ead3856 Mon Sep 17 00:00:00 2001 From: thandayuthapani Date: Thu, 30 May 2019 17:03:49 +0530 Subject: [PATCH 48/68] Add Fake Package for scheduler in Vendor --- .../versioned/fake/clientset_generated.go | 82 ++++++++++ .../client/clientset/versioned/fake/doc.go | 20 +++ .../clientset/versioned/fake/register.go | 56 +++++++ .../typed/scheduling/v1alpha1/fake/doc.go | 20 +++ .../scheduling/v1alpha1/fake/fake_podgroup.go | 140 ++++++++++++++++++ .../scheduling/v1alpha1/fake/fake_queue.go | 131 ++++++++++++++++ .../v1alpha1/fake/fake_scheduling_client.go | 44 ++++++ 7 files changed, 493 insertions(+) create mode 100644 vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake/clientset_generated.go create mode 100644 vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake/doc.go create mode 100644 vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake/register.go create mode 100644 vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/doc.go create mode 100644 vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/fake_podgroup.go create mode 100644 vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/fake_queue.go create mode 100644 vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/fake_scheduling_client.go diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake/clientset_generated.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake/clientset_generated.go new file mode 100644 index 0000000000..c4da589c2d --- /dev/null +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake/clientset_generated.go @@ -0,0 +1,82 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + clientset "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned" + schedulingv1alpha1 "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1" + fakeschedulingv1alpha1 "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake" + "k8s.io/apimachinery/pkg/runtime" + "k8s.io/apimachinery/pkg/watch" + "k8s.io/client-go/discovery" + fakediscovery "k8s.io/client-go/discovery/fake" + "k8s.io/client-go/testing" +) + +// NewSimpleClientset returns a clientset that will respond with the provided objects. +// It's backed by a very simple object tracker that processes creates, updates and deletions as-is, +// without applying any validations and/or defaults. It shouldn't be considered a replacement +// for a real clientset and is mostly useful in simple unit tests. +func NewSimpleClientset(objects ...runtime.Object) *Clientset { + o := testing.NewObjectTracker(scheme, codecs.UniversalDecoder()) + for _, obj := range objects { + if err := o.Add(obj); err != nil { + panic(err) + } + } + + cs := &Clientset{} + cs.discovery = &fakediscovery.FakeDiscovery{Fake: &cs.Fake} + cs.AddReactor("*", "*", testing.ObjectReaction(o)) + cs.AddWatchReactor("*", func(action testing.Action) (handled bool, ret watch.Interface, err error) { + gvr := action.GetResource() + ns := action.GetNamespace() + watch, err := o.Watch(gvr, ns) + if err != nil { + return false, nil, err + } + return true, watch, nil + }) + + return cs +} + +// Clientset implements clientset.Interface. Meant to be embedded into a +// struct to get a default implementation. This makes faking out just the method +// you want to test easier. +type Clientset struct { + testing.Fake + discovery *fakediscovery.FakeDiscovery +} + +func (c *Clientset) Discovery() discovery.DiscoveryInterface { + return c.discovery +} + +var _ clientset.Interface = &Clientset{} + +// SchedulingV1alpha1 retrieves the SchedulingV1alpha1Client +func (c *Clientset) SchedulingV1alpha1() schedulingv1alpha1.SchedulingV1alpha1Interface { + return &fakeschedulingv1alpha1.FakeSchedulingV1alpha1{Fake: &c.Fake} +} + +// Scheduling retrieves the SchedulingV1alpha1Client +func (c *Clientset) Scheduling() schedulingv1alpha1.SchedulingV1alpha1Interface { + return &fakeschedulingv1alpha1.FakeSchedulingV1alpha1{Fake: &c.Fake} +} diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake/doc.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake/doc.go new file mode 100644 index 0000000000..9b99e71670 --- /dev/null +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// This package has the automatically generated fake clientset. +package fake diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake/register.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake/register.go new file mode 100644 index 0000000000..253431f3fb --- /dev/null +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake/register.go @@ -0,0 +1,56 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + schedulingv1alpha1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + runtime "k8s.io/apimachinery/pkg/runtime" + schema "k8s.io/apimachinery/pkg/runtime/schema" + serializer "k8s.io/apimachinery/pkg/runtime/serializer" + utilruntime "k8s.io/apimachinery/pkg/util/runtime" +) + +var scheme = runtime.NewScheme() +var codecs = serializer.NewCodecFactory(scheme) +var parameterCodec = runtime.NewParameterCodec(scheme) +var localSchemeBuilder = runtime.SchemeBuilder{ + schedulingv1alpha1.AddToScheme, +} + +// AddToScheme adds all types of this clientset into the given scheme. This allows composition +// of clientsets, like in: +// +// import ( +// "k8s.io/client-go/kubernetes" +// clientsetscheme "k8s.io/client-go/kubernetes/scheme" +// aggregatorclientsetscheme "k8s.io/kube-aggregator/pkg/client/clientset_generated/clientset/scheme" +// ) +// +// kclientset, _ := kubernetes.NewForConfig(c) +// _ = aggregatorclientsetscheme.AddToScheme(clientsetscheme.Scheme) +// +// After this, RawExtensions in Kubernetes types will serialize kube-aggregator types +// correctly. +var AddToScheme = localSchemeBuilder.AddToScheme + +func init() { + v1.AddToGroupVersion(scheme, schema.GroupVersion{Version: "v1"}) + utilruntime.Must(AddToScheme(scheme)) +} diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/doc.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/fake_podgroup.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/fake_podgroup.go new file mode 100644 index 0000000000..5bc2cd95f3 --- /dev/null +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/fake_podgroup.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakePodGroups implements PodGroupInterface +type FakePodGroups struct { + Fake *FakeSchedulingV1alpha1 + ns string +} + +var podgroupsResource = schema.GroupVersionResource{Group: "scheduling", Version: "v1alpha1", Resource: "podgroups"} + +var podgroupsKind = schema.GroupVersionKind{Group: "scheduling", Version: "v1alpha1", Kind: "PodGroup"} + +// Get takes name of the podGroup, and returns the corresponding podGroup object, and an error if there is any. +func (c *FakePodGroups) Get(name string, options v1.GetOptions) (result *v1alpha1.PodGroup, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(podgroupsResource, c.ns, name), &v1alpha1.PodGroup{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PodGroup), err +} + +// List takes label and field selectors, and returns the list of PodGroups that match those selectors. +func (c *FakePodGroups) List(opts v1.ListOptions) (result *v1alpha1.PodGroupList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(podgroupsResource, podgroupsKind, c.ns, opts), &v1alpha1.PodGroupList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1alpha1.PodGroupList{ListMeta: obj.(*v1alpha1.PodGroupList).ListMeta} + for _, item := range obj.(*v1alpha1.PodGroupList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested podGroups. +func (c *FakePodGroups) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(podgroupsResource, c.ns, opts)) + +} + +// Create takes the representation of a podGroup and creates it. Returns the server's representation of the podGroup, and an error, if there is any. +func (c *FakePodGroups) Create(podGroup *v1alpha1.PodGroup) (result *v1alpha1.PodGroup, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(podgroupsResource, c.ns, podGroup), &v1alpha1.PodGroup{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PodGroup), err +} + +// Update takes the representation of a podGroup and updates it. Returns the server's representation of the podGroup, and an error, if there is any. +func (c *FakePodGroups) Update(podGroup *v1alpha1.PodGroup) (result *v1alpha1.PodGroup, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(podgroupsResource, c.ns, podGroup), &v1alpha1.PodGroup{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PodGroup), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakePodGroups) UpdateStatus(podGroup *v1alpha1.PodGroup) (*v1alpha1.PodGroup, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(podgroupsResource, "status", c.ns, podGroup), &v1alpha1.PodGroup{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PodGroup), err +} + +// Delete takes name of the podGroup and deletes it. Returns an error if one occurs. +func (c *FakePodGroups) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(podgroupsResource, c.ns, name), &v1alpha1.PodGroup{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakePodGroups) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(podgroupsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1alpha1.PodGroupList{}) + return err +} + +// Patch applies the patch and returns the patched podGroup. +func (c *FakePodGroups) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1alpha1.PodGroup, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(podgroupsResource, c.ns, name, pt, data, subresources...), &v1alpha1.PodGroup{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PodGroup), err +} diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/fake_queue.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/fake_queue.go new file mode 100644 index 0000000000..0f5932152e --- /dev/null +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/fake_queue.go @@ -0,0 +1,131 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeQueues implements QueueInterface +type FakeQueues struct { + Fake *FakeSchedulingV1alpha1 +} + +var queuesResource = schema.GroupVersionResource{Group: "scheduling", Version: "v1alpha1", Resource: "queues"} + +var queuesKind = schema.GroupVersionKind{Group: "scheduling", Version: "v1alpha1", Kind: "Queue"} + +// Get takes name of the queue, and returns the corresponding queue object, and an error if there is any. +func (c *FakeQueues) Get(name string, options v1.GetOptions) (result *v1alpha1.Queue, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(queuesResource, name), &v1alpha1.Queue{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.Queue), err +} + +// List takes label and field selectors, and returns the list of Queues that match those selectors. +func (c *FakeQueues) List(opts v1.ListOptions) (result *v1alpha1.QueueList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(queuesResource, queuesKind, opts), &v1alpha1.QueueList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1alpha1.QueueList{ListMeta: obj.(*v1alpha1.QueueList).ListMeta} + for _, item := range obj.(*v1alpha1.QueueList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested queues. +func (c *FakeQueues) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(queuesResource, opts)) +} + +// Create takes the representation of a queue and creates it. Returns the server's representation of the queue, and an error, if there is any. +func (c *FakeQueues) Create(queue *v1alpha1.Queue) (result *v1alpha1.Queue, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(queuesResource, queue), &v1alpha1.Queue{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.Queue), err +} + +// Update takes the representation of a queue and updates it. Returns the server's representation of the queue, and an error, if there is any. +func (c *FakeQueues) Update(queue *v1alpha1.Queue) (result *v1alpha1.Queue, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(queuesResource, queue), &v1alpha1.Queue{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.Queue), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeQueues) UpdateStatus(queue *v1alpha1.Queue) (*v1alpha1.Queue, error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateSubresourceAction(queuesResource, "status", queue), &v1alpha1.Queue{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.Queue), err +} + +// Delete takes name of the queue and deletes it. Returns an error if one occurs. +func (c *FakeQueues) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(queuesResource, name), &v1alpha1.Queue{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeQueues) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(queuesResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1alpha1.QueueList{}) + return err +} + +// Patch applies the patch and returns the patched queue. +func (c *FakeQueues) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1alpha1.Queue, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(queuesResource, name, pt, data, subresources...), &v1alpha1.Queue{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.Queue), err +} diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/fake_scheduling_client.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/fake_scheduling_client.go new file mode 100644 index 0000000000..f8c9e918ae --- /dev/null +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake/fake_scheduling_client.go @@ -0,0 +1,44 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/typed/scheduling/v1alpha1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeSchedulingV1alpha1 struct { + *testing.Fake +} + +func (c *FakeSchedulingV1alpha1) PodGroups(namespace string) v1alpha1.PodGroupInterface { + return &FakePodGroups{c, namespace} +} + +func (c *FakeSchedulingV1alpha1) Queues() v1alpha1.QueueInterface { + return &FakeQueues{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeSchedulingV1alpha1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} From 9aa1d9a3e1a2f7f5dae8fb287993d9faa811c155 Mon Sep 17 00:00:00 2001 From: thandayuthapani Date: Thu, 30 May 2019 18:30:06 +0530 Subject: [PATCH 49/68] Check Queue exist in admission controller --- Gopkg.lock | 6 +++++- cmd/admission/app/server.go | 9 ++++++++ cmd/admission/main.go | 2 ++ installer/chart/templates/admission.yaml | 3 +++ pkg/admission/admit_job.go | 9 ++++++++ pkg/admission/admit_job_test.go | 26 ++++++++++++++++++++++++ 6 files changed, 54 insertions(+), 1 deletion(-) diff --git a/Gopkg.lock b/Gopkg.lock index 60dff4092a..0fa26df6f3 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -204,7 +204,7 @@ [[projects]] branch = "master" - digest = "1:986c933d3d3c05ab20d42462475be937ee26238500e58f5b53b5ae77b60f49ad" + digest = "1:407852f4bc58f1ffb25131dbdce512b3c5a2dd0d880567d3189c60a3eb069f45" name = "github.com/kubernetes-sigs/kube-batch" packages = [ "cmd/kube-batch/app", @@ -212,8 +212,10 @@ "pkg/apis/scheduling/v1alpha1", "pkg/apis/utils", "pkg/client/clientset/versioned", + "pkg/client/clientset/versioned/fake", "pkg/client/clientset/versioned/scheme", "pkg/client/clientset/versioned/typed/scheduling/v1alpha1", + "pkg/client/clientset/versioned/typed/scheduling/v1alpha1/fake", "pkg/client/informers/externalversions", "pkg/client/informers/externalversions/internalinterfaces", "pkg/client/informers/externalversions/scheduling", @@ -1047,6 +1049,7 @@ "github.com/kubernetes-sigs/kube-batch/cmd/kube-batch/app/options", "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1", "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned", + "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake", "github.com/kubernetes-sigs/kube-batch/pkg/client/informers/externalversions", "github.com/kubernetes-sigs/kube-batch/pkg/client/informers/externalversions/scheduling/v1alpha1", "github.com/kubernetes-sigs/kube-batch/pkg/client/listers/scheduling/v1alpha1", @@ -1112,6 +1115,7 @@ "k8s.io/kubernetes/pkg/apis/core", "k8s.io/kubernetes/pkg/apis/core/v1", "k8s.io/kubernetes/pkg/apis/core/validation", + "k8s.io/kubernetes/pkg/controller", "k8s.io/kubernetes/pkg/scheduler/api", ] solver-name = "gps-cdcl" diff --git a/cmd/admission/app/server.go b/cmd/admission/app/server.go index 05e79f7b73..d6fbaeb76e 100644 --- a/cmd/admission/app/server.go +++ b/cmd/admission/app/server.go @@ -22,6 +22,7 @@ import ( "net/http" "github.com/golang/glog" + "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned" "k8s.io/api/admission/v1beta1" "k8s.io/apimachinery/pkg/runtime" @@ -46,6 +47,14 @@ func GetClient(restConfig *restclient.Config) *kubernetes.Clientset { return clientset } +func GetKubeBatchClient(restConfig *restclient.Config) *versioned.Clientset { + clientset, err := versioned.NewForConfig(restConfig) + if err != nil { + glog.Fatal(err) + } + return clientset +} + // ConfigTLS is a helper function that generate tls certificates from directly defined tls config or kubeconfig // These are passed in as command line for cluster certification. If tls config is passed in, we use the directly // defined tls config, else use that defined in kubeconfig diff --git a/cmd/admission/main.go b/cmd/admission/main.go index e60a6ef50f..82ab5dcd68 100644 --- a/cmd/admission/main.go +++ b/cmd/admission/main.go @@ -65,6 +65,8 @@ func main() { clientset := app.GetClient(restConfig) + admissioncontroller.KubeBatchClientSet = app.GetKubeBatchClient(restConfig) + caCertPem, err := ioutil.ReadFile(config.CaCertFile) if err != nil { fmt.Fprintf(os.Stderr, "%v\n", err) diff --git a/installer/chart/templates/admission.yaml b/installer/chart/templates/admission.yaml index 9b57314d0e..436b6b34d1 100644 --- a/installer/chart/templates/admission.yaml +++ b/installer/chart/templates/admission.yaml @@ -29,6 +29,9 @@ rules: - apiGroups: [""] resources: ["secrets"] verbs: ["create", "get", "patch"] + - apiGroups: ["scheduling.incubator.k8s.io"] + resources: ["queues"] + verbs: ["get", "list"] --- kind: ClusterRoleBinding diff --git a/pkg/admission/admit_job.go b/pkg/admission/admit_job.go index 586451d3e8..819cebfa95 100644 --- a/pkg/admission/admit_job.go +++ b/pkg/admission/admit_job.go @@ -21,6 +21,7 @@ import ( "strings" "github.com/golang/glog" + "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned" "k8s.io/api/admission/v1beta1" "k8s.io/api/core/v1" @@ -35,6 +36,9 @@ import ( "volcano.sh/volcano/pkg/controllers/job/plugins" ) +//KubeBatchClientSet is kube-batch clientset +var KubeBatchClientSet versioned.Interface + // job admit. func AdmitJobs(ar v1beta1.AdmissionReview) *v1beta1.AdmissionResponse { @@ -146,6 +150,11 @@ func validateJob(job v1alpha1.Job, reviewResponse *v1beta1.AdmissionResponse) st msg = msg + validateInfo } + // Check whether Queue already present or not + if _, err := KubeBatchClientSet.SchedulingV1alpha1().Queues().Get(job.Spec.Queue, metav1.GetOptions{}); err != nil { + msg = msg + fmt.Sprintf("Job not created with error: %v", err) + } + if msg != "" { reviewResponse.Allowed = false } diff --git a/pkg/admission/admit_job_test.go b/pkg/admission/admit_job_test.go index b62b1a94a6..43f509c4bb 100644 --- a/pkg/admission/admit_job_test.go +++ b/pkg/admission/admit_job_test.go @@ -20,10 +20,13 @@ import ( "strings" "testing" + kubebatchclient "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake" + "k8s.io/api/admission/v1beta1" v1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + kbv1aplha1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1" v1alpha1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" ) @@ -48,6 +51,7 @@ func TestValidateExecution(t *testing.T) { }, Spec: v1alpha1.JobSpec{ MinAvailable: 1, + Queue: "default", Tasks: []v1alpha1.TaskSpec{ { Name: "task-1", @@ -83,6 +87,7 @@ func TestValidateExecution(t *testing.T) { }, Spec: v1alpha1.JobSpec{ MinAvailable: 1, + Queue: "default", Tasks: []v1alpha1.TaskSpec{ { Name: "duplicated-task-1", @@ -135,6 +140,7 @@ func TestValidateExecution(t *testing.T) { }, Spec: v1alpha1.JobSpec{ MinAvailable: 1, + Queue: "default", Tasks: []v1alpha1.TaskSpec{ { Name: "task-1", @@ -180,6 +186,7 @@ func TestValidateExecution(t *testing.T) { }, Spec: v1alpha1.JobSpec{ MinAvailable: 2, + Queue: "default", Tasks: []v1alpha1.TaskSpec{ { Name: "task-1", @@ -215,6 +222,7 @@ func TestValidateExecution(t *testing.T) { }, Spec: v1alpha1.JobSpec{ MinAvailable: 1, + Queue: "default", Tasks: []v1alpha1.TaskSpec{ { Name: "task-1", @@ -253,6 +261,7 @@ func TestValidateExecution(t *testing.T) { }, Spec: v1alpha1.JobSpec{ MinAvailable: 1, + Queue: "default", Tasks: []v1alpha1.TaskSpec{ { Name: "task-1", @@ -283,6 +292,23 @@ func TestValidateExecution(t *testing.T) { for _, testCase := range testCases { + defaultqueue := kbv1aplha1.Queue{ + ObjectMeta: metav1.ObjectMeta{ + Name: "default", + }, + Spec: kbv1aplha1.QueueSpec{ + Weight: 1, + }, + } + // create fake kube-batch clientset + KubeBatchClientSet = kubebatchclient.NewSimpleClientset() + + //create default queue + _, err := KubeBatchClientSet.SchedulingV1alpha1().Queues().Create(&defaultqueue) + if err != nil { + t.Error("Queue Creation Failed") + } + ret := validateJob(testCase.Job, &testCase.reviewResponse) //fmt.Printf("test-case name:%s, ret:%v testCase.reviewResponse:%v \n", testCase.Name, ret,testCase.reviewResponse) if testCase.ExpectErr == true && ret == "" { From 248ac6fdaa2039606e1c8d9b92491a44b09dc767 Mon Sep 17 00:00:00 2001 From: Klaus Ma Date: Thu, 30 May 2019 21:51:06 +0800 Subject: [PATCH 50/68] Update README.md --- README.md | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 810a6373e9..4c997649d5 100644 --- a/README.md +++ b/README.md @@ -9,10 +9,8 @@ [![LICENSE](https://img.shields.io/github/license/volcano-sh/volcano.svg)](https://github.com/volcano-sh/volcano/blob/master/LICENSE) -Volcano is system for runnning high performance workloads on -Kubernetes. It provides a suite of mechanisms currently missing from -Kubernetes that are commonly required by many classes of high -performance workload including: +Volcano is a batch system built on Kubernetes. It provides a suite of mechanisms currently missing from +Kubernetes that are commonly required by many classes of batch & elastic workload including: 1. machine learning/deep learning, 2. bioinformatics/genomics, and From 9e4356478d396bab136f738a2b8f3ad1a9c0ad89 Mon Sep 17 00:00:00 2001 From: thandayuthapani Date: Thu, 30 May 2019 19:24:23 +0530 Subject: [PATCH 51/68] Remove reclaim and preempt by default --- hack/run-e2e-kind.sh | 2 +- installer/chart/config/kube-batch-ci.conf | 11 +++++++++++ installer/chart/config/kube-batch.conf | 2 +- installer/chart/templates/scheduler.yaml | 2 +- installer/chart/values.yaml | 1 + 5 files changed, 15 insertions(+), 3 deletions(-) create mode 100644 installer/chart/config/kube-batch-ci.conf diff --git a/hack/run-e2e-kind.sh b/hack/run-e2e-kind.sh index fe617a262f..cbcd9e198f 100755 --- a/hack/run-e2e-kind.sh +++ b/hack/run-e2e-kind.sh @@ -63,7 +63,7 @@ function install-volcano { kind load docker-image ${MPI_EXAMPLE_IMAGE} ${CLUSTER_CONTEXT} echo "Install volcano chart" - helm install installer/chart --namespace kube-system --name ${CLUSTER_NAME} --kubeconfig ${KUBECONFIG} --set basic.image_tag_version=${TAG} --wait + helm install installer/chart --namespace kube-system --name ${CLUSTER_NAME} --kubeconfig ${KUBECONFIG} --set basic.image_tag_version=${TAG} --set basic.scheduler_config_file=kube-batch-ci.conf --wait } function uninstall-volcano { diff --git a/installer/chart/config/kube-batch-ci.conf b/installer/chart/config/kube-batch-ci.conf new file mode 100644 index 0000000000..5a6c324b30 --- /dev/null +++ b/installer/chart/config/kube-batch-ci.conf @@ -0,0 +1,11 @@ +actions: "enqueue, reclaim, allocate, backfill, preempt" +tiers: +- plugins: + - name: priority + - name: gang + - name: conformance +- plugins: + - name: drf + - name: predicates + - name: proportion + - name: nodeorder diff --git a/installer/chart/config/kube-batch.conf b/installer/chart/config/kube-batch.conf index 5a6c324b30..14fa3072a2 100644 --- a/installer/chart/config/kube-batch.conf +++ b/installer/chart/config/kube-batch.conf @@ -1,4 +1,4 @@ -actions: "enqueue, reclaim, allocate, backfill, preempt" +actions: "enqueue, allocate, backfill" tiers: - plugins: - name: priority diff --git a/installer/chart/templates/scheduler.yaml b/installer/chart/templates/scheduler.yaml index bb66a79576..6d01220de4 100644 --- a/installer/chart/templates/scheduler.yaml +++ b/installer/chart/templates/scheduler.yaml @@ -102,7 +102,7 @@ spec: image: {{.Values.basic.scheduler_image_name}}:{{.Values.basic.image_tag_version}} args: - --alsologtostderr - - --scheduler-conf=/volcano.scheduler/kube-batch.conf + - --scheduler-conf=/volcano.scheduler/{{.Values.basic.scheduler_config_file}} - -v=3 - 2>&1 imagePullPolicy: "IfNotPresent" diff --git a/installer/chart/values.yaml b/installer/chart/values.yaml index 41137671eb..813da3ba24 100644 --- a/installer/chart/values.yaml +++ b/installer/chart/values.yaml @@ -4,4 +4,5 @@ basic: scheduler_image_name: "volcanosh/vk-kube-batch" admission_image_name: "volcanosh/vk-admission" admission_secret_name: "volcano-admission-secret" + scheduler_config_file: "kube-batch.conf" image_pull_secret: "" From 332ff82d0dcee5f9edb8331d31df426c1062fa94 Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Tue, 21 May 2019 14:45:01 +0530 Subject: [PATCH 52/68] Added UT for options in controller --- cmd/controllers/app/options/options_test.go | 53 +++++++++++++++++++++ 1 file changed, 53 insertions(+) create mode 100644 cmd/controllers/app/options/options_test.go diff --git a/cmd/controllers/app/options/options_test.go b/cmd/controllers/app/options/options_test.go new file mode 100644 index 0000000000..df6d5b7f22 --- /dev/null +++ b/cmd/controllers/app/options/options_test.go @@ -0,0 +1,53 @@ +/* +Copyright 2019 The Volcano Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package options + +import ( + "github.com/spf13/pflag" + "reflect" + "testing" +) + +func TestAddFlags(t *testing.T) { + fs := pflag.NewFlagSet("addflagstest", pflag.ContinueOnError) + s := NewServerOption() + s.AddFlags(fs) + + args := []string{ + "--master=127.0.0.1", + "--kube-api-burst=200", + } + fs.Parse(args) + + // This is a snapshot of expected options parsed by args. + expected := &ServerOption{ + Master: "127.0.0.1", + KubeAPIQPS: defaultQPS, + KubeAPIBurst: 200, + PrintVersion: false, + } + + if !reflect.DeepEqual(expected, s) { + t.Errorf("Got different run options than expected.\nGot: %+v\nExpected: %+v\n", s, expected) + } + + err := s.CheckOptionOrDie() + if err != nil { + t.Errorf("expected nil but got %v\n", err) + } + +} From 0891859ac2af4af0abce079ca221870d2605d1cb Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Fri, 31 May 2019 12:35:25 +0530 Subject: [PATCH 53/68] Added svc pluin name in ControlledResources of job status --- pkg/controllers/job/plugins/svc/svc.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pkg/controllers/job/plugins/svc/svc.go b/pkg/controllers/job/plugins/svc/svc.go index 88881d0440..2bbcbcebb0 100644 --- a/pkg/controllers/job/plugins/svc/svc.go +++ b/pkg/controllers/job/plugins/svc/svc.go @@ -79,6 +79,8 @@ func (sp *servicePlugin) OnJobAdd(job *vkv1.Job) error { return err } + job.Status.ControlledResources["plugin-"+sp.Name()] = sp.Name() + return nil } From e18910ae07b939d3e7fe9547e1c8d4ac0d3f8b0f Mon Sep 17 00:00:00 2001 From: "Da K. Ma" Date: Fri, 31 May 2019 21:05:10 +0800 Subject: [PATCH 54/68] Fix job controller panic --- pkg/controllers/cache/cache.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/controllers/cache/cache.go b/pkg/controllers/cache/cache.go index 4434e701b5..de3747dd48 100644 --- a/pkg/controllers/cache/cache.go +++ b/pkg/controllers/cache/cache.go @@ -229,7 +229,7 @@ func (jc *jobCache) Run(stopCh <-chan struct{}) { wait.Until(jc.worker, 0, stopCh) } -func (jc jobCache) TaskCompleted(jobKey, taskName string) bool { +func (jc *jobCache) TaskCompleted(jobKey, taskName string) bool { jc.Lock() defer jc.Unlock() From b21573e31f386b4f5de66c928ec0333cd17b0e46 Mon Sep 17 00:00:00 2001 From: thandayuthapani Date: Tue, 4 Jun 2019 12:50:46 +0530 Subject: [PATCH 55/68] Bump volcano-sh/kube-batch to volcano-sh/scheduler --- Gopkg.lock | 6 +- Gopkg.toml | 2 +- .../scheduler/actions/allocate/allocate.go | 2 +- .../pkg/scheduler/actions/enqueue/enqueue.go | 14 +- .../pkg/scheduler/actions/preempt/preempt.go | 2 +- .../kube-batch/pkg/scheduler/api/node_info.go | 6 +- .../pkg/scheduler/api/resource_info.go | 51 +++++-- .../kube-batch/pkg/scheduler/api/types.go | 16 +++ .../kube-batch/pkg/scheduler/cache/cache.go | 9 +- .../kube-batch/pkg/scheduler/cache/util.go | 5 + .../pkg/scheduler/framework/arguments.go | 20 +++ .../pkg/scheduler/framework/session.go | 58 ++++---- .../scheduler/framework/session_plugins.go | 86 ++++++++++++ .../plugins/predicates/predicates.go | 124 +++++++++++++----- .../plugins/proportion/proportion.go | 39 +++++- .../pkg/scheduler/util/scheduler_helper.go | 41 +++++- 16 files changed, 382 insertions(+), 99 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 0fa26df6f3..35bdd31b03 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -204,7 +204,7 @@ [[projects]] branch = "master" - digest = "1:407852f4bc58f1ffb25131dbdce512b3c5a2dd0d880567d3189c60a3eb069f45" + digest = "1:4281fb8a5ed4671cd7e80c85599e00912f796819716abaefd697797ec9eb4491" name = "github.com/kubernetes-sigs/kube-batch" packages = [ "cmd/kube-batch/app", @@ -247,8 +247,8 @@ "pkg/version", ] pruneopts = "UT" - revision = "2a63a4754d8289bf6491b7320ab6d832c75767ce" - source = "https://github.com/volcano-sh/kube-batch" + revision = "700381278a3e7da396507055d371dd93c1724322" + source = "https://github.com/volcano-sh/scheduler" [[projects]] digest = "1:ff5ebae34cfbf047d505ee150de27e60570e8c394b3b8fdbb720ff6ac71985fc" diff --git a/Gopkg.toml b/Gopkg.toml index 0803cc797b..d86986ed9b 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -39,7 +39,7 @@ required = [ [[constraint]] name = "github.com/kubernetes-sigs/kube-batch" branch = "master" - source = "https://github.com/volcano-sh/kube-batch" + source = "https://github.com/volcano-sh/scheduler" [[constraint]] name = "github.com/onsi/ginkgo" diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/actions/allocate/allocate.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/actions/allocate/allocate.go index 973db99cc5..18458d7cad 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/actions/allocate/allocate.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/actions/allocate/allocate.go @@ -150,7 +150,7 @@ func (alloc *allocateAction) Execute(ssn *framework.Session) { break } - nodeScores := util.PrioritizeNodes(task, predicateNodes, ssn.NodeOrderFn) + nodeScores := util.PrioritizeNodes(task, predicateNodes, ssn.NodeOrderMapFn, ssn.NodeOrderReduceFn) node := util.SelectBestNode(nodeScores) // Allocate idle resource to the task. diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/actions/enqueue/enqueue.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/actions/enqueue/enqueue.go index 70ac372914..7e1927347e 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/actions/enqueue/enqueue.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/actions/enqueue/enqueue.go @@ -100,18 +100,14 @@ func (enqueue *enqueueAction) Execute(ssn *framework.Session) { job := jobs.Pop().(*api.JobInfo) inqueue := false - if len(job.TaskStatusIndex[api.Pending]) != 0 { + + if job.PodGroup.Spec.MinResources == nil { inqueue = true } else { - if job.PodGroup.Spec.MinResources == nil { + pgResource := api.NewResource(*job.PodGroup.Spec.MinResources) + if ssn.JobEnqueueable(job) && pgResource.LessEqual(nodesIdleRes) { + nodesIdleRes.Sub(pgResource) inqueue = true - } else { - pgResource := api.NewResource(*job.PodGroup.Spec.MinResources) - - if pgResource.LessEqual(nodesIdleRes) { - nodesIdleRes.Sub(pgResource) - inqueue = true - } } } diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/actions/preempt/preempt.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/actions/preempt/preempt.go index 435696f228..4ca1b5f508 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/actions/preempt/preempt.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/actions/preempt/preempt.go @@ -186,7 +186,7 @@ func preempt( predicateNodes := util.PredicateNodes(preemptor, allNodes, ssn.PredicateFn) - nodeScores := util.PrioritizeNodes(preemptor, predicateNodes, ssn.NodeOrderFn) + nodeScores := util.PrioritizeNodes(preemptor, predicateNodes, ssn.NodeOrderMapFn, ssn.NodeOrderReduceFn) selectedNodes := util.SortNodes(nodeScores) for _, node := range selectedNodes { diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/node_info.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/node_info.go index 0e94a51f8f..16257d4140 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/node_info.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/node_info.go @@ -39,6 +39,9 @@ type NodeInfo struct { Capability *Resource Tasks map[TaskID]*TaskInfo + + // Used to store custom information + Other interface{} } // NewNodeInfo is used to create new nodeInfo object @@ -78,7 +81,7 @@ func (ni *NodeInfo) Clone() *NodeInfo { for _, p := range ni.Tasks { res.AddTask(p) } - + res.Other = ni.Other return res } @@ -90,6 +93,7 @@ func (ni *NodeInfo) SetNode(node *v1.Node) { ni.Allocatable = NewResource(node.Status.Allocatable) ni.Capability = NewResource(node.Status.Capacity) ni.Idle = NewResource(node.Status.Allocatable) + ni.Used = EmptyResource() for _, task := range ni.Tasks { if task.Status == Releasing { diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/resource_info.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/resource_info.go index 582d955576..b575e49174 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/resource_info.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/resource_info.go @@ -20,8 +20,6 @@ import ( "fmt" "math" - "k8s.io/apimachinery/pkg/api/resource" - v1 "k8s.io/api/core/v1" v1helper "k8s.io/kubernetes/pkg/apis/core/v1/helper" ) @@ -83,6 +81,7 @@ func NewResource(rl v1.ResourceList) *Resource { case v1.ResourcePods: r.MaxTaskNum += int(rQuant.Value()) default: + //NOTE: When converting this back to k8s resource, we need record the format as well as / 1000 if v1helper.IsScalarResourceName(rName) { r.AddScalar(rName, float64(rQuant.MilliValue())) } @@ -157,7 +156,7 @@ func (r *Resource) Sub(rr *Resource) *Resource { return r } - panic(fmt.Errorf("Resource is not sufficient to do operation: <%v> sub <%v>", + panic(fmt.Errorf("resource is not sufficient to do operation: <%v> sub <%v>", r, rr)) } @@ -278,6 +277,41 @@ func (r *Resource) LessEqual(rr *Resource) bool { return true } +// Diff calculate the difference between two resource +func (r *Resource) Diff(rr *Resource) (*Resource, *Resource) { + increasedVal := EmptyResource() + decreasedVal := EmptyResource() + if r.MilliCPU > rr.MilliCPU { + increasedVal.MilliCPU += r.MilliCPU - rr.MilliCPU + } else { + decreasedVal.MilliCPU += rr.MilliCPU - r.MilliCPU + } + + if r.Memory > rr.Memory { + increasedVal.Memory += r.Memory - rr.Memory + } else { + decreasedVal.Memory += rr.Memory - r.Memory + } + + for rName, rQuant := range r.ScalarResources { + rrQuant := rr.ScalarResources[rName] + + if rQuant > rrQuant { + if increasedVal.ScalarResources == nil { + increasedVal.ScalarResources = map[v1.ResourceName]float64{} + } + increasedVal.ScalarResources[rName] += rQuant - rrQuant + } else { + if decreasedVal.ScalarResources == nil { + decreasedVal.ScalarResources = map[v1.ResourceName]float64{} + } + decreasedVal.ScalarResources[rName] += rrQuant - rQuant + } + } + + return increasedVal, decreasedVal +} + // String returns resource details in string format func (r *Resource) String() string { str := fmt.Sprintf("cpu %0.2f, memory %0.2f", r.MilliCPU, r.Memory) @@ -326,14 +360,3 @@ func (r *Resource) SetScalar(name v1.ResourceName, quantity float64) { } r.ScalarResources[name] = quantity } - -func (r *Resource) Convert2K8sResource() *v1.ResourceList { - list := v1.ResourceList{ - v1.ResourceCPU: *resource.NewMilliQuantity(int64(r.MilliCPU), resource.DecimalSI), - v1.ResourceMemory: *resource.NewQuantity(int64(r.Memory), resource.BinarySI), - } - for name, value := range r.ScalarResources { - list[name] = *resource.NewQuantity(int64(value), resource.BinarySI) - } - return &list -} diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/types.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/types.go index 4d9b67f586..97a583df6c 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/types.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/types.go @@ -16,6 +16,10 @@ limitations under the License. package api +import ( + schedulerapi "k8s.io/kubernetes/pkg/scheduler/api" +) + // TaskStatus defines the status of a task/pod. type TaskStatus int @@ -106,3 +110,15 @@ type EvictableFn func(*TaskInfo, []*TaskInfo) []*TaskInfo // NodeOrderFn is the func declaration used to get priority score for a node for a particular task. type NodeOrderFn func(*TaskInfo, *NodeInfo) (float64, error) + +// NodeMapFn is the func declaration used to get priority score for a node for a particular task. +type NodeMapFn func(*TaskInfo, *NodeInfo) (float64, error) + +// NodeReduceFn is the func declaration used to reduce priority score for a node for a particular task. +type NodeReduceFn func(*TaskInfo, schedulerapi.HostPriorityList) error + +// NodeOrderMapFn is the func declaration used to get priority score of all plugins for a node for a particular task. +type NodeOrderMapFn func(*TaskInfo, *NodeInfo) (map[string]float64, float64, error) + +// NodeOrderReduceFn is the func declaration used to reduce priority score of all nodes for a plugiin for a particular task. +type NodeOrderReduceFn func(*TaskInfo, map[string]schedulerapi.HostPriorityList) (map[string]float64, error) diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/cache.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/cache.go index 13041c4c64..f6f1c04d2e 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/cache.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/cache.go @@ -18,7 +18,6 @@ package cache import ( "fmt" - "strings" "sync" "time" @@ -250,10 +249,12 @@ func newSchedulerCache(config *rest.Config, schedulerName string, defaultQueue s switch obj.(type) { case *v1.Pod: pod := obj.(*v1.Pod) - if strings.Compare(pod.Spec.SchedulerName, schedulerName) == 0 && pod.Status.Phase == v1.PodPending { - return true + if !responsibleForPod(pod, schedulerName) { + if len(pod.Spec.NodeName) == 0 { + return false + } } - return pod.Status.Phase != v1.PodPending + return true default: return false } diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/util.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/util.go index 7fdad4f2f2..665b38f168 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/util.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/util.go @@ -58,3 +58,8 @@ func createShadowPodGroup(pod *v1.Pod) *v1alpha1.PodGroup { }, } } + +// responsibleForPod returns true if the pod has asked to be scheduled by the given scheduler. +func responsibleForPod(pod *v1.Pod, schedulerName string) bool { + return schedulerName == pod.Spec.SchedulerName +} diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework/arguments.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework/arguments.go index d968e7ae32..5d803b67e7 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework/arguments.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework/arguments.go @@ -44,3 +44,23 @@ func (a Arguments) GetInt(ptr *int, key string) { *ptr = value } + +//GetBool get the bool value from string +func (a Arguments) GetBool(ptr *bool, key string) { + if ptr == nil { + return + } + + argv, ok := a[key] + if !ok || argv == "" { + return + } + + value, err := strconv.ParseBool(argv) + if err != nil { + glog.Warningf("Could not parse argument: %s for key %s, with err %v", argv, key, err) + return + } + + *ptr = value +} diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework/session.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework/session.go index e6ac75ddea..27d4ca2a61 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework/session.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework/session.go @@ -45,19 +45,22 @@ type Session struct { Backlog []*api.JobInfo Tiers []conf.Tier - plugins map[string]Plugin - eventHandlers []*EventHandler - jobOrderFns map[string]api.CompareFn - queueOrderFns map[string]api.CompareFn - taskOrderFns map[string]api.CompareFn - predicateFns map[string]api.PredicateFn - nodeOrderFns map[string]api.NodeOrderFn - preemptableFns map[string]api.EvictableFn - reclaimableFns map[string]api.EvictableFn - overusedFns map[string]api.ValidateFn - jobReadyFns map[string]api.ValidateFn - jobPipelinedFns map[string]api.ValidateFn - jobValidFns map[string]api.ValidateExFn + plugins map[string]Plugin + eventHandlers []*EventHandler + jobOrderFns map[string]api.CompareFn + queueOrderFns map[string]api.CompareFn + taskOrderFns map[string]api.CompareFn + predicateFns map[string]api.PredicateFn + nodeOrderFns map[string]api.NodeOrderFn + nodeMapFns map[string]api.NodeMapFn + nodeReduceFns map[string]api.NodeReduceFn + preemptableFns map[string]api.EvictableFn + reclaimableFns map[string]api.EvictableFn + overusedFns map[string]api.ValidateFn + jobReadyFns map[string]api.ValidateFn + jobPipelinedFns map[string]api.ValidateFn + jobValidFns map[string]api.ValidateExFn + jobEnqueueableFns map[string]api.ValidateFn } func openSession(cache cache.Cache) *Session { @@ -69,18 +72,21 @@ func openSession(cache cache.Cache) *Session { Nodes: map[string]*api.NodeInfo{}, Queues: map[api.QueueID]*api.QueueInfo{}, - plugins: map[string]Plugin{}, - jobOrderFns: map[string]api.CompareFn{}, - queueOrderFns: map[string]api.CompareFn{}, - taskOrderFns: map[string]api.CompareFn{}, - predicateFns: map[string]api.PredicateFn{}, - nodeOrderFns: map[string]api.NodeOrderFn{}, - preemptableFns: map[string]api.EvictableFn{}, - reclaimableFns: map[string]api.EvictableFn{}, - overusedFns: map[string]api.ValidateFn{}, - jobReadyFns: map[string]api.ValidateFn{}, - jobPipelinedFns: map[string]api.ValidateFn{}, - jobValidFns: map[string]api.ValidateExFn{}, + plugins: map[string]Plugin{}, + jobOrderFns: map[string]api.CompareFn{}, + queueOrderFns: map[string]api.CompareFn{}, + taskOrderFns: map[string]api.CompareFn{}, + predicateFns: map[string]api.PredicateFn{}, + nodeOrderFns: map[string]api.NodeOrderFn{}, + nodeMapFns: map[string]api.NodeMapFn{}, + nodeReduceFns: map[string]api.NodeReduceFn{}, + preemptableFns: map[string]api.EvictableFn{}, + reclaimableFns: map[string]api.EvictableFn{}, + overusedFns: map[string]api.ValidateFn{}, + jobReadyFns: map[string]api.ValidateFn{}, + jobPipelinedFns: map[string]api.ValidateFn{}, + jobValidFns: map[string]api.ValidateExFn{}, + jobEnqueueableFns: map[string]api.ValidateFn{}, } snapshot := cache.Snapshot() @@ -169,7 +175,7 @@ func jobStatus(ssn *Session, jobInfo *api.JobInfo) v1alpha1.PodGroupStatus { } // If there're enough allocated resource, it's running - if int32(allocated) > jobInfo.PodGroup.Spec.MinMember { + if int32(allocated) >= jobInfo.PodGroup.Spec.MinMember { status.Phase = v1alpha1.PodGroupRunning } else if jobInfo.PodGroup.Status.Phase != v1alpha1.PodGroupInqueue { status.Phase = v1alpha1.PodGroupPending diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework/session_plugins.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework/session_plugins.go index 230844017d..b7421b3d0c 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework/session_plugins.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework/session_plugins.go @@ -18,6 +18,7 @@ package framework import ( "github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api" + schedulerapi "k8s.io/kubernetes/pkg/scheduler/api" ) // AddJobOrderFn add job order function @@ -65,6 +66,16 @@ func (ssn *Session) AddNodeOrderFn(name string, pf api.NodeOrderFn) { ssn.nodeOrderFns[name] = pf } +// AddNodeMapFn add Node map function +func (ssn *Session) AddNodeMapFn(name string, pf api.NodeMapFn) { + ssn.nodeMapFns[name] = pf +} + +// AddNodeReduceFn add Node reduce function +func (ssn *Session) AddNodeReduceFn(name string, pf api.NodeReduceFn) { + ssn.nodeReduceFns[name] = pf +} + // AddOverusedFn add overused function func (ssn *Session) AddOverusedFn(name string, fn api.ValidateFn) { ssn.overusedFns[name] = fn @@ -75,6 +86,11 @@ func (ssn *Session) AddJobValidFn(name string, fn api.ValidateExFn) { ssn.jobValidFns[name] = fn } +// AddJobEnqueueableFn add jobenqueueable function +func (ssn *Session) AddJobEnqueueableFn(name string, fn api.ValidateFn) { + ssn.jobEnqueueableFns[name] = fn +} + // Reclaimable invoke reclaimable function of the plugins func (ssn *Session) Reclaimable(reclaimer *api.TaskInfo, reclaimees []*api.TaskInfo) []*api.TaskInfo { var victims []*api.TaskInfo @@ -238,6 +254,24 @@ func (ssn *Session) JobValid(obj interface{}) *api.ValidateResult { return nil } +// JobEnqueueable invoke jobEnqueueableFns function of the plugins +func (ssn *Session) JobEnqueueable(obj interface{}) bool { + for _, tier := range ssn.Tiers { + for _, plugin := range tier.Plugins { + fn, found := ssn.jobEnqueueableFns[plugin.Name] + if !found { + continue + } + + if res := fn(obj); !res { + return res + } + } + } + + return true +} + // JobOrderFn invoke joborder function of the plugins func (ssn *Session) JobOrderFn(l, r interface{}) bool { for _, tier := range ssn.Tiers { @@ -375,3 +409,55 @@ func (ssn *Session) NodeOrderFn(task *api.TaskInfo, node *api.NodeInfo) (float64 func isEnabled(enabled *bool) bool { return enabled != nil && *enabled } + +// NodeOrderMapFn invoke node order function of the plugins +func (ssn *Session) NodeOrderMapFn(task *api.TaskInfo, node *api.NodeInfo) (map[string]float64, float64, error) { + nodeScoreMap := map[string]float64{} + var priorityScore float64 + for _, tier := range ssn.Tiers { + for _, plugin := range tier.Plugins { + if !isEnabled(plugin.EnabledNodeOrder) { + continue + } + if pfn, found := ssn.nodeOrderFns[plugin.Name]; found { + score, err := pfn(task, node) + if err != nil { + return nodeScoreMap, priorityScore, err + } + priorityScore = priorityScore + score + } + if pfn, found := ssn.nodeMapFns[plugin.Name]; found { + score, err := pfn(task, node) + if err != nil { + return nodeScoreMap, priorityScore, err + } + nodeScoreMap[plugin.Name] = score + } + + } + } + return nodeScoreMap, priorityScore, nil +} + +// NodeOrderReduceFn invoke node order function of the plugins +func (ssn *Session) NodeOrderReduceFn(task *api.TaskInfo, pluginNodeScoreMap map[string]schedulerapi.HostPriorityList) (map[string]float64, error) { + nodeScoreMap := map[string]float64{} + for _, tier := range ssn.Tiers { + for _, plugin := range tier.Plugins { + if !isEnabled(plugin.EnabledNodeOrder) { + continue + } + pfn, found := ssn.nodeReduceFns[plugin.Name] + if !found { + continue + } + if err := pfn(task, pluginNodeScoreMap[plugin.Name]); err != nil { + return nodeScoreMap, err + } + for _, hp := range pluginNodeScoreMap[plugin.Name] { + nodeScoreMap[hp.Host] = nodeScoreMap[hp.Host] + float64(hp.Score) + } + } + } + return nodeScoreMap, nil +} diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/plugins/predicates/predicates.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/plugins/predicates/predicates.go index 57d92f128b..fa307bc3a1 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/plugins/predicates/predicates.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/plugins/predicates/predicates.go @@ -31,6 +31,15 @@ import ( "github.com/kubernetes-sigs/kube-batch/pkg/scheduler/plugins/util" ) +const ( + // MemoryPressurePredicate is the key for enabling Memory Pressure Predicate in YAML + MemoryPressurePredicate = "predicate.MemoryPressureEnable" + // DiskPressurePredicate is the key for enabling Disk Pressure Predicate in YAML + DiskPressurePredicate = "predicate.DiskPressureEnable" + // PIDPressurePredicate is the key for enabling PID Pressure Predicate in YAML + PIDPressurePredicate = "predicate.PIDPressureEnable" +) + type predicatesPlugin struct { // Arguments given for the plugin pluginArguments framework.Arguments @@ -54,6 +63,53 @@ func formatReason(reasons []algorithm.PredicateFailureReason) string { return strings.Join(reasonStrings, ", ") } +type predicateEnable struct { + memoryPressureEnable bool + diskPressureEnable bool + pidPressureEnable bool +} + +func enablePredicate(args framework.Arguments) predicateEnable { + + /* + User Should give predicatesEnable in this format(predicate.MemoryPressureEnable, predicate.DiskPressureEnable, predicate.PIDPressureEnable. + Currently supported only for MemoryPressure, DiskPressure, PIDPressure predicate checks. + + actions: "reclaim, allocate, backfill, preempt" + tiers: + - plugins: + - name: priority + - name: gang + - name: conformance + - plugins: + - name: drf + - name: predicates + arguments: + predicate.MemoryPressureEnable: true + predicate.DiskPressureEnable: true + predicate.PIDPressureEnable: true + - name: proportion + - name: nodeorder + */ + + predicate := predicateEnable{ + memoryPressureEnable: false, + diskPressureEnable: false, + pidPressureEnable: false, + } + + // Checks whether predicate.MemoryPressureEnable is provided or not, if given, modifies the value in predicateEnable struct. + args.GetBool(&predicate.memoryPressureEnable, MemoryPressurePredicate) + + // Checks whether predicate.DiskPressureEnable is provided or not, if given, modifies the value in predicateEnable struct. + args.GetBool(&predicate.diskPressureEnable, DiskPressurePredicate) + + // Checks whether predicate.PIDPressureEnable is provided or not, if given, modifies the value in predicateEnable struct. + args.GetBool(&predicate.pidPressureEnable, PIDPressurePredicate) + + return predicate +} + func (pp *predicatesPlugin) OnSessionOpen(ssn *framework.Session) { pl := &util.PodLister{ Session: ssn, @@ -63,6 +119,8 @@ func (pp *predicatesPlugin) OnSessionOpen(ssn *framework.Session) { Session: ssn, } + predicate := enablePredicate(pp.pluginArguments) + ssn.AddPredicateFn(pp.Name(), func(task *api.TaskInfo, node *api.NodeInfo) error { nodeInfo := cache.NewNodeInfo(node.Pods()...) nodeInfo.SetNode(node.Node) @@ -141,46 +199,52 @@ func (pp *predicatesPlugin) OnSessionOpen(ssn *framework.Session) { task.Namespace, task.Name, node.Name) } - // CheckNodeMemoryPressurePredicate - fit, _, err = predicates.CheckNodeMemoryPressurePredicate(task.Pod, nil, nodeInfo) - if err != nil { - return err - } + if predicate.memoryPressureEnable { + // CheckNodeMemoryPressurePredicate + fit, _, err = predicates.CheckNodeMemoryPressurePredicate(task.Pod, nil, nodeInfo) + if err != nil { + return err + } - glog.V(4).Infof("CheckNodeMemoryPressure predicates Task <%s/%s> on Node <%s>: fit %t, err %v", - task.Namespace, task.Name, node.Name, fit, err) + glog.V(4).Infof("CheckNodeMemoryPressure predicates Task <%s/%s> on Node <%s>: fit %t, err %v", + task.Namespace, task.Name, node.Name, fit, err) - if !fit { - return fmt.Errorf("node <%s> are not available to schedule task <%s/%s> due to Memory Pressure", - node.Name, task.Namespace, task.Name) + if !fit { + return fmt.Errorf("node <%s> are not available to schedule task <%s/%s> due to Memory Pressure", + node.Name, task.Namespace, task.Name) + } } - // CheckNodeDiskPressurePredicate - fit, _, err = predicates.CheckNodeDiskPressurePredicate(task.Pod, nil, nodeInfo) - if err != nil { - return err - } + if predicate.diskPressureEnable { + // CheckNodeDiskPressurePredicate + fit, _, err = predicates.CheckNodeDiskPressurePredicate(task.Pod, nil, nodeInfo) + if err != nil { + return err + } - glog.V(4).Infof("CheckNodeDiskPressure predicates Task <%s/%s> on Node <%s>: fit %t, err %v", - task.Namespace, task.Name, node.Name, fit, err) + glog.V(4).Infof("CheckNodeDiskPressure predicates Task <%s/%s> on Node <%s>: fit %t, err %v", + task.Namespace, task.Name, node.Name, fit, err) - if !fit { - return fmt.Errorf("node <%s> are not available to schedule task <%s/%s> due to Disk Pressure", - node.Name, task.Namespace, task.Name) + if !fit { + return fmt.Errorf("node <%s> are not available to schedule task <%s/%s> due to Disk Pressure", + node.Name, task.Namespace, task.Name) + } } - // CheckNodePIDPressurePredicate - fit, _, err = predicates.CheckNodePIDPressurePredicate(task.Pod, nil, nodeInfo) - if err != nil { - return err - } + if predicate.pidPressureEnable { + // CheckNodePIDPressurePredicate + fit, _, err = predicates.CheckNodePIDPressurePredicate(task.Pod, nil, nodeInfo) + if err != nil { + return err + } - glog.V(4).Infof("CheckNodePIDPressurePredicate predicates Task <%s/%s> on Node <%s>: fit %t, err %v", - task.Namespace, task.Name, node.Name, fit, err) + glog.V(4).Infof("CheckNodePIDPressurePredicate predicates Task <%s/%s> on Node <%s>: fit %t, err %v", + task.Namespace, task.Name, node.Name, fit, err) - if !fit { - return fmt.Errorf("node <%s> are not available to schedule task <%s/%s> due to PID Pressure", - node.Name, task.Namespace, task.Name) + if !fit { + return fmt.Errorf("node <%s> are not available to schedule task <%s/%s> due to PID Pressure", + node.Name, task.Namespace, task.Name) + } } // Pod Affinity/Anti-Affinity Predicate diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/plugins/proportion/proportion.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/plugins/proportion/proportion.go index 2059f78de7..06f4c1c4cc 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/plugins/proportion/proportion.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/plugins/proportion/proportion.go @@ -111,11 +111,15 @@ func (pp *proportionPlugin) OnSessionOpen(ssn *framework.Session) { // If no queues, break if totalWeight == 0 { + glog.V(4).Infof("Exiting when total weight is 0") break } // Calculates the deserved of each Queue. - deserved := api.EmptyResource() + // increasedDeserved is the increased value for attr.deserved of processed queues + // decreasedDeserved is the decreased value for attr.deserved of processed queues + increasedDeserved := api.EmptyResource() + decreasedDeserved := api.EmptyResource() for _, attr := range pp.queueOpts { glog.V(4).Infof("Considering Queue <%s>: weight <%d>, total weight <%d>.", attr.name, attr.weight, totalWeight) @@ -125,20 +129,26 @@ func (pp *proportionPlugin) OnSessionOpen(ssn *framework.Session) { oldDeserved := attr.deserved.Clone() attr.deserved.Add(remaining.Clone().Multi(float64(attr.weight) / float64(totalWeight))) - if !attr.deserved.LessEqual(attr.request) { + + if attr.request.Less(attr.deserved) { attr.deserved = helpers.Min(attr.deserved, attr.request) meet[attr.queueID] = struct{}{} + glog.V(4).Infof("queue <%s> is meet", attr.name) + } pp.updateShare(attr) glog.V(4).Infof("The attributes of queue <%s> in proportion: deserved <%v>, allocate <%v>, request <%v>, share <%0.2f>", attr.name, attr.deserved, attr.allocated, attr.request, attr.share) - deserved.Add(attr.deserved.Clone().Sub(oldDeserved)) + increased, decreased := attr.deserved.Diff(oldDeserved) + increasedDeserved.Add(increased) + decreasedDeserved.Add(decreased) } - remaining.Sub(deserved) + remaining.Sub(increasedDeserved).Add(decreasedDeserved) if remaining.IsEmpty() { + glog.V(4).Infof("Exiting when remaining is empty: <%v>", remaining) break } } @@ -171,7 +181,7 @@ func (pp *proportionPlugin) OnSessionOpen(ssn *framework.Session) { } allocated := allocations[job.Queue] if allocated.Less(reclaimee.Resreq) { - glog.Errorf("Failed to allocate resource for Task <%s/%s> in Queue <%s>, not enough resource.", + glog.V(3).Infof("Failed to allocate resource for Task <%s/%s> in Queue <%s>, not enough resource.", reclaimee.Namespace, reclaimee.Name, job.Queue) continue } @@ -198,6 +208,25 @@ func (pp *proportionPlugin) OnSessionOpen(ssn *framework.Session) { return overused }) + ssn.AddJobEnqueueableFn(pp.Name(), func(obj interface{}) bool { + job := obj.(*api.JobInfo) + queueID := job.Queue + attr := pp.queueOpts[queueID] + queue := ssn.Queues[queueID] + + // If no capability is set, always enqueue the job. + if len(queue.Queue.Spec.Capability) == 0 { + return true + } + + pgResource := api.NewResource(*job.PodGroup.Spec.MinResources) + // The queue resource quota limit has not reached + if pgResource.Clone().Add(attr.allocated).LessEqual(api.NewResource(queue.Queue.Spec.Capability)) { + return true + } + return false + }) + // Register event handlers. ssn.AddEventHandler(&framework.EventHandler{ AllocateFunc: func(event *framework.Event) { diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/util/scheduler_helper.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/util/scheduler_helper.go index 2641ba6cc4..7429dd8a51 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/util/scheduler_helper.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/util/scheduler_helper.go @@ -18,6 +18,7 @@ package util import ( "context" + "math" "math/rand" "sort" "sync" @@ -26,6 +27,7 @@ import ( "k8s.io/client-go/util/workqueue" "github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api" + schedulerapi "k8s.io/kubernetes/pkg/scheduler/api" ) // PredicateNodes returns nodes that fit task @@ -55,23 +57,54 @@ func PredicateNodes(task *api.TaskInfo, nodes []*api.NodeInfo, fn api.PredicateF } // PrioritizeNodes returns a map whose key is node's score and value are corresponding nodes -func PrioritizeNodes(task *api.TaskInfo, nodes []*api.NodeInfo, fn api.NodeOrderFn) map[float64][]*api.NodeInfo { +func PrioritizeNodes(task *api.TaskInfo, nodes []*api.NodeInfo, mapFn api.NodeOrderMapFn, reduceFn api.NodeOrderReduceFn) map[float64][]*api.NodeInfo { + pluginNodeScoreMap := map[string]schedulerapi.HostPriorityList{} + nodeOrderScoreMap := map[string]float64{} nodeScores := map[float64][]*api.NodeInfo{} - var workerLock sync.Mutex scoreNode := func(index int) { node := nodes[index] - score, err := fn(task, node) + mapScores, orderScore, err := mapFn(task, node) if err != nil { glog.Errorf("Error in Calculating Priority for the node:%v", err) return } workerLock.Lock() - nodeScores[score] = append(nodeScores[score], node) + for plugin, score := range mapScores { + nodeScoreMap, ok := pluginNodeScoreMap[plugin] + if !ok { + nodeScoreMap = schedulerapi.HostPriorityList{} + } + hp := schedulerapi.HostPriority{} + hp.Host = node.Name + hp.Score = int(math.Floor(score)) + pluginNodeScoreMap[plugin] = append(nodeScoreMap, hp) + } + nodeOrderScoreMap[node.Name] = orderScore workerLock.Unlock() } workqueue.ParallelizeUntil(context.TODO(), 16, len(nodes), scoreNode) + reduceScores, err := reduceFn(task, pluginNodeScoreMap) + if err != nil { + glog.Errorf("Error in Calculating Priority for the node:%v", err) + return nodeScores + } + for _, node := range nodes { + if score, found := reduceScores[node.Name]; found { + if orderScore, ok := nodeOrderScoreMap[node.Name]; ok { + score = score + orderScore + } + nodeScores[score] = append(nodeScores[score], node) + } else { + // If no plugin is applied to this node, the default is 0.0 + score = 0.0 + if orderScore, ok := nodeOrderScoreMap[node.Name]; ok { + score = score + orderScore + } + nodeScores[score] = append(nodeScores[score], node) + } + } return nodeScores } From 14888546cb89d74084d8043bf22a0b103d9b25d5 Mon Sep 17 00:00:00 2001 From: thandayuthapani Date: Thu, 6 Jun 2019 09:26:20 +0530 Subject: [PATCH 56/68] Add Kubernetes Fake Package for UT --- .../kubernetes/fake/clientset_generated.go | 384 ++++++++++++++++++ .../k8s.io/client-go/kubernetes/fake/doc.go | 20 + .../client-go/kubernetes/fake/register.go | 118 ++++++ .../v1alpha1/fake/doc.go | 20 + .../fake/fake_admissionregistration_client.go | 40 ++ .../fake/fake_initializerconfiguration.go | 120 ++++++ .../admissionregistration/v1beta1/fake/doc.go | 20 + .../fake/fake_admissionregistration_client.go | 44 ++ .../fake/fake_mutatingwebhookconfiguration.go | 120 ++++++ .../fake_validatingwebhookconfiguration.go | 120 ++++++ .../kubernetes/typed/apps/v1/fake/doc.go | 20 + .../typed/apps/v1/fake/fake_apps_client.go | 56 +++ .../apps/v1/fake/fake_controllerrevision.go | 128 ++++++ .../typed/apps/v1/fake/fake_daemonset.go | 140 +++++++ .../typed/apps/v1/fake/fake_deployment.go | 163 ++++++++ .../typed/apps/v1/fake/fake_replicaset.go | 163 ++++++++ .../typed/apps/v1/fake/fake_statefulset.go | 163 ++++++++ .../kubernetes/typed/apps/v1beta1/fake/doc.go | 20 + .../apps/v1beta1/fake/fake_apps_client.go | 48 +++ .../v1beta1/fake/fake_controllerrevision.go | 128 ++++++ .../apps/v1beta1/fake/fake_deployment.go | 140 +++++++ .../apps/v1beta1/fake/fake_statefulset.go | 140 +++++++ .../kubernetes/typed/apps/v1beta2/fake/doc.go | 20 + .../apps/v1beta2/fake/fake_apps_client.go | 56 +++ .../v1beta2/fake/fake_controllerrevision.go | 128 ++++++ .../typed/apps/v1beta2/fake/fake_daemonset.go | 140 +++++++ .../apps/v1beta2/fake/fake_deployment.go | 140 +++++++ .../apps/v1beta2/fake/fake_replicaset.go | 140 +++++++ .../apps/v1beta2/fake/fake_statefulset.go | 162 ++++++++ .../auditregistration/v1alpha1/fake/doc.go | 20 + .../fake/fake_auditregistration_client.go | 40 ++ .../v1alpha1/fake/fake_auditsink.go | 120 ++++++ .../typed/authentication/v1/fake/doc.go | 20 + .../v1/fake/fake_authentication_client.go | 40 ++ .../v1/fake/fake_tokenreview.go | 24 ++ .../v1/fake/fake_tokenreview_expansion.go | 27 ++ .../typed/authentication/v1beta1/fake/doc.go | 20 + .../fake/fake_authentication_client.go | 40 ++ .../v1beta1/fake/fake_tokenreview.go | 24 ++ .../fake/fake_tokenreview_expansion.go | 27 ++ .../typed/authorization/v1/fake/doc.go | 20 + .../v1/fake/fake_authorization_client.go | 52 +++ .../v1/fake/fake_localsubjectaccessreview.go | 25 ++ ...fake_localsubjectaccessreview_expansion.go | 27 ++ .../v1/fake/fake_selfsubjectaccessreview.go | 24 ++ .../fake_selfsubjectaccessreview_expansion.go | 27 ++ .../v1/fake/fake_selfsubjectrulesreview.go | 24 ++ .../fake_selfsubjectrulesreview_expansion.go | 27 ++ .../v1/fake/fake_subjectaccessreview.go | 24 ++ .../fake_subjectaccessreview_expansion.go | 30 ++ .../typed/authorization/v1beta1/fake/doc.go | 20 + .../v1beta1/fake/fake_authorization_client.go | 52 +++ .../v1beta1/fake/fake_generated_expansion.go | 17 + .../fake/fake_localsubjectaccessreview.go | 25 ++ ...fake_localsubjectaccessreview_expansion.go | 27 ++ .../fake/fake_selfsubjectaccessreview.go | 24 ++ .../fake_selfsubjectaccessreview_expansion.go | 27 ++ .../fake/fake_selfsubjectrulesreview.go | 24 ++ .../fake_selfsubjectrulesreview_expansion.go | 27 ++ .../v1beta1/fake/fake_subjectaccessreview.go | 24 ++ .../fake_subjectaccessreview_expansion.go | 27 ++ .../typed/autoscaling/v1/fake/doc.go | 20 + .../v1/fake/fake_autoscaling_client.go | 40 ++ .../v1/fake/fake_horizontalpodautoscaler.go | 140 +++++++ .../typed/autoscaling/v2beta1/fake/doc.go | 20 + .../v2beta1/fake/fake_autoscaling_client.go | 40 ++ .../fake/fake_horizontalpodautoscaler.go | 140 +++++++ .../typed/autoscaling/v2beta2/fake/doc.go | 20 + .../v2beta2/fake/fake_autoscaling_client.go | 40 ++ .../fake/fake_horizontalpodautoscaler.go | 140 +++++++ .../kubernetes/typed/batch/v1/fake/doc.go | 20 + .../typed/batch/v1/fake/fake_batch_client.go | 40 ++ .../typed/batch/v1/fake/fake_job.go | 140 +++++++ .../typed/batch/v1beta1/fake/doc.go | 20 + .../batch/v1beta1/fake/fake_batch_client.go | 40 ++ .../typed/batch/v1beta1/fake/fake_cronjob.go | 140 +++++++ .../typed/batch/v2alpha1/fake/doc.go | 20 + .../batch/v2alpha1/fake/fake_batch_client.go | 40 ++ .../typed/batch/v2alpha1/fake/fake_cronjob.go | 140 +++++++ .../typed/certificates/v1beta1/fake/doc.go | 20 + .../v1beta1/fake/fake_certificates_client.go | 40 ++ .../fake/fake_certificatesigningrequest.go | 131 ++++++ ...ake_certificatesigningrequest_expansion.go | 31 ++ .../typed/coordination/v1beta1/fake/doc.go | 20 + .../v1beta1/fake/fake_coordination_client.go | 40 ++ .../coordination/v1beta1/fake/fake_lease.go | 128 ++++++ .../kubernetes/typed/core/v1/fake/doc.go | 20 + .../core/v1/fake/fake_componentstatus.go | 120 ++++++ .../typed/core/v1/fake/fake_configmap.go | 128 ++++++ .../typed/core/v1/fake/fake_core_client.go | 100 +++++ .../typed/core/v1/fake/fake_endpoints.go | 128 ++++++ .../typed/core/v1/fake/fake_event.go | 128 ++++++ .../core/v1/fake/fake_event_expansion.go | 93 +++++ .../typed/core/v1/fake/fake_limitrange.go | 128 ++++++ .../typed/core/v1/fake/fake_namespace.go | 123 ++++++ .../core/v1/fake/fake_namespace_expansion.go | 37 ++ .../typed/core/v1/fake/fake_node.go | 131 ++++++ .../typed/core/v1/fake/fake_node_expansion.go | 36 ++ .../core/v1/fake/fake_persistentvolume.go | 131 ++++++ .../v1/fake/fake_persistentvolumeclaim.go | 140 +++++++ .../kubernetes/typed/core/v1/fake/fake_pod.go | 140 +++++++ .../typed/core/v1/fake/fake_pod_expansion.go | 69 ++++ .../typed/core/v1/fake/fake_podtemplate.go | 128 ++++++ .../v1/fake/fake_replicationcontroller.go | 163 ++++++++ .../typed/core/v1/fake/fake_resourcequota.go | 140 +++++++ .../typed/core/v1/fake/fake_secret.go | 128 ++++++ .../typed/core/v1/fake/fake_service.go | 132 ++++++ .../core/v1/fake/fake_service_expansion.go | 26 ++ .../typed/core/v1/fake/fake_serviceaccount.go | 128 ++++++ .../v1/fake/fake_serviceaccount_expansion.go | 31 ++ .../typed/events/v1beta1/fake/doc.go | 20 + .../typed/events/v1beta1/fake/fake_event.go | 128 ++++++ .../events/v1beta1/fake/fake_events_client.go | 40 ++ .../typed/extensions/v1beta1/fake/doc.go | 20 + .../extensions/v1beta1/fake/fake_daemonset.go | 140 +++++++ .../v1beta1/fake/fake_deployment.go | 162 ++++++++ .../v1beta1/fake/fake_deployment_expansion.go | 33 ++ .../v1beta1/fake/fake_extensions_client.go | 56 +++ .../extensions/v1beta1/fake/fake_ingress.go | 140 +++++++ .../v1beta1/fake/fake_podsecuritypolicy.go | 120 ++++++ .../v1beta1/fake/fake_replicaset.go | 162 ++++++++ .../typed/networking/v1/fake/doc.go | 20 + .../v1/fake/fake_networking_client.go | 40 ++ .../networking/v1/fake/fake_networkpolicy.go | 128 ++++++ .../typed/policy/v1beta1/fake/doc.go | 20 + .../policy/v1beta1/fake/fake_eviction.go | 25 ++ .../v1beta1/fake/fake_eviction_expansion.go | 34 ++ .../v1beta1/fake/fake_poddisruptionbudget.go | 140 +++++++ .../v1beta1/fake/fake_podsecuritypolicy.go | 120 ++++++ .../policy/v1beta1/fake/fake_policy_client.go | 48 +++ .../kubernetes/typed/rbac/v1/fake/doc.go | 20 + .../typed/rbac/v1/fake/fake_clusterrole.go | 120 ++++++ .../rbac/v1/fake/fake_clusterrolebinding.go | 120 ++++++ .../typed/rbac/v1/fake/fake_rbac_client.go | 52 +++ .../typed/rbac/v1/fake/fake_role.go | 128 ++++++ .../typed/rbac/v1/fake/fake_rolebinding.go | 128 ++++++ .../typed/rbac/v1alpha1/fake/doc.go | 20 + .../rbac/v1alpha1/fake/fake_clusterrole.go | 120 ++++++ .../v1alpha1/fake/fake_clusterrolebinding.go | 120 ++++++ .../rbac/v1alpha1/fake/fake_rbac_client.go | 52 +++ .../typed/rbac/v1alpha1/fake/fake_role.go | 128 ++++++ .../rbac/v1alpha1/fake/fake_rolebinding.go | 128 ++++++ .../kubernetes/typed/rbac/v1beta1/fake/doc.go | 20 + .../rbac/v1beta1/fake/fake_clusterrole.go | 120 ++++++ .../v1beta1/fake/fake_clusterrolebinding.go | 120 ++++++ .../rbac/v1beta1/fake/fake_rbac_client.go | 52 +++ .../typed/rbac/v1beta1/fake/fake_role.go | 128 ++++++ .../rbac/v1beta1/fake/fake_rolebinding.go | 128 ++++++ .../typed/scheduling/v1alpha1/fake/doc.go | 20 + .../v1alpha1/fake/fake_priorityclass.go | 120 ++++++ .../v1alpha1/fake/fake_scheduling_client.go | 40 ++ .../typed/scheduling/v1beta1/fake/doc.go | 20 + .../v1beta1/fake/fake_priorityclass.go | 120 ++++++ .../v1beta1/fake/fake_scheduling_client.go | 40 ++ .../typed/settings/v1alpha1/fake/doc.go | 20 + .../settings/v1alpha1/fake/fake_podpreset.go | 128 ++++++ .../v1alpha1/fake/fake_settings_client.go | 40 ++ .../kubernetes/typed/storage/v1/fake/doc.go | 20 + .../storage/v1/fake/fake_storage_client.go | 44 ++ .../storage/v1/fake/fake_storageclass.go | 120 ++++++ .../storage/v1/fake/fake_volumeattachment.go | 131 ++++++ .../typed/storage/v1alpha1/fake/doc.go | 20 + .../v1alpha1/fake/fake_storage_client.go | 40 ++ .../v1alpha1/fake/fake_volumeattachment.go | 131 ++++++ .../typed/storage/v1beta1/fake/doc.go | 20 + .../v1beta1/fake/fake_storage_client.go | 44 ++ .../storage/v1beta1/fake/fake_storageclass.go | 120 ++++++ .../v1beta1/fake/fake_volumeattachment.go | 131 ++++++ 168 files changed, 12884 insertions(+) create mode 100644 vendor/k8s.io/client-go/kubernetes/fake/clientset_generated.go create mode 100644 vendor/k8s.io/client-go/kubernetes/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/fake/register.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake/fake_admissionregistration_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake/fake_initializerconfiguration.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/fake_admissionregistration_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/fake_mutatingwebhookconfiguration.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/fake_validatingwebhookconfiguration.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_apps_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_controllerrevision.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_daemonset.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_deployment.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_replicaset.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_statefulset.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_apps_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_controllerrevision.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_deployment.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_statefulset.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_apps_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_controllerrevision.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_daemonset.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_deployment.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_replicaset.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_statefulset.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake/fake_auditregistration_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake/fake_auditsink.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/fake_authentication_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/fake_tokenreview.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/fake_tokenreview_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/fake_authentication_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/fake_tokenreview.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/fake_tokenreview_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_authorization_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_localsubjectaccessreview.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_localsubjectaccessreview_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectaccessreview.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectaccessreview_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectrulesreview.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectrulesreview_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_subjectaccessreview.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_subjectaccessreview_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_authorization_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_generated_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_localsubjectaccessreview.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_localsubjectaccessreview_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectaccessreview.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectaccessreview_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectrulesreview.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectrulesreview_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_subjectaccessreview.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_subjectaccessreview_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake/fake_autoscaling_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake/fake_horizontalpodautoscaler.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake/fake_autoscaling_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake/fake_horizontalpodautoscaler.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake/fake_autoscaling_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake/fake_horizontalpodautoscaler.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/batch/v1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/batch/v1/fake/fake_batch_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/batch/v1/fake/fake_job.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake/fake_batch_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake/fake_cronjob.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake/fake_batch_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake/fake_cronjob.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/fake_certificates_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/fake_certificatesigningrequest.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/fake_certificatesigningrequest_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake/fake_coordination_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake/fake_lease.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_componentstatus.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_configmap.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_core_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_endpoints.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_event.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_event_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_limitrange.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_namespace.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_namespace_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_node.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_node_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_persistentvolume.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_persistentvolumeclaim.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_pod.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_pod_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_podtemplate.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_replicationcontroller.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_resourcequota.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_secret.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_service.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_service_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_serviceaccount.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_serviceaccount_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/events/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/events/v1beta1/fake/fake_event.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/events/v1beta1/fake/fake_events_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_daemonset.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_deployment.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_deployment_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_extensions_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_ingress.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_podsecuritypolicy.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_replicaset.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/networking/v1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/networking/v1/fake/fake_networking_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/networking/v1/fake/fake_networkpolicy.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_eviction.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_eviction_expansion.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_poddisruptionbudget.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_podsecuritypolicy.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_policy_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_clusterrole.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_clusterrolebinding.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_rbac_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_role.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_rolebinding.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_clusterrole.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_clusterrolebinding.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_rbac_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_role.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_rolebinding.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_clusterrole.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_clusterrolebinding.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_rbac_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_role.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_rolebinding.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake/fake_priorityclass.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake/fake_scheduling_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake/fake_priorityclass.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake/fake_scheduling_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake/fake_podpreset.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake/fake_settings_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/fake_storage_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/fake_storageclass.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/fake_volumeattachment.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake/fake_storage_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake/fake_volumeattachment.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/doc.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/fake_storage_client.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/fake_storageclass.go create mode 100644 vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/fake_volumeattachment.go diff --git a/vendor/k8s.io/client-go/kubernetes/fake/clientset_generated.go b/vendor/k8s.io/client-go/kubernetes/fake/clientset_generated.go new file mode 100644 index 0000000000..47b63ffaed --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/fake/clientset_generated.go @@ -0,0 +1,384 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + "k8s.io/apimachinery/pkg/runtime" + "k8s.io/apimachinery/pkg/watch" + "k8s.io/client-go/discovery" + fakediscovery "k8s.io/client-go/discovery/fake" + clientset "k8s.io/client-go/kubernetes" + admissionregistrationv1alpha1 "k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1" + fakeadmissionregistrationv1alpha1 "k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake" + admissionregistrationv1beta1 "k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1" + fakeadmissionregistrationv1beta1 "k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake" + appsv1 "k8s.io/client-go/kubernetes/typed/apps/v1" + fakeappsv1 "k8s.io/client-go/kubernetes/typed/apps/v1/fake" + appsv1beta1 "k8s.io/client-go/kubernetes/typed/apps/v1beta1" + fakeappsv1beta1 "k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake" + appsv1beta2 "k8s.io/client-go/kubernetes/typed/apps/v1beta2" + fakeappsv1beta2 "k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake" + auditregistrationv1alpha1 "k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1" + fakeauditregistrationv1alpha1 "k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake" + authenticationv1 "k8s.io/client-go/kubernetes/typed/authentication/v1" + fakeauthenticationv1 "k8s.io/client-go/kubernetes/typed/authentication/v1/fake" + authenticationv1beta1 "k8s.io/client-go/kubernetes/typed/authentication/v1beta1" + fakeauthenticationv1beta1 "k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake" + authorizationv1 "k8s.io/client-go/kubernetes/typed/authorization/v1" + fakeauthorizationv1 "k8s.io/client-go/kubernetes/typed/authorization/v1/fake" + authorizationv1beta1 "k8s.io/client-go/kubernetes/typed/authorization/v1beta1" + fakeauthorizationv1beta1 "k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake" + autoscalingv1 "k8s.io/client-go/kubernetes/typed/autoscaling/v1" + fakeautoscalingv1 "k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake" + autoscalingv2beta1 "k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1" + fakeautoscalingv2beta1 "k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake" + autoscalingv2beta2 "k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2" + fakeautoscalingv2beta2 "k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake" + batchv1 "k8s.io/client-go/kubernetes/typed/batch/v1" + fakebatchv1 "k8s.io/client-go/kubernetes/typed/batch/v1/fake" + batchv1beta1 "k8s.io/client-go/kubernetes/typed/batch/v1beta1" + fakebatchv1beta1 "k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake" + batchv2alpha1 "k8s.io/client-go/kubernetes/typed/batch/v2alpha1" + fakebatchv2alpha1 "k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake" + certificatesv1beta1 "k8s.io/client-go/kubernetes/typed/certificates/v1beta1" + fakecertificatesv1beta1 "k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake" + coordinationv1beta1 "k8s.io/client-go/kubernetes/typed/coordination/v1beta1" + fakecoordinationv1beta1 "k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake" + corev1 "k8s.io/client-go/kubernetes/typed/core/v1" + fakecorev1 "k8s.io/client-go/kubernetes/typed/core/v1/fake" + eventsv1beta1 "k8s.io/client-go/kubernetes/typed/events/v1beta1" + fakeeventsv1beta1 "k8s.io/client-go/kubernetes/typed/events/v1beta1/fake" + extensionsv1beta1 "k8s.io/client-go/kubernetes/typed/extensions/v1beta1" + fakeextensionsv1beta1 "k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake" + networkingv1 "k8s.io/client-go/kubernetes/typed/networking/v1" + fakenetworkingv1 "k8s.io/client-go/kubernetes/typed/networking/v1/fake" + policyv1beta1 "k8s.io/client-go/kubernetes/typed/policy/v1beta1" + fakepolicyv1beta1 "k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake" + rbacv1 "k8s.io/client-go/kubernetes/typed/rbac/v1" + fakerbacv1 "k8s.io/client-go/kubernetes/typed/rbac/v1/fake" + rbacv1alpha1 "k8s.io/client-go/kubernetes/typed/rbac/v1alpha1" + fakerbacv1alpha1 "k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake" + rbacv1beta1 "k8s.io/client-go/kubernetes/typed/rbac/v1beta1" + fakerbacv1beta1 "k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake" + schedulingv1alpha1 "k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1" + fakeschedulingv1alpha1 "k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake" + schedulingv1beta1 "k8s.io/client-go/kubernetes/typed/scheduling/v1beta1" + fakeschedulingv1beta1 "k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake" + settingsv1alpha1 "k8s.io/client-go/kubernetes/typed/settings/v1alpha1" + fakesettingsv1alpha1 "k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake" + storagev1 "k8s.io/client-go/kubernetes/typed/storage/v1" + fakestoragev1 "k8s.io/client-go/kubernetes/typed/storage/v1/fake" + storagev1alpha1 "k8s.io/client-go/kubernetes/typed/storage/v1alpha1" + fakestoragev1alpha1 "k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake" + storagev1beta1 "k8s.io/client-go/kubernetes/typed/storage/v1beta1" + fakestoragev1beta1 "k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake" + "k8s.io/client-go/testing" +) + +// NewSimpleClientset returns a clientset that will respond with the provided objects. +// It's backed by a very simple object tracker that processes creates, updates and deletions as-is, +// without applying any validations and/or defaults. It shouldn't be considered a replacement +// for a real clientset and is mostly useful in simple unit tests. +func NewSimpleClientset(objects ...runtime.Object) *Clientset { + o := testing.NewObjectTracker(scheme, codecs.UniversalDecoder()) + for _, obj := range objects { + if err := o.Add(obj); err != nil { + panic(err) + } + } + + cs := &Clientset{} + cs.discovery = &fakediscovery.FakeDiscovery{Fake: &cs.Fake} + cs.AddReactor("*", "*", testing.ObjectReaction(o)) + cs.AddWatchReactor("*", func(action testing.Action) (handled bool, ret watch.Interface, err error) { + gvr := action.GetResource() + ns := action.GetNamespace() + watch, err := o.Watch(gvr, ns) + if err != nil { + return false, nil, err + } + return true, watch, nil + }) + + return cs +} + +// Clientset implements clientset.Interface. Meant to be embedded into a +// struct to get a default implementation. This makes faking out just the method +// you want to test easier. +type Clientset struct { + testing.Fake + discovery *fakediscovery.FakeDiscovery +} + +func (c *Clientset) Discovery() discovery.DiscoveryInterface { + return c.discovery +} + +var _ clientset.Interface = &Clientset{} + +// AdmissionregistrationV1alpha1 retrieves the AdmissionregistrationV1alpha1Client +func (c *Clientset) AdmissionregistrationV1alpha1() admissionregistrationv1alpha1.AdmissionregistrationV1alpha1Interface { + return &fakeadmissionregistrationv1alpha1.FakeAdmissionregistrationV1alpha1{Fake: &c.Fake} +} + +// AdmissionregistrationV1beta1 retrieves the AdmissionregistrationV1beta1Client +func (c *Clientset) AdmissionregistrationV1beta1() admissionregistrationv1beta1.AdmissionregistrationV1beta1Interface { + return &fakeadmissionregistrationv1beta1.FakeAdmissionregistrationV1beta1{Fake: &c.Fake} +} + +// Admissionregistration retrieves the AdmissionregistrationV1beta1Client +func (c *Clientset) Admissionregistration() admissionregistrationv1beta1.AdmissionregistrationV1beta1Interface { + return &fakeadmissionregistrationv1beta1.FakeAdmissionregistrationV1beta1{Fake: &c.Fake} +} + +// AppsV1beta1 retrieves the AppsV1beta1Client +func (c *Clientset) AppsV1beta1() appsv1beta1.AppsV1beta1Interface { + return &fakeappsv1beta1.FakeAppsV1beta1{Fake: &c.Fake} +} + +// AppsV1beta2 retrieves the AppsV1beta2Client +func (c *Clientset) AppsV1beta2() appsv1beta2.AppsV1beta2Interface { + return &fakeappsv1beta2.FakeAppsV1beta2{Fake: &c.Fake} +} + +// AppsV1 retrieves the AppsV1Client +func (c *Clientset) AppsV1() appsv1.AppsV1Interface { + return &fakeappsv1.FakeAppsV1{Fake: &c.Fake} +} + +// Apps retrieves the AppsV1Client +func (c *Clientset) Apps() appsv1.AppsV1Interface { + return &fakeappsv1.FakeAppsV1{Fake: &c.Fake} +} + +// AuditregistrationV1alpha1 retrieves the AuditregistrationV1alpha1Client +func (c *Clientset) AuditregistrationV1alpha1() auditregistrationv1alpha1.AuditregistrationV1alpha1Interface { + return &fakeauditregistrationv1alpha1.FakeAuditregistrationV1alpha1{Fake: &c.Fake} +} + +// Auditregistration retrieves the AuditregistrationV1alpha1Client +func (c *Clientset) Auditregistration() auditregistrationv1alpha1.AuditregistrationV1alpha1Interface { + return &fakeauditregistrationv1alpha1.FakeAuditregistrationV1alpha1{Fake: &c.Fake} +} + +// AuthenticationV1 retrieves the AuthenticationV1Client +func (c *Clientset) AuthenticationV1() authenticationv1.AuthenticationV1Interface { + return &fakeauthenticationv1.FakeAuthenticationV1{Fake: &c.Fake} +} + +// Authentication retrieves the AuthenticationV1Client +func (c *Clientset) Authentication() authenticationv1.AuthenticationV1Interface { + return &fakeauthenticationv1.FakeAuthenticationV1{Fake: &c.Fake} +} + +// AuthenticationV1beta1 retrieves the AuthenticationV1beta1Client +func (c *Clientset) AuthenticationV1beta1() authenticationv1beta1.AuthenticationV1beta1Interface { + return &fakeauthenticationv1beta1.FakeAuthenticationV1beta1{Fake: &c.Fake} +} + +// AuthorizationV1 retrieves the AuthorizationV1Client +func (c *Clientset) AuthorizationV1() authorizationv1.AuthorizationV1Interface { + return &fakeauthorizationv1.FakeAuthorizationV1{Fake: &c.Fake} +} + +// Authorization retrieves the AuthorizationV1Client +func (c *Clientset) Authorization() authorizationv1.AuthorizationV1Interface { + return &fakeauthorizationv1.FakeAuthorizationV1{Fake: &c.Fake} +} + +// AuthorizationV1beta1 retrieves the AuthorizationV1beta1Client +func (c *Clientset) AuthorizationV1beta1() authorizationv1beta1.AuthorizationV1beta1Interface { + return &fakeauthorizationv1beta1.FakeAuthorizationV1beta1{Fake: &c.Fake} +} + +// AutoscalingV1 retrieves the AutoscalingV1Client +func (c *Clientset) AutoscalingV1() autoscalingv1.AutoscalingV1Interface { + return &fakeautoscalingv1.FakeAutoscalingV1{Fake: &c.Fake} +} + +// Autoscaling retrieves the AutoscalingV1Client +func (c *Clientset) Autoscaling() autoscalingv1.AutoscalingV1Interface { + return &fakeautoscalingv1.FakeAutoscalingV1{Fake: &c.Fake} +} + +// AutoscalingV2beta1 retrieves the AutoscalingV2beta1Client +func (c *Clientset) AutoscalingV2beta1() autoscalingv2beta1.AutoscalingV2beta1Interface { + return &fakeautoscalingv2beta1.FakeAutoscalingV2beta1{Fake: &c.Fake} +} + +// AutoscalingV2beta2 retrieves the AutoscalingV2beta2Client +func (c *Clientset) AutoscalingV2beta2() autoscalingv2beta2.AutoscalingV2beta2Interface { + return &fakeautoscalingv2beta2.FakeAutoscalingV2beta2{Fake: &c.Fake} +} + +// BatchV1 retrieves the BatchV1Client +func (c *Clientset) BatchV1() batchv1.BatchV1Interface { + return &fakebatchv1.FakeBatchV1{Fake: &c.Fake} +} + +// Batch retrieves the BatchV1Client +func (c *Clientset) Batch() batchv1.BatchV1Interface { + return &fakebatchv1.FakeBatchV1{Fake: &c.Fake} +} + +// BatchV1beta1 retrieves the BatchV1beta1Client +func (c *Clientset) BatchV1beta1() batchv1beta1.BatchV1beta1Interface { + return &fakebatchv1beta1.FakeBatchV1beta1{Fake: &c.Fake} +} + +// BatchV2alpha1 retrieves the BatchV2alpha1Client +func (c *Clientset) BatchV2alpha1() batchv2alpha1.BatchV2alpha1Interface { + return &fakebatchv2alpha1.FakeBatchV2alpha1{Fake: &c.Fake} +} + +// CertificatesV1beta1 retrieves the CertificatesV1beta1Client +func (c *Clientset) CertificatesV1beta1() certificatesv1beta1.CertificatesV1beta1Interface { + return &fakecertificatesv1beta1.FakeCertificatesV1beta1{Fake: &c.Fake} +} + +// Certificates retrieves the CertificatesV1beta1Client +func (c *Clientset) Certificates() certificatesv1beta1.CertificatesV1beta1Interface { + return &fakecertificatesv1beta1.FakeCertificatesV1beta1{Fake: &c.Fake} +} + +// CoordinationV1beta1 retrieves the CoordinationV1beta1Client +func (c *Clientset) CoordinationV1beta1() coordinationv1beta1.CoordinationV1beta1Interface { + return &fakecoordinationv1beta1.FakeCoordinationV1beta1{Fake: &c.Fake} +} + +// Coordination retrieves the CoordinationV1beta1Client +func (c *Clientset) Coordination() coordinationv1beta1.CoordinationV1beta1Interface { + return &fakecoordinationv1beta1.FakeCoordinationV1beta1{Fake: &c.Fake} +} + +// CoreV1 retrieves the CoreV1Client +func (c *Clientset) CoreV1() corev1.CoreV1Interface { + return &fakecorev1.FakeCoreV1{Fake: &c.Fake} +} + +// Core retrieves the CoreV1Client +func (c *Clientset) Core() corev1.CoreV1Interface { + return &fakecorev1.FakeCoreV1{Fake: &c.Fake} +} + +// EventsV1beta1 retrieves the EventsV1beta1Client +func (c *Clientset) EventsV1beta1() eventsv1beta1.EventsV1beta1Interface { + return &fakeeventsv1beta1.FakeEventsV1beta1{Fake: &c.Fake} +} + +// Events retrieves the EventsV1beta1Client +func (c *Clientset) Events() eventsv1beta1.EventsV1beta1Interface { + return &fakeeventsv1beta1.FakeEventsV1beta1{Fake: &c.Fake} +} + +// ExtensionsV1beta1 retrieves the ExtensionsV1beta1Client +func (c *Clientset) ExtensionsV1beta1() extensionsv1beta1.ExtensionsV1beta1Interface { + return &fakeextensionsv1beta1.FakeExtensionsV1beta1{Fake: &c.Fake} +} + +// Extensions retrieves the ExtensionsV1beta1Client +func (c *Clientset) Extensions() extensionsv1beta1.ExtensionsV1beta1Interface { + return &fakeextensionsv1beta1.FakeExtensionsV1beta1{Fake: &c.Fake} +} + +// NetworkingV1 retrieves the NetworkingV1Client +func (c *Clientset) NetworkingV1() networkingv1.NetworkingV1Interface { + return &fakenetworkingv1.FakeNetworkingV1{Fake: &c.Fake} +} + +// Networking retrieves the NetworkingV1Client +func (c *Clientset) Networking() networkingv1.NetworkingV1Interface { + return &fakenetworkingv1.FakeNetworkingV1{Fake: &c.Fake} +} + +// PolicyV1beta1 retrieves the PolicyV1beta1Client +func (c *Clientset) PolicyV1beta1() policyv1beta1.PolicyV1beta1Interface { + return &fakepolicyv1beta1.FakePolicyV1beta1{Fake: &c.Fake} +} + +// Policy retrieves the PolicyV1beta1Client +func (c *Clientset) Policy() policyv1beta1.PolicyV1beta1Interface { + return &fakepolicyv1beta1.FakePolicyV1beta1{Fake: &c.Fake} +} + +// RbacV1 retrieves the RbacV1Client +func (c *Clientset) RbacV1() rbacv1.RbacV1Interface { + return &fakerbacv1.FakeRbacV1{Fake: &c.Fake} +} + +// Rbac retrieves the RbacV1Client +func (c *Clientset) Rbac() rbacv1.RbacV1Interface { + return &fakerbacv1.FakeRbacV1{Fake: &c.Fake} +} + +// RbacV1beta1 retrieves the RbacV1beta1Client +func (c *Clientset) RbacV1beta1() rbacv1beta1.RbacV1beta1Interface { + return &fakerbacv1beta1.FakeRbacV1beta1{Fake: &c.Fake} +} + +// RbacV1alpha1 retrieves the RbacV1alpha1Client +func (c *Clientset) RbacV1alpha1() rbacv1alpha1.RbacV1alpha1Interface { + return &fakerbacv1alpha1.FakeRbacV1alpha1{Fake: &c.Fake} +} + +// SchedulingV1alpha1 retrieves the SchedulingV1alpha1Client +func (c *Clientset) SchedulingV1alpha1() schedulingv1alpha1.SchedulingV1alpha1Interface { + return &fakeschedulingv1alpha1.FakeSchedulingV1alpha1{Fake: &c.Fake} +} + +// SchedulingV1beta1 retrieves the SchedulingV1beta1Client +func (c *Clientset) SchedulingV1beta1() schedulingv1beta1.SchedulingV1beta1Interface { + return &fakeschedulingv1beta1.FakeSchedulingV1beta1{Fake: &c.Fake} +} + +// Scheduling retrieves the SchedulingV1beta1Client +func (c *Clientset) Scheduling() schedulingv1beta1.SchedulingV1beta1Interface { + return &fakeschedulingv1beta1.FakeSchedulingV1beta1{Fake: &c.Fake} +} + +// SettingsV1alpha1 retrieves the SettingsV1alpha1Client +func (c *Clientset) SettingsV1alpha1() settingsv1alpha1.SettingsV1alpha1Interface { + return &fakesettingsv1alpha1.FakeSettingsV1alpha1{Fake: &c.Fake} +} + +// Settings retrieves the SettingsV1alpha1Client +func (c *Clientset) Settings() settingsv1alpha1.SettingsV1alpha1Interface { + return &fakesettingsv1alpha1.FakeSettingsV1alpha1{Fake: &c.Fake} +} + +// StorageV1beta1 retrieves the StorageV1beta1Client +func (c *Clientset) StorageV1beta1() storagev1beta1.StorageV1beta1Interface { + return &fakestoragev1beta1.FakeStorageV1beta1{Fake: &c.Fake} +} + +// StorageV1 retrieves the StorageV1Client +func (c *Clientset) StorageV1() storagev1.StorageV1Interface { + return &fakestoragev1.FakeStorageV1{Fake: &c.Fake} +} + +// Storage retrieves the StorageV1Client +func (c *Clientset) Storage() storagev1.StorageV1Interface { + return &fakestoragev1.FakeStorageV1{Fake: &c.Fake} +} + +// StorageV1alpha1 retrieves the StorageV1alpha1Client +func (c *Clientset) StorageV1alpha1() storagev1alpha1.StorageV1alpha1Interface { + return &fakestoragev1alpha1.FakeStorageV1alpha1{Fake: &c.Fake} +} diff --git a/vendor/k8s.io/client-go/kubernetes/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/fake/doc.go new file mode 100644 index 0000000000..9b99e71670 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// This package has the automatically generated fake clientset. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/fake/register.go b/vendor/k8s.io/client-go/kubernetes/fake/register.go new file mode 100644 index 0000000000..6e1e1fb293 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/fake/register.go @@ -0,0 +1,118 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + admissionregistrationv1alpha1 "k8s.io/api/admissionregistration/v1alpha1" + admissionregistrationv1beta1 "k8s.io/api/admissionregistration/v1beta1" + appsv1 "k8s.io/api/apps/v1" + appsv1beta1 "k8s.io/api/apps/v1beta1" + appsv1beta2 "k8s.io/api/apps/v1beta2" + auditregistrationv1alpha1 "k8s.io/api/auditregistration/v1alpha1" + authenticationv1 "k8s.io/api/authentication/v1" + authenticationv1beta1 "k8s.io/api/authentication/v1beta1" + authorizationv1 "k8s.io/api/authorization/v1" + authorizationv1beta1 "k8s.io/api/authorization/v1beta1" + autoscalingv1 "k8s.io/api/autoscaling/v1" + autoscalingv2beta1 "k8s.io/api/autoscaling/v2beta1" + autoscalingv2beta2 "k8s.io/api/autoscaling/v2beta2" + batchv1 "k8s.io/api/batch/v1" + batchv1beta1 "k8s.io/api/batch/v1beta1" + batchv2alpha1 "k8s.io/api/batch/v2alpha1" + certificatesv1beta1 "k8s.io/api/certificates/v1beta1" + coordinationv1beta1 "k8s.io/api/coordination/v1beta1" + corev1 "k8s.io/api/core/v1" + eventsv1beta1 "k8s.io/api/events/v1beta1" + extensionsv1beta1 "k8s.io/api/extensions/v1beta1" + networkingv1 "k8s.io/api/networking/v1" + policyv1beta1 "k8s.io/api/policy/v1beta1" + rbacv1 "k8s.io/api/rbac/v1" + rbacv1alpha1 "k8s.io/api/rbac/v1alpha1" + rbacv1beta1 "k8s.io/api/rbac/v1beta1" + schedulingv1alpha1 "k8s.io/api/scheduling/v1alpha1" + schedulingv1beta1 "k8s.io/api/scheduling/v1beta1" + settingsv1alpha1 "k8s.io/api/settings/v1alpha1" + storagev1 "k8s.io/api/storage/v1" + storagev1alpha1 "k8s.io/api/storage/v1alpha1" + storagev1beta1 "k8s.io/api/storage/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + runtime "k8s.io/apimachinery/pkg/runtime" + schema "k8s.io/apimachinery/pkg/runtime/schema" + serializer "k8s.io/apimachinery/pkg/runtime/serializer" + utilruntime "k8s.io/apimachinery/pkg/util/runtime" +) + +var scheme = runtime.NewScheme() +var codecs = serializer.NewCodecFactory(scheme) +var parameterCodec = runtime.NewParameterCodec(scheme) +var localSchemeBuilder = runtime.SchemeBuilder{ + admissionregistrationv1alpha1.AddToScheme, + admissionregistrationv1beta1.AddToScheme, + appsv1beta1.AddToScheme, + appsv1beta2.AddToScheme, + appsv1.AddToScheme, + auditregistrationv1alpha1.AddToScheme, + authenticationv1.AddToScheme, + authenticationv1beta1.AddToScheme, + authorizationv1.AddToScheme, + authorizationv1beta1.AddToScheme, + autoscalingv1.AddToScheme, + autoscalingv2beta1.AddToScheme, + autoscalingv2beta2.AddToScheme, + batchv1.AddToScheme, + batchv1beta1.AddToScheme, + batchv2alpha1.AddToScheme, + certificatesv1beta1.AddToScheme, + coordinationv1beta1.AddToScheme, + corev1.AddToScheme, + eventsv1beta1.AddToScheme, + extensionsv1beta1.AddToScheme, + networkingv1.AddToScheme, + policyv1beta1.AddToScheme, + rbacv1.AddToScheme, + rbacv1beta1.AddToScheme, + rbacv1alpha1.AddToScheme, + schedulingv1alpha1.AddToScheme, + schedulingv1beta1.AddToScheme, + settingsv1alpha1.AddToScheme, + storagev1beta1.AddToScheme, + storagev1.AddToScheme, + storagev1alpha1.AddToScheme, +} + +// AddToScheme adds all types of this clientset into the given scheme. This allows composition +// of clientsets, like in: +// +// import ( +// "k8s.io/client-go/kubernetes" +// clientsetscheme "k8s.io/client-go/kubernetes/scheme" +// aggregatorclientsetscheme "k8s.io/kube-aggregator/pkg/client/clientset_generated/clientset/scheme" +// ) +// +// kclientset, _ := kubernetes.NewForConfig(c) +// _ = aggregatorclientsetscheme.AddToScheme(clientsetscheme.Scheme) +// +// After this, RawExtensions in Kubernetes types will serialize kube-aggregator types +// correctly. +var AddToScheme = localSchemeBuilder.AddToScheme + +func init() { + v1.AddToGroupVersion(scheme, schema.GroupVersion{Version: "v1"}) + utilruntime.Must(AddToScheme(scheme)) +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake/fake_admissionregistration_client.go b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake/fake_admissionregistration_client.go new file mode 100644 index 0000000000..8457aec276 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake/fake_admissionregistration_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAdmissionregistrationV1alpha1 struct { + *testing.Fake +} + +func (c *FakeAdmissionregistrationV1alpha1) InitializerConfigurations() v1alpha1.InitializerConfigurationInterface { + return &FakeInitializerConfigurations{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAdmissionregistrationV1alpha1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake/fake_initializerconfiguration.go b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake/fake_initializerconfiguration.go new file mode 100644 index 0000000000..4baee66104 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1alpha1/fake/fake_initializerconfiguration.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/api/admissionregistration/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeInitializerConfigurations implements InitializerConfigurationInterface +type FakeInitializerConfigurations struct { + Fake *FakeAdmissionregistrationV1alpha1 +} + +var initializerconfigurationsResource = schema.GroupVersionResource{Group: "admissionregistration.k8s.io", Version: "v1alpha1", Resource: "initializerconfigurations"} + +var initializerconfigurationsKind = schema.GroupVersionKind{Group: "admissionregistration.k8s.io", Version: "v1alpha1", Kind: "InitializerConfiguration"} + +// Get takes name of the initializerConfiguration, and returns the corresponding initializerConfiguration object, and an error if there is any. +func (c *FakeInitializerConfigurations) Get(name string, options v1.GetOptions) (result *v1alpha1.InitializerConfiguration, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(initializerconfigurationsResource, name), &v1alpha1.InitializerConfiguration{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.InitializerConfiguration), err +} + +// List takes label and field selectors, and returns the list of InitializerConfigurations that match those selectors. +func (c *FakeInitializerConfigurations) List(opts v1.ListOptions) (result *v1alpha1.InitializerConfigurationList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(initializerconfigurationsResource, initializerconfigurationsKind, opts), &v1alpha1.InitializerConfigurationList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1alpha1.InitializerConfigurationList{ListMeta: obj.(*v1alpha1.InitializerConfigurationList).ListMeta} + for _, item := range obj.(*v1alpha1.InitializerConfigurationList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested initializerConfigurations. +func (c *FakeInitializerConfigurations) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(initializerconfigurationsResource, opts)) +} + +// Create takes the representation of a initializerConfiguration and creates it. Returns the server's representation of the initializerConfiguration, and an error, if there is any. +func (c *FakeInitializerConfigurations) Create(initializerConfiguration *v1alpha1.InitializerConfiguration) (result *v1alpha1.InitializerConfiguration, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(initializerconfigurationsResource, initializerConfiguration), &v1alpha1.InitializerConfiguration{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.InitializerConfiguration), err +} + +// Update takes the representation of a initializerConfiguration and updates it. Returns the server's representation of the initializerConfiguration, and an error, if there is any. +func (c *FakeInitializerConfigurations) Update(initializerConfiguration *v1alpha1.InitializerConfiguration) (result *v1alpha1.InitializerConfiguration, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(initializerconfigurationsResource, initializerConfiguration), &v1alpha1.InitializerConfiguration{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.InitializerConfiguration), err +} + +// Delete takes name of the initializerConfiguration and deletes it. Returns an error if one occurs. +func (c *FakeInitializerConfigurations) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(initializerconfigurationsResource, name), &v1alpha1.InitializerConfiguration{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeInitializerConfigurations) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(initializerconfigurationsResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1alpha1.InitializerConfigurationList{}) + return err +} + +// Patch applies the patch and returns the patched initializerConfiguration. +func (c *FakeInitializerConfigurations) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1alpha1.InitializerConfiguration, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(initializerconfigurationsResource, name, pt, data, subresources...), &v1alpha1.InitializerConfiguration{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.InitializerConfiguration), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/fake_admissionregistration_client.go b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/fake_admissionregistration_client.go new file mode 100644 index 0000000000..1a988ddba1 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/fake_admissionregistration_client.go @@ -0,0 +1,44 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAdmissionregistrationV1beta1 struct { + *testing.Fake +} + +func (c *FakeAdmissionregistrationV1beta1) MutatingWebhookConfigurations() v1beta1.MutatingWebhookConfigurationInterface { + return &FakeMutatingWebhookConfigurations{c} +} + +func (c *FakeAdmissionregistrationV1beta1) ValidatingWebhookConfigurations() v1beta1.ValidatingWebhookConfigurationInterface { + return &FakeValidatingWebhookConfigurations{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAdmissionregistrationV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/fake_mutatingwebhookconfiguration.go b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/fake_mutatingwebhookconfiguration.go new file mode 100644 index 0000000000..d2177bad52 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/fake_mutatingwebhookconfiguration.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/admissionregistration/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeMutatingWebhookConfigurations implements MutatingWebhookConfigurationInterface +type FakeMutatingWebhookConfigurations struct { + Fake *FakeAdmissionregistrationV1beta1 +} + +var mutatingwebhookconfigurationsResource = schema.GroupVersionResource{Group: "admissionregistration.k8s.io", Version: "v1beta1", Resource: "mutatingwebhookconfigurations"} + +var mutatingwebhookconfigurationsKind = schema.GroupVersionKind{Group: "admissionregistration.k8s.io", Version: "v1beta1", Kind: "MutatingWebhookConfiguration"} + +// Get takes name of the mutatingWebhookConfiguration, and returns the corresponding mutatingWebhookConfiguration object, and an error if there is any. +func (c *FakeMutatingWebhookConfigurations) Get(name string, options v1.GetOptions) (result *v1beta1.MutatingWebhookConfiguration, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(mutatingwebhookconfigurationsResource, name), &v1beta1.MutatingWebhookConfiguration{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.MutatingWebhookConfiguration), err +} + +// List takes label and field selectors, and returns the list of MutatingWebhookConfigurations that match those selectors. +func (c *FakeMutatingWebhookConfigurations) List(opts v1.ListOptions) (result *v1beta1.MutatingWebhookConfigurationList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(mutatingwebhookconfigurationsResource, mutatingwebhookconfigurationsKind, opts), &v1beta1.MutatingWebhookConfigurationList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.MutatingWebhookConfigurationList{ListMeta: obj.(*v1beta1.MutatingWebhookConfigurationList).ListMeta} + for _, item := range obj.(*v1beta1.MutatingWebhookConfigurationList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested mutatingWebhookConfigurations. +func (c *FakeMutatingWebhookConfigurations) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(mutatingwebhookconfigurationsResource, opts)) +} + +// Create takes the representation of a mutatingWebhookConfiguration and creates it. Returns the server's representation of the mutatingWebhookConfiguration, and an error, if there is any. +func (c *FakeMutatingWebhookConfigurations) Create(mutatingWebhookConfiguration *v1beta1.MutatingWebhookConfiguration) (result *v1beta1.MutatingWebhookConfiguration, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(mutatingwebhookconfigurationsResource, mutatingWebhookConfiguration), &v1beta1.MutatingWebhookConfiguration{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.MutatingWebhookConfiguration), err +} + +// Update takes the representation of a mutatingWebhookConfiguration and updates it. Returns the server's representation of the mutatingWebhookConfiguration, and an error, if there is any. +func (c *FakeMutatingWebhookConfigurations) Update(mutatingWebhookConfiguration *v1beta1.MutatingWebhookConfiguration) (result *v1beta1.MutatingWebhookConfiguration, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(mutatingwebhookconfigurationsResource, mutatingWebhookConfiguration), &v1beta1.MutatingWebhookConfiguration{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.MutatingWebhookConfiguration), err +} + +// Delete takes name of the mutatingWebhookConfiguration and deletes it. Returns an error if one occurs. +func (c *FakeMutatingWebhookConfigurations) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(mutatingwebhookconfigurationsResource, name), &v1beta1.MutatingWebhookConfiguration{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeMutatingWebhookConfigurations) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(mutatingwebhookconfigurationsResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.MutatingWebhookConfigurationList{}) + return err +} + +// Patch applies the patch and returns the patched mutatingWebhookConfiguration. +func (c *FakeMutatingWebhookConfigurations) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.MutatingWebhookConfiguration, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(mutatingwebhookconfigurationsResource, name, pt, data, subresources...), &v1beta1.MutatingWebhookConfiguration{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.MutatingWebhookConfiguration), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/fake_validatingwebhookconfiguration.go b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/fake_validatingwebhookconfiguration.go new file mode 100644 index 0000000000..6be2b39386 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1/fake/fake_validatingwebhookconfiguration.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/admissionregistration/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeValidatingWebhookConfigurations implements ValidatingWebhookConfigurationInterface +type FakeValidatingWebhookConfigurations struct { + Fake *FakeAdmissionregistrationV1beta1 +} + +var validatingwebhookconfigurationsResource = schema.GroupVersionResource{Group: "admissionregistration.k8s.io", Version: "v1beta1", Resource: "validatingwebhookconfigurations"} + +var validatingwebhookconfigurationsKind = schema.GroupVersionKind{Group: "admissionregistration.k8s.io", Version: "v1beta1", Kind: "ValidatingWebhookConfiguration"} + +// Get takes name of the validatingWebhookConfiguration, and returns the corresponding validatingWebhookConfiguration object, and an error if there is any. +func (c *FakeValidatingWebhookConfigurations) Get(name string, options v1.GetOptions) (result *v1beta1.ValidatingWebhookConfiguration, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(validatingwebhookconfigurationsResource, name), &v1beta1.ValidatingWebhookConfiguration{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ValidatingWebhookConfiguration), err +} + +// List takes label and field selectors, and returns the list of ValidatingWebhookConfigurations that match those selectors. +func (c *FakeValidatingWebhookConfigurations) List(opts v1.ListOptions) (result *v1beta1.ValidatingWebhookConfigurationList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(validatingwebhookconfigurationsResource, validatingwebhookconfigurationsKind, opts), &v1beta1.ValidatingWebhookConfigurationList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.ValidatingWebhookConfigurationList{ListMeta: obj.(*v1beta1.ValidatingWebhookConfigurationList).ListMeta} + for _, item := range obj.(*v1beta1.ValidatingWebhookConfigurationList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested validatingWebhookConfigurations. +func (c *FakeValidatingWebhookConfigurations) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(validatingwebhookconfigurationsResource, opts)) +} + +// Create takes the representation of a validatingWebhookConfiguration and creates it. Returns the server's representation of the validatingWebhookConfiguration, and an error, if there is any. +func (c *FakeValidatingWebhookConfigurations) Create(validatingWebhookConfiguration *v1beta1.ValidatingWebhookConfiguration) (result *v1beta1.ValidatingWebhookConfiguration, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(validatingwebhookconfigurationsResource, validatingWebhookConfiguration), &v1beta1.ValidatingWebhookConfiguration{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ValidatingWebhookConfiguration), err +} + +// Update takes the representation of a validatingWebhookConfiguration and updates it. Returns the server's representation of the validatingWebhookConfiguration, and an error, if there is any. +func (c *FakeValidatingWebhookConfigurations) Update(validatingWebhookConfiguration *v1beta1.ValidatingWebhookConfiguration) (result *v1beta1.ValidatingWebhookConfiguration, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(validatingwebhookconfigurationsResource, validatingWebhookConfiguration), &v1beta1.ValidatingWebhookConfiguration{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ValidatingWebhookConfiguration), err +} + +// Delete takes name of the validatingWebhookConfiguration and deletes it. Returns an error if one occurs. +func (c *FakeValidatingWebhookConfigurations) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(validatingwebhookconfigurationsResource, name), &v1beta1.ValidatingWebhookConfiguration{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeValidatingWebhookConfigurations) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(validatingwebhookconfigurationsResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.ValidatingWebhookConfigurationList{}) + return err +} + +// Patch applies the patch and returns the patched validatingWebhookConfiguration. +func (c *FakeValidatingWebhookConfigurations) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.ValidatingWebhookConfiguration, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(validatingwebhookconfigurationsResource, name, pt, data, subresources...), &v1beta1.ValidatingWebhookConfiguration{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ValidatingWebhookConfiguration), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_apps_client.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_apps_client.go new file mode 100644 index 0000000000..458df0fa33 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_apps_client.go @@ -0,0 +1,56 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1 "k8s.io/client-go/kubernetes/typed/apps/v1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAppsV1 struct { + *testing.Fake +} + +func (c *FakeAppsV1) ControllerRevisions(namespace string) v1.ControllerRevisionInterface { + return &FakeControllerRevisions{c, namespace} +} + +func (c *FakeAppsV1) DaemonSets(namespace string) v1.DaemonSetInterface { + return &FakeDaemonSets{c, namespace} +} + +func (c *FakeAppsV1) Deployments(namespace string) v1.DeploymentInterface { + return &FakeDeployments{c, namespace} +} + +func (c *FakeAppsV1) ReplicaSets(namespace string) v1.ReplicaSetInterface { + return &FakeReplicaSets{c, namespace} +} + +func (c *FakeAppsV1) StatefulSets(namespace string) v1.StatefulSetInterface { + return &FakeStatefulSets{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAppsV1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_controllerrevision.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_controllerrevision.go new file mode 100644 index 0000000000..eb38bca41b --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_controllerrevision.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + appsv1 "k8s.io/api/apps/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeControllerRevisions implements ControllerRevisionInterface +type FakeControllerRevisions struct { + Fake *FakeAppsV1 + ns string +} + +var controllerrevisionsResource = schema.GroupVersionResource{Group: "apps", Version: "v1", Resource: "controllerrevisions"} + +var controllerrevisionsKind = schema.GroupVersionKind{Group: "apps", Version: "v1", Kind: "ControllerRevision"} + +// Get takes name of the controllerRevision, and returns the corresponding controllerRevision object, and an error if there is any. +func (c *FakeControllerRevisions) Get(name string, options v1.GetOptions) (result *appsv1.ControllerRevision, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(controllerrevisionsResource, c.ns, name), &appsv1.ControllerRevision{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.ControllerRevision), err +} + +// List takes label and field selectors, and returns the list of ControllerRevisions that match those selectors. +func (c *FakeControllerRevisions) List(opts v1.ListOptions) (result *appsv1.ControllerRevisionList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(controllerrevisionsResource, controllerrevisionsKind, c.ns, opts), &appsv1.ControllerRevisionList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &appsv1.ControllerRevisionList{ListMeta: obj.(*appsv1.ControllerRevisionList).ListMeta} + for _, item := range obj.(*appsv1.ControllerRevisionList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested controllerRevisions. +func (c *FakeControllerRevisions) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(controllerrevisionsResource, c.ns, opts)) + +} + +// Create takes the representation of a controllerRevision and creates it. Returns the server's representation of the controllerRevision, and an error, if there is any. +func (c *FakeControllerRevisions) Create(controllerRevision *appsv1.ControllerRevision) (result *appsv1.ControllerRevision, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(controllerrevisionsResource, c.ns, controllerRevision), &appsv1.ControllerRevision{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.ControllerRevision), err +} + +// Update takes the representation of a controllerRevision and updates it. Returns the server's representation of the controllerRevision, and an error, if there is any. +func (c *FakeControllerRevisions) Update(controllerRevision *appsv1.ControllerRevision) (result *appsv1.ControllerRevision, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(controllerrevisionsResource, c.ns, controllerRevision), &appsv1.ControllerRevision{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.ControllerRevision), err +} + +// Delete takes name of the controllerRevision and deletes it. Returns an error if one occurs. +func (c *FakeControllerRevisions) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(controllerrevisionsResource, c.ns, name), &appsv1.ControllerRevision{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeControllerRevisions) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(controllerrevisionsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &appsv1.ControllerRevisionList{}) + return err +} + +// Patch applies the patch and returns the patched controllerRevision. +func (c *FakeControllerRevisions) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *appsv1.ControllerRevision, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(controllerrevisionsResource, c.ns, name, pt, data, subresources...), &appsv1.ControllerRevision{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.ControllerRevision), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_daemonset.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_daemonset.go new file mode 100644 index 0000000000..c06336e970 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_daemonset.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + appsv1 "k8s.io/api/apps/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeDaemonSets implements DaemonSetInterface +type FakeDaemonSets struct { + Fake *FakeAppsV1 + ns string +} + +var daemonsetsResource = schema.GroupVersionResource{Group: "apps", Version: "v1", Resource: "daemonsets"} + +var daemonsetsKind = schema.GroupVersionKind{Group: "apps", Version: "v1", Kind: "DaemonSet"} + +// Get takes name of the daemonSet, and returns the corresponding daemonSet object, and an error if there is any. +func (c *FakeDaemonSets) Get(name string, options v1.GetOptions) (result *appsv1.DaemonSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(daemonsetsResource, c.ns, name), &appsv1.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.DaemonSet), err +} + +// List takes label and field selectors, and returns the list of DaemonSets that match those selectors. +func (c *FakeDaemonSets) List(opts v1.ListOptions) (result *appsv1.DaemonSetList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(daemonsetsResource, daemonsetsKind, c.ns, opts), &appsv1.DaemonSetList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &appsv1.DaemonSetList{ListMeta: obj.(*appsv1.DaemonSetList).ListMeta} + for _, item := range obj.(*appsv1.DaemonSetList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested daemonSets. +func (c *FakeDaemonSets) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(daemonsetsResource, c.ns, opts)) + +} + +// Create takes the representation of a daemonSet and creates it. Returns the server's representation of the daemonSet, and an error, if there is any. +func (c *FakeDaemonSets) Create(daemonSet *appsv1.DaemonSet) (result *appsv1.DaemonSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(daemonsetsResource, c.ns, daemonSet), &appsv1.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.DaemonSet), err +} + +// Update takes the representation of a daemonSet and updates it. Returns the server's representation of the daemonSet, and an error, if there is any. +func (c *FakeDaemonSets) Update(daemonSet *appsv1.DaemonSet) (result *appsv1.DaemonSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(daemonsetsResource, c.ns, daemonSet), &appsv1.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.DaemonSet), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeDaemonSets) UpdateStatus(daemonSet *appsv1.DaemonSet) (*appsv1.DaemonSet, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(daemonsetsResource, "status", c.ns, daemonSet), &appsv1.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.DaemonSet), err +} + +// Delete takes name of the daemonSet and deletes it. Returns an error if one occurs. +func (c *FakeDaemonSets) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(daemonsetsResource, c.ns, name), &appsv1.DaemonSet{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeDaemonSets) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(daemonsetsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &appsv1.DaemonSetList{}) + return err +} + +// Patch applies the patch and returns the patched daemonSet. +func (c *FakeDaemonSets) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *appsv1.DaemonSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(daemonsetsResource, c.ns, name, pt, data, subresources...), &appsv1.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.DaemonSet), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_deployment.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_deployment.go new file mode 100644 index 0000000000..6a8cb379da --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_deployment.go @@ -0,0 +1,163 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + appsv1 "k8s.io/api/apps/v1" + autoscalingv1 "k8s.io/api/autoscaling/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeDeployments implements DeploymentInterface +type FakeDeployments struct { + Fake *FakeAppsV1 + ns string +} + +var deploymentsResource = schema.GroupVersionResource{Group: "apps", Version: "v1", Resource: "deployments"} + +var deploymentsKind = schema.GroupVersionKind{Group: "apps", Version: "v1", Kind: "Deployment"} + +// Get takes name of the deployment, and returns the corresponding deployment object, and an error if there is any. +func (c *FakeDeployments) Get(name string, options v1.GetOptions) (result *appsv1.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(deploymentsResource, c.ns, name), &appsv1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.Deployment), err +} + +// List takes label and field selectors, and returns the list of Deployments that match those selectors. +func (c *FakeDeployments) List(opts v1.ListOptions) (result *appsv1.DeploymentList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(deploymentsResource, deploymentsKind, c.ns, opts), &appsv1.DeploymentList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &appsv1.DeploymentList{ListMeta: obj.(*appsv1.DeploymentList).ListMeta} + for _, item := range obj.(*appsv1.DeploymentList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested deployments. +func (c *FakeDeployments) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(deploymentsResource, c.ns, opts)) + +} + +// Create takes the representation of a deployment and creates it. Returns the server's representation of the deployment, and an error, if there is any. +func (c *FakeDeployments) Create(deployment *appsv1.Deployment) (result *appsv1.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(deploymentsResource, c.ns, deployment), &appsv1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.Deployment), err +} + +// Update takes the representation of a deployment and updates it. Returns the server's representation of the deployment, and an error, if there is any. +func (c *FakeDeployments) Update(deployment *appsv1.Deployment) (result *appsv1.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(deploymentsResource, c.ns, deployment), &appsv1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.Deployment), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeDeployments) UpdateStatus(deployment *appsv1.Deployment) (*appsv1.Deployment, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(deploymentsResource, "status", c.ns, deployment), &appsv1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.Deployment), err +} + +// Delete takes name of the deployment and deletes it. Returns an error if one occurs. +func (c *FakeDeployments) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(deploymentsResource, c.ns, name), &appsv1.Deployment{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeDeployments) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(deploymentsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &appsv1.DeploymentList{}) + return err +} + +// Patch applies the patch and returns the patched deployment. +func (c *FakeDeployments) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *appsv1.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(deploymentsResource, c.ns, name, pt, data, subresources...), &appsv1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.Deployment), err +} + +// GetScale takes name of the deployment, and returns the corresponding scale object, and an error if there is any. +func (c *FakeDeployments) GetScale(deploymentName string, options v1.GetOptions) (result *autoscalingv1.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetSubresourceAction(deploymentsResource, c.ns, "scale", deploymentName), &autoscalingv1.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.Scale), err +} + +// UpdateScale takes the representation of a scale and updates it. Returns the server's representation of the scale, and an error, if there is any. +func (c *FakeDeployments) UpdateScale(deploymentName string, scale *autoscalingv1.Scale) (result *autoscalingv1.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(deploymentsResource, "scale", c.ns, scale), &autoscalingv1.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.Scale), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_replicaset.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_replicaset.go new file mode 100644 index 0000000000..e871f82f76 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_replicaset.go @@ -0,0 +1,163 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + appsv1 "k8s.io/api/apps/v1" + autoscalingv1 "k8s.io/api/autoscaling/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeReplicaSets implements ReplicaSetInterface +type FakeReplicaSets struct { + Fake *FakeAppsV1 + ns string +} + +var replicasetsResource = schema.GroupVersionResource{Group: "apps", Version: "v1", Resource: "replicasets"} + +var replicasetsKind = schema.GroupVersionKind{Group: "apps", Version: "v1", Kind: "ReplicaSet"} + +// Get takes name of the replicaSet, and returns the corresponding replicaSet object, and an error if there is any. +func (c *FakeReplicaSets) Get(name string, options v1.GetOptions) (result *appsv1.ReplicaSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(replicasetsResource, c.ns, name), &appsv1.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.ReplicaSet), err +} + +// List takes label and field selectors, and returns the list of ReplicaSets that match those selectors. +func (c *FakeReplicaSets) List(opts v1.ListOptions) (result *appsv1.ReplicaSetList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(replicasetsResource, replicasetsKind, c.ns, opts), &appsv1.ReplicaSetList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &appsv1.ReplicaSetList{ListMeta: obj.(*appsv1.ReplicaSetList).ListMeta} + for _, item := range obj.(*appsv1.ReplicaSetList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested replicaSets. +func (c *FakeReplicaSets) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(replicasetsResource, c.ns, opts)) + +} + +// Create takes the representation of a replicaSet and creates it. Returns the server's representation of the replicaSet, and an error, if there is any. +func (c *FakeReplicaSets) Create(replicaSet *appsv1.ReplicaSet) (result *appsv1.ReplicaSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(replicasetsResource, c.ns, replicaSet), &appsv1.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.ReplicaSet), err +} + +// Update takes the representation of a replicaSet and updates it. Returns the server's representation of the replicaSet, and an error, if there is any. +func (c *FakeReplicaSets) Update(replicaSet *appsv1.ReplicaSet) (result *appsv1.ReplicaSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(replicasetsResource, c.ns, replicaSet), &appsv1.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.ReplicaSet), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeReplicaSets) UpdateStatus(replicaSet *appsv1.ReplicaSet) (*appsv1.ReplicaSet, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(replicasetsResource, "status", c.ns, replicaSet), &appsv1.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.ReplicaSet), err +} + +// Delete takes name of the replicaSet and deletes it. Returns an error if one occurs. +func (c *FakeReplicaSets) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(replicasetsResource, c.ns, name), &appsv1.ReplicaSet{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeReplicaSets) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(replicasetsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &appsv1.ReplicaSetList{}) + return err +} + +// Patch applies the patch and returns the patched replicaSet. +func (c *FakeReplicaSets) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *appsv1.ReplicaSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(replicasetsResource, c.ns, name, pt, data, subresources...), &appsv1.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.ReplicaSet), err +} + +// GetScale takes name of the replicaSet, and returns the corresponding scale object, and an error if there is any. +func (c *FakeReplicaSets) GetScale(replicaSetName string, options v1.GetOptions) (result *autoscalingv1.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetSubresourceAction(replicasetsResource, c.ns, "scale", replicaSetName), &autoscalingv1.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.Scale), err +} + +// UpdateScale takes the representation of a scale and updates it. Returns the server's representation of the scale, and an error, if there is any. +func (c *FakeReplicaSets) UpdateScale(replicaSetName string, scale *autoscalingv1.Scale) (result *autoscalingv1.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(replicasetsResource, "scale", c.ns, scale), &autoscalingv1.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.Scale), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_statefulset.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_statefulset.go new file mode 100644 index 0000000000..83e80bff49 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1/fake/fake_statefulset.go @@ -0,0 +1,163 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + appsv1 "k8s.io/api/apps/v1" + autoscalingv1 "k8s.io/api/autoscaling/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeStatefulSets implements StatefulSetInterface +type FakeStatefulSets struct { + Fake *FakeAppsV1 + ns string +} + +var statefulsetsResource = schema.GroupVersionResource{Group: "apps", Version: "v1", Resource: "statefulsets"} + +var statefulsetsKind = schema.GroupVersionKind{Group: "apps", Version: "v1", Kind: "StatefulSet"} + +// Get takes name of the statefulSet, and returns the corresponding statefulSet object, and an error if there is any. +func (c *FakeStatefulSets) Get(name string, options v1.GetOptions) (result *appsv1.StatefulSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(statefulsetsResource, c.ns, name), &appsv1.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.StatefulSet), err +} + +// List takes label and field selectors, and returns the list of StatefulSets that match those selectors. +func (c *FakeStatefulSets) List(opts v1.ListOptions) (result *appsv1.StatefulSetList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(statefulsetsResource, statefulsetsKind, c.ns, opts), &appsv1.StatefulSetList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &appsv1.StatefulSetList{ListMeta: obj.(*appsv1.StatefulSetList).ListMeta} + for _, item := range obj.(*appsv1.StatefulSetList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested statefulSets. +func (c *FakeStatefulSets) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(statefulsetsResource, c.ns, opts)) + +} + +// Create takes the representation of a statefulSet and creates it. Returns the server's representation of the statefulSet, and an error, if there is any. +func (c *FakeStatefulSets) Create(statefulSet *appsv1.StatefulSet) (result *appsv1.StatefulSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(statefulsetsResource, c.ns, statefulSet), &appsv1.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.StatefulSet), err +} + +// Update takes the representation of a statefulSet and updates it. Returns the server's representation of the statefulSet, and an error, if there is any. +func (c *FakeStatefulSets) Update(statefulSet *appsv1.StatefulSet) (result *appsv1.StatefulSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(statefulsetsResource, c.ns, statefulSet), &appsv1.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.StatefulSet), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeStatefulSets) UpdateStatus(statefulSet *appsv1.StatefulSet) (*appsv1.StatefulSet, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(statefulsetsResource, "status", c.ns, statefulSet), &appsv1.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.StatefulSet), err +} + +// Delete takes name of the statefulSet and deletes it. Returns an error if one occurs. +func (c *FakeStatefulSets) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(statefulsetsResource, c.ns, name), &appsv1.StatefulSet{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeStatefulSets) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(statefulsetsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &appsv1.StatefulSetList{}) + return err +} + +// Patch applies the patch and returns the patched statefulSet. +func (c *FakeStatefulSets) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *appsv1.StatefulSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(statefulsetsResource, c.ns, name, pt, data, subresources...), &appsv1.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*appsv1.StatefulSet), err +} + +// GetScale takes name of the statefulSet, and returns the corresponding scale object, and an error if there is any. +func (c *FakeStatefulSets) GetScale(statefulSetName string, options v1.GetOptions) (result *autoscalingv1.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetSubresourceAction(statefulsetsResource, c.ns, "scale", statefulSetName), &autoscalingv1.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.Scale), err +} + +// UpdateScale takes the representation of a scale and updates it. Returns the server's representation of the scale, and an error, if there is any. +func (c *FakeStatefulSets) UpdateScale(statefulSetName string, scale *autoscalingv1.Scale) (result *autoscalingv1.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(statefulsetsResource, "scale", c.ns, scale), &autoscalingv1.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.Scale), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_apps_client.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_apps_client.go new file mode 100644 index 0000000000..8e65d78d29 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_apps_client.go @@ -0,0 +1,48 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/apps/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAppsV1beta1 struct { + *testing.Fake +} + +func (c *FakeAppsV1beta1) ControllerRevisions(namespace string) v1beta1.ControllerRevisionInterface { + return &FakeControllerRevisions{c, namespace} +} + +func (c *FakeAppsV1beta1) Deployments(namespace string) v1beta1.DeploymentInterface { + return &FakeDeployments{c, namespace} +} + +func (c *FakeAppsV1beta1) StatefulSets(namespace string) v1beta1.StatefulSetInterface { + return &FakeStatefulSets{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAppsV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_controllerrevision.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_controllerrevision.go new file mode 100644 index 0000000000..8e339d78b0 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_controllerrevision.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/apps/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeControllerRevisions implements ControllerRevisionInterface +type FakeControllerRevisions struct { + Fake *FakeAppsV1beta1 + ns string +} + +var controllerrevisionsResource = schema.GroupVersionResource{Group: "apps", Version: "v1beta1", Resource: "controllerrevisions"} + +var controllerrevisionsKind = schema.GroupVersionKind{Group: "apps", Version: "v1beta1", Kind: "ControllerRevision"} + +// Get takes name of the controllerRevision, and returns the corresponding controllerRevision object, and an error if there is any. +func (c *FakeControllerRevisions) Get(name string, options v1.GetOptions) (result *v1beta1.ControllerRevision, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(controllerrevisionsResource, c.ns, name), &v1beta1.ControllerRevision{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ControllerRevision), err +} + +// List takes label and field selectors, and returns the list of ControllerRevisions that match those selectors. +func (c *FakeControllerRevisions) List(opts v1.ListOptions) (result *v1beta1.ControllerRevisionList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(controllerrevisionsResource, controllerrevisionsKind, c.ns, opts), &v1beta1.ControllerRevisionList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.ControllerRevisionList{ListMeta: obj.(*v1beta1.ControllerRevisionList).ListMeta} + for _, item := range obj.(*v1beta1.ControllerRevisionList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested controllerRevisions. +func (c *FakeControllerRevisions) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(controllerrevisionsResource, c.ns, opts)) + +} + +// Create takes the representation of a controllerRevision and creates it. Returns the server's representation of the controllerRevision, and an error, if there is any. +func (c *FakeControllerRevisions) Create(controllerRevision *v1beta1.ControllerRevision) (result *v1beta1.ControllerRevision, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(controllerrevisionsResource, c.ns, controllerRevision), &v1beta1.ControllerRevision{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ControllerRevision), err +} + +// Update takes the representation of a controllerRevision and updates it. Returns the server's representation of the controllerRevision, and an error, if there is any. +func (c *FakeControllerRevisions) Update(controllerRevision *v1beta1.ControllerRevision) (result *v1beta1.ControllerRevision, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(controllerrevisionsResource, c.ns, controllerRevision), &v1beta1.ControllerRevision{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ControllerRevision), err +} + +// Delete takes name of the controllerRevision and deletes it. Returns an error if one occurs. +func (c *FakeControllerRevisions) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(controllerrevisionsResource, c.ns, name), &v1beta1.ControllerRevision{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeControllerRevisions) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(controllerrevisionsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.ControllerRevisionList{}) + return err +} + +// Patch applies the patch and returns the patched controllerRevision. +func (c *FakeControllerRevisions) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.ControllerRevision, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(controllerrevisionsResource, c.ns, name, pt, data, subresources...), &v1beta1.ControllerRevision{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ControllerRevision), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_deployment.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_deployment.go new file mode 100644 index 0000000000..c33baba589 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_deployment.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/apps/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeDeployments implements DeploymentInterface +type FakeDeployments struct { + Fake *FakeAppsV1beta1 + ns string +} + +var deploymentsResource = schema.GroupVersionResource{Group: "apps", Version: "v1beta1", Resource: "deployments"} + +var deploymentsKind = schema.GroupVersionKind{Group: "apps", Version: "v1beta1", Kind: "Deployment"} + +// Get takes name of the deployment, and returns the corresponding deployment object, and an error if there is any. +func (c *FakeDeployments) Get(name string, options v1.GetOptions) (result *v1beta1.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(deploymentsResource, c.ns, name), &v1beta1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Deployment), err +} + +// List takes label and field selectors, and returns the list of Deployments that match those selectors. +func (c *FakeDeployments) List(opts v1.ListOptions) (result *v1beta1.DeploymentList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(deploymentsResource, deploymentsKind, c.ns, opts), &v1beta1.DeploymentList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.DeploymentList{ListMeta: obj.(*v1beta1.DeploymentList).ListMeta} + for _, item := range obj.(*v1beta1.DeploymentList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested deployments. +func (c *FakeDeployments) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(deploymentsResource, c.ns, opts)) + +} + +// Create takes the representation of a deployment and creates it. Returns the server's representation of the deployment, and an error, if there is any. +func (c *FakeDeployments) Create(deployment *v1beta1.Deployment) (result *v1beta1.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(deploymentsResource, c.ns, deployment), &v1beta1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Deployment), err +} + +// Update takes the representation of a deployment and updates it. Returns the server's representation of the deployment, and an error, if there is any. +func (c *FakeDeployments) Update(deployment *v1beta1.Deployment) (result *v1beta1.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(deploymentsResource, c.ns, deployment), &v1beta1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Deployment), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeDeployments) UpdateStatus(deployment *v1beta1.Deployment) (*v1beta1.Deployment, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(deploymentsResource, "status", c.ns, deployment), &v1beta1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Deployment), err +} + +// Delete takes name of the deployment and deletes it. Returns an error if one occurs. +func (c *FakeDeployments) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(deploymentsResource, c.ns, name), &v1beta1.Deployment{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeDeployments) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(deploymentsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.DeploymentList{}) + return err +} + +// Patch applies the patch and returns the patched deployment. +func (c *FakeDeployments) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(deploymentsResource, c.ns, name, pt, data, subresources...), &v1beta1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Deployment), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_statefulset.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_statefulset.go new file mode 100644 index 0000000000..754da5fba6 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta1/fake/fake_statefulset.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/apps/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeStatefulSets implements StatefulSetInterface +type FakeStatefulSets struct { + Fake *FakeAppsV1beta1 + ns string +} + +var statefulsetsResource = schema.GroupVersionResource{Group: "apps", Version: "v1beta1", Resource: "statefulsets"} + +var statefulsetsKind = schema.GroupVersionKind{Group: "apps", Version: "v1beta1", Kind: "StatefulSet"} + +// Get takes name of the statefulSet, and returns the corresponding statefulSet object, and an error if there is any. +func (c *FakeStatefulSets) Get(name string, options v1.GetOptions) (result *v1beta1.StatefulSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(statefulsetsResource, c.ns, name), &v1beta1.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.StatefulSet), err +} + +// List takes label and field selectors, and returns the list of StatefulSets that match those selectors. +func (c *FakeStatefulSets) List(opts v1.ListOptions) (result *v1beta1.StatefulSetList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(statefulsetsResource, statefulsetsKind, c.ns, opts), &v1beta1.StatefulSetList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.StatefulSetList{ListMeta: obj.(*v1beta1.StatefulSetList).ListMeta} + for _, item := range obj.(*v1beta1.StatefulSetList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested statefulSets. +func (c *FakeStatefulSets) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(statefulsetsResource, c.ns, opts)) + +} + +// Create takes the representation of a statefulSet and creates it. Returns the server's representation of the statefulSet, and an error, if there is any. +func (c *FakeStatefulSets) Create(statefulSet *v1beta1.StatefulSet) (result *v1beta1.StatefulSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(statefulsetsResource, c.ns, statefulSet), &v1beta1.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.StatefulSet), err +} + +// Update takes the representation of a statefulSet and updates it. Returns the server's representation of the statefulSet, and an error, if there is any. +func (c *FakeStatefulSets) Update(statefulSet *v1beta1.StatefulSet) (result *v1beta1.StatefulSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(statefulsetsResource, c.ns, statefulSet), &v1beta1.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.StatefulSet), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeStatefulSets) UpdateStatus(statefulSet *v1beta1.StatefulSet) (*v1beta1.StatefulSet, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(statefulsetsResource, "status", c.ns, statefulSet), &v1beta1.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.StatefulSet), err +} + +// Delete takes name of the statefulSet and deletes it. Returns an error if one occurs. +func (c *FakeStatefulSets) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(statefulsetsResource, c.ns, name), &v1beta1.StatefulSet{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeStatefulSets) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(statefulsetsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.StatefulSetList{}) + return err +} + +// Patch applies the patch and returns the patched statefulSet. +func (c *FakeStatefulSets) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.StatefulSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(statefulsetsResource, c.ns, name, pt, data, subresources...), &v1beta1.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.StatefulSet), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_apps_client.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_apps_client.go new file mode 100644 index 0000000000..0ec34a2cdb --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_apps_client.go @@ -0,0 +1,56 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta2 "k8s.io/client-go/kubernetes/typed/apps/v1beta2" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAppsV1beta2 struct { + *testing.Fake +} + +func (c *FakeAppsV1beta2) ControllerRevisions(namespace string) v1beta2.ControllerRevisionInterface { + return &FakeControllerRevisions{c, namespace} +} + +func (c *FakeAppsV1beta2) DaemonSets(namespace string) v1beta2.DaemonSetInterface { + return &FakeDaemonSets{c, namespace} +} + +func (c *FakeAppsV1beta2) Deployments(namespace string) v1beta2.DeploymentInterface { + return &FakeDeployments{c, namespace} +} + +func (c *FakeAppsV1beta2) ReplicaSets(namespace string) v1beta2.ReplicaSetInterface { + return &FakeReplicaSets{c, namespace} +} + +func (c *FakeAppsV1beta2) StatefulSets(namespace string) v1beta2.StatefulSetInterface { + return &FakeStatefulSets{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAppsV1beta2) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_controllerrevision.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_controllerrevision.go new file mode 100644 index 0000000000..197f190cbd --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_controllerrevision.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta2 "k8s.io/api/apps/v1beta2" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeControllerRevisions implements ControllerRevisionInterface +type FakeControllerRevisions struct { + Fake *FakeAppsV1beta2 + ns string +} + +var controllerrevisionsResource = schema.GroupVersionResource{Group: "apps", Version: "v1beta2", Resource: "controllerrevisions"} + +var controllerrevisionsKind = schema.GroupVersionKind{Group: "apps", Version: "v1beta2", Kind: "ControllerRevision"} + +// Get takes name of the controllerRevision, and returns the corresponding controllerRevision object, and an error if there is any. +func (c *FakeControllerRevisions) Get(name string, options v1.GetOptions) (result *v1beta2.ControllerRevision, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(controllerrevisionsResource, c.ns, name), &v1beta2.ControllerRevision{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.ControllerRevision), err +} + +// List takes label and field selectors, and returns the list of ControllerRevisions that match those selectors. +func (c *FakeControllerRevisions) List(opts v1.ListOptions) (result *v1beta2.ControllerRevisionList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(controllerrevisionsResource, controllerrevisionsKind, c.ns, opts), &v1beta2.ControllerRevisionList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta2.ControllerRevisionList{ListMeta: obj.(*v1beta2.ControllerRevisionList).ListMeta} + for _, item := range obj.(*v1beta2.ControllerRevisionList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested controllerRevisions. +func (c *FakeControllerRevisions) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(controllerrevisionsResource, c.ns, opts)) + +} + +// Create takes the representation of a controllerRevision and creates it. Returns the server's representation of the controllerRevision, and an error, if there is any. +func (c *FakeControllerRevisions) Create(controllerRevision *v1beta2.ControllerRevision) (result *v1beta2.ControllerRevision, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(controllerrevisionsResource, c.ns, controllerRevision), &v1beta2.ControllerRevision{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.ControllerRevision), err +} + +// Update takes the representation of a controllerRevision and updates it. Returns the server's representation of the controllerRevision, and an error, if there is any. +func (c *FakeControllerRevisions) Update(controllerRevision *v1beta2.ControllerRevision) (result *v1beta2.ControllerRevision, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(controllerrevisionsResource, c.ns, controllerRevision), &v1beta2.ControllerRevision{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.ControllerRevision), err +} + +// Delete takes name of the controllerRevision and deletes it. Returns an error if one occurs. +func (c *FakeControllerRevisions) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(controllerrevisionsResource, c.ns, name), &v1beta2.ControllerRevision{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeControllerRevisions) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(controllerrevisionsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta2.ControllerRevisionList{}) + return err +} + +// Patch applies the patch and returns the patched controllerRevision. +func (c *FakeControllerRevisions) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta2.ControllerRevision, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(controllerrevisionsResource, c.ns, name, pt, data, subresources...), &v1beta2.ControllerRevision{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.ControllerRevision), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_daemonset.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_daemonset.go new file mode 100644 index 0000000000..b50747fdc9 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_daemonset.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta2 "k8s.io/api/apps/v1beta2" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeDaemonSets implements DaemonSetInterface +type FakeDaemonSets struct { + Fake *FakeAppsV1beta2 + ns string +} + +var daemonsetsResource = schema.GroupVersionResource{Group: "apps", Version: "v1beta2", Resource: "daemonsets"} + +var daemonsetsKind = schema.GroupVersionKind{Group: "apps", Version: "v1beta2", Kind: "DaemonSet"} + +// Get takes name of the daemonSet, and returns the corresponding daemonSet object, and an error if there is any. +func (c *FakeDaemonSets) Get(name string, options v1.GetOptions) (result *v1beta2.DaemonSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(daemonsetsResource, c.ns, name), &v1beta2.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.DaemonSet), err +} + +// List takes label and field selectors, and returns the list of DaemonSets that match those selectors. +func (c *FakeDaemonSets) List(opts v1.ListOptions) (result *v1beta2.DaemonSetList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(daemonsetsResource, daemonsetsKind, c.ns, opts), &v1beta2.DaemonSetList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta2.DaemonSetList{ListMeta: obj.(*v1beta2.DaemonSetList).ListMeta} + for _, item := range obj.(*v1beta2.DaemonSetList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested daemonSets. +func (c *FakeDaemonSets) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(daemonsetsResource, c.ns, opts)) + +} + +// Create takes the representation of a daemonSet and creates it. Returns the server's representation of the daemonSet, and an error, if there is any. +func (c *FakeDaemonSets) Create(daemonSet *v1beta2.DaemonSet) (result *v1beta2.DaemonSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(daemonsetsResource, c.ns, daemonSet), &v1beta2.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.DaemonSet), err +} + +// Update takes the representation of a daemonSet and updates it. Returns the server's representation of the daemonSet, and an error, if there is any. +func (c *FakeDaemonSets) Update(daemonSet *v1beta2.DaemonSet) (result *v1beta2.DaemonSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(daemonsetsResource, c.ns, daemonSet), &v1beta2.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.DaemonSet), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeDaemonSets) UpdateStatus(daemonSet *v1beta2.DaemonSet) (*v1beta2.DaemonSet, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(daemonsetsResource, "status", c.ns, daemonSet), &v1beta2.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.DaemonSet), err +} + +// Delete takes name of the daemonSet and deletes it. Returns an error if one occurs. +func (c *FakeDaemonSets) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(daemonsetsResource, c.ns, name), &v1beta2.DaemonSet{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeDaemonSets) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(daemonsetsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta2.DaemonSetList{}) + return err +} + +// Patch applies the patch and returns the patched daemonSet. +func (c *FakeDaemonSets) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta2.DaemonSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(daemonsetsResource, c.ns, name, pt, data, subresources...), &v1beta2.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.DaemonSet), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_deployment.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_deployment.go new file mode 100644 index 0000000000..b74d24ed7c --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_deployment.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta2 "k8s.io/api/apps/v1beta2" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeDeployments implements DeploymentInterface +type FakeDeployments struct { + Fake *FakeAppsV1beta2 + ns string +} + +var deploymentsResource = schema.GroupVersionResource{Group: "apps", Version: "v1beta2", Resource: "deployments"} + +var deploymentsKind = schema.GroupVersionKind{Group: "apps", Version: "v1beta2", Kind: "Deployment"} + +// Get takes name of the deployment, and returns the corresponding deployment object, and an error if there is any. +func (c *FakeDeployments) Get(name string, options v1.GetOptions) (result *v1beta2.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(deploymentsResource, c.ns, name), &v1beta2.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.Deployment), err +} + +// List takes label and field selectors, and returns the list of Deployments that match those selectors. +func (c *FakeDeployments) List(opts v1.ListOptions) (result *v1beta2.DeploymentList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(deploymentsResource, deploymentsKind, c.ns, opts), &v1beta2.DeploymentList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta2.DeploymentList{ListMeta: obj.(*v1beta2.DeploymentList).ListMeta} + for _, item := range obj.(*v1beta2.DeploymentList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested deployments. +func (c *FakeDeployments) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(deploymentsResource, c.ns, opts)) + +} + +// Create takes the representation of a deployment and creates it. Returns the server's representation of the deployment, and an error, if there is any. +func (c *FakeDeployments) Create(deployment *v1beta2.Deployment) (result *v1beta2.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(deploymentsResource, c.ns, deployment), &v1beta2.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.Deployment), err +} + +// Update takes the representation of a deployment and updates it. Returns the server's representation of the deployment, and an error, if there is any. +func (c *FakeDeployments) Update(deployment *v1beta2.Deployment) (result *v1beta2.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(deploymentsResource, c.ns, deployment), &v1beta2.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.Deployment), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeDeployments) UpdateStatus(deployment *v1beta2.Deployment) (*v1beta2.Deployment, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(deploymentsResource, "status", c.ns, deployment), &v1beta2.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.Deployment), err +} + +// Delete takes name of the deployment and deletes it. Returns an error if one occurs. +func (c *FakeDeployments) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(deploymentsResource, c.ns, name), &v1beta2.Deployment{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeDeployments) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(deploymentsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta2.DeploymentList{}) + return err +} + +// Patch applies the patch and returns the patched deployment. +func (c *FakeDeployments) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta2.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(deploymentsResource, c.ns, name, pt, data, subresources...), &v1beta2.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.Deployment), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_replicaset.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_replicaset.go new file mode 100644 index 0000000000..ba1de33ecf --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_replicaset.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta2 "k8s.io/api/apps/v1beta2" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeReplicaSets implements ReplicaSetInterface +type FakeReplicaSets struct { + Fake *FakeAppsV1beta2 + ns string +} + +var replicasetsResource = schema.GroupVersionResource{Group: "apps", Version: "v1beta2", Resource: "replicasets"} + +var replicasetsKind = schema.GroupVersionKind{Group: "apps", Version: "v1beta2", Kind: "ReplicaSet"} + +// Get takes name of the replicaSet, and returns the corresponding replicaSet object, and an error if there is any. +func (c *FakeReplicaSets) Get(name string, options v1.GetOptions) (result *v1beta2.ReplicaSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(replicasetsResource, c.ns, name), &v1beta2.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.ReplicaSet), err +} + +// List takes label and field selectors, and returns the list of ReplicaSets that match those selectors. +func (c *FakeReplicaSets) List(opts v1.ListOptions) (result *v1beta2.ReplicaSetList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(replicasetsResource, replicasetsKind, c.ns, opts), &v1beta2.ReplicaSetList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta2.ReplicaSetList{ListMeta: obj.(*v1beta2.ReplicaSetList).ListMeta} + for _, item := range obj.(*v1beta2.ReplicaSetList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested replicaSets. +func (c *FakeReplicaSets) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(replicasetsResource, c.ns, opts)) + +} + +// Create takes the representation of a replicaSet and creates it. Returns the server's representation of the replicaSet, and an error, if there is any. +func (c *FakeReplicaSets) Create(replicaSet *v1beta2.ReplicaSet) (result *v1beta2.ReplicaSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(replicasetsResource, c.ns, replicaSet), &v1beta2.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.ReplicaSet), err +} + +// Update takes the representation of a replicaSet and updates it. Returns the server's representation of the replicaSet, and an error, if there is any. +func (c *FakeReplicaSets) Update(replicaSet *v1beta2.ReplicaSet) (result *v1beta2.ReplicaSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(replicasetsResource, c.ns, replicaSet), &v1beta2.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.ReplicaSet), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeReplicaSets) UpdateStatus(replicaSet *v1beta2.ReplicaSet) (*v1beta2.ReplicaSet, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(replicasetsResource, "status", c.ns, replicaSet), &v1beta2.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.ReplicaSet), err +} + +// Delete takes name of the replicaSet and deletes it. Returns an error if one occurs. +func (c *FakeReplicaSets) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(replicasetsResource, c.ns, name), &v1beta2.ReplicaSet{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeReplicaSets) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(replicasetsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta2.ReplicaSetList{}) + return err +} + +// Patch applies the patch and returns the patched replicaSet. +func (c *FakeReplicaSets) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta2.ReplicaSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(replicasetsResource, c.ns, name, pt, data, subresources...), &v1beta2.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.ReplicaSet), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_statefulset.go b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_statefulset.go new file mode 100644 index 0000000000..652c7cbc5d --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/apps/v1beta2/fake/fake_statefulset.go @@ -0,0 +1,162 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta2 "k8s.io/api/apps/v1beta2" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeStatefulSets implements StatefulSetInterface +type FakeStatefulSets struct { + Fake *FakeAppsV1beta2 + ns string +} + +var statefulsetsResource = schema.GroupVersionResource{Group: "apps", Version: "v1beta2", Resource: "statefulsets"} + +var statefulsetsKind = schema.GroupVersionKind{Group: "apps", Version: "v1beta2", Kind: "StatefulSet"} + +// Get takes name of the statefulSet, and returns the corresponding statefulSet object, and an error if there is any. +func (c *FakeStatefulSets) Get(name string, options v1.GetOptions) (result *v1beta2.StatefulSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(statefulsetsResource, c.ns, name), &v1beta2.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.StatefulSet), err +} + +// List takes label and field selectors, and returns the list of StatefulSets that match those selectors. +func (c *FakeStatefulSets) List(opts v1.ListOptions) (result *v1beta2.StatefulSetList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(statefulsetsResource, statefulsetsKind, c.ns, opts), &v1beta2.StatefulSetList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta2.StatefulSetList{ListMeta: obj.(*v1beta2.StatefulSetList).ListMeta} + for _, item := range obj.(*v1beta2.StatefulSetList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested statefulSets. +func (c *FakeStatefulSets) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(statefulsetsResource, c.ns, opts)) + +} + +// Create takes the representation of a statefulSet and creates it. Returns the server's representation of the statefulSet, and an error, if there is any. +func (c *FakeStatefulSets) Create(statefulSet *v1beta2.StatefulSet) (result *v1beta2.StatefulSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(statefulsetsResource, c.ns, statefulSet), &v1beta2.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.StatefulSet), err +} + +// Update takes the representation of a statefulSet and updates it. Returns the server's representation of the statefulSet, and an error, if there is any. +func (c *FakeStatefulSets) Update(statefulSet *v1beta2.StatefulSet) (result *v1beta2.StatefulSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(statefulsetsResource, c.ns, statefulSet), &v1beta2.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.StatefulSet), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeStatefulSets) UpdateStatus(statefulSet *v1beta2.StatefulSet) (*v1beta2.StatefulSet, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(statefulsetsResource, "status", c.ns, statefulSet), &v1beta2.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.StatefulSet), err +} + +// Delete takes name of the statefulSet and deletes it. Returns an error if one occurs. +func (c *FakeStatefulSets) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(statefulsetsResource, c.ns, name), &v1beta2.StatefulSet{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeStatefulSets) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(statefulsetsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta2.StatefulSetList{}) + return err +} + +// Patch applies the patch and returns the patched statefulSet. +func (c *FakeStatefulSets) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta2.StatefulSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(statefulsetsResource, c.ns, name, pt, data, subresources...), &v1beta2.StatefulSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.StatefulSet), err +} + +// GetScale takes name of the statefulSet, and returns the corresponding scale object, and an error if there is any. +func (c *FakeStatefulSets) GetScale(statefulSetName string, options v1.GetOptions) (result *v1beta2.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetSubresourceAction(statefulsetsResource, c.ns, "scale", statefulSetName), &v1beta2.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.Scale), err +} + +// UpdateScale takes the representation of a scale and updates it. Returns the server's representation of the scale, and an error, if there is any. +func (c *FakeStatefulSets) UpdateScale(statefulSetName string, scale *v1beta2.Scale) (result *v1beta2.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(statefulsetsResource, "scale", c.ns, scale), &v1beta2.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta2.Scale), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake/fake_auditregistration_client.go b/vendor/k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake/fake_auditregistration_client.go new file mode 100644 index 0000000000..c22acabcf4 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake/fake_auditregistration_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAuditregistrationV1alpha1 struct { + *testing.Fake +} + +func (c *FakeAuditregistrationV1alpha1) AuditSinks() v1alpha1.AuditSinkInterface { + return &FakeAuditSinks{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAuditregistrationV1alpha1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake/fake_auditsink.go b/vendor/k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake/fake_auditsink.go new file mode 100644 index 0000000000..d0bb9fd000 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/auditregistration/v1alpha1/fake/fake_auditsink.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/api/auditregistration/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeAuditSinks implements AuditSinkInterface +type FakeAuditSinks struct { + Fake *FakeAuditregistrationV1alpha1 +} + +var auditsinksResource = schema.GroupVersionResource{Group: "auditregistration.k8s.io", Version: "v1alpha1", Resource: "auditsinks"} + +var auditsinksKind = schema.GroupVersionKind{Group: "auditregistration.k8s.io", Version: "v1alpha1", Kind: "AuditSink"} + +// Get takes name of the auditSink, and returns the corresponding auditSink object, and an error if there is any. +func (c *FakeAuditSinks) Get(name string, options v1.GetOptions) (result *v1alpha1.AuditSink, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(auditsinksResource, name), &v1alpha1.AuditSink{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.AuditSink), err +} + +// List takes label and field selectors, and returns the list of AuditSinks that match those selectors. +func (c *FakeAuditSinks) List(opts v1.ListOptions) (result *v1alpha1.AuditSinkList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(auditsinksResource, auditsinksKind, opts), &v1alpha1.AuditSinkList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1alpha1.AuditSinkList{ListMeta: obj.(*v1alpha1.AuditSinkList).ListMeta} + for _, item := range obj.(*v1alpha1.AuditSinkList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested auditSinks. +func (c *FakeAuditSinks) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(auditsinksResource, opts)) +} + +// Create takes the representation of a auditSink and creates it. Returns the server's representation of the auditSink, and an error, if there is any. +func (c *FakeAuditSinks) Create(auditSink *v1alpha1.AuditSink) (result *v1alpha1.AuditSink, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(auditsinksResource, auditSink), &v1alpha1.AuditSink{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.AuditSink), err +} + +// Update takes the representation of a auditSink and updates it. Returns the server's representation of the auditSink, and an error, if there is any. +func (c *FakeAuditSinks) Update(auditSink *v1alpha1.AuditSink) (result *v1alpha1.AuditSink, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(auditsinksResource, auditSink), &v1alpha1.AuditSink{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.AuditSink), err +} + +// Delete takes name of the auditSink and deletes it. Returns an error if one occurs. +func (c *FakeAuditSinks) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(auditsinksResource, name), &v1alpha1.AuditSink{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeAuditSinks) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(auditsinksResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1alpha1.AuditSinkList{}) + return err +} + +// Patch applies the patch and returns the patched auditSink. +func (c *FakeAuditSinks) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1alpha1.AuditSink, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(auditsinksResource, name, pt, data, subresources...), &v1alpha1.AuditSink{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.AuditSink), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/fake_authentication_client.go b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/fake_authentication_client.go new file mode 100644 index 0000000000..ee06a6cdd6 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/fake_authentication_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1 "k8s.io/client-go/kubernetes/typed/authentication/v1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAuthenticationV1 struct { + *testing.Fake +} + +func (c *FakeAuthenticationV1) TokenReviews() v1.TokenReviewInterface { + return &FakeTokenReviews{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAuthenticationV1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/fake_tokenreview.go b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/fake_tokenreview.go new file mode 100644 index 0000000000..e2a7f72b66 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/fake_tokenreview.go @@ -0,0 +1,24 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +// FakeTokenReviews implements TokenReviewInterface +type FakeTokenReviews struct { + Fake *FakeAuthenticationV1 +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/fake_tokenreview_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/fake_tokenreview_expansion.go new file mode 100644 index 0000000000..7008c927cd --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1/fake/fake_tokenreview_expansion.go @@ -0,0 +1,27 @@ +/* +Copyright 2017 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 fake + +import ( + authenticationapi "k8s.io/api/authentication/v1" + core "k8s.io/client-go/testing" +) + +func (c *FakeTokenReviews) Create(tokenReview *authenticationapi.TokenReview) (result *authenticationapi.TokenReview, err error) { + obj, err := c.Fake.Invokes(core.NewRootCreateAction(authenticationapi.SchemeGroupVersion.WithResource("tokenreviews"), tokenReview), &authenticationapi.TokenReview{}) + return obj.(*authenticationapi.TokenReview), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/fake_authentication_client.go b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/fake_authentication_client.go new file mode 100644 index 0000000000..7299653ca2 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/fake_authentication_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/authentication/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAuthenticationV1beta1 struct { + *testing.Fake +} + +func (c *FakeAuthenticationV1beta1) TokenReviews() v1beta1.TokenReviewInterface { + return &FakeTokenReviews{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAuthenticationV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/fake_tokenreview.go b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/fake_tokenreview.go new file mode 100644 index 0000000000..63b6b6a853 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/fake_tokenreview.go @@ -0,0 +1,24 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +// FakeTokenReviews implements TokenReviewInterface +type FakeTokenReviews struct { + Fake *FakeAuthenticationV1beta1 +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/fake_tokenreview_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/fake_tokenreview_expansion.go new file mode 100644 index 0000000000..92ef5d1a15 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authentication/v1beta1/fake/fake_tokenreview_expansion.go @@ -0,0 +1,27 @@ +/* +Copyright 2016 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 fake + +import ( + authenticationapi "k8s.io/api/authentication/v1beta1" + core "k8s.io/client-go/testing" +) + +func (c *FakeTokenReviews) Create(tokenReview *authenticationapi.TokenReview) (result *authenticationapi.TokenReview, err error) { + obj, err := c.Fake.Invokes(core.NewRootCreateAction(authenticationapi.SchemeGroupVersion.WithResource("tokenreviews"), tokenReview), &authenticationapi.TokenReview{}) + return obj.(*authenticationapi.TokenReview), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_authorization_client.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_authorization_client.go new file mode 100644 index 0000000000..f7e8234509 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_authorization_client.go @@ -0,0 +1,52 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1 "k8s.io/client-go/kubernetes/typed/authorization/v1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAuthorizationV1 struct { + *testing.Fake +} + +func (c *FakeAuthorizationV1) LocalSubjectAccessReviews(namespace string) v1.LocalSubjectAccessReviewInterface { + return &FakeLocalSubjectAccessReviews{c, namespace} +} + +func (c *FakeAuthorizationV1) SelfSubjectAccessReviews() v1.SelfSubjectAccessReviewInterface { + return &FakeSelfSubjectAccessReviews{c} +} + +func (c *FakeAuthorizationV1) SelfSubjectRulesReviews() v1.SelfSubjectRulesReviewInterface { + return &FakeSelfSubjectRulesReviews{c} +} + +func (c *FakeAuthorizationV1) SubjectAccessReviews() v1.SubjectAccessReviewInterface { + return &FakeSubjectAccessReviews{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAuthorizationV1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_localsubjectaccessreview.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_localsubjectaccessreview.go new file mode 100644 index 0000000000..778ba9cea0 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_localsubjectaccessreview.go @@ -0,0 +1,25 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +// FakeLocalSubjectAccessReviews implements LocalSubjectAccessReviewInterface +type FakeLocalSubjectAccessReviews struct { + Fake *FakeAuthorizationV1 + ns string +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_localsubjectaccessreview_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_localsubjectaccessreview_expansion.go new file mode 100644 index 0000000000..a01e415c8f --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_localsubjectaccessreview_expansion.go @@ -0,0 +1,27 @@ +/* +Copyright 2017 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 fake + +import ( + authorizationapi "k8s.io/api/authorization/v1" + core "k8s.io/client-go/testing" +) + +func (c *FakeLocalSubjectAccessReviews) Create(sar *authorizationapi.LocalSubjectAccessReview) (result *authorizationapi.LocalSubjectAccessReview, err error) { + obj, err := c.Fake.Invokes(core.NewCreateAction(authorizationapi.SchemeGroupVersion.WithResource("localsubjectaccessreviews"), c.ns, sar), &authorizationapi.SubjectAccessReview{}) + return obj.(*authorizationapi.LocalSubjectAccessReview), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectaccessreview.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectaccessreview.go new file mode 100644 index 0000000000..a43a980baf --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectaccessreview.go @@ -0,0 +1,24 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +// FakeSelfSubjectAccessReviews implements SelfSubjectAccessReviewInterface +type FakeSelfSubjectAccessReviews struct { + Fake *FakeAuthorizationV1 +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectaccessreview_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectaccessreview_expansion.go new file mode 100644 index 0000000000..91acbe029e --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectaccessreview_expansion.go @@ -0,0 +1,27 @@ +/* +Copyright 2017 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 fake + +import ( + authorizationapi "k8s.io/api/authorization/v1" + core "k8s.io/client-go/testing" +) + +func (c *FakeSelfSubjectAccessReviews) Create(sar *authorizationapi.SelfSubjectAccessReview) (result *authorizationapi.SelfSubjectAccessReview, err error) { + obj, err := c.Fake.Invokes(core.NewRootCreateAction(authorizationapi.SchemeGroupVersion.WithResource("selfsubjectaccessreviews"), sar), &authorizationapi.SelfSubjectAccessReview{}) + return obj.(*authorizationapi.SelfSubjectAccessReview), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectrulesreview.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectrulesreview.go new file mode 100644 index 0000000000..243f2e89ee --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectrulesreview.go @@ -0,0 +1,24 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +// FakeSelfSubjectRulesReviews implements SelfSubjectRulesReviewInterface +type FakeSelfSubjectRulesReviews struct { + Fake *FakeAuthorizationV1 +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectrulesreview_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectrulesreview_expansion.go new file mode 100644 index 0000000000..a6dc951349 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_selfsubjectrulesreview_expansion.go @@ -0,0 +1,27 @@ +/* +Copyright 2017 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 fake + +import ( + authorizationapi "k8s.io/api/authorization/v1" + core "k8s.io/client-go/testing" +) + +func (c *FakeSelfSubjectRulesReviews) Create(srr *authorizationapi.SelfSubjectRulesReview) (result *authorizationapi.SelfSubjectRulesReview, err error) { + obj, err := c.Fake.Invokes(core.NewRootCreateAction(authorizationapi.SchemeGroupVersion.WithResource("selfsubjectrulesreviews"), srr), &authorizationapi.SelfSubjectRulesReview{}) + return obj.(*authorizationapi.SelfSubjectRulesReview), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_subjectaccessreview.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_subjectaccessreview.go new file mode 100644 index 0000000000..d07e562546 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_subjectaccessreview.go @@ -0,0 +1,24 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +// FakeSubjectAccessReviews implements SubjectAccessReviewInterface +type FakeSubjectAccessReviews struct { + Fake *FakeAuthorizationV1 +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_subjectaccessreview_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_subjectaccessreview_expansion.go new file mode 100644 index 0000000000..a2a2f0697e --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1/fake/fake_subjectaccessreview_expansion.go @@ -0,0 +1,30 @@ +/* +Copyright 2017 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 fake + +import ( + authorizationapi "k8s.io/api/authorization/v1" + core "k8s.io/client-go/testing" +) + +func (c *FakeSubjectAccessReviews) Create(sar *authorizationapi.SubjectAccessReview) (result *authorizationapi.SubjectAccessReview, err error) { + obj, err := c.Fake.Invokes(core.NewRootCreateAction(authorizationapi.SchemeGroupVersion.WithResource("subjectaccessreviews"), sar), &authorizationapi.SubjectAccessReview{}) + if obj == nil { + return nil, err + } + return obj.(*authorizationapi.SubjectAccessReview), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_authorization_client.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_authorization_client.go new file mode 100644 index 0000000000..8e328a57bc --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_authorization_client.go @@ -0,0 +1,52 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/authorization/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAuthorizationV1beta1 struct { + *testing.Fake +} + +func (c *FakeAuthorizationV1beta1) LocalSubjectAccessReviews(namespace string) v1beta1.LocalSubjectAccessReviewInterface { + return &FakeLocalSubjectAccessReviews{c, namespace} +} + +func (c *FakeAuthorizationV1beta1) SelfSubjectAccessReviews() v1beta1.SelfSubjectAccessReviewInterface { + return &FakeSelfSubjectAccessReviews{c} +} + +func (c *FakeAuthorizationV1beta1) SelfSubjectRulesReviews() v1beta1.SelfSubjectRulesReviewInterface { + return &FakeSelfSubjectRulesReviews{c} +} + +func (c *FakeAuthorizationV1beta1) SubjectAccessReviews() v1beta1.SubjectAccessReviewInterface { + return &FakeSubjectAccessReviews{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAuthorizationV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_generated_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_generated_expansion.go new file mode 100644 index 0000000000..8754e39d87 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_generated_expansion.go @@ -0,0 +1,17 @@ +/* +Copyright 2016 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 fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_localsubjectaccessreview.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_localsubjectaccessreview.go new file mode 100644 index 0000000000..d02d05e5d1 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_localsubjectaccessreview.go @@ -0,0 +1,25 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +// FakeLocalSubjectAccessReviews implements LocalSubjectAccessReviewInterface +type FakeLocalSubjectAccessReviews struct { + Fake *FakeAuthorizationV1beta1 + ns string +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_localsubjectaccessreview_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_localsubjectaccessreview_expansion.go new file mode 100644 index 0000000000..5211628f26 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_localsubjectaccessreview_expansion.go @@ -0,0 +1,27 @@ +/* +Copyright 2016 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 fake + +import ( + authorizationapi "k8s.io/api/authorization/v1beta1" + core "k8s.io/client-go/testing" +) + +func (c *FakeLocalSubjectAccessReviews) Create(sar *authorizationapi.LocalSubjectAccessReview) (result *authorizationapi.LocalSubjectAccessReview, err error) { + obj, err := c.Fake.Invokes(core.NewCreateAction(authorizationapi.SchemeGroupVersion.WithResource("localsubjectaccessreviews"), c.ns, sar), &authorizationapi.SubjectAccessReview{}) + return obj.(*authorizationapi.LocalSubjectAccessReview), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectaccessreview.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectaccessreview.go new file mode 100644 index 0000000000..8f98ce7a3c --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectaccessreview.go @@ -0,0 +1,24 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +// FakeSelfSubjectAccessReviews implements SelfSubjectAccessReviewInterface +type FakeSelfSubjectAccessReviews struct { + Fake *FakeAuthorizationV1beta1 +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectaccessreview_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectaccessreview_expansion.go new file mode 100644 index 0000000000..6e3af12a78 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectaccessreview_expansion.go @@ -0,0 +1,27 @@ +/* +Copyright 2016 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 fake + +import ( + authorizationapi "k8s.io/api/authorization/v1beta1" + core "k8s.io/client-go/testing" +) + +func (c *FakeSelfSubjectAccessReviews) Create(sar *authorizationapi.SelfSubjectAccessReview) (result *authorizationapi.SelfSubjectAccessReview, err error) { + obj, err := c.Fake.Invokes(core.NewRootCreateAction(authorizationapi.SchemeGroupVersion.WithResource("selfsubjectaccessreviews"), sar), &authorizationapi.SelfSubjectAccessReview{}) + return obj.(*authorizationapi.SelfSubjectAccessReview), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectrulesreview.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectrulesreview.go new file mode 100644 index 0000000000..d8466b4c8d --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectrulesreview.go @@ -0,0 +1,24 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +// FakeSelfSubjectRulesReviews implements SelfSubjectRulesReviewInterface +type FakeSelfSubjectRulesReviews struct { + Fake *FakeAuthorizationV1beta1 +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectrulesreview_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectrulesreview_expansion.go new file mode 100644 index 0000000000..f92ffd717d --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_selfsubjectrulesreview_expansion.go @@ -0,0 +1,27 @@ +/* +Copyright 2017 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 fake + +import ( + authorizationapi "k8s.io/api/authorization/v1beta1" + core "k8s.io/client-go/testing" +) + +func (c *FakeSelfSubjectRulesReviews) Create(srr *authorizationapi.SelfSubjectRulesReview) (result *authorizationapi.SelfSubjectRulesReview, err error) { + obj, err := c.Fake.Invokes(core.NewRootCreateAction(authorizationapi.SchemeGroupVersion.WithResource("selfsubjectrulesreviews"), srr), &authorizationapi.SelfSubjectRulesReview{}) + return obj.(*authorizationapi.SelfSubjectRulesReview), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_subjectaccessreview.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_subjectaccessreview.go new file mode 100644 index 0000000000..0d0abdb72a --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_subjectaccessreview.go @@ -0,0 +1,24 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +// FakeSubjectAccessReviews implements SubjectAccessReviewInterface +type FakeSubjectAccessReviews struct { + Fake *FakeAuthorizationV1beta1 +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_subjectaccessreview_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_subjectaccessreview_expansion.go new file mode 100644 index 0000000000..b0b18b099c --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/authorization/v1beta1/fake/fake_subjectaccessreview_expansion.go @@ -0,0 +1,27 @@ +/* +Copyright 2016 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 fake + +import ( + authorizationapi "k8s.io/api/authorization/v1beta1" + core "k8s.io/client-go/testing" +) + +func (c *FakeSubjectAccessReviews) Create(sar *authorizationapi.SubjectAccessReview) (result *authorizationapi.SubjectAccessReview, err error) { + obj, err := c.Fake.Invokes(core.NewRootCreateAction(authorizationapi.SchemeGroupVersion.WithResource("subjectaccessreviews"), sar), &authorizationapi.SubjectAccessReview{}) + return obj.(*authorizationapi.SubjectAccessReview), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake/fake_autoscaling_client.go b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake/fake_autoscaling_client.go new file mode 100644 index 0000000000..99e26fcf39 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake/fake_autoscaling_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1 "k8s.io/client-go/kubernetes/typed/autoscaling/v1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAutoscalingV1 struct { + *testing.Fake +} + +func (c *FakeAutoscalingV1) HorizontalPodAutoscalers(namespace string) v1.HorizontalPodAutoscalerInterface { + return &FakeHorizontalPodAutoscalers{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAutoscalingV1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake/fake_horizontalpodautoscaler.go b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake/fake_horizontalpodautoscaler.go new file mode 100644 index 0000000000..6a4bf98810 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v1/fake/fake_horizontalpodautoscaler.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + autoscalingv1 "k8s.io/api/autoscaling/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeHorizontalPodAutoscalers implements HorizontalPodAutoscalerInterface +type FakeHorizontalPodAutoscalers struct { + Fake *FakeAutoscalingV1 + ns string +} + +var horizontalpodautoscalersResource = schema.GroupVersionResource{Group: "autoscaling", Version: "v1", Resource: "horizontalpodautoscalers"} + +var horizontalpodautoscalersKind = schema.GroupVersionKind{Group: "autoscaling", Version: "v1", Kind: "HorizontalPodAutoscaler"} + +// Get takes name of the horizontalPodAutoscaler, and returns the corresponding horizontalPodAutoscaler object, and an error if there is any. +func (c *FakeHorizontalPodAutoscalers) Get(name string, options v1.GetOptions) (result *autoscalingv1.HorizontalPodAutoscaler, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(horizontalpodautoscalersResource, c.ns, name), &autoscalingv1.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.HorizontalPodAutoscaler), err +} + +// List takes label and field selectors, and returns the list of HorizontalPodAutoscalers that match those selectors. +func (c *FakeHorizontalPodAutoscalers) List(opts v1.ListOptions) (result *autoscalingv1.HorizontalPodAutoscalerList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(horizontalpodautoscalersResource, horizontalpodautoscalersKind, c.ns, opts), &autoscalingv1.HorizontalPodAutoscalerList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &autoscalingv1.HorizontalPodAutoscalerList{ListMeta: obj.(*autoscalingv1.HorizontalPodAutoscalerList).ListMeta} + for _, item := range obj.(*autoscalingv1.HorizontalPodAutoscalerList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested horizontalPodAutoscalers. +func (c *FakeHorizontalPodAutoscalers) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(horizontalpodautoscalersResource, c.ns, opts)) + +} + +// Create takes the representation of a horizontalPodAutoscaler and creates it. Returns the server's representation of the horizontalPodAutoscaler, and an error, if there is any. +func (c *FakeHorizontalPodAutoscalers) Create(horizontalPodAutoscaler *autoscalingv1.HorizontalPodAutoscaler) (result *autoscalingv1.HorizontalPodAutoscaler, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(horizontalpodautoscalersResource, c.ns, horizontalPodAutoscaler), &autoscalingv1.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.HorizontalPodAutoscaler), err +} + +// Update takes the representation of a horizontalPodAutoscaler and updates it. Returns the server's representation of the horizontalPodAutoscaler, and an error, if there is any. +func (c *FakeHorizontalPodAutoscalers) Update(horizontalPodAutoscaler *autoscalingv1.HorizontalPodAutoscaler) (result *autoscalingv1.HorizontalPodAutoscaler, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(horizontalpodautoscalersResource, c.ns, horizontalPodAutoscaler), &autoscalingv1.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.HorizontalPodAutoscaler), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeHorizontalPodAutoscalers) UpdateStatus(horizontalPodAutoscaler *autoscalingv1.HorizontalPodAutoscaler) (*autoscalingv1.HorizontalPodAutoscaler, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(horizontalpodautoscalersResource, "status", c.ns, horizontalPodAutoscaler), &autoscalingv1.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.HorizontalPodAutoscaler), err +} + +// Delete takes name of the horizontalPodAutoscaler and deletes it. Returns an error if one occurs. +func (c *FakeHorizontalPodAutoscalers) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(horizontalpodautoscalersResource, c.ns, name), &autoscalingv1.HorizontalPodAutoscaler{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeHorizontalPodAutoscalers) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(horizontalpodautoscalersResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &autoscalingv1.HorizontalPodAutoscalerList{}) + return err +} + +// Patch applies the patch and returns the patched horizontalPodAutoscaler. +func (c *FakeHorizontalPodAutoscalers) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *autoscalingv1.HorizontalPodAutoscaler, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(horizontalpodautoscalersResource, c.ns, name, pt, data, subresources...), &autoscalingv1.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.HorizontalPodAutoscaler), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake/fake_autoscaling_client.go b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake/fake_autoscaling_client.go new file mode 100644 index 0000000000..be8e0f48e5 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake/fake_autoscaling_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v2beta1 "k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAutoscalingV2beta1 struct { + *testing.Fake +} + +func (c *FakeAutoscalingV2beta1) HorizontalPodAutoscalers(namespace string) v2beta1.HorizontalPodAutoscalerInterface { + return &FakeHorizontalPodAutoscalers{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAutoscalingV2beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake/fake_horizontalpodautoscaler.go b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake/fake_horizontalpodautoscaler.go new file mode 100644 index 0000000000..514a787cb1 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta1/fake/fake_horizontalpodautoscaler.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v2beta1 "k8s.io/api/autoscaling/v2beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeHorizontalPodAutoscalers implements HorizontalPodAutoscalerInterface +type FakeHorizontalPodAutoscalers struct { + Fake *FakeAutoscalingV2beta1 + ns string +} + +var horizontalpodautoscalersResource = schema.GroupVersionResource{Group: "autoscaling", Version: "v2beta1", Resource: "horizontalpodautoscalers"} + +var horizontalpodautoscalersKind = schema.GroupVersionKind{Group: "autoscaling", Version: "v2beta1", Kind: "HorizontalPodAutoscaler"} + +// Get takes name of the horizontalPodAutoscaler, and returns the corresponding horizontalPodAutoscaler object, and an error if there is any. +func (c *FakeHorizontalPodAutoscalers) Get(name string, options v1.GetOptions) (result *v2beta1.HorizontalPodAutoscaler, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(horizontalpodautoscalersResource, c.ns, name), &v2beta1.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*v2beta1.HorizontalPodAutoscaler), err +} + +// List takes label and field selectors, and returns the list of HorizontalPodAutoscalers that match those selectors. +func (c *FakeHorizontalPodAutoscalers) List(opts v1.ListOptions) (result *v2beta1.HorizontalPodAutoscalerList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(horizontalpodautoscalersResource, horizontalpodautoscalersKind, c.ns, opts), &v2beta1.HorizontalPodAutoscalerList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v2beta1.HorizontalPodAutoscalerList{ListMeta: obj.(*v2beta1.HorizontalPodAutoscalerList).ListMeta} + for _, item := range obj.(*v2beta1.HorizontalPodAutoscalerList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested horizontalPodAutoscalers. +func (c *FakeHorizontalPodAutoscalers) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(horizontalpodautoscalersResource, c.ns, opts)) + +} + +// Create takes the representation of a horizontalPodAutoscaler and creates it. Returns the server's representation of the horizontalPodAutoscaler, and an error, if there is any. +func (c *FakeHorizontalPodAutoscalers) Create(horizontalPodAutoscaler *v2beta1.HorizontalPodAutoscaler) (result *v2beta1.HorizontalPodAutoscaler, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(horizontalpodautoscalersResource, c.ns, horizontalPodAutoscaler), &v2beta1.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*v2beta1.HorizontalPodAutoscaler), err +} + +// Update takes the representation of a horizontalPodAutoscaler and updates it. Returns the server's representation of the horizontalPodAutoscaler, and an error, if there is any. +func (c *FakeHorizontalPodAutoscalers) Update(horizontalPodAutoscaler *v2beta1.HorizontalPodAutoscaler) (result *v2beta1.HorizontalPodAutoscaler, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(horizontalpodautoscalersResource, c.ns, horizontalPodAutoscaler), &v2beta1.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*v2beta1.HorizontalPodAutoscaler), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeHorizontalPodAutoscalers) UpdateStatus(horizontalPodAutoscaler *v2beta1.HorizontalPodAutoscaler) (*v2beta1.HorizontalPodAutoscaler, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(horizontalpodautoscalersResource, "status", c.ns, horizontalPodAutoscaler), &v2beta1.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*v2beta1.HorizontalPodAutoscaler), err +} + +// Delete takes name of the horizontalPodAutoscaler and deletes it. Returns an error if one occurs. +func (c *FakeHorizontalPodAutoscalers) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(horizontalpodautoscalersResource, c.ns, name), &v2beta1.HorizontalPodAutoscaler{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeHorizontalPodAutoscalers) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(horizontalpodautoscalersResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v2beta1.HorizontalPodAutoscalerList{}) + return err +} + +// Patch applies the patch and returns the patched horizontalPodAutoscaler. +func (c *FakeHorizontalPodAutoscalers) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v2beta1.HorizontalPodAutoscaler, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(horizontalpodautoscalersResource, c.ns, name, pt, data, subresources...), &v2beta1.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*v2beta1.HorizontalPodAutoscaler), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake/fake_autoscaling_client.go b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake/fake_autoscaling_client.go new file mode 100644 index 0000000000..8c36e0e815 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake/fake_autoscaling_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v2beta2 "k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeAutoscalingV2beta2 struct { + *testing.Fake +} + +func (c *FakeAutoscalingV2beta2) HorizontalPodAutoscalers(namespace string) v2beta2.HorizontalPodAutoscalerInterface { + return &FakeHorizontalPodAutoscalers{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeAutoscalingV2beta2) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake/fake_horizontalpodautoscaler.go b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake/fake_horizontalpodautoscaler.go new file mode 100644 index 0000000000..c0569f00ad --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/autoscaling/v2beta2/fake/fake_horizontalpodautoscaler.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v2beta2 "k8s.io/api/autoscaling/v2beta2" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeHorizontalPodAutoscalers implements HorizontalPodAutoscalerInterface +type FakeHorizontalPodAutoscalers struct { + Fake *FakeAutoscalingV2beta2 + ns string +} + +var horizontalpodautoscalersResource = schema.GroupVersionResource{Group: "autoscaling", Version: "v2beta2", Resource: "horizontalpodautoscalers"} + +var horizontalpodautoscalersKind = schema.GroupVersionKind{Group: "autoscaling", Version: "v2beta2", Kind: "HorizontalPodAutoscaler"} + +// Get takes name of the horizontalPodAutoscaler, and returns the corresponding horizontalPodAutoscaler object, and an error if there is any. +func (c *FakeHorizontalPodAutoscalers) Get(name string, options v1.GetOptions) (result *v2beta2.HorizontalPodAutoscaler, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(horizontalpodautoscalersResource, c.ns, name), &v2beta2.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*v2beta2.HorizontalPodAutoscaler), err +} + +// List takes label and field selectors, and returns the list of HorizontalPodAutoscalers that match those selectors. +func (c *FakeHorizontalPodAutoscalers) List(opts v1.ListOptions) (result *v2beta2.HorizontalPodAutoscalerList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(horizontalpodautoscalersResource, horizontalpodautoscalersKind, c.ns, opts), &v2beta2.HorizontalPodAutoscalerList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v2beta2.HorizontalPodAutoscalerList{ListMeta: obj.(*v2beta2.HorizontalPodAutoscalerList).ListMeta} + for _, item := range obj.(*v2beta2.HorizontalPodAutoscalerList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested horizontalPodAutoscalers. +func (c *FakeHorizontalPodAutoscalers) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(horizontalpodautoscalersResource, c.ns, opts)) + +} + +// Create takes the representation of a horizontalPodAutoscaler and creates it. Returns the server's representation of the horizontalPodAutoscaler, and an error, if there is any. +func (c *FakeHorizontalPodAutoscalers) Create(horizontalPodAutoscaler *v2beta2.HorizontalPodAutoscaler) (result *v2beta2.HorizontalPodAutoscaler, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(horizontalpodautoscalersResource, c.ns, horizontalPodAutoscaler), &v2beta2.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*v2beta2.HorizontalPodAutoscaler), err +} + +// Update takes the representation of a horizontalPodAutoscaler and updates it. Returns the server's representation of the horizontalPodAutoscaler, and an error, if there is any. +func (c *FakeHorizontalPodAutoscalers) Update(horizontalPodAutoscaler *v2beta2.HorizontalPodAutoscaler) (result *v2beta2.HorizontalPodAutoscaler, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(horizontalpodautoscalersResource, c.ns, horizontalPodAutoscaler), &v2beta2.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*v2beta2.HorizontalPodAutoscaler), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeHorizontalPodAutoscalers) UpdateStatus(horizontalPodAutoscaler *v2beta2.HorizontalPodAutoscaler) (*v2beta2.HorizontalPodAutoscaler, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(horizontalpodautoscalersResource, "status", c.ns, horizontalPodAutoscaler), &v2beta2.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*v2beta2.HorizontalPodAutoscaler), err +} + +// Delete takes name of the horizontalPodAutoscaler and deletes it. Returns an error if one occurs. +func (c *FakeHorizontalPodAutoscalers) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(horizontalpodautoscalersResource, c.ns, name), &v2beta2.HorizontalPodAutoscaler{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeHorizontalPodAutoscalers) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(horizontalpodautoscalersResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v2beta2.HorizontalPodAutoscalerList{}) + return err +} + +// Patch applies the patch and returns the patched horizontalPodAutoscaler. +func (c *FakeHorizontalPodAutoscalers) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v2beta2.HorizontalPodAutoscaler, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(horizontalpodautoscalersResource, c.ns, name, pt, data, subresources...), &v2beta2.HorizontalPodAutoscaler{}) + + if obj == nil { + return nil, err + } + return obj.(*v2beta2.HorizontalPodAutoscaler), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/batch/v1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/batch/v1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/batch/v1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/batch/v1/fake/fake_batch_client.go b/vendor/k8s.io/client-go/kubernetes/typed/batch/v1/fake/fake_batch_client.go new file mode 100644 index 0000000000..c90dd75616 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/batch/v1/fake/fake_batch_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1 "k8s.io/client-go/kubernetes/typed/batch/v1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeBatchV1 struct { + *testing.Fake +} + +func (c *FakeBatchV1) Jobs(namespace string) v1.JobInterface { + return &FakeJobs{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeBatchV1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/batch/v1/fake/fake_job.go b/vendor/k8s.io/client-go/kubernetes/typed/batch/v1/fake/fake_job.go new file mode 100644 index 0000000000..06dc25c6b4 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/batch/v1/fake/fake_job.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + batchv1 "k8s.io/api/batch/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeJobs implements JobInterface +type FakeJobs struct { + Fake *FakeBatchV1 + ns string +} + +var jobsResource = schema.GroupVersionResource{Group: "batch", Version: "v1", Resource: "jobs"} + +var jobsKind = schema.GroupVersionKind{Group: "batch", Version: "v1", Kind: "Job"} + +// Get takes name of the job, and returns the corresponding job object, and an error if there is any. +func (c *FakeJobs) Get(name string, options v1.GetOptions) (result *batchv1.Job, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(jobsResource, c.ns, name), &batchv1.Job{}) + + if obj == nil { + return nil, err + } + return obj.(*batchv1.Job), err +} + +// List takes label and field selectors, and returns the list of Jobs that match those selectors. +func (c *FakeJobs) List(opts v1.ListOptions) (result *batchv1.JobList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(jobsResource, jobsKind, c.ns, opts), &batchv1.JobList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &batchv1.JobList{ListMeta: obj.(*batchv1.JobList).ListMeta} + for _, item := range obj.(*batchv1.JobList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested jobs. +func (c *FakeJobs) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(jobsResource, c.ns, opts)) + +} + +// Create takes the representation of a job and creates it. Returns the server's representation of the job, and an error, if there is any. +func (c *FakeJobs) Create(job *batchv1.Job) (result *batchv1.Job, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(jobsResource, c.ns, job), &batchv1.Job{}) + + if obj == nil { + return nil, err + } + return obj.(*batchv1.Job), err +} + +// Update takes the representation of a job and updates it. Returns the server's representation of the job, and an error, if there is any. +func (c *FakeJobs) Update(job *batchv1.Job) (result *batchv1.Job, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(jobsResource, c.ns, job), &batchv1.Job{}) + + if obj == nil { + return nil, err + } + return obj.(*batchv1.Job), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeJobs) UpdateStatus(job *batchv1.Job) (*batchv1.Job, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(jobsResource, "status", c.ns, job), &batchv1.Job{}) + + if obj == nil { + return nil, err + } + return obj.(*batchv1.Job), err +} + +// Delete takes name of the job and deletes it. Returns an error if one occurs. +func (c *FakeJobs) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(jobsResource, c.ns, name), &batchv1.Job{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeJobs) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(jobsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &batchv1.JobList{}) + return err +} + +// Patch applies the patch and returns the patched job. +func (c *FakeJobs) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *batchv1.Job, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(jobsResource, c.ns, name, pt, data, subresources...), &batchv1.Job{}) + + if obj == nil { + return nil, err + } + return obj.(*batchv1.Job), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake/fake_batch_client.go b/vendor/k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake/fake_batch_client.go new file mode 100644 index 0000000000..6f350aed9a --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake/fake_batch_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/batch/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeBatchV1beta1 struct { + *testing.Fake +} + +func (c *FakeBatchV1beta1) CronJobs(namespace string) v1beta1.CronJobInterface { + return &FakeCronJobs{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeBatchV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake/fake_cronjob.go b/vendor/k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake/fake_cronjob.go new file mode 100644 index 0000000000..3985c40374 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/batch/v1beta1/fake/fake_cronjob.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/batch/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeCronJobs implements CronJobInterface +type FakeCronJobs struct { + Fake *FakeBatchV1beta1 + ns string +} + +var cronjobsResource = schema.GroupVersionResource{Group: "batch", Version: "v1beta1", Resource: "cronjobs"} + +var cronjobsKind = schema.GroupVersionKind{Group: "batch", Version: "v1beta1", Kind: "CronJob"} + +// Get takes name of the cronJob, and returns the corresponding cronJob object, and an error if there is any. +func (c *FakeCronJobs) Get(name string, options v1.GetOptions) (result *v1beta1.CronJob, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(cronjobsResource, c.ns, name), &v1beta1.CronJob{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.CronJob), err +} + +// List takes label and field selectors, and returns the list of CronJobs that match those selectors. +func (c *FakeCronJobs) List(opts v1.ListOptions) (result *v1beta1.CronJobList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(cronjobsResource, cronjobsKind, c.ns, opts), &v1beta1.CronJobList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.CronJobList{ListMeta: obj.(*v1beta1.CronJobList).ListMeta} + for _, item := range obj.(*v1beta1.CronJobList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested cronJobs. +func (c *FakeCronJobs) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(cronjobsResource, c.ns, opts)) + +} + +// Create takes the representation of a cronJob and creates it. Returns the server's representation of the cronJob, and an error, if there is any. +func (c *FakeCronJobs) Create(cronJob *v1beta1.CronJob) (result *v1beta1.CronJob, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(cronjobsResource, c.ns, cronJob), &v1beta1.CronJob{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.CronJob), err +} + +// Update takes the representation of a cronJob and updates it. Returns the server's representation of the cronJob, and an error, if there is any. +func (c *FakeCronJobs) Update(cronJob *v1beta1.CronJob) (result *v1beta1.CronJob, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(cronjobsResource, c.ns, cronJob), &v1beta1.CronJob{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.CronJob), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeCronJobs) UpdateStatus(cronJob *v1beta1.CronJob) (*v1beta1.CronJob, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(cronjobsResource, "status", c.ns, cronJob), &v1beta1.CronJob{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.CronJob), err +} + +// Delete takes name of the cronJob and deletes it. Returns an error if one occurs. +func (c *FakeCronJobs) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(cronjobsResource, c.ns, name), &v1beta1.CronJob{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeCronJobs) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(cronjobsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.CronJobList{}) + return err +} + +// Patch applies the patch and returns the patched cronJob. +func (c *FakeCronJobs) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.CronJob, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(cronjobsResource, c.ns, name, pt, data, subresources...), &v1beta1.CronJob{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.CronJob), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake/fake_batch_client.go b/vendor/k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake/fake_batch_client.go new file mode 100644 index 0000000000..3e478cde9d --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake/fake_batch_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v2alpha1 "k8s.io/client-go/kubernetes/typed/batch/v2alpha1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeBatchV2alpha1 struct { + *testing.Fake +} + +func (c *FakeBatchV2alpha1) CronJobs(namespace string) v2alpha1.CronJobInterface { + return &FakeCronJobs{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeBatchV2alpha1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake/fake_cronjob.go b/vendor/k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake/fake_cronjob.go new file mode 100644 index 0000000000..2195027d27 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/batch/v2alpha1/fake/fake_cronjob.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v2alpha1 "k8s.io/api/batch/v2alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeCronJobs implements CronJobInterface +type FakeCronJobs struct { + Fake *FakeBatchV2alpha1 + ns string +} + +var cronjobsResource = schema.GroupVersionResource{Group: "batch", Version: "v2alpha1", Resource: "cronjobs"} + +var cronjobsKind = schema.GroupVersionKind{Group: "batch", Version: "v2alpha1", Kind: "CronJob"} + +// Get takes name of the cronJob, and returns the corresponding cronJob object, and an error if there is any. +func (c *FakeCronJobs) Get(name string, options v1.GetOptions) (result *v2alpha1.CronJob, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(cronjobsResource, c.ns, name), &v2alpha1.CronJob{}) + + if obj == nil { + return nil, err + } + return obj.(*v2alpha1.CronJob), err +} + +// List takes label and field selectors, and returns the list of CronJobs that match those selectors. +func (c *FakeCronJobs) List(opts v1.ListOptions) (result *v2alpha1.CronJobList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(cronjobsResource, cronjobsKind, c.ns, opts), &v2alpha1.CronJobList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v2alpha1.CronJobList{ListMeta: obj.(*v2alpha1.CronJobList).ListMeta} + for _, item := range obj.(*v2alpha1.CronJobList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested cronJobs. +func (c *FakeCronJobs) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(cronjobsResource, c.ns, opts)) + +} + +// Create takes the representation of a cronJob and creates it. Returns the server's representation of the cronJob, and an error, if there is any. +func (c *FakeCronJobs) Create(cronJob *v2alpha1.CronJob) (result *v2alpha1.CronJob, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(cronjobsResource, c.ns, cronJob), &v2alpha1.CronJob{}) + + if obj == nil { + return nil, err + } + return obj.(*v2alpha1.CronJob), err +} + +// Update takes the representation of a cronJob and updates it. Returns the server's representation of the cronJob, and an error, if there is any. +func (c *FakeCronJobs) Update(cronJob *v2alpha1.CronJob) (result *v2alpha1.CronJob, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(cronjobsResource, c.ns, cronJob), &v2alpha1.CronJob{}) + + if obj == nil { + return nil, err + } + return obj.(*v2alpha1.CronJob), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeCronJobs) UpdateStatus(cronJob *v2alpha1.CronJob) (*v2alpha1.CronJob, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(cronjobsResource, "status", c.ns, cronJob), &v2alpha1.CronJob{}) + + if obj == nil { + return nil, err + } + return obj.(*v2alpha1.CronJob), err +} + +// Delete takes name of the cronJob and deletes it. Returns an error if one occurs. +func (c *FakeCronJobs) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(cronjobsResource, c.ns, name), &v2alpha1.CronJob{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeCronJobs) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(cronjobsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v2alpha1.CronJobList{}) + return err +} + +// Patch applies the patch and returns the patched cronJob. +func (c *FakeCronJobs) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v2alpha1.CronJob, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(cronjobsResource, c.ns, name, pt, data, subresources...), &v2alpha1.CronJob{}) + + if obj == nil { + return nil, err + } + return obj.(*v2alpha1.CronJob), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/fake_certificates_client.go b/vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/fake_certificates_client.go new file mode 100644 index 0000000000..29d8b088ea --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/fake_certificates_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/certificates/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeCertificatesV1beta1 struct { + *testing.Fake +} + +func (c *FakeCertificatesV1beta1) CertificateSigningRequests() v1beta1.CertificateSigningRequestInterface { + return &FakeCertificateSigningRequests{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeCertificatesV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/fake_certificatesigningrequest.go b/vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/fake_certificatesigningrequest.go new file mode 100644 index 0000000000..aa45c88033 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/fake_certificatesigningrequest.go @@ -0,0 +1,131 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/certificates/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeCertificateSigningRequests implements CertificateSigningRequestInterface +type FakeCertificateSigningRequests struct { + Fake *FakeCertificatesV1beta1 +} + +var certificatesigningrequestsResource = schema.GroupVersionResource{Group: "certificates.k8s.io", Version: "v1beta1", Resource: "certificatesigningrequests"} + +var certificatesigningrequestsKind = schema.GroupVersionKind{Group: "certificates.k8s.io", Version: "v1beta1", Kind: "CertificateSigningRequest"} + +// Get takes name of the certificateSigningRequest, and returns the corresponding certificateSigningRequest object, and an error if there is any. +func (c *FakeCertificateSigningRequests) Get(name string, options v1.GetOptions) (result *v1beta1.CertificateSigningRequest, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(certificatesigningrequestsResource, name), &v1beta1.CertificateSigningRequest{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.CertificateSigningRequest), err +} + +// List takes label and field selectors, and returns the list of CertificateSigningRequests that match those selectors. +func (c *FakeCertificateSigningRequests) List(opts v1.ListOptions) (result *v1beta1.CertificateSigningRequestList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(certificatesigningrequestsResource, certificatesigningrequestsKind, opts), &v1beta1.CertificateSigningRequestList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.CertificateSigningRequestList{ListMeta: obj.(*v1beta1.CertificateSigningRequestList).ListMeta} + for _, item := range obj.(*v1beta1.CertificateSigningRequestList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested certificateSigningRequests. +func (c *FakeCertificateSigningRequests) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(certificatesigningrequestsResource, opts)) +} + +// Create takes the representation of a certificateSigningRequest and creates it. Returns the server's representation of the certificateSigningRequest, and an error, if there is any. +func (c *FakeCertificateSigningRequests) Create(certificateSigningRequest *v1beta1.CertificateSigningRequest) (result *v1beta1.CertificateSigningRequest, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(certificatesigningrequestsResource, certificateSigningRequest), &v1beta1.CertificateSigningRequest{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.CertificateSigningRequest), err +} + +// Update takes the representation of a certificateSigningRequest and updates it. Returns the server's representation of the certificateSigningRequest, and an error, if there is any. +func (c *FakeCertificateSigningRequests) Update(certificateSigningRequest *v1beta1.CertificateSigningRequest) (result *v1beta1.CertificateSigningRequest, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(certificatesigningrequestsResource, certificateSigningRequest), &v1beta1.CertificateSigningRequest{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.CertificateSigningRequest), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeCertificateSigningRequests) UpdateStatus(certificateSigningRequest *v1beta1.CertificateSigningRequest) (*v1beta1.CertificateSigningRequest, error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateSubresourceAction(certificatesigningrequestsResource, "status", certificateSigningRequest), &v1beta1.CertificateSigningRequest{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.CertificateSigningRequest), err +} + +// Delete takes name of the certificateSigningRequest and deletes it. Returns an error if one occurs. +func (c *FakeCertificateSigningRequests) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(certificatesigningrequestsResource, name), &v1beta1.CertificateSigningRequest{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeCertificateSigningRequests) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(certificatesigningrequestsResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.CertificateSigningRequestList{}) + return err +} + +// Patch applies the patch and returns the patched certificateSigningRequest. +func (c *FakeCertificateSigningRequests) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.CertificateSigningRequest, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(certificatesigningrequestsResource, name, pt, data, subresources...), &v1beta1.CertificateSigningRequest{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.CertificateSigningRequest), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/fake_certificatesigningrequest_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/fake_certificatesigningrequest_expansion.go new file mode 100644 index 0000000000..8af33e62ad --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/certificates/v1beta1/fake/fake_certificatesigningrequest_expansion.go @@ -0,0 +1,31 @@ +/* +Copyright 2017 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 fake + +import ( + certificates "k8s.io/api/certificates/v1beta1" + core "k8s.io/client-go/testing" +) + +func (c *FakeCertificateSigningRequests) UpdateApproval(certificateSigningRequest *certificates.CertificateSigningRequest) (result *certificates.CertificateSigningRequest, err error) { + obj, err := c.Fake. + Invokes(core.NewRootUpdateSubresourceAction(certificatesigningrequestsResource, "approval", certificateSigningRequest), &certificates.CertificateSigningRequest{}) + if obj == nil { + return nil, err + } + return obj.(*certificates.CertificateSigningRequest), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake/fake_coordination_client.go b/vendor/k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake/fake_coordination_client.go new file mode 100644 index 0000000000..f583b466e2 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake/fake_coordination_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/coordination/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeCoordinationV1beta1 struct { + *testing.Fake +} + +func (c *FakeCoordinationV1beta1) Leases(namespace string) v1beta1.LeaseInterface { + return &FakeLeases{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeCoordinationV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake/fake_lease.go b/vendor/k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake/fake_lease.go new file mode 100644 index 0000000000..0ebf3bffc2 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/coordination/v1beta1/fake/fake_lease.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/coordination/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeLeases implements LeaseInterface +type FakeLeases struct { + Fake *FakeCoordinationV1beta1 + ns string +} + +var leasesResource = schema.GroupVersionResource{Group: "coordination.k8s.io", Version: "v1beta1", Resource: "leases"} + +var leasesKind = schema.GroupVersionKind{Group: "coordination.k8s.io", Version: "v1beta1", Kind: "Lease"} + +// Get takes name of the lease, and returns the corresponding lease object, and an error if there is any. +func (c *FakeLeases) Get(name string, options v1.GetOptions) (result *v1beta1.Lease, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(leasesResource, c.ns, name), &v1beta1.Lease{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Lease), err +} + +// List takes label and field selectors, and returns the list of Leases that match those selectors. +func (c *FakeLeases) List(opts v1.ListOptions) (result *v1beta1.LeaseList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(leasesResource, leasesKind, c.ns, opts), &v1beta1.LeaseList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.LeaseList{ListMeta: obj.(*v1beta1.LeaseList).ListMeta} + for _, item := range obj.(*v1beta1.LeaseList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested leases. +func (c *FakeLeases) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(leasesResource, c.ns, opts)) + +} + +// Create takes the representation of a lease and creates it. Returns the server's representation of the lease, and an error, if there is any. +func (c *FakeLeases) Create(lease *v1beta1.Lease) (result *v1beta1.Lease, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(leasesResource, c.ns, lease), &v1beta1.Lease{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Lease), err +} + +// Update takes the representation of a lease and updates it. Returns the server's representation of the lease, and an error, if there is any. +func (c *FakeLeases) Update(lease *v1beta1.Lease) (result *v1beta1.Lease, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(leasesResource, c.ns, lease), &v1beta1.Lease{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Lease), err +} + +// Delete takes name of the lease and deletes it. Returns an error if one occurs. +func (c *FakeLeases) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(leasesResource, c.ns, name), &v1beta1.Lease{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeLeases) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(leasesResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.LeaseList{}) + return err +} + +// Patch applies the patch and returns the patched lease. +func (c *FakeLeases) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.Lease, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(leasesResource, c.ns, name, pt, data, subresources...), &v1beta1.Lease{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Lease), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_componentstatus.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_componentstatus.go new file mode 100644 index 0000000000..18beedc2d3 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_componentstatus.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeComponentStatuses implements ComponentStatusInterface +type FakeComponentStatuses struct { + Fake *FakeCoreV1 +} + +var componentstatusesResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "componentstatuses"} + +var componentstatusesKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "ComponentStatus"} + +// Get takes name of the componentStatus, and returns the corresponding componentStatus object, and an error if there is any. +func (c *FakeComponentStatuses) Get(name string, options v1.GetOptions) (result *corev1.ComponentStatus, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(componentstatusesResource, name), &corev1.ComponentStatus{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.ComponentStatus), err +} + +// List takes label and field selectors, and returns the list of ComponentStatuses that match those selectors. +func (c *FakeComponentStatuses) List(opts v1.ListOptions) (result *corev1.ComponentStatusList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(componentstatusesResource, componentstatusesKind, opts), &corev1.ComponentStatusList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.ComponentStatusList{ListMeta: obj.(*corev1.ComponentStatusList).ListMeta} + for _, item := range obj.(*corev1.ComponentStatusList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested componentStatuses. +func (c *FakeComponentStatuses) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(componentstatusesResource, opts)) +} + +// Create takes the representation of a componentStatus and creates it. Returns the server's representation of the componentStatus, and an error, if there is any. +func (c *FakeComponentStatuses) Create(componentStatus *corev1.ComponentStatus) (result *corev1.ComponentStatus, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(componentstatusesResource, componentStatus), &corev1.ComponentStatus{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.ComponentStatus), err +} + +// Update takes the representation of a componentStatus and updates it. Returns the server's representation of the componentStatus, and an error, if there is any. +func (c *FakeComponentStatuses) Update(componentStatus *corev1.ComponentStatus) (result *corev1.ComponentStatus, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(componentstatusesResource, componentStatus), &corev1.ComponentStatus{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.ComponentStatus), err +} + +// Delete takes name of the componentStatus and deletes it. Returns an error if one occurs. +func (c *FakeComponentStatuses) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(componentstatusesResource, name), &corev1.ComponentStatus{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeComponentStatuses) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(componentstatusesResource, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.ComponentStatusList{}) + return err +} + +// Patch applies the patch and returns the patched componentStatus. +func (c *FakeComponentStatuses) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.ComponentStatus, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(componentstatusesResource, name, pt, data, subresources...), &corev1.ComponentStatus{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.ComponentStatus), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_configmap.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_configmap.go new file mode 100644 index 0000000000..2361ac3fe9 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_configmap.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeConfigMaps implements ConfigMapInterface +type FakeConfigMaps struct { + Fake *FakeCoreV1 + ns string +} + +var configmapsResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "configmaps"} + +var configmapsKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "ConfigMap"} + +// Get takes name of the configMap, and returns the corresponding configMap object, and an error if there is any. +func (c *FakeConfigMaps) Get(name string, options v1.GetOptions) (result *corev1.ConfigMap, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(configmapsResource, c.ns, name), &corev1.ConfigMap{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ConfigMap), err +} + +// List takes label and field selectors, and returns the list of ConfigMaps that match those selectors. +func (c *FakeConfigMaps) List(opts v1.ListOptions) (result *corev1.ConfigMapList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(configmapsResource, configmapsKind, c.ns, opts), &corev1.ConfigMapList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.ConfigMapList{ListMeta: obj.(*corev1.ConfigMapList).ListMeta} + for _, item := range obj.(*corev1.ConfigMapList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested configMaps. +func (c *FakeConfigMaps) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(configmapsResource, c.ns, opts)) + +} + +// Create takes the representation of a configMap and creates it. Returns the server's representation of the configMap, and an error, if there is any. +func (c *FakeConfigMaps) Create(configMap *corev1.ConfigMap) (result *corev1.ConfigMap, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(configmapsResource, c.ns, configMap), &corev1.ConfigMap{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ConfigMap), err +} + +// Update takes the representation of a configMap and updates it. Returns the server's representation of the configMap, and an error, if there is any. +func (c *FakeConfigMaps) Update(configMap *corev1.ConfigMap) (result *corev1.ConfigMap, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(configmapsResource, c.ns, configMap), &corev1.ConfigMap{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ConfigMap), err +} + +// Delete takes name of the configMap and deletes it. Returns an error if one occurs. +func (c *FakeConfigMaps) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(configmapsResource, c.ns, name), &corev1.ConfigMap{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeConfigMaps) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(configmapsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.ConfigMapList{}) + return err +} + +// Patch applies the patch and returns the patched configMap. +func (c *FakeConfigMaps) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.ConfigMap, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(configmapsResource, c.ns, name, pt, data, subresources...), &corev1.ConfigMap{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ConfigMap), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_core_client.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_core_client.go new file mode 100644 index 0000000000..5ad90943c9 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_core_client.go @@ -0,0 +1,100 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1 "k8s.io/client-go/kubernetes/typed/core/v1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeCoreV1 struct { + *testing.Fake +} + +func (c *FakeCoreV1) ComponentStatuses() v1.ComponentStatusInterface { + return &FakeComponentStatuses{c} +} + +func (c *FakeCoreV1) ConfigMaps(namespace string) v1.ConfigMapInterface { + return &FakeConfigMaps{c, namespace} +} + +func (c *FakeCoreV1) Endpoints(namespace string) v1.EndpointsInterface { + return &FakeEndpoints{c, namespace} +} + +func (c *FakeCoreV1) Events(namespace string) v1.EventInterface { + return &FakeEvents{c, namespace} +} + +func (c *FakeCoreV1) LimitRanges(namespace string) v1.LimitRangeInterface { + return &FakeLimitRanges{c, namespace} +} + +func (c *FakeCoreV1) Namespaces() v1.NamespaceInterface { + return &FakeNamespaces{c} +} + +func (c *FakeCoreV1) Nodes() v1.NodeInterface { + return &FakeNodes{c} +} + +func (c *FakeCoreV1) PersistentVolumes() v1.PersistentVolumeInterface { + return &FakePersistentVolumes{c} +} + +func (c *FakeCoreV1) PersistentVolumeClaims(namespace string) v1.PersistentVolumeClaimInterface { + return &FakePersistentVolumeClaims{c, namespace} +} + +func (c *FakeCoreV1) Pods(namespace string) v1.PodInterface { + return &FakePods{c, namespace} +} + +func (c *FakeCoreV1) PodTemplates(namespace string) v1.PodTemplateInterface { + return &FakePodTemplates{c, namespace} +} + +func (c *FakeCoreV1) ReplicationControllers(namespace string) v1.ReplicationControllerInterface { + return &FakeReplicationControllers{c, namespace} +} + +func (c *FakeCoreV1) ResourceQuotas(namespace string) v1.ResourceQuotaInterface { + return &FakeResourceQuotas{c, namespace} +} + +func (c *FakeCoreV1) Secrets(namespace string) v1.SecretInterface { + return &FakeSecrets{c, namespace} +} + +func (c *FakeCoreV1) Services(namespace string) v1.ServiceInterface { + return &FakeServices{c, namespace} +} + +func (c *FakeCoreV1) ServiceAccounts(namespace string) v1.ServiceAccountInterface { + return &FakeServiceAccounts{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeCoreV1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_endpoints.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_endpoints.go new file mode 100644 index 0000000000..d521af4083 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_endpoints.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeEndpoints implements EndpointsInterface +type FakeEndpoints struct { + Fake *FakeCoreV1 + ns string +} + +var endpointsResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "endpoints"} + +var endpointsKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "Endpoints"} + +// Get takes name of the endpoints, and returns the corresponding endpoints object, and an error if there is any. +func (c *FakeEndpoints) Get(name string, options v1.GetOptions) (result *corev1.Endpoints, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(endpointsResource, c.ns, name), &corev1.Endpoints{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Endpoints), err +} + +// List takes label and field selectors, and returns the list of Endpoints that match those selectors. +func (c *FakeEndpoints) List(opts v1.ListOptions) (result *corev1.EndpointsList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(endpointsResource, endpointsKind, c.ns, opts), &corev1.EndpointsList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.EndpointsList{ListMeta: obj.(*corev1.EndpointsList).ListMeta} + for _, item := range obj.(*corev1.EndpointsList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested endpoints. +func (c *FakeEndpoints) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(endpointsResource, c.ns, opts)) + +} + +// Create takes the representation of a endpoints and creates it. Returns the server's representation of the endpoints, and an error, if there is any. +func (c *FakeEndpoints) Create(endpoints *corev1.Endpoints) (result *corev1.Endpoints, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(endpointsResource, c.ns, endpoints), &corev1.Endpoints{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Endpoints), err +} + +// Update takes the representation of a endpoints and updates it. Returns the server's representation of the endpoints, and an error, if there is any. +func (c *FakeEndpoints) Update(endpoints *corev1.Endpoints) (result *corev1.Endpoints, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(endpointsResource, c.ns, endpoints), &corev1.Endpoints{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Endpoints), err +} + +// Delete takes name of the endpoints and deletes it. Returns an error if one occurs. +func (c *FakeEndpoints) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(endpointsResource, c.ns, name), &corev1.Endpoints{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeEndpoints) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(endpointsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.EndpointsList{}) + return err +} + +// Patch applies the patch and returns the patched endpoints. +func (c *FakeEndpoints) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.Endpoints, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(endpointsResource, c.ns, name, pt, data, subresources...), &corev1.Endpoints{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Endpoints), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_event.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_event.go new file mode 100644 index 0000000000..3444f4be96 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_event.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeEvents implements EventInterface +type FakeEvents struct { + Fake *FakeCoreV1 + ns string +} + +var eventsResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "events"} + +var eventsKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "Event"} + +// Get takes name of the event, and returns the corresponding event object, and an error if there is any. +func (c *FakeEvents) Get(name string, options v1.GetOptions) (result *corev1.Event, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(eventsResource, c.ns, name), &corev1.Event{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Event), err +} + +// List takes label and field selectors, and returns the list of Events that match those selectors. +func (c *FakeEvents) List(opts v1.ListOptions) (result *corev1.EventList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(eventsResource, eventsKind, c.ns, opts), &corev1.EventList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.EventList{ListMeta: obj.(*corev1.EventList).ListMeta} + for _, item := range obj.(*corev1.EventList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested events. +func (c *FakeEvents) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(eventsResource, c.ns, opts)) + +} + +// Create takes the representation of a event and creates it. Returns the server's representation of the event, and an error, if there is any. +func (c *FakeEvents) Create(event *corev1.Event) (result *corev1.Event, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(eventsResource, c.ns, event), &corev1.Event{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Event), err +} + +// Update takes the representation of a event and updates it. Returns the server's representation of the event, and an error, if there is any. +func (c *FakeEvents) Update(event *corev1.Event) (result *corev1.Event, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(eventsResource, c.ns, event), &corev1.Event{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Event), err +} + +// Delete takes name of the event and deletes it. Returns an error if one occurs. +func (c *FakeEvents) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(eventsResource, c.ns, name), &corev1.Event{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeEvents) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(eventsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.EventList{}) + return err +} + +// Patch applies the patch and returns the patched event. +func (c *FakeEvents) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.Event, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(eventsResource, c.ns, name, pt, data, subresources...), &corev1.Event{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Event), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_event_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_event_expansion.go new file mode 100644 index 0000000000..4b4c90d7d1 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_event_expansion.go @@ -0,0 +1,93 @@ +/* +Copyright 2014 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 fake + +import ( + "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/fields" + "k8s.io/apimachinery/pkg/runtime" + types "k8s.io/apimachinery/pkg/types" + core "k8s.io/client-go/testing" +) + +func (c *FakeEvents) CreateWithEventNamespace(event *v1.Event) (*v1.Event, error) { + action := core.NewRootCreateAction(eventsResource, event) + if c.ns != "" { + action = core.NewCreateAction(eventsResource, c.ns, event) + } + obj, err := c.Fake.Invokes(action, event) + if obj == nil { + return nil, err + } + + return obj.(*v1.Event), err +} + +// Update replaces an existing event. Returns the copy of the event the server returns, or an error. +func (c *FakeEvents) UpdateWithEventNamespace(event *v1.Event) (*v1.Event, error) { + action := core.NewRootUpdateAction(eventsResource, event) + if c.ns != "" { + action = core.NewUpdateAction(eventsResource, c.ns, event) + } + obj, err := c.Fake.Invokes(action, event) + if obj == nil { + return nil, err + } + + return obj.(*v1.Event), err +} + +// PatchWithEventNamespace patches an existing event. Returns the copy of the event the server returns, or an error. +// TODO: Should take a PatchType as an argument probably. +func (c *FakeEvents) PatchWithEventNamespace(event *v1.Event, data []byte) (*v1.Event, error) { + // TODO: Should be configurable to support additional patch strategies. + pt := types.StrategicMergePatchType + action := core.NewRootPatchAction(eventsResource, event.Name, pt, data) + if c.ns != "" { + action = core.NewPatchAction(eventsResource, c.ns, event.Name, pt, data) + } + obj, err := c.Fake.Invokes(action, event) + if obj == nil { + return nil, err + } + + return obj.(*v1.Event), err +} + +// Search returns a list of events matching the specified object. +func (c *FakeEvents) Search(scheme *runtime.Scheme, objOrRef runtime.Object) (*v1.EventList, error) { + action := core.NewRootListAction(eventsResource, eventsKind, metav1.ListOptions{}) + if c.ns != "" { + action = core.NewListAction(eventsResource, eventsKind, c.ns, metav1.ListOptions{}) + } + obj, err := c.Fake.Invokes(action, &v1.EventList{}) + if obj == nil { + return nil, err + } + + return obj.(*v1.EventList), err +} + +func (c *FakeEvents) GetFieldSelector(involvedObjectName, involvedObjectNamespace, involvedObjectKind, involvedObjectUID *string) fields.Selector { + action := core.GenericActionImpl{} + action.Verb = "get-field-selector" + action.Resource = eventsResource + + c.Fake.Invokes(action, nil) + return fields.Everything() +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_limitrange.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_limitrange.go new file mode 100644 index 0000000000..d110031f83 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_limitrange.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeLimitRanges implements LimitRangeInterface +type FakeLimitRanges struct { + Fake *FakeCoreV1 + ns string +} + +var limitrangesResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "limitranges"} + +var limitrangesKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "LimitRange"} + +// Get takes name of the limitRange, and returns the corresponding limitRange object, and an error if there is any. +func (c *FakeLimitRanges) Get(name string, options v1.GetOptions) (result *corev1.LimitRange, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(limitrangesResource, c.ns, name), &corev1.LimitRange{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.LimitRange), err +} + +// List takes label and field selectors, and returns the list of LimitRanges that match those selectors. +func (c *FakeLimitRanges) List(opts v1.ListOptions) (result *corev1.LimitRangeList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(limitrangesResource, limitrangesKind, c.ns, opts), &corev1.LimitRangeList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.LimitRangeList{ListMeta: obj.(*corev1.LimitRangeList).ListMeta} + for _, item := range obj.(*corev1.LimitRangeList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested limitRanges. +func (c *FakeLimitRanges) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(limitrangesResource, c.ns, opts)) + +} + +// Create takes the representation of a limitRange and creates it. Returns the server's representation of the limitRange, and an error, if there is any. +func (c *FakeLimitRanges) Create(limitRange *corev1.LimitRange) (result *corev1.LimitRange, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(limitrangesResource, c.ns, limitRange), &corev1.LimitRange{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.LimitRange), err +} + +// Update takes the representation of a limitRange and updates it. Returns the server's representation of the limitRange, and an error, if there is any. +func (c *FakeLimitRanges) Update(limitRange *corev1.LimitRange) (result *corev1.LimitRange, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(limitrangesResource, c.ns, limitRange), &corev1.LimitRange{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.LimitRange), err +} + +// Delete takes name of the limitRange and deletes it. Returns an error if one occurs. +func (c *FakeLimitRanges) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(limitrangesResource, c.ns, name), &corev1.LimitRange{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeLimitRanges) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(limitrangesResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.LimitRangeList{}) + return err +} + +// Patch applies the patch and returns the patched limitRange. +func (c *FakeLimitRanges) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.LimitRange, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(limitrangesResource, c.ns, name, pt, data, subresources...), &corev1.LimitRange{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.LimitRange), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_namespace.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_namespace.go new file mode 100644 index 0000000000..21387b5e25 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_namespace.go @@ -0,0 +1,123 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeNamespaces implements NamespaceInterface +type FakeNamespaces struct { + Fake *FakeCoreV1 +} + +var namespacesResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "namespaces"} + +var namespacesKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "Namespace"} + +// Get takes name of the namespace, and returns the corresponding namespace object, and an error if there is any. +func (c *FakeNamespaces) Get(name string, options v1.GetOptions) (result *corev1.Namespace, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(namespacesResource, name), &corev1.Namespace{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.Namespace), err +} + +// List takes label and field selectors, and returns the list of Namespaces that match those selectors. +func (c *FakeNamespaces) List(opts v1.ListOptions) (result *corev1.NamespaceList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(namespacesResource, namespacesKind, opts), &corev1.NamespaceList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.NamespaceList{ListMeta: obj.(*corev1.NamespaceList).ListMeta} + for _, item := range obj.(*corev1.NamespaceList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested namespaces. +func (c *FakeNamespaces) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(namespacesResource, opts)) +} + +// Create takes the representation of a namespace and creates it. Returns the server's representation of the namespace, and an error, if there is any. +func (c *FakeNamespaces) Create(namespace *corev1.Namespace) (result *corev1.Namespace, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(namespacesResource, namespace), &corev1.Namespace{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.Namespace), err +} + +// Update takes the representation of a namespace and updates it. Returns the server's representation of the namespace, and an error, if there is any. +func (c *FakeNamespaces) Update(namespace *corev1.Namespace) (result *corev1.Namespace, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(namespacesResource, namespace), &corev1.Namespace{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.Namespace), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeNamespaces) UpdateStatus(namespace *corev1.Namespace) (*corev1.Namespace, error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateSubresourceAction(namespacesResource, "status", namespace), &corev1.Namespace{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.Namespace), err +} + +// Delete takes name of the namespace and deletes it. Returns an error if one occurs. +func (c *FakeNamespaces) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(namespacesResource, name), &corev1.Namespace{}) + return err +} + +// Patch applies the patch and returns the patched namespace. +func (c *FakeNamespaces) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.Namespace, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(namespacesResource, name, pt, data, subresources...), &corev1.Namespace{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.Namespace), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_namespace_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_namespace_expansion.go new file mode 100644 index 0000000000..a0eae34904 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_namespace_expansion.go @@ -0,0 +1,37 @@ +/* +Copyright 2014 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 fake + +import ( + "k8s.io/api/core/v1" + core "k8s.io/client-go/testing" +) + +func (c *FakeNamespaces) Finalize(namespace *v1.Namespace) (*v1.Namespace, error) { + action := core.CreateActionImpl{} + action.Verb = "create" + action.Resource = namespacesResource + action.Subresource = "finalize" + action.Object = namespace + + obj, err := c.Fake.Invokes(action, namespace) + if obj == nil { + return nil, err + } + + return obj.(*v1.Namespace), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_node.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_node.go new file mode 100644 index 0000000000..bcde116a4e --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_node.go @@ -0,0 +1,131 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeNodes implements NodeInterface +type FakeNodes struct { + Fake *FakeCoreV1 +} + +var nodesResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "nodes"} + +var nodesKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "Node"} + +// Get takes name of the node, and returns the corresponding node object, and an error if there is any. +func (c *FakeNodes) Get(name string, options v1.GetOptions) (result *corev1.Node, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(nodesResource, name), &corev1.Node{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.Node), err +} + +// List takes label and field selectors, and returns the list of Nodes that match those selectors. +func (c *FakeNodes) List(opts v1.ListOptions) (result *corev1.NodeList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(nodesResource, nodesKind, opts), &corev1.NodeList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.NodeList{ListMeta: obj.(*corev1.NodeList).ListMeta} + for _, item := range obj.(*corev1.NodeList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested nodes. +func (c *FakeNodes) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(nodesResource, opts)) +} + +// Create takes the representation of a node and creates it. Returns the server's representation of the node, and an error, if there is any. +func (c *FakeNodes) Create(node *corev1.Node) (result *corev1.Node, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(nodesResource, node), &corev1.Node{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.Node), err +} + +// Update takes the representation of a node and updates it. Returns the server's representation of the node, and an error, if there is any. +func (c *FakeNodes) Update(node *corev1.Node) (result *corev1.Node, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(nodesResource, node), &corev1.Node{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.Node), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeNodes) UpdateStatus(node *corev1.Node) (*corev1.Node, error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateSubresourceAction(nodesResource, "status", node), &corev1.Node{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.Node), err +} + +// Delete takes name of the node and deletes it. Returns an error if one occurs. +func (c *FakeNodes) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(nodesResource, name), &corev1.Node{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeNodes) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(nodesResource, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.NodeList{}) + return err +} + +// Patch applies the patch and returns the patched node. +func (c *FakeNodes) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.Node, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(nodesResource, name, pt, data, subresources...), &corev1.Node{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.Node), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_node_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_node_expansion.go new file mode 100644 index 0000000000..a39022c83f --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_node_expansion.go @@ -0,0 +1,36 @@ +/* +Copyright 2016 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 fake + +import ( + "k8s.io/api/core/v1" + types "k8s.io/apimachinery/pkg/types" + core "k8s.io/client-go/testing" +) + +// TODO: Should take a PatchType as an argument probably. +func (c *FakeNodes) PatchStatus(nodeName string, data []byte) (*v1.Node, error) { + // TODO: Should be configurable to support additional patch strategies. + pt := types.StrategicMergePatchType + obj, err := c.Fake.Invokes( + core.NewRootPatchSubresourceAction(nodesResource, nodeName, pt, data, "status"), &v1.Node{}) + if obj == nil { + return nil, err + } + + return obj.(*v1.Node), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_persistentvolume.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_persistentvolume.go new file mode 100644 index 0000000000..843f323075 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_persistentvolume.go @@ -0,0 +1,131 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakePersistentVolumes implements PersistentVolumeInterface +type FakePersistentVolumes struct { + Fake *FakeCoreV1 +} + +var persistentvolumesResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "persistentvolumes"} + +var persistentvolumesKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "PersistentVolume"} + +// Get takes name of the persistentVolume, and returns the corresponding persistentVolume object, and an error if there is any. +func (c *FakePersistentVolumes) Get(name string, options v1.GetOptions) (result *corev1.PersistentVolume, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(persistentvolumesResource, name), &corev1.PersistentVolume{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.PersistentVolume), err +} + +// List takes label and field selectors, and returns the list of PersistentVolumes that match those selectors. +func (c *FakePersistentVolumes) List(opts v1.ListOptions) (result *corev1.PersistentVolumeList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(persistentvolumesResource, persistentvolumesKind, opts), &corev1.PersistentVolumeList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.PersistentVolumeList{ListMeta: obj.(*corev1.PersistentVolumeList).ListMeta} + for _, item := range obj.(*corev1.PersistentVolumeList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested persistentVolumes. +func (c *FakePersistentVolumes) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(persistentvolumesResource, opts)) +} + +// Create takes the representation of a persistentVolume and creates it. Returns the server's representation of the persistentVolume, and an error, if there is any. +func (c *FakePersistentVolumes) Create(persistentVolume *corev1.PersistentVolume) (result *corev1.PersistentVolume, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(persistentvolumesResource, persistentVolume), &corev1.PersistentVolume{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.PersistentVolume), err +} + +// Update takes the representation of a persistentVolume and updates it. Returns the server's representation of the persistentVolume, and an error, if there is any. +func (c *FakePersistentVolumes) Update(persistentVolume *corev1.PersistentVolume) (result *corev1.PersistentVolume, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(persistentvolumesResource, persistentVolume), &corev1.PersistentVolume{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.PersistentVolume), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakePersistentVolumes) UpdateStatus(persistentVolume *corev1.PersistentVolume) (*corev1.PersistentVolume, error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateSubresourceAction(persistentvolumesResource, "status", persistentVolume), &corev1.PersistentVolume{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.PersistentVolume), err +} + +// Delete takes name of the persistentVolume and deletes it. Returns an error if one occurs. +func (c *FakePersistentVolumes) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(persistentvolumesResource, name), &corev1.PersistentVolume{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakePersistentVolumes) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(persistentvolumesResource, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.PersistentVolumeList{}) + return err +} + +// Patch applies the patch and returns the patched persistentVolume. +func (c *FakePersistentVolumes) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.PersistentVolume, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(persistentvolumesResource, name, pt, data, subresources...), &corev1.PersistentVolume{}) + if obj == nil { + return nil, err + } + return obj.(*corev1.PersistentVolume), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_persistentvolumeclaim.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_persistentvolumeclaim.go new file mode 100644 index 0000000000..d2557c4c83 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_persistentvolumeclaim.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakePersistentVolumeClaims implements PersistentVolumeClaimInterface +type FakePersistentVolumeClaims struct { + Fake *FakeCoreV1 + ns string +} + +var persistentvolumeclaimsResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "persistentvolumeclaims"} + +var persistentvolumeclaimsKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "PersistentVolumeClaim"} + +// Get takes name of the persistentVolumeClaim, and returns the corresponding persistentVolumeClaim object, and an error if there is any. +func (c *FakePersistentVolumeClaims) Get(name string, options v1.GetOptions) (result *corev1.PersistentVolumeClaim, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(persistentvolumeclaimsResource, c.ns, name), &corev1.PersistentVolumeClaim{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.PersistentVolumeClaim), err +} + +// List takes label and field selectors, and returns the list of PersistentVolumeClaims that match those selectors. +func (c *FakePersistentVolumeClaims) List(opts v1.ListOptions) (result *corev1.PersistentVolumeClaimList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(persistentvolumeclaimsResource, persistentvolumeclaimsKind, c.ns, opts), &corev1.PersistentVolumeClaimList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.PersistentVolumeClaimList{ListMeta: obj.(*corev1.PersistentVolumeClaimList).ListMeta} + for _, item := range obj.(*corev1.PersistentVolumeClaimList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested persistentVolumeClaims. +func (c *FakePersistentVolumeClaims) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(persistentvolumeclaimsResource, c.ns, opts)) + +} + +// Create takes the representation of a persistentVolumeClaim and creates it. Returns the server's representation of the persistentVolumeClaim, and an error, if there is any. +func (c *FakePersistentVolumeClaims) Create(persistentVolumeClaim *corev1.PersistentVolumeClaim) (result *corev1.PersistentVolumeClaim, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(persistentvolumeclaimsResource, c.ns, persistentVolumeClaim), &corev1.PersistentVolumeClaim{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.PersistentVolumeClaim), err +} + +// Update takes the representation of a persistentVolumeClaim and updates it. Returns the server's representation of the persistentVolumeClaim, and an error, if there is any. +func (c *FakePersistentVolumeClaims) Update(persistentVolumeClaim *corev1.PersistentVolumeClaim) (result *corev1.PersistentVolumeClaim, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(persistentvolumeclaimsResource, c.ns, persistentVolumeClaim), &corev1.PersistentVolumeClaim{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.PersistentVolumeClaim), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakePersistentVolumeClaims) UpdateStatus(persistentVolumeClaim *corev1.PersistentVolumeClaim) (*corev1.PersistentVolumeClaim, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(persistentvolumeclaimsResource, "status", c.ns, persistentVolumeClaim), &corev1.PersistentVolumeClaim{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.PersistentVolumeClaim), err +} + +// Delete takes name of the persistentVolumeClaim and deletes it. Returns an error if one occurs. +func (c *FakePersistentVolumeClaims) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(persistentvolumeclaimsResource, c.ns, name), &corev1.PersistentVolumeClaim{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakePersistentVolumeClaims) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(persistentvolumeclaimsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.PersistentVolumeClaimList{}) + return err +} + +// Patch applies the patch and returns the patched persistentVolumeClaim. +func (c *FakePersistentVolumeClaims) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.PersistentVolumeClaim, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(persistentvolumeclaimsResource, c.ns, name, pt, data, subresources...), &corev1.PersistentVolumeClaim{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.PersistentVolumeClaim), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_pod.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_pod.go new file mode 100644 index 0000000000..2dbecbbaac --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_pod.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakePods implements PodInterface +type FakePods struct { + Fake *FakeCoreV1 + ns string +} + +var podsResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "pods"} + +var podsKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "Pod"} + +// Get takes name of the pod, and returns the corresponding pod object, and an error if there is any. +func (c *FakePods) Get(name string, options v1.GetOptions) (result *corev1.Pod, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(podsResource, c.ns, name), &corev1.Pod{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Pod), err +} + +// List takes label and field selectors, and returns the list of Pods that match those selectors. +func (c *FakePods) List(opts v1.ListOptions) (result *corev1.PodList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(podsResource, podsKind, c.ns, opts), &corev1.PodList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.PodList{ListMeta: obj.(*corev1.PodList).ListMeta} + for _, item := range obj.(*corev1.PodList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested pods. +func (c *FakePods) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(podsResource, c.ns, opts)) + +} + +// Create takes the representation of a pod and creates it. Returns the server's representation of the pod, and an error, if there is any. +func (c *FakePods) Create(pod *corev1.Pod) (result *corev1.Pod, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(podsResource, c.ns, pod), &corev1.Pod{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Pod), err +} + +// Update takes the representation of a pod and updates it. Returns the server's representation of the pod, and an error, if there is any. +func (c *FakePods) Update(pod *corev1.Pod) (result *corev1.Pod, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(podsResource, c.ns, pod), &corev1.Pod{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Pod), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakePods) UpdateStatus(pod *corev1.Pod) (*corev1.Pod, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(podsResource, "status", c.ns, pod), &corev1.Pod{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Pod), err +} + +// Delete takes name of the pod and deletes it. Returns an error if one occurs. +func (c *FakePods) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(podsResource, c.ns, name), &corev1.Pod{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakePods) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(podsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.PodList{}) + return err +} + +// Patch applies the patch and returns the patched pod. +func (c *FakePods) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.Pod, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(podsResource, c.ns, name, pt, data, subresources...), &corev1.Pod{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Pod), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_pod_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_pod_expansion.go new file mode 100644 index 0000000000..9c4b09d3e9 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_pod_expansion.go @@ -0,0 +1,69 @@ +/* +Copyright 2014 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 fake + +import ( + "k8s.io/api/core/v1" + policy "k8s.io/api/policy/v1beta1" + restclient "k8s.io/client-go/rest" + core "k8s.io/client-go/testing" +) + +func (c *FakePods) Bind(binding *v1.Binding) error { + action := core.CreateActionImpl{} + action.Verb = "create" + action.Namespace = binding.Namespace + action.Resource = podsResource + action.Subresource = "binding" + action.Object = binding + + _, err := c.Fake.Invokes(action, binding) + return err +} + +func (c *FakePods) GetBinding(name string) (result *v1.Binding, err error) { + obj, err := c.Fake. + Invokes(core.NewGetSubresourceAction(podsResource, c.ns, "binding", name), &v1.Binding{}) + + if obj == nil { + return nil, err + } + return obj.(*v1.Binding), err +} + +func (c *FakePods) GetLogs(name string, opts *v1.PodLogOptions) *restclient.Request { + action := core.GenericActionImpl{} + action.Verb = "get" + action.Namespace = c.ns + action.Resource = podsResource + action.Subresource = "log" + action.Value = opts + + _, _ = c.Fake.Invokes(action, &v1.Pod{}) + return &restclient.Request{} +} + +func (c *FakePods) Evict(eviction *policy.Eviction) error { + action := core.CreateActionImpl{} + action.Verb = "create" + action.Resource = podsResource + action.Subresource = "eviction" + action.Object = eviction + + _, err := c.Fake.Invokes(action, eviction) + return err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_podtemplate.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_podtemplate.go new file mode 100644 index 0000000000..307f30594e --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_podtemplate.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakePodTemplates implements PodTemplateInterface +type FakePodTemplates struct { + Fake *FakeCoreV1 + ns string +} + +var podtemplatesResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "podtemplates"} + +var podtemplatesKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "PodTemplate"} + +// Get takes name of the podTemplate, and returns the corresponding podTemplate object, and an error if there is any. +func (c *FakePodTemplates) Get(name string, options v1.GetOptions) (result *corev1.PodTemplate, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(podtemplatesResource, c.ns, name), &corev1.PodTemplate{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.PodTemplate), err +} + +// List takes label and field selectors, and returns the list of PodTemplates that match those selectors. +func (c *FakePodTemplates) List(opts v1.ListOptions) (result *corev1.PodTemplateList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(podtemplatesResource, podtemplatesKind, c.ns, opts), &corev1.PodTemplateList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.PodTemplateList{ListMeta: obj.(*corev1.PodTemplateList).ListMeta} + for _, item := range obj.(*corev1.PodTemplateList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested podTemplates. +func (c *FakePodTemplates) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(podtemplatesResource, c.ns, opts)) + +} + +// Create takes the representation of a podTemplate and creates it. Returns the server's representation of the podTemplate, and an error, if there is any. +func (c *FakePodTemplates) Create(podTemplate *corev1.PodTemplate) (result *corev1.PodTemplate, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(podtemplatesResource, c.ns, podTemplate), &corev1.PodTemplate{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.PodTemplate), err +} + +// Update takes the representation of a podTemplate and updates it. Returns the server's representation of the podTemplate, and an error, if there is any. +func (c *FakePodTemplates) Update(podTemplate *corev1.PodTemplate) (result *corev1.PodTemplate, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(podtemplatesResource, c.ns, podTemplate), &corev1.PodTemplate{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.PodTemplate), err +} + +// Delete takes name of the podTemplate and deletes it. Returns an error if one occurs. +func (c *FakePodTemplates) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(podtemplatesResource, c.ns, name), &corev1.PodTemplate{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakePodTemplates) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(podtemplatesResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.PodTemplateList{}) + return err +} + +// Patch applies the patch and returns the patched podTemplate. +func (c *FakePodTemplates) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.PodTemplate, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(podtemplatesResource, c.ns, name, pt, data, subresources...), &corev1.PodTemplate{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.PodTemplate), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_replicationcontroller.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_replicationcontroller.go new file mode 100644 index 0000000000..6de94c1482 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_replicationcontroller.go @@ -0,0 +1,163 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + autoscalingv1 "k8s.io/api/autoscaling/v1" + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeReplicationControllers implements ReplicationControllerInterface +type FakeReplicationControllers struct { + Fake *FakeCoreV1 + ns string +} + +var replicationcontrollersResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "replicationcontrollers"} + +var replicationcontrollersKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "ReplicationController"} + +// Get takes name of the replicationController, and returns the corresponding replicationController object, and an error if there is any. +func (c *FakeReplicationControllers) Get(name string, options v1.GetOptions) (result *corev1.ReplicationController, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(replicationcontrollersResource, c.ns, name), &corev1.ReplicationController{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ReplicationController), err +} + +// List takes label and field selectors, and returns the list of ReplicationControllers that match those selectors. +func (c *FakeReplicationControllers) List(opts v1.ListOptions) (result *corev1.ReplicationControllerList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(replicationcontrollersResource, replicationcontrollersKind, c.ns, opts), &corev1.ReplicationControllerList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.ReplicationControllerList{ListMeta: obj.(*corev1.ReplicationControllerList).ListMeta} + for _, item := range obj.(*corev1.ReplicationControllerList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested replicationControllers. +func (c *FakeReplicationControllers) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(replicationcontrollersResource, c.ns, opts)) + +} + +// Create takes the representation of a replicationController and creates it. Returns the server's representation of the replicationController, and an error, if there is any. +func (c *FakeReplicationControllers) Create(replicationController *corev1.ReplicationController) (result *corev1.ReplicationController, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(replicationcontrollersResource, c.ns, replicationController), &corev1.ReplicationController{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ReplicationController), err +} + +// Update takes the representation of a replicationController and updates it. Returns the server's representation of the replicationController, and an error, if there is any. +func (c *FakeReplicationControllers) Update(replicationController *corev1.ReplicationController) (result *corev1.ReplicationController, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(replicationcontrollersResource, c.ns, replicationController), &corev1.ReplicationController{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ReplicationController), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeReplicationControllers) UpdateStatus(replicationController *corev1.ReplicationController) (*corev1.ReplicationController, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(replicationcontrollersResource, "status", c.ns, replicationController), &corev1.ReplicationController{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ReplicationController), err +} + +// Delete takes name of the replicationController and deletes it. Returns an error if one occurs. +func (c *FakeReplicationControllers) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(replicationcontrollersResource, c.ns, name), &corev1.ReplicationController{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeReplicationControllers) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(replicationcontrollersResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.ReplicationControllerList{}) + return err +} + +// Patch applies the patch and returns the patched replicationController. +func (c *FakeReplicationControllers) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.ReplicationController, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(replicationcontrollersResource, c.ns, name, pt, data, subresources...), &corev1.ReplicationController{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ReplicationController), err +} + +// GetScale takes name of the replicationController, and returns the corresponding scale object, and an error if there is any. +func (c *FakeReplicationControllers) GetScale(replicationControllerName string, options v1.GetOptions) (result *autoscalingv1.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetSubresourceAction(replicationcontrollersResource, c.ns, "scale", replicationControllerName), &autoscalingv1.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.Scale), err +} + +// UpdateScale takes the representation of a scale and updates it. Returns the server's representation of the scale, and an error, if there is any. +func (c *FakeReplicationControllers) UpdateScale(replicationControllerName string, scale *autoscalingv1.Scale) (result *autoscalingv1.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(replicationcontrollersResource, "scale", c.ns, scale), &autoscalingv1.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*autoscalingv1.Scale), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_resourcequota.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_resourcequota.go new file mode 100644 index 0000000000..b521f7120b --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_resourcequota.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeResourceQuotas implements ResourceQuotaInterface +type FakeResourceQuotas struct { + Fake *FakeCoreV1 + ns string +} + +var resourcequotasResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "resourcequotas"} + +var resourcequotasKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "ResourceQuota"} + +// Get takes name of the resourceQuota, and returns the corresponding resourceQuota object, and an error if there is any. +func (c *FakeResourceQuotas) Get(name string, options v1.GetOptions) (result *corev1.ResourceQuota, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(resourcequotasResource, c.ns, name), &corev1.ResourceQuota{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ResourceQuota), err +} + +// List takes label and field selectors, and returns the list of ResourceQuotas that match those selectors. +func (c *FakeResourceQuotas) List(opts v1.ListOptions) (result *corev1.ResourceQuotaList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(resourcequotasResource, resourcequotasKind, c.ns, opts), &corev1.ResourceQuotaList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.ResourceQuotaList{ListMeta: obj.(*corev1.ResourceQuotaList).ListMeta} + for _, item := range obj.(*corev1.ResourceQuotaList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested resourceQuotas. +func (c *FakeResourceQuotas) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(resourcequotasResource, c.ns, opts)) + +} + +// Create takes the representation of a resourceQuota and creates it. Returns the server's representation of the resourceQuota, and an error, if there is any. +func (c *FakeResourceQuotas) Create(resourceQuota *corev1.ResourceQuota) (result *corev1.ResourceQuota, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(resourcequotasResource, c.ns, resourceQuota), &corev1.ResourceQuota{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ResourceQuota), err +} + +// Update takes the representation of a resourceQuota and updates it. Returns the server's representation of the resourceQuota, and an error, if there is any. +func (c *FakeResourceQuotas) Update(resourceQuota *corev1.ResourceQuota) (result *corev1.ResourceQuota, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(resourcequotasResource, c.ns, resourceQuota), &corev1.ResourceQuota{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ResourceQuota), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeResourceQuotas) UpdateStatus(resourceQuota *corev1.ResourceQuota) (*corev1.ResourceQuota, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(resourcequotasResource, "status", c.ns, resourceQuota), &corev1.ResourceQuota{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ResourceQuota), err +} + +// Delete takes name of the resourceQuota and deletes it. Returns an error if one occurs. +func (c *FakeResourceQuotas) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(resourcequotasResource, c.ns, name), &corev1.ResourceQuota{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeResourceQuotas) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(resourcequotasResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.ResourceQuotaList{}) + return err +} + +// Patch applies the patch and returns the patched resourceQuota. +func (c *FakeResourceQuotas) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.ResourceQuota, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(resourcequotasResource, c.ns, name, pt, data, subresources...), &corev1.ResourceQuota{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ResourceQuota), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_secret.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_secret.go new file mode 100644 index 0000000000..47dba9eff4 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_secret.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeSecrets implements SecretInterface +type FakeSecrets struct { + Fake *FakeCoreV1 + ns string +} + +var secretsResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "secrets"} + +var secretsKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "Secret"} + +// Get takes name of the secret, and returns the corresponding secret object, and an error if there is any. +func (c *FakeSecrets) Get(name string, options v1.GetOptions) (result *corev1.Secret, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(secretsResource, c.ns, name), &corev1.Secret{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Secret), err +} + +// List takes label and field selectors, and returns the list of Secrets that match those selectors. +func (c *FakeSecrets) List(opts v1.ListOptions) (result *corev1.SecretList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(secretsResource, secretsKind, c.ns, opts), &corev1.SecretList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.SecretList{ListMeta: obj.(*corev1.SecretList).ListMeta} + for _, item := range obj.(*corev1.SecretList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested secrets. +func (c *FakeSecrets) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(secretsResource, c.ns, opts)) + +} + +// Create takes the representation of a secret and creates it. Returns the server's representation of the secret, and an error, if there is any. +func (c *FakeSecrets) Create(secret *corev1.Secret) (result *corev1.Secret, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(secretsResource, c.ns, secret), &corev1.Secret{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Secret), err +} + +// Update takes the representation of a secret and updates it. Returns the server's representation of the secret, and an error, if there is any. +func (c *FakeSecrets) Update(secret *corev1.Secret) (result *corev1.Secret, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(secretsResource, c.ns, secret), &corev1.Secret{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Secret), err +} + +// Delete takes name of the secret and deletes it. Returns an error if one occurs. +func (c *FakeSecrets) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(secretsResource, c.ns, name), &corev1.Secret{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeSecrets) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(secretsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.SecretList{}) + return err +} + +// Patch applies the patch and returns the patched secret. +func (c *FakeSecrets) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.Secret, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(secretsResource, c.ns, name, pt, data, subresources...), &corev1.Secret{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Secret), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_service.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_service.go new file mode 100644 index 0000000000..a65de49911 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_service.go @@ -0,0 +1,132 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeServices implements ServiceInterface +type FakeServices struct { + Fake *FakeCoreV1 + ns string +} + +var servicesResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "services"} + +var servicesKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "Service"} + +// Get takes name of the service, and returns the corresponding service object, and an error if there is any. +func (c *FakeServices) Get(name string, options v1.GetOptions) (result *corev1.Service, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(servicesResource, c.ns, name), &corev1.Service{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Service), err +} + +// List takes label and field selectors, and returns the list of Services that match those selectors. +func (c *FakeServices) List(opts v1.ListOptions) (result *corev1.ServiceList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(servicesResource, servicesKind, c.ns, opts), &corev1.ServiceList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.ServiceList{ListMeta: obj.(*corev1.ServiceList).ListMeta} + for _, item := range obj.(*corev1.ServiceList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested services. +func (c *FakeServices) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(servicesResource, c.ns, opts)) + +} + +// Create takes the representation of a service and creates it. Returns the server's representation of the service, and an error, if there is any. +func (c *FakeServices) Create(service *corev1.Service) (result *corev1.Service, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(servicesResource, c.ns, service), &corev1.Service{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Service), err +} + +// Update takes the representation of a service and updates it. Returns the server's representation of the service, and an error, if there is any. +func (c *FakeServices) Update(service *corev1.Service) (result *corev1.Service, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(servicesResource, c.ns, service), &corev1.Service{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Service), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeServices) UpdateStatus(service *corev1.Service) (*corev1.Service, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(servicesResource, "status", c.ns, service), &corev1.Service{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Service), err +} + +// Delete takes name of the service and deletes it. Returns an error if one occurs. +func (c *FakeServices) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(servicesResource, c.ns, name), &corev1.Service{}) + + return err +} + +// Patch applies the patch and returns the patched service. +func (c *FakeServices) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.Service, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(servicesResource, c.ns, name, pt, data, subresources...), &corev1.Service{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.Service), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_service_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_service_expansion.go new file mode 100644 index 0000000000..92e4930d71 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_service_expansion.go @@ -0,0 +1,26 @@ +/* +Copyright 2014 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 fake + +import ( + restclient "k8s.io/client-go/rest" + core "k8s.io/client-go/testing" +) + +func (c *FakeServices) ProxyGet(scheme, name, port, path string, params map[string]string) restclient.ResponseWrapper { + return c.Fake.InvokesProxy(core.NewProxyGetAction(servicesResource, c.ns, scheme, name, port, path, params)) +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_serviceaccount.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_serviceaccount.go new file mode 100644 index 0000000000..5b6d8f8be5 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_serviceaccount.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeServiceAccounts implements ServiceAccountInterface +type FakeServiceAccounts struct { + Fake *FakeCoreV1 + ns string +} + +var serviceaccountsResource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "serviceaccounts"} + +var serviceaccountsKind = schema.GroupVersionKind{Group: "", Version: "v1", Kind: "ServiceAccount"} + +// Get takes name of the serviceAccount, and returns the corresponding serviceAccount object, and an error if there is any. +func (c *FakeServiceAccounts) Get(name string, options v1.GetOptions) (result *corev1.ServiceAccount, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(serviceaccountsResource, c.ns, name), &corev1.ServiceAccount{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ServiceAccount), err +} + +// List takes label and field selectors, and returns the list of ServiceAccounts that match those selectors. +func (c *FakeServiceAccounts) List(opts v1.ListOptions) (result *corev1.ServiceAccountList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(serviceaccountsResource, serviceaccountsKind, c.ns, opts), &corev1.ServiceAccountList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &corev1.ServiceAccountList{ListMeta: obj.(*corev1.ServiceAccountList).ListMeta} + for _, item := range obj.(*corev1.ServiceAccountList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested serviceAccounts. +func (c *FakeServiceAccounts) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(serviceaccountsResource, c.ns, opts)) + +} + +// Create takes the representation of a serviceAccount and creates it. Returns the server's representation of the serviceAccount, and an error, if there is any. +func (c *FakeServiceAccounts) Create(serviceAccount *corev1.ServiceAccount) (result *corev1.ServiceAccount, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(serviceaccountsResource, c.ns, serviceAccount), &corev1.ServiceAccount{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ServiceAccount), err +} + +// Update takes the representation of a serviceAccount and updates it. Returns the server's representation of the serviceAccount, and an error, if there is any. +func (c *FakeServiceAccounts) Update(serviceAccount *corev1.ServiceAccount) (result *corev1.ServiceAccount, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(serviceaccountsResource, c.ns, serviceAccount), &corev1.ServiceAccount{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ServiceAccount), err +} + +// Delete takes name of the serviceAccount and deletes it. Returns an error if one occurs. +func (c *FakeServiceAccounts) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(serviceaccountsResource, c.ns, name), &corev1.ServiceAccount{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeServiceAccounts) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(serviceaccountsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &corev1.ServiceAccountList{}) + return err +} + +// Patch applies the patch and returns the patched serviceAccount. +func (c *FakeServiceAccounts) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *corev1.ServiceAccount, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(serviceaccountsResource, c.ns, name, pt, data, subresources...), &corev1.ServiceAccount{}) + + if obj == nil { + return nil, err + } + return obj.(*corev1.ServiceAccount), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_serviceaccount_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_serviceaccount_expansion.go new file mode 100644 index 0000000000..a0efbcc2fe --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/core/v1/fake/fake_serviceaccount_expansion.go @@ -0,0 +1,31 @@ +/* +Copyright 2018 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 fake + +import ( + authenticationv1 "k8s.io/api/authentication/v1" + core "k8s.io/client-go/testing" +) + +func (c *FakeServiceAccounts) CreateToken(name string, tr *authenticationv1.TokenRequest) (*authenticationv1.TokenRequest, error) { + obj, err := c.Fake.Invokes(core.NewCreateSubresourceAction(serviceaccountsResource, name, "token", c.ns, tr), &authenticationv1.TokenRequest{}) + + if obj == nil { + return nil, err + } + return obj.(*authenticationv1.TokenRequest), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/events/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/events/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/events/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/events/v1beta1/fake/fake_event.go b/vendor/k8s.io/client-go/kubernetes/typed/events/v1beta1/fake/fake_event.go new file mode 100644 index 0000000000..ef76ec1318 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/events/v1beta1/fake/fake_event.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/events/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeEvents implements EventInterface +type FakeEvents struct { + Fake *FakeEventsV1beta1 + ns string +} + +var eventsResource = schema.GroupVersionResource{Group: "events.k8s.io", Version: "v1beta1", Resource: "events"} + +var eventsKind = schema.GroupVersionKind{Group: "events.k8s.io", Version: "v1beta1", Kind: "Event"} + +// Get takes name of the event, and returns the corresponding event object, and an error if there is any. +func (c *FakeEvents) Get(name string, options v1.GetOptions) (result *v1beta1.Event, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(eventsResource, c.ns, name), &v1beta1.Event{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Event), err +} + +// List takes label and field selectors, and returns the list of Events that match those selectors. +func (c *FakeEvents) List(opts v1.ListOptions) (result *v1beta1.EventList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(eventsResource, eventsKind, c.ns, opts), &v1beta1.EventList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.EventList{ListMeta: obj.(*v1beta1.EventList).ListMeta} + for _, item := range obj.(*v1beta1.EventList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested events. +func (c *FakeEvents) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(eventsResource, c.ns, opts)) + +} + +// Create takes the representation of a event and creates it. Returns the server's representation of the event, and an error, if there is any. +func (c *FakeEvents) Create(event *v1beta1.Event) (result *v1beta1.Event, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(eventsResource, c.ns, event), &v1beta1.Event{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Event), err +} + +// Update takes the representation of a event and updates it. Returns the server's representation of the event, and an error, if there is any. +func (c *FakeEvents) Update(event *v1beta1.Event) (result *v1beta1.Event, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(eventsResource, c.ns, event), &v1beta1.Event{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Event), err +} + +// Delete takes name of the event and deletes it. Returns an error if one occurs. +func (c *FakeEvents) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(eventsResource, c.ns, name), &v1beta1.Event{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeEvents) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(eventsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.EventList{}) + return err +} + +// Patch applies the patch and returns the patched event. +func (c *FakeEvents) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.Event, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(eventsResource, c.ns, name, pt, data, subresources...), &v1beta1.Event{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Event), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/events/v1beta1/fake/fake_events_client.go b/vendor/k8s.io/client-go/kubernetes/typed/events/v1beta1/fake/fake_events_client.go new file mode 100644 index 0000000000..875c774e38 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/events/v1beta1/fake/fake_events_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/events/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeEventsV1beta1 struct { + *testing.Fake +} + +func (c *FakeEventsV1beta1) Events(namespace string) v1beta1.EventInterface { + return &FakeEvents{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeEventsV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_daemonset.go b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_daemonset.go new file mode 100644 index 0000000000..4c98660607 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_daemonset.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/extensions/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeDaemonSets implements DaemonSetInterface +type FakeDaemonSets struct { + Fake *FakeExtensionsV1beta1 + ns string +} + +var daemonsetsResource = schema.GroupVersionResource{Group: "extensions", Version: "v1beta1", Resource: "daemonsets"} + +var daemonsetsKind = schema.GroupVersionKind{Group: "extensions", Version: "v1beta1", Kind: "DaemonSet"} + +// Get takes name of the daemonSet, and returns the corresponding daemonSet object, and an error if there is any. +func (c *FakeDaemonSets) Get(name string, options v1.GetOptions) (result *v1beta1.DaemonSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(daemonsetsResource, c.ns, name), &v1beta1.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.DaemonSet), err +} + +// List takes label and field selectors, and returns the list of DaemonSets that match those selectors. +func (c *FakeDaemonSets) List(opts v1.ListOptions) (result *v1beta1.DaemonSetList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(daemonsetsResource, daemonsetsKind, c.ns, opts), &v1beta1.DaemonSetList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.DaemonSetList{ListMeta: obj.(*v1beta1.DaemonSetList).ListMeta} + for _, item := range obj.(*v1beta1.DaemonSetList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested daemonSets. +func (c *FakeDaemonSets) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(daemonsetsResource, c.ns, opts)) + +} + +// Create takes the representation of a daemonSet and creates it. Returns the server's representation of the daemonSet, and an error, if there is any. +func (c *FakeDaemonSets) Create(daemonSet *v1beta1.DaemonSet) (result *v1beta1.DaemonSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(daemonsetsResource, c.ns, daemonSet), &v1beta1.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.DaemonSet), err +} + +// Update takes the representation of a daemonSet and updates it. Returns the server's representation of the daemonSet, and an error, if there is any. +func (c *FakeDaemonSets) Update(daemonSet *v1beta1.DaemonSet) (result *v1beta1.DaemonSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(daemonsetsResource, c.ns, daemonSet), &v1beta1.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.DaemonSet), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeDaemonSets) UpdateStatus(daemonSet *v1beta1.DaemonSet) (*v1beta1.DaemonSet, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(daemonsetsResource, "status", c.ns, daemonSet), &v1beta1.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.DaemonSet), err +} + +// Delete takes name of the daemonSet and deletes it. Returns an error if one occurs. +func (c *FakeDaemonSets) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(daemonsetsResource, c.ns, name), &v1beta1.DaemonSet{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeDaemonSets) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(daemonsetsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.DaemonSetList{}) + return err +} + +// Patch applies the patch and returns the patched daemonSet. +func (c *FakeDaemonSets) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.DaemonSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(daemonsetsResource, c.ns, name, pt, data, subresources...), &v1beta1.DaemonSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.DaemonSet), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_deployment.go b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_deployment.go new file mode 100644 index 0000000000..7b7df45cc3 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_deployment.go @@ -0,0 +1,162 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/extensions/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeDeployments implements DeploymentInterface +type FakeDeployments struct { + Fake *FakeExtensionsV1beta1 + ns string +} + +var deploymentsResource = schema.GroupVersionResource{Group: "extensions", Version: "v1beta1", Resource: "deployments"} + +var deploymentsKind = schema.GroupVersionKind{Group: "extensions", Version: "v1beta1", Kind: "Deployment"} + +// Get takes name of the deployment, and returns the corresponding deployment object, and an error if there is any. +func (c *FakeDeployments) Get(name string, options v1.GetOptions) (result *v1beta1.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(deploymentsResource, c.ns, name), &v1beta1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Deployment), err +} + +// List takes label and field selectors, and returns the list of Deployments that match those selectors. +func (c *FakeDeployments) List(opts v1.ListOptions) (result *v1beta1.DeploymentList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(deploymentsResource, deploymentsKind, c.ns, opts), &v1beta1.DeploymentList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.DeploymentList{ListMeta: obj.(*v1beta1.DeploymentList).ListMeta} + for _, item := range obj.(*v1beta1.DeploymentList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested deployments. +func (c *FakeDeployments) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(deploymentsResource, c.ns, opts)) + +} + +// Create takes the representation of a deployment and creates it. Returns the server's representation of the deployment, and an error, if there is any. +func (c *FakeDeployments) Create(deployment *v1beta1.Deployment) (result *v1beta1.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(deploymentsResource, c.ns, deployment), &v1beta1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Deployment), err +} + +// Update takes the representation of a deployment and updates it. Returns the server's representation of the deployment, and an error, if there is any. +func (c *FakeDeployments) Update(deployment *v1beta1.Deployment) (result *v1beta1.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(deploymentsResource, c.ns, deployment), &v1beta1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Deployment), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeDeployments) UpdateStatus(deployment *v1beta1.Deployment) (*v1beta1.Deployment, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(deploymentsResource, "status", c.ns, deployment), &v1beta1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Deployment), err +} + +// Delete takes name of the deployment and deletes it. Returns an error if one occurs. +func (c *FakeDeployments) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(deploymentsResource, c.ns, name), &v1beta1.Deployment{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeDeployments) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(deploymentsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.DeploymentList{}) + return err +} + +// Patch applies the patch and returns the patched deployment. +func (c *FakeDeployments) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.Deployment, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(deploymentsResource, c.ns, name, pt, data, subresources...), &v1beta1.Deployment{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Deployment), err +} + +// GetScale takes name of the deployment, and returns the corresponding scale object, and an error if there is any. +func (c *FakeDeployments) GetScale(deploymentName string, options v1.GetOptions) (result *v1beta1.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetSubresourceAction(deploymentsResource, c.ns, "scale", deploymentName), &v1beta1.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Scale), err +} + +// UpdateScale takes the representation of a scale and updates it. Returns the server's representation of the scale, and an error, if there is any. +func (c *FakeDeployments) UpdateScale(deploymentName string, scale *v1beta1.Scale) (result *v1beta1.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(deploymentsResource, "scale", c.ns, scale), &v1beta1.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Scale), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_deployment_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_deployment_expansion.go new file mode 100644 index 0000000000..af2bc0f713 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_deployment_expansion.go @@ -0,0 +1,33 @@ +/* +Copyright 2014 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 fake + +import ( + "k8s.io/api/extensions/v1beta1" + core "k8s.io/client-go/testing" +) + +func (c *FakeDeployments) Rollback(deploymentRollback *v1beta1.DeploymentRollback) error { + action := core.CreateActionImpl{} + action.Verb = "create" + action.Resource = deploymentsResource + action.Subresource = "rollback" + action.Object = deploymentRollback + + _, err := c.Fake.Invokes(action, deploymentRollback) + return err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_extensions_client.go b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_extensions_client.go new file mode 100644 index 0000000000..0282c0b499 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_extensions_client.go @@ -0,0 +1,56 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/extensions/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeExtensionsV1beta1 struct { + *testing.Fake +} + +func (c *FakeExtensionsV1beta1) DaemonSets(namespace string) v1beta1.DaemonSetInterface { + return &FakeDaemonSets{c, namespace} +} + +func (c *FakeExtensionsV1beta1) Deployments(namespace string) v1beta1.DeploymentInterface { + return &FakeDeployments{c, namespace} +} + +func (c *FakeExtensionsV1beta1) Ingresses(namespace string) v1beta1.IngressInterface { + return &FakeIngresses{c, namespace} +} + +func (c *FakeExtensionsV1beta1) PodSecurityPolicies() v1beta1.PodSecurityPolicyInterface { + return &FakePodSecurityPolicies{c} +} + +func (c *FakeExtensionsV1beta1) ReplicaSets(namespace string) v1beta1.ReplicaSetInterface { + return &FakeReplicaSets{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeExtensionsV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_ingress.go b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_ingress.go new file mode 100644 index 0000000000..01c2521401 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_ingress.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/extensions/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeIngresses implements IngressInterface +type FakeIngresses struct { + Fake *FakeExtensionsV1beta1 + ns string +} + +var ingressesResource = schema.GroupVersionResource{Group: "extensions", Version: "v1beta1", Resource: "ingresses"} + +var ingressesKind = schema.GroupVersionKind{Group: "extensions", Version: "v1beta1", Kind: "Ingress"} + +// Get takes name of the ingress, and returns the corresponding ingress object, and an error if there is any. +func (c *FakeIngresses) Get(name string, options v1.GetOptions) (result *v1beta1.Ingress, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(ingressesResource, c.ns, name), &v1beta1.Ingress{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Ingress), err +} + +// List takes label and field selectors, and returns the list of Ingresses that match those selectors. +func (c *FakeIngresses) List(opts v1.ListOptions) (result *v1beta1.IngressList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(ingressesResource, ingressesKind, c.ns, opts), &v1beta1.IngressList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.IngressList{ListMeta: obj.(*v1beta1.IngressList).ListMeta} + for _, item := range obj.(*v1beta1.IngressList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested ingresses. +func (c *FakeIngresses) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(ingressesResource, c.ns, opts)) + +} + +// Create takes the representation of a ingress and creates it. Returns the server's representation of the ingress, and an error, if there is any. +func (c *FakeIngresses) Create(ingress *v1beta1.Ingress) (result *v1beta1.Ingress, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(ingressesResource, c.ns, ingress), &v1beta1.Ingress{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Ingress), err +} + +// Update takes the representation of a ingress and updates it. Returns the server's representation of the ingress, and an error, if there is any. +func (c *FakeIngresses) Update(ingress *v1beta1.Ingress) (result *v1beta1.Ingress, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(ingressesResource, c.ns, ingress), &v1beta1.Ingress{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Ingress), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeIngresses) UpdateStatus(ingress *v1beta1.Ingress) (*v1beta1.Ingress, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(ingressesResource, "status", c.ns, ingress), &v1beta1.Ingress{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Ingress), err +} + +// Delete takes name of the ingress and deletes it. Returns an error if one occurs. +func (c *FakeIngresses) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(ingressesResource, c.ns, name), &v1beta1.Ingress{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeIngresses) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(ingressesResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.IngressList{}) + return err +} + +// Patch applies the patch and returns the patched ingress. +func (c *FakeIngresses) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.Ingress, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(ingressesResource, c.ns, name, pt, data, subresources...), &v1beta1.Ingress{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Ingress), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_podsecuritypolicy.go b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_podsecuritypolicy.go new file mode 100644 index 0000000000..b97a34416e --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_podsecuritypolicy.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/extensions/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakePodSecurityPolicies implements PodSecurityPolicyInterface +type FakePodSecurityPolicies struct { + Fake *FakeExtensionsV1beta1 +} + +var podsecuritypoliciesResource = schema.GroupVersionResource{Group: "extensions", Version: "v1beta1", Resource: "podsecuritypolicies"} + +var podsecuritypoliciesKind = schema.GroupVersionKind{Group: "extensions", Version: "v1beta1", Kind: "PodSecurityPolicy"} + +// Get takes name of the podSecurityPolicy, and returns the corresponding podSecurityPolicy object, and an error if there is any. +func (c *FakePodSecurityPolicies) Get(name string, options v1.GetOptions) (result *v1beta1.PodSecurityPolicy, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(podsecuritypoliciesResource, name), &v1beta1.PodSecurityPolicy{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodSecurityPolicy), err +} + +// List takes label and field selectors, and returns the list of PodSecurityPolicies that match those selectors. +func (c *FakePodSecurityPolicies) List(opts v1.ListOptions) (result *v1beta1.PodSecurityPolicyList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(podsecuritypoliciesResource, podsecuritypoliciesKind, opts), &v1beta1.PodSecurityPolicyList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.PodSecurityPolicyList{ListMeta: obj.(*v1beta1.PodSecurityPolicyList).ListMeta} + for _, item := range obj.(*v1beta1.PodSecurityPolicyList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested podSecurityPolicies. +func (c *FakePodSecurityPolicies) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(podsecuritypoliciesResource, opts)) +} + +// Create takes the representation of a podSecurityPolicy and creates it. Returns the server's representation of the podSecurityPolicy, and an error, if there is any. +func (c *FakePodSecurityPolicies) Create(podSecurityPolicy *v1beta1.PodSecurityPolicy) (result *v1beta1.PodSecurityPolicy, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(podsecuritypoliciesResource, podSecurityPolicy), &v1beta1.PodSecurityPolicy{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodSecurityPolicy), err +} + +// Update takes the representation of a podSecurityPolicy and updates it. Returns the server's representation of the podSecurityPolicy, and an error, if there is any. +func (c *FakePodSecurityPolicies) Update(podSecurityPolicy *v1beta1.PodSecurityPolicy) (result *v1beta1.PodSecurityPolicy, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(podsecuritypoliciesResource, podSecurityPolicy), &v1beta1.PodSecurityPolicy{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodSecurityPolicy), err +} + +// Delete takes name of the podSecurityPolicy and deletes it. Returns an error if one occurs. +func (c *FakePodSecurityPolicies) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(podsecuritypoliciesResource, name), &v1beta1.PodSecurityPolicy{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakePodSecurityPolicies) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(podsecuritypoliciesResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.PodSecurityPolicyList{}) + return err +} + +// Patch applies the patch and returns the patched podSecurityPolicy. +func (c *FakePodSecurityPolicies) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.PodSecurityPolicy, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(podsecuritypoliciesResource, name, pt, data, subresources...), &v1beta1.PodSecurityPolicy{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodSecurityPolicy), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_replicaset.go b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_replicaset.go new file mode 100644 index 0000000000..7ed16af904 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/extensions/v1beta1/fake/fake_replicaset.go @@ -0,0 +1,162 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/extensions/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeReplicaSets implements ReplicaSetInterface +type FakeReplicaSets struct { + Fake *FakeExtensionsV1beta1 + ns string +} + +var replicasetsResource = schema.GroupVersionResource{Group: "extensions", Version: "v1beta1", Resource: "replicasets"} + +var replicasetsKind = schema.GroupVersionKind{Group: "extensions", Version: "v1beta1", Kind: "ReplicaSet"} + +// Get takes name of the replicaSet, and returns the corresponding replicaSet object, and an error if there is any. +func (c *FakeReplicaSets) Get(name string, options v1.GetOptions) (result *v1beta1.ReplicaSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(replicasetsResource, c.ns, name), &v1beta1.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ReplicaSet), err +} + +// List takes label and field selectors, and returns the list of ReplicaSets that match those selectors. +func (c *FakeReplicaSets) List(opts v1.ListOptions) (result *v1beta1.ReplicaSetList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(replicasetsResource, replicasetsKind, c.ns, opts), &v1beta1.ReplicaSetList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.ReplicaSetList{ListMeta: obj.(*v1beta1.ReplicaSetList).ListMeta} + for _, item := range obj.(*v1beta1.ReplicaSetList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested replicaSets. +func (c *FakeReplicaSets) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(replicasetsResource, c.ns, opts)) + +} + +// Create takes the representation of a replicaSet and creates it. Returns the server's representation of the replicaSet, and an error, if there is any. +func (c *FakeReplicaSets) Create(replicaSet *v1beta1.ReplicaSet) (result *v1beta1.ReplicaSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(replicasetsResource, c.ns, replicaSet), &v1beta1.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ReplicaSet), err +} + +// Update takes the representation of a replicaSet and updates it. Returns the server's representation of the replicaSet, and an error, if there is any. +func (c *FakeReplicaSets) Update(replicaSet *v1beta1.ReplicaSet) (result *v1beta1.ReplicaSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(replicasetsResource, c.ns, replicaSet), &v1beta1.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ReplicaSet), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeReplicaSets) UpdateStatus(replicaSet *v1beta1.ReplicaSet) (*v1beta1.ReplicaSet, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(replicasetsResource, "status", c.ns, replicaSet), &v1beta1.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ReplicaSet), err +} + +// Delete takes name of the replicaSet and deletes it. Returns an error if one occurs. +func (c *FakeReplicaSets) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(replicasetsResource, c.ns, name), &v1beta1.ReplicaSet{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeReplicaSets) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(replicasetsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.ReplicaSetList{}) + return err +} + +// Patch applies the patch and returns the patched replicaSet. +func (c *FakeReplicaSets) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.ReplicaSet, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(replicasetsResource, c.ns, name, pt, data, subresources...), &v1beta1.ReplicaSet{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ReplicaSet), err +} + +// GetScale takes name of the replicaSet, and returns the corresponding scale object, and an error if there is any. +func (c *FakeReplicaSets) GetScale(replicaSetName string, options v1.GetOptions) (result *v1beta1.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetSubresourceAction(replicasetsResource, c.ns, "scale", replicaSetName), &v1beta1.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Scale), err +} + +// UpdateScale takes the representation of a scale and updates it. Returns the server's representation of the scale, and an error, if there is any. +func (c *FakeReplicaSets) UpdateScale(replicaSetName string, scale *v1beta1.Scale) (result *v1beta1.Scale, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(replicasetsResource, "scale", c.ns, scale), &v1beta1.Scale{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Scale), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/networking/v1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/networking/v1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/networking/v1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/networking/v1/fake/fake_networking_client.go b/vendor/k8s.io/client-go/kubernetes/typed/networking/v1/fake/fake_networking_client.go new file mode 100644 index 0000000000..6b135c6369 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/networking/v1/fake/fake_networking_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1 "k8s.io/client-go/kubernetes/typed/networking/v1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeNetworkingV1 struct { + *testing.Fake +} + +func (c *FakeNetworkingV1) NetworkPolicies(namespace string) v1.NetworkPolicyInterface { + return &FakeNetworkPolicies{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeNetworkingV1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/networking/v1/fake/fake_networkpolicy.go b/vendor/k8s.io/client-go/kubernetes/typed/networking/v1/fake/fake_networkpolicy.go new file mode 100644 index 0000000000..58667c481a --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/networking/v1/fake/fake_networkpolicy.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + networkingv1 "k8s.io/api/networking/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeNetworkPolicies implements NetworkPolicyInterface +type FakeNetworkPolicies struct { + Fake *FakeNetworkingV1 + ns string +} + +var networkpoliciesResource = schema.GroupVersionResource{Group: "networking.k8s.io", Version: "v1", Resource: "networkpolicies"} + +var networkpoliciesKind = schema.GroupVersionKind{Group: "networking.k8s.io", Version: "v1", Kind: "NetworkPolicy"} + +// Get takes name of the networkPolicy, and returns the corresponding networkPolicy object, and an error if there is any. +func (c *FakeNetworkPolicies) Get(name string, options v1.GetOptions) (result *networkingv1.NetworkPolicy, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(networkpoliciesResource, c.ns, name), &networkingv1.NetworkPolicy{}) + + if obj == nil { + return nil, err + } + return obj.(*networkingv1.NetworkPolicy), err +} + +// List takes label and field selectors, and returns the list of NetworkPolicies that match those selectors. +func (c *FakeNetworkPolicies) List(opts v1.ListOptions) (result *networkingv1.NetworkPolicyList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(networkpoliciesResource, networkpoliciesKind, c.ns, opts), &networkingv1.NetworkPolicyList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &networkingv1.NetworkPolicyList{ListMeta: obj.(*networkingv1.NetworkPolicyList).ListMeta} + for _, item := range obj.(*networkingv1.NetworkPolicyList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested networkPolicies. +func (c *FakeNetworkPolicies) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(networkpoliciesResource, c.ns, opts)) + +} + +// Create takes the representation of a networkPolicy and creates it. Returns the server's representation of the networkPolicy, and an error, if there is any. +func (c *FakeNetworkPolicies) Create(networkPolicy *networkingv1.NetworkPolicy) (result *networkingv1.NetworkPolicy, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(networkpoliciesResource, c.ns, networkPolicy), &networkingv1.NetworkPolicy{}) + + if obj == nil { + return nil, err + } + return obj.(*networkingv1.NetworkPolicy), err +} + +// Update takes the representation of a networkPolicy and updates it. Returns the server's representation of the networkPolicy, and an error, if there is any. +func (c *FakeNetworkPolicies) Update(networkPolicy *networkingv1.NetworkPolicy) (result *networkingv1.NetworkPolicy, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(networkpoliciesResource, c.ns, networkPolicy), &networkingv1.NetworkPolicy{}) + + if obj == nil { + return nil, err + } + return obj.(*networkingv1.NetworkPolicy), err +} + +// Delete takes name of the networkPolicy and deletes it. Returns an error if one occurs. +func (c *FakeNetworkPolicies) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(networkpoliciesResource, c.ns, name), &networkingv1.NetworkPolicy{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeNetworkPolicies) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(networkpoliciesResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &networkingv1.NetworkPolicyList{}) + return err +} + +// Patch applies the patch and returns the patched networkPolicy. +func (c *FakeNetworkPolicies) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *networkingv1.NetworkPolicy, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(networkpoliciesResource, c.ns, name, pt, data, subresources...), &networkingv1.NetworkPolicy{}) + + if obj == nil { + return nil, err + } + return obj.(*networkingv1.NetworkPolicy), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_eviction.go b/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_eviction.go new file mode 100644 index 0000000000..b8f6f3eae2 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_eviction.go @@ -0,0 +1,25 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +// FakeEvictions implements EvictionInterface +type FakeEvictions struct { + Fake *FakePolicyV1beta1 + ns string +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_eviction_expansion.go b/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_eviction_expansion.go new file mode 100644 index 0000000000..f3b5e93ab0 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_eviction_expansion.go @@ -0,0 +1,34 @@ +/* +Copyright 2016 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 fake + +import ( + policy "k8s.io/api/policy/v1beta1" + "k8s.io/apimachinery/pkg/runtime/schema" + core "k8s.io/client-go/testing" +) + +func (c *FakeEvictions) Evict(eviction *policy.Eviction) error { + action := core.GetActionImpl{} + action.Verb = "post" + action.Namespace = c.ns + action.Resource = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "pods"} + action.Subresource = "eviction" + action.Name = eviction.Name + _, err := c.Fake.Invokes(action, eviction) + return err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_poddisruptionbudget.go b/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_poddisruptionbudget.go new file mode 100644 index 0000000000..5bfbbca47f --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_poddisruptionbudget.go @@ -0,0 +1,140 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/policy/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakePodDisruptionBudgets implements PodDisruptionBudgetInterface +type FakePodDisruptionBudgets struct { + Fake *FakePolicyV1beta1 + ns string +} + +var poddisruptionbudgetsResource = schema.GroupVersionResource{Group: "policy", Version: "v1beta1", Resource: "poddisruptionbudgets"} + +var poddisruptionbudgetsKind = schema.GroupVersionKind{Group: "policy", Version: "v1beta1", Kind: "PodDisruptionBudget"} + +// Get takes name of the podDisruptionBudget, and returns the corresponding podDisruptionBudget object, and an error if there is any. +func (c *FakePodDisruptionBudgets) Get(name string, options v1.GetOptions) (result *v1beta1.PodDisruptionBudget, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(poddisruptionbudgetsResource, c.ns, name), &v1beta1.PodDisruptionBudget{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodDisruptionBudget), err +} + +// List takes label and field selectors, and returns the list of PodDisruptionBudgets that match those selectors. +func (c *FakePodDisruptionBudgets) List(opts v1.ListOptions) (result *v1beta1.PodDisruptionBudgetList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(poddisruptionbudgetsResource, poddisruptionbudgetsKind, c.ns, opts), &v1beta1.PodDisruptionBudgetList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.PodDisruptionBudgetList{ListMeta: obj.(*v1beta1.PodDisruptionBudgetList).ListMeta} + for _, item := range obj.(*v1beta1.PodDisruptionBudgetList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested podDisruptionBudgets. +func (c *FakePodDisruptionBudgets) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(poddisruptionbudgetsResource, c.ns, opts)) + +} + +// Create takes the representation of a podDisruptionBudget and creates it. Returns the server's representation of the podDisruptionBudget, and an error, if there is any. +func (c *FakePodDisruptionBudgets) Create(podDisruptionBudget *v1beta1.PodDisruptionBudget) (result *v1beta1.PodDisruptionBudget, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(poddisruptionbudgetsResource, c.ns, podDisruptionBudget), &v1beta1.PodDisruptionBudget{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodDisruptionBudget), err +} + +// Update takes the representation of a podDisruptionBudget and updates it. Returns the server's representation of the podDisruptionBudget, and an error, if there is any. +func (c *FakePodDisruptionBudgets) Update(podDisruptionBudget *v1beta1.PodDisruptionBudget) (result *v1beta1.PodDisruptionBudget, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(poddisruptionbudgetsResource, c.ns, podDisruptionBudget), &v1beta1.PodDisruptionBudget{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodDisruptionBudget), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakePodDisruptionBudgets) UpdateStatus(podDisruptionBudget *v1beta1.PodDisruptionBudget) (*v1beta1.PodDisruptionBudget, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(poddisruptionbudgetsResource, "status", c.ns, podDisruptionBudget), &v1beta1.PodDisruptionBudget{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodDisruptionBudget), err +} + +// Delete takes name of the podDisruptionBudget and deletes it. Returns an error if one occurs. +func (c *FakePodDisruptionBudgets) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(poddisruptionbudgetsResource, c.ns, name), &v1beta1.PodDisruptionBudget{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakePodDisruptionBudgets) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(poddisruptionbudgetsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.PodDisruptionBudgetList{}) + return err +} + +// Patch applies the patch and returns the patched podDisruptionBudget. +func (c *FakePodDisruptionBudgets) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.PodDisruptionBudget, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(poddisruptionbudgetsResource, c.ns, name, pt, data, subresources...), &v1beta1.PodDisruptionBudget{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodDisruptionBudget), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_podsecuritypolicy.go b/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_podsecuritypolicy.go new file mode 100644 index 0000000000..32d1989f33 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_podsecuritypolicy.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/policy/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakePodSecurityPolicies implements PodSecurityPolicyInterface +type FakePodSecurityPolicies struct { + Fake *FakePolicyV1beta1 +} + +var podsecuritypoliciesResource = schema.GroupVersionResource{Group: "policy", Version: "v1beta1", Resource: "podsecuritypolicies"} + +var podsecuritypoliciesKind = schema.GroupVersionKind{Group: "policy", Version: "v1beta1", Kind: "PodSecurityPolicy"} + +// Get takes name of the podSecurityPolicy, and returns the corresponding podSecurityPolicy object, and an error if there is any. +func (c *FakePodSecurityPolicies) Get(name string, options v1.GetOptions) (result *v1beta1.PodSecurityPolicy, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(podsecuritypoliciesResource, name), &v1beta1.PodSecurityPolicy{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodSecurityPolicy), err +} + +// List takes label and field selectors, and returns the list of PodSecurityPolicies that match those selectors. +func (c *FakePodSecurityPolicies) List(opts v1.ListOptions) (result *v1beta1.PodSecurityPolicyList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(podsecuritypoliciesResource, podsecuritypoliciesKind, opts), &v1beta1.PodSecurityPolicyList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.PodSecurityPolicyList{ListMeta: obj.(*v1beta1.PodSecurityPolicyList).ListMeta} + for _, item := range obj.(*v1beta1.PodSecurityPolicyList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested podSecurityPolicies. +func (c *FakePodSecurityPolicies) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(podsecuritypoliciesResource, opts)) +} + +// Create takes the representation of a podSecurityPolicy and creates it. Returns the server's representation of the podSecurityPolicy, and an error, if there is any. +func (c *FakePodSecurityPolicies) Create(podSecurityPolicy *v1beta1.PodSecurityPolicy) (result *v1beta1.PodSecurityPolicy, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(podsecuritypoliciesResource, podSecurityPolicy), &v1beta1.PodSecurityPolicy{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodSecurityPolicy), err +} + +// Update takes the representation of a podSecurityPolicy and updates it. Returns the server's representation of the podSecurityPolicy, and an error, if there is any. +func (c *FakePodSecurityPolicies) Update(podSecurityPolicy *v1beta1.PodSecurityPolicy) (result *v1beta1.PodSecurityPolicy, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(podsecuritypoliciesResource, podSecurityPolicy), &v1beta1.PodSecurityPolicy{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodSecurityPolicy), err +} + +// Delete takes name of the podSecurityPolicy and deletes it. Returns an error if one occurs. +func (c *FakePodSecurityPolicies) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(podsecuritypoliciesResource, name), &v1beta1.PodSecurityPolicy{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakePodSecurityPolicies) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(podsecuritypoliciesResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.PodSecurityPolicyList{}) + return err +} + +// Patch applies the patch and returns the patched podSecurityPolicy. +func (c *FakePodSecurityPolicies) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.PodSecurityPolicy, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(podsecuritypoliciesResource, name, pt, data, subresources...), &v1beta1.PodSecurityPolicy{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PodSecurityPolicy), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_policy_client.go b/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_policy_client.go new file mode 100644 index 0000000000..9c780bf1f0 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/policy/v1beta1/fake/fake_policy_client.go @@ -0,0 +1,48 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/policy/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakePolicyV1beta1 struct { + *testing.Fake +} + +func (c *FakePolicyV1beta1) Evictions(namespace string) v1beta1.EvictionInterface { + return &FakeEvictions{c, namespace} +} + +func (c *FakePolicyV1beta1) PodDisruptionBudgets(namespace string) v1beta1.PodDisruptionBudgetInterface { + return &FakePodDisruptionBudgets{c, namespace} +} + +func (c *FakePolicyV1beta1) PodSecurityPolicies() v1beta1.PodSecurityPolicyInterface { + return &FakePodSecurityPolicies{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakePolicyV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_clusterrole.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_clusterrole.go new file mode 100644 index 0000000000..d57f339390 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_clusterrole.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + rbacv1 "k8s.io/api/rbac/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeClusterRoles implements ClusterRoleInterface +type FakeClusterRoles struct { + Fake *FakeRbacV1 +} + +var clusterrolesResource = schema.GroupVersionResource{Group: "rbac.authorization.k8s.io", Version: "v1", Resource: "clusterroles"} + +var clusterrolesKind = schema.GroupVersionKind{Group: "rbac.authorization.k8s.io", Version: "v1", Kind: "ClusterRole"} + +// Get takes name of the clusterRole, and returns the corresponding clusterRole object, and an error if there is any. +func (c *FakeClusterRoles) Get(name string, options v1.GetOptions) (result *rbacv1.ClusterRole, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(clusterrolesResource, name), &rbacv1.ClusterRole{}) + if obj == nil { + return nil, err + } + return obj.(*rbacv1.ClusterRole), err +} + +// List takes label and field selectors, and returns the list of ClusterRoles that match those selectors. +func (c *FakeClusterRoles) List(opts v1.ListOptions) (result *rbacv1.ClusterRoleList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(clusterrolesResource, clusterrolesKind, opts), &rbacv1.ClusterRoleList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &rbacv1.ClusterRoleList{ListMeta: obj.(*rbacv1.ClusterRoleList).ListMeta} + for _, item := range obj.(*rbacv1.ClusterRoleList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested clusterRoles. +func (c *FakeClusterRoles) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(clusterrolesResource, opts)) +} + +// Create takes the representation of a clusterRole and creates it. Returns the server's representation of the clusterRole, and an error, if there is any. +func (c *FakeClusterRoles) Create(clusterRole *rbacv1.ClusterRole) (result *rbacv1.ClusterRole, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(clusterrolesResource, clusterRole), &rbacv1.ClusterRole{}) + if obj == nil { + return nil, err + } + return obj.(*rbacv1.ClusterRole), err +} + +// Update takes the representation of a clusterRole and updates it. Returns the server's representation of the clusterRole, and an error, if there is any. +func (c *FakeClusterRoles) Update(clusterRole *rbacv1.ClusterRole) (result *rbacv1.ClusterRole, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(clusterrolesResource, clusterRole), &rbacv1.ClusterRole{}) + if obj == nil { + return nil, err + } + return obj.(*rbacv1.ClusterRole), err +} + +// Delete takes name of the clusterRole and deletes it. Returns an error if one occurs. +func (c *FakeClusterRoles) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(clusterrolesResource, name), &rbacv1.ClusterRole{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeClusterRoles) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(clusterrolesResource, listOptions) + + _, err := c.Fake.Invokes(action, &rbacv1.ClusterRoleList{}) + return err +} + +// Patch applies the patch and returns the patched clusterRole. +func (c *FakeClusterRoles) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *rbacv1.ClusterRole, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(clusterrolesResource, name, pt, data, subresources...), &rbacv1.ClusterRole{}) + if obj == nil { + return nil, err + } + return obj.(*rbacv1.ClusterRole), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_clusterrolebinding.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_clusterrolebinding.go new file mode 100644 index 0000000000..878473ef35 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_clusterrolebinding.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + rbacv1 "k8s.io/api/rbac/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeClusterRoleBindings implements ClusterRoleBindingInterface +type FakeClusterRoleBindings struct { + Fake *FakeRbacV1 +} + +var clusterrolebindingsResource = schema.GroupVersionResource{Group: "rbac.authorization.k8s.io", Version: "v1", Resource: "clusterrolebindings"} + +var clusterrolebindingsKind = schema.GroupVersionKind{Group: "rbac.authorization.k8s.io", Version: "v1", Kind: "ClusterRoleBinding"} + +// Get takes name of the clusterRoleBinding, and returns the corresponding clusterRoleBinding object, and an error if there is any. +func (c *FakeClusterRoleBindings) Get(name string, options v1.GetOptions) (result *rbacv1.ClusterRoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(clusterrolebindingsResource, name), &rbacv1.ClusterRoleBinding{}) + if obj == nil { + return nil, err + } + return obj.(*rbacv1.ClusterRoleBinding), err +} + +// List takes label and field selectors, and returns the list of ClusterRoleBindings that match those selectors. +func (c *FakeClusterRoleBindings) List(opts v1.ListOptions) (result *rbacv1.ClusterRoleBindingList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(clusterrolebindingsResource, clusterrolebindingsKind, opts), &rbacv1.ClusterRoleBindingList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &rbacv1.ClusterRoleBindingList{ListMeta: obj.(*rbacv1.ClusterRoleBindingList).ListMeta} + for _, item := range obj.(*rbacv1.ClusterRoleBindingList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested clusterRoleBindings. +func (c *FakeClusterRoleBindings) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(clusterrolebindingsResource, opts)) +} + +// Create takes the representation of a clusterRoleBinding and creates it. Returns the server's representation of the clusterRoleBinding, and an error, if there is any. +func (c *FakeClusterRoleBindings) Create(clusterRoleBinding *rbacv1.ClusterRoleBinding) (result *rbacv1.ClusterRoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(clusterrolebindingsResource, clusterRoleBinding), &rbacv1.ClusterRoleBinding{}) + if obj == nil { + return nil, err + } + return obj.(*rbacv1.ClusterRoleBinding), err +} + +// Update takes the representation of a clusterRoleBinding and updates it. Returns the server's representation of the clusterRoleBinding, and an error, if there is any. +func (c *FakeClusterRoleBindings) Update(clusterRoleBinding *rbacv1.ClusterRoleBinding) (result *rbacv1.ClusterRoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(clusterrolebindingsResource, clusterRoleBinding), &rbacv1.ClusterRoleBinding{}) + if obj == nil { + return nil, err + } + return obj.(*rbacv1.ClusterRoleBinding), err +} + +// Delete takes name of the clusterRoleBinding and deletes it. Returns an error if one occurs. +func (c *FakeClusterRoleBindings) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(clusterrolebindingsResource, name), &rbacv1.ClusterRoleBinding{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeClusterRoleBindings) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(clusterrolebindingsResource, listOptions) + + _, err := c.Fake.Invokes(action, &rbacv1.ClusterRoleBindingList{}) + return err +} + +// Patch applies the patch and returns the patched clusterRoleBinding. +func (c *FakeClusterRoleBindings) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *rbacv1.ClusterRoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(clusterrolebindingsResource, name, pt, data, subresources...), &rbacv1.ClusterRoleBinding{}) + if obj == nil { + return nil, err + } + return obj.(*rbacv1.ClusterRoleBinding), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_rbac_client.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_rbac_client.go new file mode 100644 index 0000000000..426fd70d6c --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_rbac_client.go @@ -0,0 +1,52 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1 "k8s.io/client-go/kubernetes/typed/rbac/v1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeRbacV1 struct { + *testing.Fake +} + +func (c *FakeRbacV1) ClusterRoles() v1.ClusterRoleInterface { + return &FakeClusterRoles{c} +} + +func (c *FakeRbacV1) ClusterRoleBindings() v1.ClusterRoleBindingInterface { + return &FakeClusterRoleBindings{c} +} + +func (c *FakeRbacV1) Roles(namespace string) v1.RoleInterface { + return &FakeRoles{c, namespace} +} + +func (c *FakeRbacV1) RoleBindings(namespace string) v1.RoleBindingInterface { + return &FakeRoleBindings{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeRbacV1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_role.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_role.go new file mode 100644 index 0000000000..78ef3192f3 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_role.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + rbacv1 "k8s.io/api/rbac/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeRoles implements RoleInterface +type FakeRoles struct { + Fake *FakeRbacV1 + ns string +} + +var rolesResource = schema.GroupVersionResource{Group: "rbac.authorization.k8s.io", Version: "v1", Resource: "roles"} + +var rolesKind = schema.GroupVersionKind{Group: "rbac.authorization.k8s.io", Version: "v1", Kind: "Role"} + +// Get takes name of the role, and returns the corresponding role object, and an error if there is any. +func (c *FakeRoles) Get(name string, options v1.GetOptions) (result *rbacv1.Role, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(rolesResource, c.ns, name), &rbacv1.Role{}) + + if obj == nil { + return nil, err + } + return obj.(*rbacv1.Role), err +} + +// List takes label and field selectors, and returns the list of Roles that match those selectors. +func (c *FakeRoles) List(opts v1.ListOptions) (result *rbacv1.RoleList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(rolesResource, rolesKind, c.ns, opts), &rbacv1.RoleList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &rbacv1.RoleList{ListMeta: obj.(*rbacv1.RoleList).ListMeta} + for _, item := range obj.(*rbacv1.RoleList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested roles. +func (c *FakeRoles) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(rolesResource, c.ns, opts)) + +} + +// Create takes the representation of a role and creates it. Returns the server's representation of the role, and an error, if there is any. +func (c *FakeRoles) Create(role *rbacv1.Role) (result *rbacv1.Role, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(rolesResource, c.ns, role), &rbacv1.Role{}) + + if obj == nil { + return nil, err + } + return obj.(*rbacv1.Role), err +} + +// Update takes the representation of a role and updates it. Returns the server's representation of the role, and an error, if there is any. +func (c *FakeRoles) Update(role *rbacv1.Role) (result *rbacv1.Role, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(rolesResource, c.ns, role), &rbacv1.Role{}) + + if obj == nil { + return nil, err + } + return obj.(*rbacv1.Role), err +} + +// Delete takes name of the role and deletes it. Returns an error if one occurs. +func (c *FakeRoles) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(rolesResource, c.ns, name), &rbacv1.Role{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeRoles) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(rolesResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &rbacv1.RoleList{}) + return err +} + +// Patch applies the patch and returns the patched role. +func (c *FakeRoles) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *rbacv1.Role, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(rolesResource, c.ns, name, pt, data, subresources...), &rbacv1.Role{}) + + if obj == nil { + return nil, err + } + return obj.(*rbacv1.Role), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_rolebinding.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_rolebinding.go new file mode 100644 index 0000000000..6c344cadff --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1/fake/fake_rolebinding.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + rbacv1 "k8s.io/api/rbac/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeRoleBindings implements RoleBindingInterface +type FakeRoleBindings struct { + Fake *FakeRbacV1 + ns string +} + +var rolebindingsResource = schema.GroupVersionResource{Group: "rbac.authorization.k8s.io", Version: "v1", Resource: "rolebindings"} + +var rolebindingsKind = schema.GroupVersionKind{Group: "rbac.authorization.k8s.io", Version: "v1", Kind: "RoleBinding"} + +// Get takes name of the roleBinding, and returns the corresponding roleBinding object, and an error if there is any. +func (c *FakeRoleBindings) Get(name string, options v1.GetOptions) (result *rbacv1.RoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(rolebindingsResource, c.ns, name), &rbacv1.RoleBinding{}) + + if obj == nil { + return nil, err + } + return obj.(*rbacv1.RoleBinding), err +} + +// List takes label and field selectors, and returns the list of RoleBindings that match those selectors. +func (c *FakeRoleBindings) List(opts v1.ListOptions) (result *rbacv1.RoleBindingList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(rolebindingsResource, rolebindingsKind, c.ns, opts), &rbacv1.RoleBindingList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &rbacv1.RoleBindingList{ListMeta: obj.(*rbacv1.RoleBindingList).ListMeta} + for _, item := range obj.(*rbacv1.RoleBindingList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested roleBindings. +func (c *FakeRoleBindings) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(rolebindingsResource, c.ns, opts)) + +} + +// Create takes the representation of a roleBinding and creates it. Returns the server's representation of the roleBinding, and an error, if there is any. +func (c *FakeRoleBindings) Create(roleBinding *rbacv1.RoleBinding) (result *rbacv1.RoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(rolebindingsResource, c.ns, roleBinding), &rbacv1.RoleBinding{}) + + if obj == nil { + return nil, err + } + return obj.(*rbacv1.RoleBinding), err +} + +// Update takes the representation of a roleBinding and updates it. Returns the server's representation of the roleBinding, and an error, if there is any. +func (c *FakeRoleBindings) Update(roleBinding *rbacv1.RoleBinding) (result *rbacv1.RoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(rolebindingsResource, c.ns, roleBinding), &rbacv1.RoleBinding{}) + + if obj == nil { + return nil, err + } + return obj.(*rbacv1.RoleBinding), err +} + +// Delete takes name of the roleBinding and deletes it. Returns an error if one occurs. +func (c *FakeRoleBindings) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(rolebindingsResource, c.ns, name), &rbacv1.RoleBinding{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeRoleBindings) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(rolebindingsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &rbacv1.RoleBindingList{}) + return err +} + +// Patch applies the patch and returns the patched roleBinding. +func (c *FakeRoleBindings) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *rbacv1.RoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(rolebindingsResource, c.ns, name, pt, data, subresources...), &rbacv1.RoleBinding{}) + + if obj == nil { + return nil, err + } + return obj.(*rbacv1.RoleBinding), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_clusterrole.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_clusterrole.go new file mode 100644 index 0000000000..d2d1b1c74c --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_clusterrole.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/api/rbac/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeClusterRoles implements ClusterRoleInterface +type FakeClusterRoles struct { + Fake *FakeRbacV1alpha1 +} + +var clusterrolesResource = schema.GroupVersionResource{Group: "rbac.authorization.k8s.io", Version: "v1alpha1", Resource: "clusterroles"} + +var clusterrolesKind = schema.GroupVersionKind{Group: "rbac.authorization.k8s.io", Version: "v1alpha1", Kind: "ClusterRole"} + +// Get takes name of the clusterRole, and returns the corresponding clusterRole object, and an error if there is any. +func (c *FakeClusterRoles) Get(name string, options v1.GetOptions) (result *v1alpha1.ClusterRole, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(clusterrolesResource, name), &v1alpha1.ClusterRole{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.ClusterRole), err +} + +// List takes label and field selectors, and returns the list of ClusterRoles that match those selectors. +func (c *FakeClusterRoles) List(opts v1.ListOptions) (result *v1alpha1.ClusterRoleList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(clusterrolesResource, clusterrolesKind, opts), &v1alpha1.ClusterRoleList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1alpha1.ClusterRoleList{ListMeta: obj.(*v1alpha1.ClusterRoleList).ListMeta} + for _, item := range obj.(*v1alpha1.ClusterRoleList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested clusterRoles. +func (c *FakeClusterRoles) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(clusterrolesResource, opts)) +} + +// Create takes the representation of a clusterRole and creates it. Returns the server's representation of the clusterRole, and an error, if there is any. +func (c *FakeClusterRoles) Create(clusterRole *v1alpha1.ClusterRole) (result *v1alpha1.ClusterRole, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(clusterrolesResource, clusterRole), &v1alpha1.ClusterRole{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.ClusterRole), err +} + +// Update takes the representation of a clusterRole and updates it. Returns the server's representation of the clusterRole, and an error, if there is any. +func (c *FakeClusterRoles) Update(clusterRole *v1alpha1.ClusterRole) (result *v1alpha1.ClusterRole, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(clusterrolesResource, clusterRole), &v1alpha1.ClusterRole{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.ClusterRole), err +} + +// Delete takes name of the clusterRole and deletes it. Returns an error if one occurs. +func (c *FakeClusterRoles) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(clusterrolesResource, name), &v1alpha1.ClusterRole{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeClusterRoles) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(clusterrolesResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1alpha1.ClusterRoleList{}) + return err +} + +// Patch applies the patch and returns the patched clusterRole. +func (c *FakeClusterRoles) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1alpha1.ClusterRole, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(clusterrolesResource, name, pt, data, subresources...), &v1alpha1.ClusterRole{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.ClusterRole), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_clusterrolebinding.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_clusterrolebinding.go new file mode 100644 index 0000000000..3e23e5f657 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_clusterrolebinding.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/api/rbac/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeClusterRoleBindings implements ClusterRoleBindingInterface +type FakeClusterRoleBindings struct { + Fake *FakeRbacV1alpha1 +} + +var clusterrolebindingsResource = schema.GroupVersionResource{Group: "rbac.authorization.k8s.io", Version: "v1alpha1", Resource: "clusterrolebindings"} + +var clusterrolebindingsKind = schema.GroupVersionKind{Group: "rbac.authorization.k8s.io", Version: "v1alpha1", Kind: "ClusterRoleBinding"} + +// Get takes name of the clusterRoleBinding, and returns the corresponding clusterRoleBinding object, and an error if there is any. +func (c *FakeClusterRoleBindings) Get(name string, options v1.GetOptions) (result *v1alpha1.ClusterRoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(clusterrolebindingsResource, name), &v1alpha1.ClusterRoleBinding{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.ClusterRoleBinding), err +} + +// List takes label and field selectors, and returns the list of ClusterRoleBindings that match those selectors. +func (c *FakeClusterRoleBindings) List(opts v1.ListOptions) (result *v1alpha1.ClusterRoleBindingList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(clusterrolebindingsResource, clusterrolebindingsKind, opts), &v1alpha1.ClusterRoleBindingList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1alpha1.ClusterRoleBindingList{ListMeta: obj.(*v1alpha1.ClusterRoleBindingList).ListMeta} + for _, item := range obj.(*v1alpha1.ClusterRoleBindingList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested clusterRoleBindings. +func (c *FakeClusterRoleBindings) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(clusterrolebindingsResource, opts)) +} + +// Create takes the representation of a clusterRoleBinding and creates it. Returns the server's representation of the clusterRoleBinding, and an error, if there is any. +func (c *FakeClusterRoleBindings) Create(clusterRoleBinding *v1alpha1.ClusterRoleBinding) (result *v1alpha1.ClusterRoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(clusterrolebindingsResource, clusterRoleBinding), &v1alpha1.ClusterRoleBinding{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.ClusterRoleBinding), err +} + +// Update takes the representation of a clusterRoleBinding and updates it. Returns the server's representation of the clusterRoleBinding, and an error, if there is any. +func (c *FakeClusterRoleBindings) Update(clusterRoleBinding *v1alpha1.ClusterRoleBinding) (result *v1alpha1.ClusterRoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(clusterrolebindingsResource, clusterRoleBinding), &v1alpha1.ClusterRoleBinding{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.ClusterRoleBinding), err +} + +// Delete takes name of the clusterRoleBinding and deletes it. Returns an error if one occurs. +func (c *FakeClusterRoleBindings) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(clusterrolebindingsResource, name), &v1alpha1.ClusterRoleBinding{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeClusterRoleBindings) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(clusterrolebindingsResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1alpha1.ClusterRoleBindingList{}) + return err +} + +// Patch applies the patch and returns the patched clusterRoleBinding. +func (c *FakeClusterRoleBindings) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1alpha1.ClusterRoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(clusterrolebindingsResource, name, pt, data, subresources...), &v1alpha1.ClusterRoleBinding{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.ClusterRoleBinding), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_rbac_client.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_rbac_client.go new file mode 100644 index 0000000000..3447e9be83 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_rbac_client.go @@ -0,0 +1,52 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/client-go/kubernetes/typed/rbac/v1alpha1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeRbacV1alpha1 struct { + *testing.Fake +} + +func (c *FakeRbacV1alpha1) ClusterRoles() v1alpha1.ClusterRoleInterface { + return &FakeClusterRoles{c} +} + +func (c *FakeRbacV1alpha1) ClusterRoleBindings() v1alpha1.ClusterRoleBindingInterface { + return &FakeClusterRoleBindings{c} +} + +func (c *FakeRbacV1alpha1) Roles(namespace string) v1alpha1.RoleInterface { + return &FakeRoles{c, namespace} +} + +func (c *FakeRbacV1alpha1) RoleBindings(namespace string) v1alpha1.RoleBindingInterface { + return &FakeRoleBindings{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeRbacV1alpha1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_role.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_role.go new file mode 100644 index 0000000000..7bd52373fa --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_role.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/api/rbac/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeRoles implements RoleInterface +type FakeRoles struct { + Fake *FakeRbacV1alpha1 + ns string +} + +var rolesResource = schema.GroupVersionResource{Group: "rbac.authorization.k8s.io", Version: "v1alpha1", Resource: "roles"} + +var rolesKind = schema.GroupVersionKind{Group: "rbac.authorization.k8s.io", Version: "v1alpha1", Kind: "Role"} + +// Get takes name of the role, and returns the corresponding role object, and an error if there is any. +func (c *FakeRoles) Get(name string, options v1.GetOptions) (result *v1alpha1.Role, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(rolesResource, c.ns, name), &v1alpha1.Role{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.Role), err +} + +// List takes label and field selectors, and returns the list of Roles that match those selectors. +func (c *FakeRoles) List(opts v1.ListOptions) (result *v1alpha1.RoleList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(rolesResource, rolesKind, c.ns, opts), &v1alpha1.RoleList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1alpha1.RoleList{ListMeta: obj.(*v1alpha1.RoleList).ListMeta} + for _, item := range obj.(*v1alpha1.RoleList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested roles. +func (c *FakeRoles) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(rolesResource, c.ns, opts)) + +} + +// Create takes the representation of a role and creates it. Returns the server's representation of the role, and an error, if there is any. +func (c *FakeRoles) Create(role *v1alpha1.Role) (result *v1alpha1.Role, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(rolesResource, c.ns, role), &v1alpha1.Role{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.Role), err +} + +// Update takes the representation of a role and updates it. Returns the server's representation of the role, and an error, if there is any. +func (c *FakeRoles) Update(role *v1alpha1.Role) (result *v1alpha1.Role, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(rolesResource, c.ns, role), &v1alpha1.Role{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.Role), err +} + +// Delete takes name of the role and deletes it. Returns an error if one occurs. +func (c *FakeRoles) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(rolesResource, c.ns, name), &v1alpha1.Role{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeRoles) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(rolesResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1alpha1.RoleList{}) + return err +} + +// Patch applies the patch and returns the patched role. +func (c *FakeRoles) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1alpha1.Role, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(rolesResource, c.ns, name, pt, data, subresources...), &v1alpha1.Role{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.Role), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_rolebinding.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_rolebinding.go new file mode 100644 index 0000000000..0150503115 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1alpha1/fake/fake_rolebinding.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/api/rbac/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeRoleBindings implements RoleBindingInterface +type FakeRoleBindings struct { + Fake *FakeRbacV1alpha1 + ns string +} + +var rolebindingsResource = schema.GroupVersionResource{Group: "rbac.authorization.k8s.io", Version: "v1alpha1", Resource: "rolebindings"} + +var rolebindingsKind = schema.GroupVersionKind{Group: "rbac.authorization.k8s.io", Version: "v1alpha1", Kind: "RoleBinding"} + +// Get takes name of the roleBinding, and returns the corresponding roleBinding object, and an error if there is any. +func (c *FakeRoleBindings) Get(name string, options v1.GetOptions) (result *v1alpha1.RoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(rolebindingsResource, c.ns, name), &v1alpha1.RoleBinding{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.RoleBinding), err +} + +// List takes label and field selectors, and returns the list of RoleBindings that match those selectors. +func (c *FakeRoleBindings) List(opts v1.ListOptions) (result *v1alpha1.RoleBindingList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(rolebindingsResource, rolebindingsKind, c.ns, opts), &v1alpha1.RoleBindingList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1alpha1.RoleBindingList{ListMeta: obj.(*v1alpha1.RoleBindingList).ListMeta} + for _, item := range obj.(*v1alpha1.RoleBindingList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested roleBindings. +func (c *FakeRoleBindings) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(rolebindingsResource, c.ns, opts)) + +} + +// Create takes the representation of a roleBinding and creates it. Returns the server's representation of the roleBinding, and an error, if there is any. +func (c *FakeRoleBindings) Create(roleBinding *v1alpha1.RoleBinding) (result *v1alpha1.RoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(rolebindingsResource, c.ns, roleBinding), &v1alpha1.RoleBinding{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.RoleBinding), err +} + +// Update takes the representation of a roleBinding and updates it. Returns the server's representation of the roleBinding, and an error, if there is any. +func (c *FakeRoleBindings) Update(roleBinding *v1alpha1.RoleBinding) (result *v1alpha1.RoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(rolebindingsResource, c.ns, roleBinding), &v1alpha1.RoleBinding{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.RoleBinding), err +} + +// Delete takes name of the roleBinding and deletes it. Returns an error if one occurs. +func (c *FakeRoleBindings) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(rolebindingsResource, c.ns, name), &v1alpha1.RoleBinding{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeRoleBindings) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(rolebindingsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1alpha1.RoleBindingList{}) + return err +} + +// Patch applies the patch and returns the patched roleBinding. +func (c *FakeRoleBindings) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1alpha1.RoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(rolebindingsResource, c.ns, name, pt, data, subresources...), &v1alpha1.RoleBinding{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.RoleBinding), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_clusterrole.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_clusterrole.go new file mode 100644 index 0000000000..2dbc3f6166 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_clusterrole.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/rbac/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeClusterRoles implements ClusterRoleInterface +type FakeClusterRoles struct { + Fake *FakeRbacV1beta1 +} + +var clusterrolesResource = schema.GroupVersionResource{Group: "rbac.authorization.k8s.io", Version: "v1beta1", Resource: "clusterroles"} + +var clusterrolesKind = schema.GroupVersionKind{Group: "rbac.authorization.k8s.io", Version: "v1beta1", Kind: "ClusterRole"} + +// Get takes name of the clusterRole, and returns the corresponding clusterRole object, and an error if there is any. +func (c *FakeClusterRoles) Get(name string, options v1.GetOptions) (result *v1beta1.ClusterRole, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(clusterrolesResource, name), &v1beta1.ClusterRole{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ClusterRole), err +} + +// List takes label and field selectors, and returns the list of ClusterRoles that match those selectors. +func (c *FakeClusterRoles) List(opts v1.ListOptions) (result *v1beta1.ClusterRoleList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(clusterrolesResource, clusterrolesKind, opts), &v1beta1.ClusterRoleList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.ClusterRoleList{ListMeta: obj.(*v1beta1.ClusterRoleList).ListMeta} + for _, item := range obj.(*v1beta1.ClusterRoleList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested clusterRoles. +func (c *FakeClusterRoles) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(clusterrolesResource, opts)) +} + +// Create takes the representation of a clusterRole and creates it. Returns the server's representation of the clusterRole, and an error, if there is any. +func (c *FakeClusterRoles) Create(clusterRole *v1beta1.ClusterRole) (result *v1beta1.ClusterRole, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(clusterrolesResource, clusterRole), &v1beta1.ClusterRole{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ClusterRole), err +} + +// Update takes the representation of a clusterRole and updates it. Returns the server's representation of the clusterRole, and an error, if there is any. +func (c *FakeClusterRoles) Update(clusterRole *v1beta1.ClusterRole) (result *v1beta1.ClusterRole, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(clusterrolesResource, clusterRole), &v1beta1.ClusterRole{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ClusterRole), err +} + +// Delete takes name of the clusterRole and deletes it. Returns an error if one occurs. +func (c *FakeClusterRoles) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(clusterrolesResource, name), &v1beta1.ClusterRole{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeClusterRoles) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(clusterrolesResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.ClusterRoleList{}) + return err +} + +// Patch applies the patch and returns the patched clusterRole. +func (c *FakeClusterRoles) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.ClusterRole, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(clusterrolesResource, name, pt, data, subresources...), &v1beta1.ClusterRole{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ClusterRole), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_clusterrolebinding.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_clusterrolebinding.go new file mode 100644 index 0000000000..14e20bc28c --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_clusterrolebinding.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/rbac/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeClusterRoleBindings implements ClusterRoleBindingInterface +type FakeClusterRoleBindings struct { + Fake *FakeRbacV1beta1 +} + +var clusterrolebindingsResource = schema.GroupVersionResource{Group: "rbac.authorization.k8s.io", Version: "v1beta1", Resource: "clusterrolebindings"} + +var clusterrolebindingsKind = schema.GroupVersionKind{Group: "rbac.authorization.k8s.io", Version: "v1beta1", Kind: "ClusterRoleBinding"} + +// Get takes name of the clusterRoleBinding, and returns the corresponding clusterRoleBinding object, and an error if there is any. +func (c *FakeClusterRoleBindings) Get(name string, options v1.GetOptions) (result *v1beta1.ClusterRoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(clusterrolebindingsResource, name), &v1beta1.ClusterRoleBinding{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ClusterRoleBinding), err +} + +// List takes label and field selectors, and returns the list of ClusterRoleBindings that match those selectors. +func (c *FakeClusterRoleBindings) List(opts v1.ListOptions) (result *v1beta1.ClusterRoleBindingList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(clusterrolebindingsResource, clusterrolebindingsKind, opts), &v1beta1.ClusterRoleBindingList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.ClusterRoleBindingList{ListMeta: obj.(*v1beta1.ClusterRoleBindingList).ListMeta} + for _, item := range obj.(*v1beta1.ClusterRoleBindingList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested clusterRoleBindings. +func (c *FakeClusterRoleBindings) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(clusterrolebindingsResource, opts)) +} + +// Create takes the representation of a clusterRoleBinding and creates it. Returns the server's representation of the clusterRoleBinding, and an error, if there is any. +func (c *FakeClusterRoleBindings) Create(clusterRoleBinding *v1beta1.ClusterRoleBinding) (result *v1beta1.ClusterRoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(clusterrolebindingsResource, clusterRoleBinding), &v1beta1.ClusterRoleBinding{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ClusterRoleBinding), err +} + +// Update takes the representation of a clusterRoleBinding and updates it. Returns the server's representation of the clusterRoleBinding, and an error, if there is any. +func (c *FakeClusterRoleBindings) Update(clusterRoleBinding *v1beta1.ClusterRoleBinding) (result *v1beta1.ClusterRoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(clusterrolebindingsResource, clusterRoleBinding), &v1beta1.ClusterRoleBinding{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ClusterRoleBinding), err +} + +// Delete takes name of the clusterRoleBinding and deletes it. Returns an error if one occurs. +func (c *FakeClusterRoleBindings) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(clusterrolebindingsResource, name), &v1beta1.ClusterRoleBinding{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeClusterRoleBindings) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(clusterrolebindingsResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.ClusterRoleBindingList{}) + return err +} + +// Patch applies the patch and returns the patched clusterRoleBinding. +func (c *FakeClusterRoleBindings) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.ClusterRoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(clusterrolebindingsResource, name, pt, data, subresources...), &v1beta1.ClusterRoleBinding{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.ClusterRoleBinding), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_rbac_client.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_rbac_client.go new file mode 100644 index 0000000000..bdbc246b7a --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_rbac_client.go @@ -0,0 +1,52 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/rbac/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeRbacV1beta1 struct { + *testing.Fake +} + +func (c *FakeRbacV1beta1) ClusterRoles() v1beta1.ClusterRoleInterface { + return &FakeClusterRoles{c} +} + +func (c *FakeRbacV1beta1) ClusterRoleBindings() v1beta1.ClusterRoleBindingInterface { + return &FakeClusterRoleBindings{c} +} + +func (c *FakeRbacV1beta1) Roles(namespace string) v1beta1.RoleInterface { + return &FakeRoles{c, namespace} +} + +func (c *FakeRbacV1beta1) RoleBindings(namespace string) v1beta1.RoleBindingInterface { + return &FakeRoleBindings{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeRbacV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_role.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_role.go new file mode 100644 index 0000000000..e31768e4e5 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_role.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/rbac/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeRoles implements RoleInterface +type FakeRoles struct { + Fake *FakeRbacV1beta1 + ns string +} + +var rolesResource = schema.GroupVersionResource{Group: "rbac.authorization.k8s.io", Version: "v1beta1", Resource: "roles"} + +var rolesKind = schema.GroupVersionKind{Group: "rbac.authorization.k8s.io", Version: "v1beta1", Kind: "Role"} + +// Get takes name of the role, and returns the corresponding role object, and an error if there is any. +func (c *FakeRoles) Get(name string, options v1.GetOptions) (result *v1beta1.Role, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(rolesResource, c.ns, name), &v1beta1.Role{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Role), err +} + +// List takes label and field selectors, and returns the list of Roles that match those selectors. +func (c *FakeRoles) List(opts v1.ListOptions) (result *v1beta1.RoleList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(rolesResource, rolesKind, c.ns, opts), &v1beta1.RoleList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.RoleList{ListMeta: obj.(*v1beta1.RoleList).ListMeta} + for _, item := range obj.(*v1beta1.RoleList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested roles. +func (c *FakeRoles) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(rolesResource, c.ns, opts)) + +} + +// Create takes the representation of a role and creates it. Returns the server's representation of the role, and an error, if there is any. +func (c *FakeRoles) Create(role *v1beta1.Role) (result *v1beta1.Role, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(rolesResource, c.ns, role), &v1beta1.Role{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Role), err +} + +// Update takes the representation of a role and updates it. Returns the server's representation of the role, and an error, if there is any. +func (c *FakeRoles) Update(role *v1beta1.Role) (result *v1beta1.Role, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(rolesResource, c.ns, role), &v1beta1.Role{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Role), err +} + +// Delete takes name of the role and deletes it. Returns an error if one occurs. +func (c *FakeRoles) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(rolesResource, c.ns, name), &v1beta1.Role{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeRoles) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(rolesResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.RoleList{}) + return err +} + +// Patch applies the patch and returns the patched role. +func (c *FakeRoles) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.Role, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(rolesResource, c.ns, name, pt, data, subresources...), &v1beta1.Role{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.Role), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_rolebinding.go b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_rolebinding.go new file mode 100644 index 0000000000..06b93c93f6 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/rbac/v1beta1/fake/fake_rolebinding.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/rbac/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeRoleBindings implements RoleBindingInterface +type FakeRoleBindings struct { + Fake *FakeRbacV1beta1 + ns string +} + +var rolebindingsResource = schema.GroupVersionResource{Group: "rbac.authorization.k8s.io", Version: "v1beta1", Resource: "rolebindings"} + +var rolebindingsKind = schema.GroupVersionKind{Group: "rbac.authorization.k8s.io", Version: "v1beta1", Kind: "RoleBinding"} + +// Get takes name of the roleBinding, and returns the corresponding roleBinding object, and an error if there is any. +func (c *FakeRoleBindings) Get(name string, options v1.GetOptions) (result *v1beta1.RoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(rolebindingsResource, c.ns, name), &v1beta1.RoleBinding{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.RoleBinding), err +} + +// List takes label and field selectors, and returns the list of RoleBindings that match those selectors. +func (c *FakeRoleBindings) List(opts v1.ListOptions) (result *v1beta1.RoleBindingList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(rolebindingsResource, rolebindingsKind, c.ns, opts), &v1beta1.RoleBindingList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.RoleBindingList{ListMeta: obj.(*v1beta1.RoleBindingList).ListMeta} + for _, item := range obj.(*v1beta1.RoleBindingList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested roleBindings. +func (c *FakeRoleBindings) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(rolebindingsResource, c.ns, opts)) + +} + +// Create takes the representation of a roleBinding and creates it. Returns the server's representation of the roleBinding, and an error, if there is any. +func (c *FakeRoleBindings) Create(roleBinding *v1beta1.RoleBinding) (result *v1beta1.RoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(rolebindingsResource, c.ns, roleBinding), &v1beta1.RoleBinding{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.RoleBinding), err +} + +// Update takes the representation of a roleBinding and updates it. Returns the server's representation of the roleBinding, and an error, if there is any. +func (c *FakeRoleBindings) Update(roleBinding *v1beta1.RoleBinding) (result *v1beta1.RoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(rolebindingsResource, c.ns, roleBinding), &v1beta1.RoleBinding{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.RoleBinding), err +} + +// Delete takes name of the roleBinding and deletes it. Returns an error if one occurs. +func (c *FakeRoleBindings) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(rolebindingsResource, c.ns, name), &v1beta1.RoleBinding{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeRoleBindings) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(rolebindingsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.RoleBindingList{}) + return err +} + +// Patch applies the patch and returns the patched roleBinding. +func (c *FakeRoleBindings) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.RoleBinding, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(rolebindingsResource, c.ns, name, pt, data, subresources...), &v1beta1.RoleBinding{}) + + if obj == nil { + return nil, err + } + return obj.(*v1beta1.RoleBinding), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake/fake_priorityclass.go b/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake/fake_priorityclass.go new file mode 100644 index 0000000000..e592ed137f --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake/fake_priorityclass.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/api/scheduling/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakePriorityClasses implements PriorityClassInterface +type FakePriorityClasses struct { + Fake *FakeSchedulingV1alpha1 +} + +var priorityclassesResource = schema.GroupVersionResource{Group: "scheduling.k8s.io", Version: "v1alpha1", Resource: "priorityclasses"} + +var priorityclassesKind = schema.GroupVersionKind{Group: "scheduling.k8s.io", Version: "v1alpha1", Kind: "PriorityClass"} + +// Get takes name of the priorityClass, and returns the corresponding priorityClass object, and an error if there is any. +func (c *FakePriorityClasses) Get(name string, options v1.GetOptions) (result *v1alpha1.PriorityClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(priorityclassesResource, name), &v1alpha1.PriorityClass{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PriorityClass), err +} + +// List takes label and field selectors, and returns the list of PriorityClasses that match those selectors. +func (c *FakePriorityClasses) List(opts v1.ListOptions) (result *v1alpha1.PriorityClassList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(priorityclassesResource, priorityclassesKind, opts), &v1alpha1.PriorityClassList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1alpha1.PriorityClassList{ListMeta: obj.(*v1alpha1.PriorityClassList).ListMeta} + for _, item := range obj.(*v1alpha1.PriorityClassList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested priorityClasses. +func (c *FakePriorityClasses) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(priorityclassesResource, opts)) +} + +// Create takes the representation of a priorityClass and creates it. Returns the server's representation of the priorityClass, and an error, if there is any. +func (c *FakePriorityClasses) Create(priorityClass *v1alpha1.PriorityClass) (result *v1alpha1.PriorityClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(priorityclassesResource, priorityClass), &v1alpha1.PriorityClass{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PriorityClass), err +} + +// Update takes the representation of a priorityClass and updates it. Returns the server's representation of the priorityClass, and an error, if there is any. +func (c *FakePriorityClasses) Update(priorityClass *v1alpha1.PriorityClass) (result *v1alpha1.PriorityClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(priorityclassesResource, priorityClass), &v1alpha1.PriorityClass{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PriorityClass), err +} + +// Delete takes name of the priorityClass and deletes it. Returns an error if one occurs. +func (c *FakePriorityClasses) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(priorityclassesResource, name), &v1alpha1.PriorityClass{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakePriorityClasses) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(priorityclassesResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1alpha1.PriorityClassList{}) + return err +} + +// Patch applies the patch and returns the patched priorityClass. +func (c *FakePriorityClasses) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1alpha1.PriorityClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(priorityclassesResource, name, pt, data, subresources...), &v1alpha1.PriorityClass{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PriorityClass), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake/fake_scheduling_client.go b/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake/fake_scheduling_client.go new file mode 100644 index 0000000000..974ba193f7 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1/fake/fake_scheduling_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/client-go/kubernetes/typed/scheduling/v1alpha1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeSchedulingV1alpha1 struct { + *testing.Fake +} + +func (c *FakeSchedulingV1alpha1) PriorityClasses() v1alpha1.PriorityClassInterface { + return &FakePriorityClasses{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeSchedulingV1alpha1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake/fake_priorityclass.go b/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake/fake_priorityclass.go new file mode 100644 index 0000000000..44ce64b5ce --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake/fake_priorityclass.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/scheduling/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakePriorityClasses implements PriorityClassInterface +type FakePriorityClasses struct { + Fake *FakeSchedulingV1beta1 +} + +var priorityclassesResource = schema.GroupVersionResource{Group: "scheduling.k8s.io", Version: "v1beta1", Resource: "priorityclasses"} + +var priorityclassesKind = schema.GroupVersionKind{Group: "scheduling.k8s.io", Version: "v1beta1", Kind: "PriorityClass"} + +// Get takes name of the priorityClass, and returns the corresponding priorityClass object, and an error if there is any. +func (c *FakePriorityClasses) Get(name string, options v1.GetOptions) (result *v1beta1.PriorityClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(priorityclassesResource, name), &v1beta1.PriorityClass{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PriorityClass), err +} + +// List takes label and field selectors, and returns the list of PriorityClasses that match those selectors. +func (c *FakePriorityClasses) List(opts v1.ListOptions) (result *v1beta1.PriorityClassList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(priorityclassesResource, priorityclassesKind, opts), &v1beta1.PriorityClassList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.PriorityClassList{ListMeta: obj.(*v1beta1.PriorityClassList).ListMeta} + for _, item := range obj.(*v1beta1.PriorityClassList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested priorityClasses. +func (c *FakePriorityClasses) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(priorityclassesResource, opts)) +} + +// Create takes the representation of a priorityClass and creates it. Returns the server's representation of the priorityClass, and an error, if there is any. +func (c *FakePriorityClasses) Create(priorityClass *v1beta1.PriorityClass) (result *v1beta1.PriorityClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(priorityclassesResource, priorityClass), &v1beta1.PriorityClass{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PriorityClass), err +} + +// Update takes the representation of a priorityClass and updates it. Returns the server's representation of the priorityClass, and an error, if there is any. +func (c *FakePriorityClasses) Update(priorityClass *v1beta1.PriorityClass) (result *v1beta1.PriorityClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(priorityclassesResource, priorityClass), &v1beta1.PriorityClass{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PriorityClass), err +} + +// Delete takes name of the priorityClass and deletes it. Returns an error if one occurs. +func (c *FakePriorityClasses) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(priorityclassesResource, name), &v1beta1.PriorityClass{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakePriorityClasses) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(priorityclassesResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.PriorityClassList{}) + return err +} + +// Patch applies the patch and returns the patched priorityClass. +func (c *FakePriorityClasses) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.PriorityClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(priorityclassesResource, name, pt, data, subresources...), &v1beta1.PriorityClass{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.PriorityClass), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake/fake_scheduling_client.go b/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake/fake_scheduling_client.go new file mode 100644 index 0000000000..4a6878a45e --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/scheduling/v1beta1/fake/fake_scheduling_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/scheduling/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeSchedulingV1beta1 struct { + *testing.Fake +} + +func (c *FakeSchedulingV1beta1) PriorityClasses() v1beta1.PriorityClassInterface { + return &FakePriorityClasses{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeSchedulingV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake/fake_podpreset.go b/vendor/k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake/fake_podpreset.go new file mode 100644 index 0000000000..273a027fad --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake/fake_podpreset.go @@ -0,0 +1,128 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/api/settings/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakePodPresets implements PodPresetInterface +type FakePodPresets struct { + Fake *FakeSettingsV1alpha1 + ns string +} + +var podpresetsResource = schema.GroupVersionResource{Group: "settings.k8s.io", Version: "v1alpha1", Resource: "podpresets"} + +var podpresetsKind = schema.GroupVersionKind{Group: "settings.k8s.io", Version: "v1alpha1", Kind: "PodPreset"} + +// Get takes name of the podPreset, and returns the corresponding podPreset object, and an error if there is any. +func (c *FakePodPresets) Get(name string, options v1.GetOptions) (result *v1alpha1.PodPreset, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(podpresetsResource, c.ns, name), &v1alpha1.PodPreset{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PodPreset), err +} + +// List takes label and field selectors, and returns the list of PodPresets that match those selectors. +func (c *FakePodPresets) List(opts v1.ListOptions) (result *v1alpha1.PodPresetList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(podpresetsResource, podpresetsKind, c.ns, opts), &v1alpha1.PodPresetList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1alpha1.PodPresetList{ListMeta: obj.(*v1alpha1.PodPresetList).ListMeta} + for _, item := range obj.(*v1alpha1.PodPresetList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested podPresets. +func (c *FakePodPresets) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(podpresetsResource, c.ns, opts)) + +} + +// Create takes the representation of a podPreset and creates it. Returns the server's representation of the podPreset, and an error, if there is any. +func (c *FakePodPresets) Create(podPreset *v1alpha1.PodPreset) (result *v1alpha1.PodPreset, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(podpresetsResource, c.ns, podPreset), &v1alpha1.PodPreset{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PodPreset), err +} + +// Update takes the representation of a podPreset and updates it. Returns the server's representation of the podPreset, and an error, if there is any. +func (c *FakePodPresets) Update(podPreset *v1alpha1.PodPreset) (result *v1alpha1.PodPreset, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(podpresetsResource, c.ns, podPreset), &v1alpha1.PodPreset{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PodPreset), err +} + +// Delete takes name of the podPreset and deletes it. Returns an error if one occurs. +func (c *FakePodPresets) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteAction(podpresetsResource, c.ns, name), &v1alpha1.PodPreset{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakePodPresets) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(podpresetsResource, c.ns, listOptions) + + _, err := c.Fake.Invokes(action, &v1alpha1.PodPresetList{}) + return err +} + +// Patch applies the patch and returns the patched podPreset. +func (c *FakePodPresets) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1alpha1.PodPreset, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(podpresetsResource, c.ns, name, pt, data, subresources...), &v1alpha1.PodPreset{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.PodPreset), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake/fake_settings_client.go b/vendor/k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake/fake_settings_client.go new file mode 100644 index 0000000000..a142edfed0 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/settings/v1alpha1/fake/fake_settings_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/client-go/kubernetes/typed/settings/v1alpha1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeSettingsV1alpha1 struct { + *testing.Fake +} + +func (c *FakeSettingsV1alpha1) PodPresets(namespace string) v1alpha1.PodPresetInterface { + return &FakePodPresets{c, namespace} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeSettingsV1alpha1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/fake_storage_client.go b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/fake_storage_client.go new file mode 100644 index 0000000000..967a528500 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/fake_storage_client.go @@ -0,0 +1,44 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1 "k8s.io/client-go/kubernetes/typed/storage/v1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeStorageV1 struct { + *testing.Fake +} + +func (c *FakeStorageV1) StorageClasses() v1.StorageClassInterface { + return &FakeStorageClasses{c} +} + +func (c *FakeStorageV1) VolumeAttachments() v1.VolumeAttachmentInterface { + return &FakeVolumeAttachments{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeStorageV1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/fake_storageclass.go b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/fake_storageclass.go new file mode 100644 index 0000000000..c7531d8793 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/fake_storageclass.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + storagev1 "k8s.io/api/storage/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeStorageClasses implements StorageClassInterface +type FakeStorageClasses struct { + Fake *FakeStorageV1 +} + +var storageclassesResource = schema.GroupVersionResource{Group: "storage.k8s.io", Version: "v1", Resource: "storageclasses"} + +var storageclassesKind = schema.GroupVersionKind{Group: "storage.k8s.io", Version: "v1", Kind: "StorageClass"} + +// Get takes name of the storageClass, and returns the corresponding storageClass object, and an error if there is any. +func (c *FakeStorageClasses) Get(name string, options v1.GetOptions) (result *storagev1.StorageClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(storageclassesResource, name), &storagev1.StorageClass{}) + if obj == nil { + return nil, err + } + return obj.(*storagev1.StorageClass), err +} + +// List takes label and field selectors, and returns the list of StorageClasses that match those selectors. +func (c *FakeStorageClasses) List(opts v1.ListOptions) (result *storagev1.StorageClassList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(storageclassesResource, storageclassesKind, opts), &storagev1.StorageClassList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &storagev1.StorageClassList{ListMeta: obj.(*storagev1.StorageClassList).ListMeta} + for _, item := range obj.(*storagev1.StorageClassList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested storageClasses. +func (c *FakeStorageClasses) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(storageclassesResource, opts)) +} + +// Create takes the representation of a storageClass and creates it. Returns the server's representation of the storageClass, and an error, if there is any. +func (c *FakeStorageClasses) Create(storageClass *storagev1.StorageClass) (result *storagev1.StorageClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(storageclassesResource, storageClass), &storagev1.StorageClass{}) + if obj == nil { + return nil, err + } + return obj.(*storagev1.StorageClass), err +} + +// Update takes the representation of a storageClass and updates it. Returns the server's representation of the storageClass, and an error, if there is any. +func (c *FakeStorageClasses) Update(storageClass *storagev1.StorageClass) (result *storagev1.StorageClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(storageclassesResource, storageClass), &storagev1.StorageClass{}) + if obj == nil { + return nil, err + } + return obj.(*storagev1.StorageClass), err +} + +// Delete takes name of the storageClass and deletes it. Returns an error if one occurs. +func (c *FakeStorageClasses) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(storageclassesResource, name), &storagev1.StorageClass{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeStorageClasses) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(storageclassesResource, listOptions) + + _, err := c.Fake.Invokes(action, &storagev1.StorageClassList{}) + return err +} + +// Patch applies the patch and returns the patched storageClass. +func (c *FakeStorageClasses) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *storagev1.StorageClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(storageclassesResource, name, pt, data, subresources...), &storagev1.StorageClass{}) + if obj == nil { + return nil, err + } + return obj.(*storagev1.StorageClass), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/fake_volumeattachment.go b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/fake_volumeattachment.go new file mode 100644 index 0000000000..58e09da46b --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1/fake/fake_volumeattachment.go @@ -0,0 +1,131 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + storagev1 "k8s.io/api/storage/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeVolumeAttachments implements VolumeAttachmentInterface +type FakeVolumeAttachments struct { + Fake *FakeStorageV1 +} + +var volumeattachmentsResource = schema.GroupVersionResource{Group: "storage.k8s.io", Version: "v1", Resource: "volumeattachments"} + +var volumeattachmentsKind = schema.GroupVersionKind{Group: "storage.k8s.io", Version: "v1", Kind: "VolumeAttachment"} + +// Get takes name of the volumeAttachment, and returns the corresponding volumeAttachment object, and an error if there is any. +func (c *FakeVolumeAttachments) Get(name string, options v1.GetOptions) (result *storagev1.VolumeAttachment, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(volumeattachmentsResource, name), &storagev1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*storagev1.VolumeAttachment), err +} + +// List takes label and field selectors, and returns the list of VolumeAttachments that match those selectors. +func (c *FakeVolumeAttachments) List(opts v1.ListOptions) (result *storagev1.VolumeAttachmentList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(volumeattachmentsResource, volumeattachmentsKind, opts), &storagev1.VolumeAttachmentList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &storagev1.VolumeAttachmentList{ListMeta: obj.(*storagev1.VolumeAttachmentList).ListMeta} + for _, item := range obj.(*storagev1.VolumeAttachmentList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested volumeAttachments. +func (c *FakeVolumeAttachments) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(volumeattachmentsResource, opts)) +} + +// Create takes the representation of a volumeAttachment and creates it. Returns the server's representation of the volumeAttachment, and an error, if there is any. +func (c *FakeVolumeAttachments) Create(volumeAttachment *storagev1.VolumeAttachment) (result *storagev1.VolumeAttachment, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(volumeattachmentsResource, volumeAttachment), &storagev1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*storagev1.VolumeAttachment), err +} + +// Update takes the representation of a volumeAttachment and updates it. Returns the server's representation of the volumeAttachment, and an error, if there is any. +func (c *FakeVolumeAttachments) Update(volumeAttachment *storagev1.VolumeAttachment) (result *storagev1.VolumeAttachment, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(volumeattachmentsResource, volumeAttachment), &storagev1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*storagev1.VolumeAttachment), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeVolumeAttachments) UpdateStatus(volumeAttachment *storagev1.VolumeAttachment) (*storagev1.VolumeAttachment, error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateSubresourceAction(volumeattachmentsResource, "status", volumeAttachment), &storagev1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*storagev1.VolumeAttachment), err +} + +// Delete takes name of the volumeAttachment and deletes it. Returns an error if one occurs. +func (c *FakeVolumeAttachments) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(volumeattachmentsResource, name), &storagev1.VolumeAttachment{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeVolumeAttachments) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(volumeattachmentsResource, listOptions) + + _, err := c.Fake.Invokes(action, &storagev1.VolumeAttachmentList{}) + return err +} + +// Patch applies the patch and returns the patched volumeAttachment. +func (c *FakeVolumeAttachments) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *storagev1.VolumeAttachment, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(volumeattachmentsResource, name, pt, data, subresources...), &storagev1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*storagev1.VolumeAttachment), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake/fake_storage_client.go b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake/fake_storage_client.go new file mode 100644 index 0000000000..1a4d9f56fc --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake/fake_storage_client.go @@ -0,0 +1,40 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/client-go/kubernetes/typed/storage/v1alpha1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeStorageV1alpha1 struct { + *testing.Fake +} + +func (c *FakeStorageV1alpha1) VolumeAttachments() v1alpha1.VolumeAttachmentInterface { + return &FakeVolumeAttachments{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeStorageV1alpha1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake/fake_volumeattachment.go b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake/fake_volumeattachment.go new file mode 100644 index 0000000000..86f53e2d4d --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1alpha1/fake/fake_volumeattachment.go @@ -0,0 +1,131 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1alpha1 "k8s.io/api/storage/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeVolumeAttachments implements VolumeAttachmentInterface +type FakeVolumeAttachments struct { + Fake *FakeStorageV1alpha1 +} + +var volumeattachmentsResource = schema.GroupVersionResource{Group: "storage.k8s.io", Version: "v1alpha1", Resource: "volumeattachments"} + +var volumeattachmentsKind = schema.GroupVersionKind{Group: "storage.k8s.io", Version: "v1alpha1", Kind: "VolumeAttachment"} + +// Get takes name of the volumeAttachment, and returns the corresponding volumeAttachment object, and an error if there is any. +func (c *FakeVolumeAttachments) Get(name string, options v1.GetOptions) (result *v1alpha1.VolumeAttachment, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(volumeattachmentsResource, name), &v1alpha1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.VolumeAttachment), err +} + +// List takes label and field selectors, and returns the list of VolumeAttachments that match those selectors. +func (c *FakeVolumeAttachments) List(opts v1.ListOptions) (result *v1alpha1.VolumeAttachmentList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(volumeattachmentsResource, volumeattachmentsKind, opts), &v1alpha1.VolumeAttachmentList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1alpha1.VolumeAttachmentList{ListMeta: obj.(*v1alpha1.VolumeAttachmentList).ListMeta} + for _, item := range obj.(*v1alpha1.VolumeAttachmentList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested volumeAttachments. +func (c *FakeVolumeAttachments) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(volumeattachmentsResource, opts)) +} + +// Create takes the representation of a volumeAttachment and creates it. Returns the server's representation of the volumeAttachment, and an error, if there is any. +func (c *FakeVolumeAttachments) Create(volumeAttachment *v1alpha1.VolumeAttachment) (result *v1alpha1.VolumeAttachment, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(volumeattachmentsResource, volumeAttachment), &v1alpha1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.VolumeAttachment), err +} + +// Update takes the representation of a volumeAttachment and updates it. Returns the server's representation of the volumeAttachment, and an error, if there is any. +func (c *FakeVolumeAttachments) Update(volumeAttachment *v1alpha1.VolumeAttachment) (result *v1alpha1.VolumeAttachment, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(volumeattachmentsResource, volumeAttachment), &v1alpha1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.VolumeAttachment), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeVolumeAttachments) UpdateStatus(volumeAttachment *v1alpha1.VolumeAttachment) (*v1alpha1.VolumeAttachment, error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateSubresourceAction(volumeattachmentsResource, "status", volumeAttachment), &v1alpha1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.VolumeAttachment), err +} + +// Delete takes name of the volumeAttachment and deletes it. Returns an error if one occurs. +func (c *FakeVolumeAttachments) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(volumeattachmentsResource, name), &v1alpha1.VolumeAttachment{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeVolumeAttachments) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(volumeattachmentsResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1alpha1.VolumeAttachmentList{}) + return err +} + +// Patch applies the patch and returns the patched volumeAttachment. +func (c *FakeVolumeAttachments) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1alpha1.VolumeAttachment, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(volumeattachmentsResource, name, pt, data, subresources...), &v1alpha1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.VolumeAttachment), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/doc.go b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/doc.go new file mode 100644 index 0000000000..16f4439906 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/doc.go @@ -0,0 +1,20 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +// Package fake has the automatically generated clients. +package fake diff --git a/vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/fake_storage_client.go b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/fake_storage_client.go new file mode 100644 index 0000000000..e0e3f1d78b --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/fake_storage_client.go @@ -0,0 +1,44 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/client-go/kubernetes/typed/storage/v1beta1" + rest "k8s.io/client-go/rest" + testing "k8s.io/client-go/testing" +) + +type FakeStorageV1beta1 struct { + *testing.Fake +} + +func (c *FakeStorageV1beta1) StorageClasses() v1beta1.StorageClassInterface { + return &FakeStorageClasses{c} +} + +func (c *FakeStorageV1beta1) VolumeAttachments() v1beta1.VolumeAttachmentInterface { + return &FakeVolumeAttachments{c} +} + +// RESTClient returns a RESTClient that is used to communicate +// with API server by this client implementation. +func (c *FakeStorageV1beta1) RESTClient() rest.Interface { + var ret *rest.RESTClient + return ret +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/fake_storageclass.go b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/fake_storageclass.go new file mode 100644 index 0000000000..9fc8ca991e --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/fake_storageclass.go @@ -0,0 +1,120 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/storage/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeStorageClasses implements StorageClassInterface +type FakeStorageClasses struct { + Fake *FakeStorageV1beta1 +} + +var storageclassesResource = schema.GroupVersionResource{Group: "storage.k8s.io", Version: "v1beta1", Resource: "storageclasses"} + +var storageclassesKind = schema.GroupVersionKind{Group: "storage.k8s.io", Version: "v1beta1", Kind: "StorageClass"} + +// Get takes name of the storageClass, and returns the corresponding storageClass object, and an error if there is any. +func (c *FakeStorageClasses) Get(name string, options v1.GetOptions) (result *v1beta1.StorageClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(storageclassesResource, name), &v1beta1.StorageClass{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.StorageClass), err +} + +// List takes label and field selectors, and returns the list of StorageClasses that match those selectors. +func (c *FakeStorageClasses) List(opts v1.ListOptions) (result *v1beta1.StorageClassList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(storageclassesResource, storageclassesKind, opts), &v1beta1.StorageClassList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.StorageClassList{ListMeta: obj.(*v1beta1.StorageClassList).ListMeta} + for _, item := range obj.(*v1beta1.StorageClassList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested storageClasses. +func (c *FakeStorageClasses) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(storageclassesResource, opts)) +} + +// Create takes the representation of a storageClass and creates it. Returns the server's representation of the storageClass, and an error, if there is any. +func (c *FakeStorageClasses) Create(storageClass *v1beta1.StorageClass) (result *v1beta1.StorageClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(storageclassesResource, storageClass), &v1beta1.StorageClass{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.StorageClass), err +} + +// Update takes the representation of a storageClass and updates it. Returns the server's representation of the storageClass, and an error, if there is any. +func (c *FakeStorageClasses) Update(storageClass *v1beta1.StorageClass) (result *v1beta1.StorageClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(storageclassesResource, storageClass), &v1beta1.StorageClass{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.StorageClass), err +} + +// Delete takes name of the storageClass and deletes it. Returns an error if one occurs. +func (c *FakeStorageClasses) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(storageclassesResource, name), &v1beta1.StorageClass{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeStorageClasses) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(storageclassesResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.StorageClassList{}) + return err +} + +// Patch applies the patch and returns the patched storageClass. +func (c *FakeStorageClasses) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.StorageClass, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(storageclassesResource, name, pt, data, subresources...), &v1beta1.StorageClass{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.StorageClass), err +} diff --git a/vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/fake_volumeattachment.go b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/fake_volumeattachment.go new file mode 100644 index 0000000000..043098f455 --- /dev/null +++ b/vendor/k8s.io/client-go/kubernetes/typed/storage/v1beta1/fake/fake_volumeattachment.go @@ -0,0 +1,131 @@ +/* +Copyright 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. +*/ + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + v1beta1 "k8s.io/api/storage/v1beta1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + schema "k8s.io/apimachinery/pkg/runtime/schema" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeVolumeAttachments implements VolumeAttachmentInterface +type FakeVolumeAttachments struct { + Fake *FakeStorageV1beta1 +} + +var volumeattachmentsResource = schema.GroupVersionResource{Group: "storage.k8s.io", Version: "v1beta1", Resource: "volumeattachments"} + +var volumeattachmentsKind = schema.GroupVersionKind{Group: "storage.k8s.io", Version: "v1beta1", Kind: "VolumeAttachment"} + +// Get takes name of the volumeAttachment, and returns the corresponding volumeAttachment object, and an error if there is any. +func (c *FakeVolumeAttachments) Get(name string, options v1.GetOptions) (result *v1beta1.VolumeAttachment, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootGetAction(volumeattachmentsResource, name), &v1beta1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.VolumeAttachment), err +} + +// List takes label and field selectors, and returns the list of VolumeAttachments that match those selectors. +func (c *FakeVolumeAttachments) List(opts v1.ListOptions) (result *v1beta1.VolumeAttachmentList, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootListAction(volumeattachmentsResource, volumeattachmentsKind, opts), &v1beta1.VolumeAttachmentList{}) + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1beta1.VolumeAttachmentList{ListMeta: obj.(*v1beta1.VolumeAttachmentList).ListMeta} + for _, item := range obj.(*v1beta1.VolumeAttachmentList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested volumeAttachments. +func (c *FakeVolumeAttachments) Watch(opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewRootWatchAction(volumeattachmentsResource, opts)) +} + +// Create takes the representation of a volumeAttachment and creates it. Returns the server's representation of the volumeAttachment, and an error, if there is any. +func (c *FakeVolumeAttachments) Create(volumeAttachment *v1beta1.VolumeAttachment) (result *v1beta1.VolumeAttachment, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootCreateAction(volumeattachmentsResource, volumeAttachment), &v1beta1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.VolumeAttachment), err +} + +// Update takes the representation of a volumeAttachment and updates it. Returns the server's representation of the volumeAttachment, and an error, if there is any. +func (c *FakeVolumeAttachments) Update(volumeAttachment *v1beta1.VolumeAttachment) (result *v1beta1.VolumeAttachment, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateAction(volumeattachmentsResource, volumeAttachment), &v1beta1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.VolumeAttachment), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeVolumeAttachments) UpdateStatus(volumeAttachment *v1beta1.VolumeAttachment) (*v1beta1.VolumeAttachment, error) { + obj, err := c.Fake. + Invokes(testing.NewRootUpdateSubresourceAction(volumeattachmentsResource, "status", volumeAttachment), &v1beta1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.VolumeAttachment), err +} + +// Delete takes name of the volumeAttachment and deletes it. Returns an error if one occurs. +func (c *FakeVolumeAttachments) Delete(name string, options *v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewRootDeleteAction(volumeattachmentsResource, name), &v1beta1.VolumeAttachment{}) + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeVolumeAttachments) DeleteCollection(options *v1.DeleteOptions, listOptions v1.ListOptions) error { + action := testing.NewRootDeleteCollectionAction(volumeattachmentsResource, listOptions) + + _, err := c.Fake.Invokes(action, &v1beta1.VolumeAttachmentList{}) + return err +} + +// Patch applies the patch and returns the patched volumeAttachment. +func (c *FakeVolumeAttachments) Patch(name string, pt types.PatchType, data []byte, subresources ...string) (result *v1beta1.VolumeAttachment, err error) { + obj, err := c.Fake. + Invokes(testing.NewRootPatchSubresourceAction(volumeattachmentsResource, name, pt, data, subresources...), &v1beta1.VolumeAttachment{}) + if obj == nil { + return nil, err + } + return obj.(*v1beta1.VolumeAttachment), err +} From 1272d46191fe05c31154811cb0577ffb14374e4e Mon Sep 17 00:00:00 2001 From: thandayuthapani Date: Thu, 6 Jun 2019 09:27:14 +0530 Subject: [PATCH 57/68] Add UT cases for Job package - job_controller_plugins --- Gopkg.lock | 36 ++- pkg/apis/helpers/helpers.go | 4 +- pkg/controllers/job/job_controller.go | 12 +- .../job/job_controller_plugins_test.go | 287 ++++++++++++++++++ .../job/plugins/interface/interface.go | 2 +- 5 files changed, 331 insertions(+), 10 deletions(-) create mode 100644 pkg/controllers/job/job_controller_plugins_test.go diff --git a/Gopkg.lock b/Gopkg.lock index 35bdd31b03..8165d21a8f 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -726,7 +726,7 @@ version = "kubernetes-1.13.2" [[projects]] - digest = "1:f2da632d588dfece5f2699e26a55c3873b04d07ed562170033686cb0e33bdb1d" + digest = "1:c542cfd25f35d41d6d7818d5710684c83c5112a2fd9ca0123693d5eff2c0b62d" name = "k8s.io/client-go" packages = [ "discovery", @@ -778,39 +778,72 @@ "informers/storage/v1alpha1", "informers/storage/v1beta1", "kubernetes", + "kubernetes/fake", "kubernetes/scheme", "kubernetes/typed/admissionregistration/v1alpha1", + "kubernetes/typed/admissionregistration/v1alpha1/fake", "kubernetes/typed/admissionregistration/v1beta1", + "kubernetes/typed/admissionregistration/v1beta1/fake", "kubernetes/typed/apps/v1", + "kubernetes/typed/apps/v1/fake", "kubernetes/typed/apps/v1beta1", + "kubernetes/typed/apps/v1beta1/fake", "kubernetes/typed/apps/v1beta2", + "kubernetes/typed/apps/v1beta2/fake", "kubernetes/typed/auditregistration/v1alpha1", + "kubernetes/typed/auditregistration/v1alpha1/fake", "kubernetes/typed/authentication/v1", + "kubernetes/typed/authentication/v1/fake", "kubernetes/typed/authentication/v1beta1", + "kubernetes/typed/authentication/v1beta1/fake", "kubernetes/typed/authorization/v1", + "kubernetes/typed/authorization/v1/fake", "kubernetes/typed/authorization/v1beta1", + "kubernetes/typed/authorization/v1beta1/fake", "kubernetes/typed/autoscaling/v1", + "kubernetes/typed/autoscaling/v1/fake", "kubernetes/typed/autoscaling/v2beta1", + "kubernetes/typed/autoscaling/v2beta1/fake", "kubernetes/typed/autoscaling/v2beta2", + "kubernetes/typed/autoscaling/v2beta2/fake", "kubernetes/typed/batch/v1", + "kubernetes/typed/batch/v1/fake", "kubernetes/typed/batch/v1beta1", + "kubernetes/typed/batch/v1beta1/fake", "kubernetes/typed/batch/v2alpha1", + "kubernetes/typed/batch/v2alpha1/fake", "kubernetes/typed/certificates/v1beta1", + "kubernetes/typed/certificates/v1beta1/fake", "kubernetes/typed/coordination/v1beta1", + "kubernetes/typed/coordination/v1beta1/fake", "kubernetes/typed/core/v1", + "kubernetes/typed/core/v1/fake", "kubernetes/typed/events/v1beta1", + "kubernetes/typed/events/v1beta1/fake", "kubernetes/typed/extensions/v1beta1", + "kubernetes/typed/extensions/v1beta1/fake", "kubernetes/typed/networking/v1", + "kubernetes/typed/networking/v1/fake", "kubernetes/typed/policy/v1beta1", + "kubernetes/typed/policy/v1beta1/fake", "kubernetes/typed/rbac/v1", + "kubernetes/typed/rbac/v1/fake", "kubernetes/typed/rbac/v1alpha1", + "kubernetes/typed/rbac/v1alpha1/fake", "kubernetes/typed/rbac/v1beta1", + "kubernetes/typed/rbac/v1beta1/fake", "kubernetes/typed/scheduling/v1alpha1", + "kubernetes/typed/scheduling/v1alpha1/fake", "kubernetes/typed/scheduling/v1beta1", + "kubernetes/typed/scheduling/v1beta1/fake", "kubernetes/typed/settings/v1alpha1", + "kubernetes/typed/settings/v1alpha1/fake", "kubernetes/typed/storage/v1", + "kubernetes/typed/storage/v1/fake", "kubernetes/typed/storage/v1alpha1", + "kubernetes/typed/storage/v1alpha1/fake", "kubernetes/typed/storage/v1beta1", + "kubernetes/typed/storage/v1beta1/fake", "listers/admissionregistration/v1alpha1", "listers/admissionregistration/v1beta1", "listers/apps/v1", @@ -1090,6 +1123,7 @@ "k8s.io/client-go/informers/core/v1", "k8s.io/client-go/informers/scheduling/v1beta1", "k8s.io/client-go/kubernetes", + "k8s.io/client-go/kubernetes/fake", "k8s.io/client-go/kubernetes/scheme", "k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1", "k8s.io/client-go/kubernetes/typed/core/v1", diff --git a/pkg/apis/helpers/helpers.go b/pkg/apis/helpers/helpers.go index 3e2d21c67b..571d5edb70 100644 --- a/pkg/apis/helpers/helpers.go +++ b/pkg/apis/helpers/helpers.go @@ -63,7 +63,7 @@ func ControlledBy(obj interface{}, gvk schema.GroupVersionKind) bool { return false } -func CreateConfigMapIfNotExist(job *vkv1.Job, kubeClients *kubernetes.Clientset, data map[string]string, cmName string) error { +func CreateConfigMapIfNotExist(job *vkv1.Job, kubeClients kubernetes.Interface, data map[string]string, cmName string) error { // If ConfigMap does not exist, create one for Job. cmOld, err := kubeClients.CoreV1().ConfigMaps(job.Namespace).Get(cmName, metav1.GetOptions{}) if err != nil { @@ -102,7 +102,7 @@ func CreateConfigMapIfNotExist(job *vkv1.Job, kubeClients *kubernetes.Clientset, return nil } -func DeleteConfigmap(job *vkv1.Job, kubeClients *kubernetes.Clientset, cmName string) error { +func DeleteConfigmap(job *vkv1.Job, kubeClients kubernetes.Interface, cmName string) error { if _, err := kubeClients.CoreV1().ConfigMaps(job.Namespace).Get(cmName, metav1.GetOptions{}); err != nil { if !apierrors.IsNotFound(err) { glog.V(3).Infof("Failed to get Configmap for Job <%s/%s>: %v", diff --git a/pkg/controllers/job/job_controller.go b/pkg/controllers/job/job_controller.go index 76f73f09bb..8f6eb06dea 100644 --- a/pkg/controllers/job/job_controller.go +++ b/pkg/controllers/job/job_controller.go @@ -52,9 +52,9 @@ import ( // Controller the Job Controller type type Controller struct { - kubeClients *kubernetes.Clientset - vkClients *vkver.Clientset - kbClients *kbver.Clientset + kubeClients kubernetes.Interface + vkClients vkver.Interface + kbClients kbver.Interface jobInformer vkbatchinfo.JobInformer podInformer coreinformers.PodInformer @@ -101,9 +101,9 @@ type Controller struct { // NewJobController create new Job Controller func NewJobController( - kubeClient *kubernetes.Clientset, - kbClient *kbver.Clientset, - vkClient *vkver.Clientset, + kubeClient kubernetes.Interface, + kbClient kbver.Interface, + vkClient vkver.Interface, ) *Controller { //Initialize event client diff --git a/pkg/controllers/job/job_controller_plugins_test.go b/pkg/controllers/job/job_controller_plugins_test.go new file mode 100644 index 0000000000..2d3d3141aa --- /dev/null +++ b/pkg/controllers/job/job_controller_plugins_test.go @@ -0,0 +1,287 @@ +/* +Copyright 2019 The Volcano 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 job + +import ( + "fmt" + "testing" + + "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + kubeclient "k8s.io/client-go/kubernetes/fake" + + vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" + volcanoclient "volcano.sh/volcano/pkg/client/clientset/versioned/fake" + + kubebatchclient "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/fake" +) + +func newFakeController() *Controller { + KubeBatchClientSet := kubebatchclient.NewSimpleClientset() + VolcanoClientSet := volcanoclient.NewSimpleClientset() + KubeClientSet := kubeclient.NewSimpleClientset() + + controller := NewJobController(KubeClientSet, KubeBatchClientSet, VolcanoClientSet) + return controller +} + +func TestPluginOnPodCreate(t *testing.T) { + namespace := "test" + + testcases := []struct { + Name string + Job *vkv1.Job + Pod *v1.Pod + Plugins []string + RetVal error + }{ + { + Name: "All Plugin", + Job: &vkv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "Job1", + Namespace: namespace, + }, + }, + Pod: buildPod(namespace, "pod1", v1.PodPending, nil), + Plugins: []string{"env", "svc", "ssh"}, + RetVal: nil, + }, + { + Name: "Wrong Plugin", + Job: &vkv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "Job1", + }, + }, + Pod: buildPod(namespace, "pod1", v1.PodPending, nil), + Plugins: []string{"new"}, + RetVal: fmt.Errorf("failed to get plugin %s", "new"), + }, + } + + for i, testcase := range testcases { + + fakeController := newFakeController() + jobPlugins := make(map[string][]string) + + for _, plugin := range testcase.Plugins { + jobPlugins[plugin] = make([]string, 0) + } + + testcase.Job.Spec.Plugins = jobPlugins + + err := fakeController.pluginOnPodCreate(testcase.Job, testcase.Pod) + if testcase.RetVal != nil && err.Error() != testcase.RetVal.Error() { + t.Errorf("case %d (%s): expected: %v, got %v ", i, testcase.Name, testcase.RetVal, err) + } + + for _, plugin := range testcase.Plugins { + if plugin == "env" { + for _, container := range testcase.Pod.Spec.Containers { + if len(container.Env) == 0 { + t.Errorf("case %d (%s): expected: Env Length not to be zero", i, testcase.Name) + } + } + } + + if plugin == "svc" { + for _, container := range testcase.Pod.Spec.Containers { + if len(container.VolumeMounts) == 0 { + t.Errorf("case %d (%s): expected: VolumeMount Length not to be zero", i, testcase.Name) + } + exist := false + for _, volume := range container.VolumeMounts { + if volume.Name == fmt.Sprint(testcase.Job.Name, "-svc") { + exist = true + } + } + if !exist { + t.Errorf("case %d (%s): expected: VolumeMount not created", i, testcase.Name) + } + } + } + + if plugin == "ssh" { + for _, container := range testcase.Pod.Spec.Containers { + if len(container.VolumeMounts) == 0 { + t.Errorf("case %d (%s): expected: VolumeMount Length not to be zero", i, testcase.Name) + } + exist := false + for _, volume := range container.VolumeMounts { + if volume.Name == fmt.Sprint(testcase.Job.Name, "-ssh") { + exist = true + } + } + if !exist { + t.Errorf("case %d (%s): expected: VolumeMount not created", i, testcase.Name) + } + } + } + } + } +} + +func TestPluginOnJobAdd(t *testing.T) { + namespace := "test" + + testcases := []struct { + Name string + Job *vkv1.Job + Plugins []string + RetVal error + }{ + { + Name: "Plugins", + Job: &vkv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job1", + Namespace: namespace, + }, + }, + Plugins: []string{"svc", "ssh", "env"}, + RetVal: nil, + }, + { + Name: "Wrong Plugin", + Job: &vkv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "Job1", + }, + }, + Plugins: []string{"new"}, + RetVal: fmt.Errorf("failed to get plugin %s", "new"), + }, + } + + for i, testcase := range testcases { + + fakeController := newFakeController() + jobPlugins := make(map[string][]string) + + for _, plugin := range testcase.Plugins { + jobPlugins[plugin] = make([]string, 0) + } + + testcase.Job.Spec.Plugins = jobPlugins + + err := fakeController.pluginOnJobAdd(testcase.Job) + if testcase.RetVal != nil && err.Error() != testcase.RetVal.Error() { + t.Errorf("case %d (%s): expected: %v, got %v ", i, testcase.Name, testcase.RetVal, err) + } + + for _, plugin := range testcase.Plugins { + + if plugin == "svc" { + _, err := fakeController.kubeClients.CoreV1().ConfigMaps(namespace).Get(fmt.Sprint(testcase.Job.Name, "-svc"), metav1.GetOptions{}) + if err != nil { + t.Errorf("Case %d (%s): expected: ConfigMap to be created, but not created because of error %s", i, testcase.Name, err.Error()) + } + + _, err = fakeController.kubeClients.CoreV1().Services(namespace).Get(testcase.Job.Name, metav1.GetOptions{}) + if err != nil { + t.Errorf("Case %d (%s): expected: Service to be created, but not created because of error %s", i, testcase.Name, err.Error()) + } + } + + if plugin == "ssh" { + _, err := fakeController.kubeClients.CoreV1().ConfigMaps(namespace).Get(fmt.Sprint(testcase.Job.Name, "-ssh"), metav1.GetOptions{}) + if err != nil { + t.Errorf("Case %d (%s): expected: ConfigMap to be created, but not created because of error %s", i, testcase.Name, err.Error()) + } + } + + if plugin == "env" { + if testcase.Job.Status.ControlledResources["plugin-env"] == "" { + t.Errorf("Case %d (%s): expected: to find controlled resource, but not found because of error %s", i, testcase.Name, err.Error()) + } + } + } + } +} + +func TestPluginOnJobDelete(t *testing.T) { + namespace := "test" + + testcases := []struct { + Name string + Job *vkv1.Job + Plugins []string + RetVal error + }{ + { + Name: "Plugins", + Job: &vkv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "job1", + Namespace: namespace, + }, + }, + Plugins: []string{"svc", "ssh", "env"}, + RetVal: nil, + }, + { + Name: "Wrong Plugin", + Job: &vkv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: "Job1", + }, + }, + Plugins: []string{"new"}, + RetVal: fmt.Errorf("failed to get plugin %s", "new"), + }, + } + + for i, testcase := range testcases { + + fakeController := newFakeController() + jobPlugins := make(map[string][]string) + + for _, plugin := range testcase.Plugins { + jobPlugins[plugin] = make([]string, 0) + } + + testcase.Job.Spec.Plugins = jobPlugins + + err := fakeController.pluginOnJobDelete(testcase.Job) + if testcase.RetVal != nil && err.Error() != testcase.RetVal.Error() { + t.Errorf("case %d (%s): expected: %v, got %v ", i, testcase.Name, testcase.RetVal, err) + } + + for _, plugin := range testcase.Plugins { + + if plugin == "svc" { + _, err := fakeController.kubeClients.CoreV1().ConfigMaps(namespace).Get(fmt.Sprint(testcase.Job.Name, "-svc"), metav1.GetOptions{}) + if err == nil { + t.Errorf("Case %d (%s): expected: ConfigMap to be deleted, but not deleted because of error %s", i, testcase.Name, err.Error()) + } + + _, err = fakeController.kubeClients.CoreV1().Services(namespace).Get(testcase.Job.Name, metav1.GetOptions{}) + if err == nil { + t.Errorf("Case %d (%s): expected: Service to be deleted, but not deleted because of error %s", i, testcase.Name, err.Error()) + } + } + + if plugin == "ssh" { + _, err := fakeController.kubeClients.CoreV1().ConfigMaps(namespace).Get(fmt.Sprint(testcase.Job.Name, "-ssh"), metav1.GetOptions{}) + if err == nil { + t.Errorf("Case %d (%s): expected: ConfigMap to be deleted, but not deleted because of error %s", i, testcase.Name, err.Error()) + } + } + } + } +} diff --git a/pkg/controllers/job/plugins/interface/interface.go b/pkg/controllers/job/plugins/interface/interface.go index 2e9274a154..8481e70327 100644 --- a/pkg/controllers/job/plugins/interface/interface.go +++ b/pkg/controllers/job/plugins/interface/interface.go @@ -24,7 +24,7 @@ import ( ) type PluginClientset struct { - KubeClients *kubernetes.Clientset + KubeClients kubernetes.Interface } type PluginInterface interface { From a480827454e1bf68013537c4b68cefc77f885636 Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Sat, 8 Jun 2019 11:55:37 +0530 Subject: [PATCH 58/68] updated readme --- README.md | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/README.md b/README.md index 4c997649d5..677d5c2563 100644 --- a/README.md +++ b/README.md @@ -88,16 +88,6 @@ example, if you are using [kind cluster](https://github.com/kubernetes-sigs/kind try command ```kind load docker-image : ``` for each of the images. ### 2. Helm charts -Second, install the required helm plugin and generate valid -certificate, volcano uses a helm plugin **gen-admission-secret** to -generate certificate for admission service to communicate with -kubernetes API server. - -``` -#1. Install helm plugin -helm plugin install installer/chart/plugins/gen-admission-secret - -``` Secondly, install helm chart. @@ -118,6 +108,7 @@ NAME READY STATUS RESTARTS -admission-84fd9b9dd8-9trxn 1/1 Running 0 43s -controllers-75dcc8ff89-42v6r 1/1 Running 0 43s -scheduler-b94cdb867-89pm2 1/1 Running 0 43s +--admission-init-qbtmb 0/1 Completed 0 43s #2. Verify the Services # kubectl get services --namespace From b5ab5cbce6446809865651ffdf775a31a273f1e8 Mon Sep 17 00:00:00 2001 From: Rajadeepan D Ramesh Date: Tue, 11 Jun 2019 14:29:21 +0530 Subject: [PATCH 59/68] Fix lint error in cmd package --- cmd/admission/app/configure/configure.go | 6 +++++- cmd/admission/app/server.go | 10 ++++++++-- cmd/controllers/app/options/options.go | 1 + cmd/controllers/app/server.go | 3 +++ hack/.golint_failures | 4 ---- 5 files changed, 17 insertions(+), 7 deletions(-) diff --git a/cmd/admission/app/configure/configure.go b/cmd/admission/app/configure/configure.go index 1875841463..981af32231 100644 --- a/cmd/admission/app/configure/configure.go +++ b/cmd/admission/app/configure/configure.go @@ -13,6 +13,7 @@ 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 configure import ( @@ -28,7 +29,7 @@ import ( admissionregistrationv1beta1client "k8s.io/client-go/kubernetes/typed/admissionregistration/v1beta1" ) -// admission-controller server config. +// Config admission-controller server config. type Config struct { Master string Kubeconfig string @@ -43,11 +44,13 @@ type Config struct { PrintVersion bool } +// NewConfig create new config func NewConfig() *Config { c := Config{} return &c } +// AddFlags add flags func (c *Config) AddFlags() { flag.StringVar(&c.Master, "master", c.Master, "The address of the Kubernetes API server (overrides any value in kubeconfig)") flag.StringVar(&c.Kubeconfig, "kubeconfig", c.Kubeconfig, "Path to kubeconfig file with authorization and master location information.") @@ -68,6 +71,7 @@ func (c *Config) AddFlags() { flag.BoolVar(&c.PrintVersion, "version", false, "Show version and quit") } +// CheckPortOrDie check valid port range func (c *Config) CheckPortOrDie() error { if c.Port < 1 || c.Port > 65535 { return fmt.Errorf("the port should be in the range of 1 and 65535") diff --git a/cmd/admission/app/server.go b/cmd/admission/app/server.go index d6fbaeb76e..23669438ba 100644 --- a/cmd/admission/app/server.go +++ b/cmd/admission/app/server.go @@ -13,6 +13,7 @@ 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 app import ( @@ -34,11 +35,14 @@ import ( ) const ( - CONTENTTYPE = "Content-Type" + //CONTENTTYPE http content-type + CONTENTTYPE = "Content-Type" + + //APPLICATIONJSON json content APPLICATIONJSON = "application/json" ) -// Get a clientset with restConfig. +// GetClient Get a clientset with restConfig. func GetClient(restConfig *restclient.Config) *kubernetes.Clientset { clientset, err := kubernetes.NewForConfig(restConfig) if err != nil { @@ -47,6 +51,7 @@ func GetClient(restConfig *restclient.Config) *kubernetes.Clientset { return clientset } +//GetKubeBatchClient get a clientset for kubebatch func GetKubeBatchClient(restConfig *restclient.Config) *versioned.Clientset { clientset, err := versioned.NewForConfig(restConfig) if err != nil { @@ -85,6 +90,7 @@ func ConfigTLS(config *appConf.Config, restConfig *restclient.Config) *tls.Confi return &tls.Config{} } +//Serve the http request func Serve(w http.ResponseWriter, r *http.Request, admit admissioncontroller.AdmitFunc) { var body []byte if r.Body != nil { diff --git a/cmd/controllers/app/options/options.go b/cmd/controllers/app/options/options.go index dfb2017e86..d0ea3fe9ca 100644 --- a/cmd/controllers/app/options/options.go +++ b/cmd/controllers/app/options/options.go @@ -56,6 +56,7 @@ func (s *ServerOption) AddFlags(fs *pflag.FlagSet) { fs.BoolVar(&s.PrintVersion, "version", false, "Show version and quit") } +// CheckOptionOrDie checks the LockObjectNamespace func (s *ServerOption) CheckOptionOrDie() error { if s.EnableLeaderElection && s.LockObjectNamespace == "" { return fmt.Errorf("lock-object-namespace must not be nil when LeaderElection is enabled") diff --git a/cmd/controllers/app/server.go b/cmd/controllers/app/server.go index 57e85b1cf6..f3748e3555 100644 --- a/cmd/controllers/app/server.go +++ b/cmd/controllers/app/server.go @@ -29,6 +29,8 @@ import ( clientset "k8s.io/client-go/kubernetes" "k8s.io/client-go/kubernetes/scheme" corev1 "k8s.io/client-go/kubernetes/typed/core/v1" + + // Initialize client auth plugin. _ "k8s.io/client-go/plugin/pkg/client/auth/gcp" "k8s.io/client-go/rest" "k8s.io/client-go/tools/clientcmd" @@ -71,6 +73,7 @@ func buildConfig(opt *options.ServerOption) (*rest.Config, error) { return cfg, nil } +//Run the controller func Run(opt *options.ServerOption) error { config, err := buildConfig(opt) if err != nil { diff --git a/hack/.golint_failures b/hack/.golint_failures index 3ce6bcb7e3..0ce0e14469 100644 --- a/hack/.golint_failures +++ b/hack/.golint_failures @@ -1,7 +1,3 @@ -volcano.sh/volcano/cmd/admission/app -volcano.sh/volcano/cmd/admission/app/configure -volcano.sh/volcano/cmd/controllers/app -volcano.sh/volcano/cmd/controllers/app/options volcano.sh/volcano/pkg/admission volcano.sh/volcano/pkg/apis/batch/v1alpha1 volcano.sh/volcano/pkg/apis/bus/v1alpha1 From 0e9cb5df4f819c15fece6fae9d9f236c62d57ffb Mon Sep 17 00:00:00 2001 From: asifdxtreme Date: Tue, 11 Jun 2019 15:12:16 +0530 Subject: [PATCH 60/68] Bump volcano-sh/scheduler --- Gopkg.lock | 4 +- .../kube-batch/cmd/kube-batch/app/server.go | 2 +- .../kube-batch/pkg/scheduler/api/node_info.go | 88 +++++++++++++++---- .../kube-batch/pkg/scheduler/api/types.go | 21 +++++ .../kube-batch/pkg/scheduler/cache/cache.go | 11 ++- .../pkg/scheduler/cache/event_handlers.go | 12 +-- 6 files changed, 106 insertions(+), 32 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 8165d21a8f..9560e925f9 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -204,7 +204,7 @@ [[projects]] branch = "master" - digest = "1:4281fb8a5ed4671cd7e80c85599e00912f796819716abaefd697797ec9eb4491" + digest = "1:ea789d69109979142a6fbd569095f7678ea62b633b9b5a284f2fd1ff673da48f" name = "github.com/kubernetes-sigs/kube-batch" packages = [ "cmd/kube-batch/app", @@ -247,7 +247,7 @@ "pkg/version", ] pruneopts = "UT" - revision = "700381278a3e7da396507055d371dd93c1724322" + revision = "4b391ab34b53779e47243217006d8772cb86d8d8" source = "https://github.com/volcano-sh/scheduler" [[projects]] diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/cmd/kube-batch/app/server.go b/vendor/github.com/kubernetes-sigs/kube-batch/cmd/kube-batch/app/server.go index f0e1491d72..1fbe6f415e 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/cmd/kube-batch/app/server.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/cmd/kube-batch/app/server.go @@ -103,7 +103,7 @@ func Run(opt *options.ServerOption) error { // Prepare event clients. broadcaster := record.NewBroadcaster() broadcaster.StartRecordingToSink(&corev1.EventSinkImpl{Interface: leaderElectionClient.CoreV1().Events(opt.LockObjectNamespace)}) - eventRecorder := broadcaster.NewRecorder(scheme.Scheme, v1.EventSource{Component: "kube-batch"}) + eventRecorder := broadcaster.NewRecorder(scheme.Scheme, v1.EventSource{Component: opt.SchedulerName}) hostname, err := os.Hostname() if err != nil { diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/node_info.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/node_info.go index 16257d4140..679818db51 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/node_info.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/node_info.go @@ -19,6 +19,8 @@ package api import ( "fmt" + "github.com/golang/glog" + v1 "k8s.io/api/core/v1" ) @@ -27,6 +29,9 @@ type NodeInfo struct { Name string Node *v1.Node + // The state of node + State NodeState + // The releasing resource on that node Releasing *Resource // The idle resource on that node @@ -44,10 +49,18 @@ type NodeInfo struct { Other interface{} } +// NodeState defines the current state of node. +type NodeState struct { + Phase NodePhase + Reason string +} + // NewNodeInfo is used to create new nodeInfo object func NewNodeInfo(node *v1.Node) *NodeInfo { + var ni *NodeInfo + if node == nil { - return &NodeInfo{ + ni = &NodeInfo{ Releasing: EmptyResource(), Idle: EmptyResource(), Used: EmptyResource(), @@ -57,21 +70,25 @@ func NewNodeInfo(node *v1.Node) *NodeInfo { Tasks: make(map[TaskID]*TaskInfo), } - } - - return &NodeInfo{ - Name: node.Name, - Node: node, + } else { + ni = &NodeInfo{ + Name: node.Name, + Node: node, - Releasing: EmptyResource(), - Idle: NewResource(node.Status.Allocatable), - Used: EmptyResource(), + Releasing: EmptyResource(), + Idle: NewResource(node.Status.Allocatable), + Used: EmptyResource(), - Allocatable: NewResource(node.Status.Allocatable), - Capability: NewResource(node.Status.Capacity), + Allocatable: NewResource(node.Status.Allocatable), + Capability: NewResource(node.Status.Capacity), - Tasks: make(map[TaskID]*TaskInfo), + Tasks: make(map[TaskID]*TaskInfo), + } } + + ni.setNodeState(node) + + return ni } // Clone used to clone nodeInfo Object @@ -85,8 +102,47 @@ func (ni *NodeInfo) Clone() *NodeInfo { return res } +// Ready returns whether node is ready for scheduling +func (ni *NodeInfo) Ready() bool { + return ni.State.Phase == Ready +} + +func (ni *NodeInfo) setNodeState(node *v1.Node) { + // If node is nil, the node is un-initialized in cache + if node == nil { + ni.State = NodeState{ + Phase: NotReady, + Reason: "UnInitialized", + } + return + } + + // set NodeState according to resources + if !ni.Used.LessEqual(NewResource(node.Status.Allocatable)) { + ni.State = NodeState{ + Phase: NotReady, + Reason: "OutOfSync", + } + return + } + + // Node is ready (ignore node conditions because of taint/toleration) + ni.State = NodeState{ + Phase: Ready, + Reason: "", + } +} + // SetNode sets kubernetes node object to nodeInfo object func (ni *NodeInfo) SetNode(node *v1.Node) { + ni.setNodeState(node) + + if !ni.Ready() { + glog.Warningf("Failed to set node info, phase: %s, reason: %s", + ni.State.Phase, ni.State.Reason) + return + } + ni.Name = node.Name ni.Node = node @@ -176,16 +232,16 @@ func (ni *NodeInfo) UpdateTask(ti *TaskInfo) error { // String returns nodeInfo details in string format func (ni NodeInfo) String() string { - res := "" + tasks := "" i := 0 for _, task := range ni.Tasks { - res = res + fmt.Sprintf("\n\t %d: %v", i, task) + tasks = tasks + fmt.Sprintf("\n\t %d: %v", i, task) i++ } - return fmt.Sprintf("Node (%s): idle <%v>, used <%v>, releasing <%v>, taints <%v>%s", - ni.Name, ni.Idle, ni.Used, ni.Releasing, ni.Node.Spec.Taints, res) + return fmt.Sprintf("Node (%s): idle <%v>, used <%v>, releasing <%v>, state , taints <%v>%s", + ni.Name, ni.Idle, ni.Used, ni.Releasing, ni.State.Phase, ni.State.Reason, ni.Node.Spec.Taints, tasks) } diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/types.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/types.go index 97a583df6c..a88439e451 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/types.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api/types.go @@ -78,6 +78,27 @@ func (ts TaskStatus) String() string { } } +// NodePhase defines the phase of node +type NodePhase int + +const ( + // Ready means the node is ready for scheduling + Ready NodePhase = 1 << iota + // NotReady means the node is not ready for scheduling + NotReady +) + +func (np NodePhase) String() string { + switch np { + case Ready: + return "Ready" + case NotReady: + return "NotReady" + } + + return "Unknown" +} + // validateStatusUpdate validates whether the status transfer is valid. func validateStatusUpdate(oldStatus, newStatus TaskStatus) error { return nil diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/cache.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/cache.go index f6f1c04d2e..322d3b4d02 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/cache.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/cache.go @@ -200,7 +200,7 @@ func newSchedulerCache(config *rest.Config, schedulerName string, defaultQueue s // Prepare event clients. broadcaster := record.NewBroadcaster() broadcaster.StartRecordingToSink(&corev1.EventSinkImpl{Interface: sc.kubeclient.CoreV1().Events("")}) - sc.Recorder = broadcaster.NewRecorder(scheme.Scheme, v1.EventSource{Component: "kube-batch"}) + sc.Recorder = broadcaster.NewRecorder(scheme.Scheme, v1.EventSource{Component: schedulerName}) sc.Binder = &defaultBinder{ kubeclient: sc.kubeclient, @@ -465,7 +465,10 @@ func (sc *SchedulerCache) taskUnschedulable(task *api.TaskInfo, message string) pod := task.Pod.DeepCopy() - sc.Recorder.Eventf(pod, v1.EventTypeWarning, string(v1.PodReasonUnschedulable), message) + // The reason field in 'Events' should be "FailedScheduling", there is not constants defined for this in + // k8s core, so using the same string here. + // The reason field in PodCondition should be "Unschedulable" + sc.Recorder.Eventf(pod, v1.EventTypeWarning, "FailedScheduling", message) if _, err := sc.StatusUpdater.UpdatePodCondition(pod, &v1.PodCondition{ Type: v1.PodScheduled, Status: v1.ConditionFalse, @@ -546,6 +549,10 @@ func (sc *SchedulerCache) Snapshot() *kbapi.ClusterInfo { } for _, value := range sc.Nodes { + if !value.Ready() { + continue + } + snapshot.Nodes[value.Name] = value.Clone() } diff --git a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/event_handlers.go b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/event_handlers.go index 4260521fdc..3b005a2fee 100644 --- a/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/event_handlers.go +++ b/vendor/github.com/kubernetes-sigs/kube-batch/pkg/scheduler/cache/event_handlers.go @@ -18,7 +18,6 @@ package cache import ( "fmt" - "reflect" "github.com/golang/glog" @@ -269,19 +268,10 @@ func (sc *SchedulerCache) addNode(node *v1.Node) error { return nil } -func isNodeInfoUpdated(oldNode, newNode *v1.Node) bool { - return !reflect.DeepEqual(oldNode.Status.Allocatable, newNode.Status.Allocatable) || - !reflect.DeepEqual(oldNode.Spec.Taints, newNode.Spec.Taints) || - !reflect.DeepEqual(oldNode.Labels, newNode.Labels) || - !reflect.DeepEqual(oldNode.Spec.Unschedulable, newNode.Spec.Unschedulable) -} - // Assumes that lock is already acquired. func (sc *SchedulerCache) updateNode(oldNode, newNode *v1.Node) error { if sc.Nodes[newNode.Name] != nil { - if isNodeInfoUpdated(oldNode, newNode) { - sc.Nodes[newNode.Name].SetNode(newNode) - } + sc.Nodes[newNode.Name].SetNode(newNode) return nil } From b1fc7ceb51cd8df61f3ce4e8aa023f694f968dd8 Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Tue, 11 Jun 2019 15:26:24 +0530 Subject: [PATCH 61/68] Added imagePullSecrete for scheduler in helm chart --- installer/chart/templates/scheduler.yaml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/installer/chart/templates/scheduler.yaml b/installer/chart/templates/scheduler.yaml index 6d01220de4..9617d431ae 100644 --- a/installer/chart/templates/scheduler.yaml +++ b/installer/chart/templates/scheduler.yaml @@ -97,6 +97,10 @@ spec: app: volcano-scheduler spec: serviceAccount: {{ .Release.Name }}-scheduler + {{ if .Values.basic.image_pull_secret }} + imagePullSecrets: + - name: {{ .Values.basic.image_pull_secret }} + {{ end }} containers: - name: {{ .Release.Name }}-scheduler image: {{.Values.basic.scheduler_image_name}}:{{.Values.basic.image_tag_version}} From d87cc807c186a67aa9454f66e553bedb21c3829d Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Tue, 11 Jun 2019 23:32:51 +0530 Subject: [PATCH 62/68] Lint Fixes --- hack/.golint_failures | 10 ----- pkg/apis/batch/v1alpha1/doc.go | 1 + pkg/apis/batch/v1alpha1/job.go | 18 ++++++-- pkg/apis/batch/v1alpha1/labels.go | 14 ++++-- pkg/apis/batch/v1alpha1/register.go | 4 +- pkg/apis/bus/v1alpha1/doc.go | 1 + pkg/apis/bus/v1alpha1/register.go | 4 +- pkg/apis/bus/v1alpha1/types.go | 4 ++ pkg/apis/helpers/helpers.go | 11 ++++- pkg/cli/job/common.go | 1 + pkg/cli/job/delete.go | 2 + pkg/cli/job/list.go | 44 +++++++++++++------ pkg/cli/job/resume.go | 3 ++ pkg/cli/job/run.go | 3 ++ pkg/cli/job/suspend.go | 3 ++ pkg/cli/job/util.go | 1 + pkg/cli/job/view.go | 19 ++++++++ pkg/controllers/job/helpers/helpers.go | 7 ++- pkg/controllers/job/plugins/env/env.go | 1 + pkg/controllers/job/plugins/env/types.go | 3 ++ pkg/controllers/job/plugins/factory.go | 7 ++- .../job/plugins/interface/interface.go | 4 +- pkg/controllers/job/plugins/ssh/ssh.go | 1 + pkg/controllers/job/plugins/ssh/types.go | 16 +++++-- pkg/controllers/job/plugins/svc/svc.go | 7 +-- pkg/controllers/job/plugins/svc/types.go | 2 + 26 files changed, 146 insertions(+), 45 deletions(-) diff --git a/hack/.golint_failures b/hack/.golint_failures index 0ce0e14469..7ea2ff67b7 100644 --- a/hack/.golint_failures +++ b/hack/.golint_failures @@ -1,16 +1,6 @@ volcano.sh/volcano/pkg/admission -volcano.sh/volcano/pkg/apis/batch/v1alpha1 -volcano.sh/volcano/pkg/apis/bus/v1alpha1 -volcano.sh/volcano/pkg/apis/helpers -volcano.sh/volcano/pkg/cli/job volcano.sh/volcano/pkg/controllers/apis volcano.sh/volcano/pkg/controllers/cache volcano.sh/volcano/pkg/controllers/job -volcano.sh/volcano/pkg/controllers/job/helpers -volcano.sh/volcano/pkg/controllers/job/plugins -volcano.sh/volcano/pkg/controllers/job/plugins/env -volcano.sh/volcano/pkg/controllers/job/plugins/interface -volcano.sh/volcano/pkg/controllers/job/plugins/ssh -volcano.sh/volcano/pkg/controllers/job/plugins/svc volcano.sh/volcano/pkg/controllers/job/state volcano.sh/volcano/test/e2e diff --git a/pkg/apis/batch/v1alpha1/doc.go b/pkg/apis/batch/v1alpha1/doc.go index 1c873db349..890d2ecd95 100644 --- a/pkg/apis/batch/v1alpha1/doc.go +++ b/pkg/apis/batch/v1alpha1/doc.go @@ -15,4 +15,5 @@ limitations under the License. */ // +k8s:deepcopy-gen=package + package v1alpha1 diff --git a/pkg/apis/batch/v1alpha1/job.go b/pkg/apis/batch/v1alpha1/job.go index d636eabd2c..2ce3a28c00 100644 --- a/pkg/apis/batch/v1alpha1/job.go +++ b/pkg/apis/batch/v1alpha1/job.go @@ -23,6 +23,8 @@ import ( // +genclient // +k8s:deepcopy-gen:interfaces=k8s.io/apimachinery/pkg/runtime.Object + +// Job volcano job struct type Job struct { metav1.TypeMeta `json:",inline"` @@ -98,12 +100,17 @@ type VolumeSpec struct { VolumeClaim *v1.PersistentVolumeClaimSpec `json:"volumeClaim,omitempty" protobuf:"bytes,3,opt,name=volumeClaim"` } +// JobEvent job event type JobEvent string const ( + // CommandIssued command issued CommandIssued JobEvent = "CommandIssued" - PluginError JobEvent = "PluginError" - PVCError JobEvent = "PVCError" + // PluginError plugin error + PluginError JobEvent = "PluginError" + // PVCError pvc error + PVCError JobEvent = "PVCError" + // PodGroupError pod grp error PodGroupError JobEvent = "PodGroupError" ) @@ -111,13 +118,13 @@ const ( type Event string const ( - // AllEvent means all event + // AnyEvent means all event AnyEvent Event = "*" // PodFailedEvent is triggered if Pod was failed PodFailedEvent Event = "PodFailed" // PodEvictedEvent is triggered if Pod was deleted PodEvictedEvent Event = "PodEvicted" - // These below are several events can lead to job 'Unknown' + // JobUnknownEvent These below are several events can lead to job 'Unknown' // 1. Task Unschedulable, this is triggered when part of // pods can't be scheduled while some are already running in gang-scheduling case. JobUnknownEvent Event = "Unknown" @@ -198,6 +205,7 @@ type TaskSpec struct { Policies []LifecyclePolicy `json:"policies,omitempty" protobuf:"bytes,4,opt,name=policies"` } +// JobPhase phase of the job type JobPhase string const ( @@ -285,6 +293,8 @@ type JobStatus struct { } // +k8s:deepcopy-gen:interfaces=k8s.io/apimachinery/pkg/runtime.Object + +// JobList list of jobs type JobList struct { metav1.TypeMeta `json:",inline"` metav1.ListMeta `json:"metadata,omitempty" protobuf:"bytes,1,opt,name=metadata"` diff --git a/pkg/apis/batch/v1alpha1/labels.go b/pkg/apis/batch/v1alpha1/labels.go index a0fdfbd1f2..be1a505533 100644 --- a/pkg/apis/batch/v1alpha1/labels.go +++ b/pkg/apis/batch/v1alpha1/labels.go @@ -17,10 +17,16 @@ limitations under the License. package v1alpha1 const ( - TaskSpecKey = "volcano.sh/task-spec" - JobNameKey = "volcano.sh/job-name" + // TaskSpecKey task spec key + TaskSpecKey = "volcano.sh/task-spec" + // JobNameKey job name key + JobNameKey = "volcano.sh/job-name" + // JobNamespaceKey job namespace key JobNamespaceKey = "volcano.sh/job-namespace" + // DefaultTaskSpec dfeault task spec DefaultTaskSpec = "default" - JobVersion = "volcano.sh/job-version" - JobTypeKey = "volcano.sh/job-type" + // JobVersion job version key + JobVersion = "volcano.sh/job-version" + // JobTypeKey job type key + JobTypeKey = "volcano.sh/job-type" ) diff --git a/pkg/apis/batch/v1alpha1/register.go b/pkg/apis/batch/v1alpha1/register.go index 0c5b9e96fd..ec3f6aeba7 100644 --- a/pkg/apis/batch/v1alpha1/register.go +++ b/pkg/apis/batch/v1alpha1/register.go @@ -23,8 +23,10 @@ import ( ) var ( + // SchemeBuilder points to a list of functions added to Scheme. SchemeBuilder = runtime.NewSchemeBuilder(addKnownTypes) - AddToScheme = SchemeBuilder.AddToScheme + // AddToScheme applies all the stored functions to the scheme. + AddToScheme = SchemeBuilder.AddToScheme ) // GroupName is the group name used in this package. diff --git a/pkg/apis/bus/v1alpha1/doc.go b/pkg/apis/bus/v1alpha1/doc.go index 1c873db349..890d2ecd95 100644 --- a/pkg/apis/bus/v1alpha1/doc.go +++ b/pkg/apis/bus/v1alpha1/doc.go @@ -15,4 +15,5 @@ limitations under the License. */ // +k8s:deepcopy-gen=package + package v1alpha1 diff --git a/pkg/apis/bus/v1alpha1/register.go b/pkg/apis/bus/v1alpha1/register.go index 5c03343078..9165cb374d 100644 --- a/pkg/apis/bus/v1alpha1/register.go +++ b/pkg/apis/bus/v1alpha1/register.go @@ -23,8 +23,10 @@ import ( ) var ( + // SchemeBuilder points to a list of functions added to Scheme. SchemeBuilder = runtime.NewSchemeBuilder(addKnownTypes) - AddToScheme = SchemeBuilder.AddToScheme + // AddToScheme applies all the stored functions to the scheme. + AddToScheme = SchemeBuilder.AddToScheme ) // GroupName is the group name used in this package. diff --git a/pkg/apis/bus/v1alpha1/types.go b/pkg/apis/bus/v1alpha1/types.go index cddb9954da..b725af492f 100644 --- a/pkg/apis/bus/v1alpha1/types.go +++ b/pkg/apis/bus/v1alpha1/types.go @@ -6,6 +6,8 @@ import ( // +genclient // +k8s:deepcopy-gen:interfaces=k8s.io/apimachinery/pkg/runtime.Object + +// Command command object type Command struct { metav1.TypeMeta `json:",inline"` metav1.ObjectMeta `json:"metadata,omitempty" protobuf:"bytes,1,opt,name=metadata"` @@ -26,6 +28,8 @@ type Command struct { } // +k8s:deepcopy-gen:interfaces=k8s.io/apimachinery/pkg/runtime.Object + +// CommandList list of commands type CommandList struct { metav1.TypeMeta `json:",inline"` metav1.ListMeta `json:"metadata,omitempty" protobuf:"bytes,1,opt,name=metadata"` diff --git a/pkg/apis/helpers/helpers.go b/pkg/apis/helpers/helpers.go index 571d5edb70..7f0f74fdcd 100644 --- a/pkg/apis/helpers/helpers.go +++ b/pkg/apis/helpers/helpers.go @@ -32,9 +32,13 @@ import ( vkcorev1 "volcano.sh/volcano/pkg/apis/bus/v1alpha1" ) +// JobKind job keind var JobKind = vkbatchv1.SchemeGroupVersion.WithKind("Job") + +// CommandKind command kid var CommandKind = vkcorev1.SchemeGroupVersion.WithKind("Command") +// GetController get the controller uid func GetController(obj interface{}) types.UID { accessor, err := meta.Accessor(obj) if err != nil { @@ -49,6 +53,7 @@ func GetController(obj interface{}) types.UID { return "" } +// ControlledBy controlled by func ControlledBy(obj interface{}, gvk schema.GroupVersionKind) bool { accessor, err := meta.Accessor(obj) if err != nil { @@ -63,6 +68,7 @@ func ControlledBy(obj interface{}, gvk schema.GroupVersionKind) bool { return false } +// CreateConfigMapIfNotExist create config map if not there func CreateConfigMapIfNotExist(job *vkv1.Job, kubeClients kubernetes.Interface, data map[string]string, cmName string) error { // If ConfigMap does not exist, create one for Job. cmOld, err := kubeClients.CoreV1().ConfigMaps(job.Namespace).Get(cmName, metav1.GetOptions{}) @@ -102,15 +108,16 @@ func CreateConfigMapIfNotExist(job *vkv1.Job, kubeClients kubernetes.Interface, return nil } +// DeleteConfigmap delete the config map func DeleteConfigmap(job *vkv1.Job, kubeClients kubernetes.Interface, cmName string) error { if _, err := kubeClients.CoreV1().ConfigMaps(job.Namespace).Get(cmName, metav1.GetOptions{}); err != nil { if !apierrors.IsNotFound(err) { glog.V(3).Infof("Failed to get Configmap for Job <%s/%s>: %v", job.Namespace, job.Name, err) return err - } else { - return nil } + return nil + } if err := kubeClients.CoreV1().ConfigMaps(job.Namespace).Delete(cmName, nil); err != nil { diff --git a/pkg/cli/job/common.go b/pkg/cli/job/common.go index 040b566112..9fa6b209de 100644 --- a/pkg/cli/job/common.go +++ b/pkg/cli/job/common.go @@ -13,6 +13,7 @@ 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 job import ( diff --git a/pkg/cli/job/delete.go b/pkg/cli/job/delete.go index e6f697b500..f2ed0fa152 100644 --- a/pkg/cli/job/delete.go +++ b/pkg/cli/job/delete.go @@ -34,6 +34,7 @@ type deleteFlags struct { var deleteJobFlags = &deleteFlags{} +// InitDeleteFlags init the delete command flags func InitDeleteFlags(cmd *cobra.Command) { initFlags(cmd, &deleteJobFlags.commonFlags) @@ -41,6 +42,7 @@ func InitDeleteFlags(cmd *cobra.Command) { cmd.Flags().StringVarP(&deleteJobFlags.JobName, "name", "n", "", "the name of job") } +// DeleteJob delete the job func DeleteJob() error { config, err := buildConfig(deleteJobFlags.Master, deleteJobFlags.Kubeconfig) if err != nil { diff --git a/pkg/cli/job/list.go b/pkg/cli/job/list.go index 534fe6db3d..7baf415514 100644 --- a/pkg/cli/job/list.go +++ b/pkg/cli/job/list.go @@ -13,6 +13,7 @@ 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 job import ( @@ -36,24 +37,39 @@ type listFlags struct { } const ( - Name string = "Name" - Creation string = "Creation" - Phase string = "Phase" - Replicas string = "Replicas" - Min string = "Min" - Scheduler string = "Scheduler" - Pending string = "Pending" - Running string = "Running" - Succeeded string = "Succeeded" + // Name name + Name string = "Name" + // Creation create + Creation string = "Creation" + // Phase phase + Phase string = "Phase" + // Replicas replicas + Replicas string = "Replicas" + // Min minimum + Min string = "Min" + // Scheduler scheduler + Scheduler string = "Scheduler" + // Pending pending + Pending string = "Pending" + // Running running + Running string = "Running" + // Succeeded success + Succeeded string = "Succeeded" + // Terminating terminating Terminating string = "Terminating" - Version string = "Version" - Failed string = "Failed" - RetryCount string = "RetryCount" - JobType string = "JobType" + // Version version + Version string = "Version" + // Failed failed + Failed string = "Failed" + // RetryCount retry count + RetryCount string = "RetryCount" + // JobType job type + JobType string = "JobType" ) var listJobFlags = &listFlags{} +// InitListFlags init list command flags func InitListFlags(cmd *cobra.Command) { initFlags(cmd, &listJobFlags.commonFlags) @@ -61,6 +77,7 @@ func InitListFlags(cmd *cobra.Command) { cmd.Flags().StringVarP(&listJobFlags.SchedulerName, "scheduler", "S", "", "list job with specified scheduler name") } +// ListJobs list all jobs func ListJobs() error { config, err := buildConfig(listJobFlags.Master, listJobFlags.Kubeconfig) if err != nil { @@ -82,6 +99,7 @@ func ListJobs() error { return nil } +// PrintJobs print all jobs func PrintJobs(jobs *v1alpha1.JobList, writer io.Writer) { maxNameLen := getMaxNameLen(jobs) _, err := fmt.Fprintf(writer, fmt.Sprintf("%%-%ds%%-25s%%-12s%%-12s%%-12s%%-6s%%-10s%%-10s%%-12s%%-10s%%-12s\n", maxNameLen), diff --git a/pkg/cli/job/resume.go b/pkg/cli/job/resume.go index d73c1d6dcd..025c1ccf44 100644 --- a/pkg/cli/job/resume.go +++ b/pkg/cli/job/resume.go @@ -13,6 +13,7 @@ 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 job import ( @@ -31,6 +32,7 @@ type resumeFlags struct { var resumeJobFlags = &resumeFlags{} +// InitResumeFlags init resume flags func InitResumeFlags(cmd *cobra.Command) { initFlags(cmd, &resumeJobFlags.commonFlags) @@ -38,6 +40,7 @@ func InitResumeFlags(cmd *cobra.Command) { cmd.Flags().StringVarP(&resumeJobFlags.JobName, "name", "n", "", "the name of job") } +// ResumeJob resume the job func ResumeJob() error { config, err := buildConfig(resumeJobFlags.Master, resumeJobFlags.Kubeconfig) if err != nil { diff --git a/pkg/cli/job/run.go b/pkg/cli/job/run.go index 4fc8c28afb..16efc6889d 100644 --- a/pkg/cli/job/run.go +++ b/pkg/cli/job/run.go @@ -13,6 +13,7 @@ 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 job import ( @@ -41,6 +42,7 @@ type runFlags struct { var launchJobFlags = &runFlags{} +// InitRunFlags init the run flags func InitRunFlags(cmd *cobra.Command) { initFlags(cmd, &launchJobFlags.commonFlags) @@ -56,6 +58,7 @@ func InitRunFlags(cmd *cobra.Command) { var jobName = "job.volcano.sh" +// RunJob run the job command func RunJob() error { config, err := buildConfig(launchJobFlags.Master, launchJobFlags.Kubeconfig) if err != nil { diff --git a/pkg/cli/job/suspend.go b/pkg/cli/job/suspend.go index 504cdd2381..5cc7d5d3b4 100644 --- a/pkg/cli/job/suspend.go +++ b/pkg/cli/job/suspend.go @@ -13,6 +13,7 @@ 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 job import ( @@ -31,6 +32,7 @@ type suspendFlags struct { var suspendJobFlags = &suspendFlags{} +// InitSuspendFlags init suspend related flags func InitSuspendFlags(cmd *cobra.Command) { initFlags(cmd, &suspendJobFlags.commonFlags) @@ -38,6 +40,7 @@ func InitSuspendFlags(cmd *cobra.Command) { cmd.Flags().StringVarP(&suspendJobFlags.JobName, "name", "n", "", "the name of job") } +// SuspendJob suspends the job func SuspendJob() error { config, err := buildConfig(suspendJobFlags.Master, suspendJobFlags.Kubeconfig) if err != nil { diff --git a/pkg/cli/job/util.go b/pkg/cli/job/util.go index a3c919e609..8a5db6a313 100644 --- a/pkg/cli/job/util.go +++ b/pkg/cli/job/util.go @@ -13,6 +13,7 @@ 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 job import ( diff --git a/pkg/cli/job/view.go b/pkg/cli/job/view.go index 65c11a37d5..1316ad5866 100644 --- a/pkg/cli/job/view.go +++ b/pkg/cli/job/view.go @@ -1,3 +1,19 @@ +/* +Copyright 2019 The Volcano 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 job import ( @@ -23,6 +39,7 @@ type viewFlags struct { var viewJobFlags = &viewFlags{} +// InitViewFlags init the view command flags func InitViewFlags(cmd *cobra.Command) { initFlags(cmd, &viewJobFlags.commonFlags) @@ -30,6 +47,7 @@ func InitViewFlags(cmd *cobra.Command) { cmd.Flags().StringVarP(&viewJobFlags.JobName, "name", "n", "", "the name of job") } +// ViewJob view the job func ViewJob() error { config, err := buildConfig(viewJobFlags.Master, viewJobFlags.Kubeconfig) if err != nil { @@ -54,6 +72,7 @@ func ViewJob() error { return nil } +// PrintJob print the job func PrintJob(job *v1alpha1.Job, writer io.Writer) { replicas := int32(0) for _, ts := range job.Spec.Tasks { diff --git a/pkg/controllers/job/helpers/helpers.go b/pkg/controllers/job/helpers/helpers.go index 10fd9a7dde..2df8081905 100644 --- a/pkg/controllers/job/helpers/helpers.go +++ b/pkg/controllers/job/helpers/helpers.go @@ -25,10 +25,13 @@ import ( ) const ( - PodNameFmt = "%s-%s-%d" + // PodNameFmt pod nameformat + PodNameFmt = "%s-%s-%d" + // VolumeClaimFmt volumeclaim name format VolumeClaimFmt = "%s-volume-%s" ) +// GetTaskIndex get task Index func GetTaskIndex(pod *v1.Pod) string { num := strings.Split(pod.Name, "-") if len(num) >= 3 { @@ -38,6 +41,7 @@ func GetTaskIndex(pod *v1.Pod) string { return "" } +// MakePodName construct pod name func MakePodName(jobName string, taskName string, index int) string { return fmt.Sprintf(PodNameFmt, jobName, taskName, index) } @@ -53,6 +57,7 @@ func genRandomStr(l int) string { return string(result) } +// MakeVolumeClaimName construct volume claim name func MakeVolumeClaimName(jobName string) string { return fmt.Sprintf(VolumeClaimFmt, jobName, genRandomStr(12)) } diff --git a/pkg/controllers/job/plugins/env/env.go b/pkg/controllers/job/plugins/env/env.go index 027fe3ce6f..c247b2d3ac 100644 --- a/pkg/controllers/job/plugins/env/env.go +++ b/pkg/controllers/job/plugins/env/env.go @@ -31,6 +31,7 @@ type envPlugin struct { Clientset vkinterface.PluginClientset } +// New construct plugin func New(client vkinterface.PluginClientset, arguments []string) vkinterface.PluginInterface { envPlugin := envPlugin{pluginArguments: arguments, Clientset: client} diff --git a/pkg/controllers/job/plugins/env/types.go b/pkg/controllers/job/plugins/env/types.go index 301b95746b..84d4f2614d 100644 --- a/pkg/controllers/job/plugins/env/types.go +++ b/pkg/controllers/job/plugins/env/types.go @@ -17,9 +17,12 @@ limitations under the License. package env const ( + // ConfigMapTaskHostFmt host format in config map ConfigMapTaskHostFmt = "%s.host" + // ConfigMapMountPath mount path ConfigMapMountPath = "/etc/volcano" + // TaskVkIndex index TaskVkIndex = "VK_TASK_INDEX" ) diff --git a/pkg/controllers/job/plugins/factory.go b/pkg/controllers/job/plugins/factory.go index dc6d2b9ef7..f227a67a06 100644 --- a/pkg/controllers/job/plugins/factory.go +++ b/pkg/controllers/job/plugins/factory.go @@ -36,15 +36,18 @@ var pluginMutex sync.Mutex // Plugin management var pluginBuilders = map[string]PluginBuilder{} -type PluginBuilder func(_interface.PluginClientset, []string) _interface.PluginInterface +// PluginBuilder func type +type PluginBuilder func(pluginsinterface.PluginClientset, []string) pluginsinterface.PluginInterface -func RegisterPluginBuilder(name string, pc func(_interface.PluginClientset, []string) _interface.PluginInterface) { +// RegisterPluginBuilder register plugin builders +func RegisterPluginBuilder(name string, pc func(pluginsinterface.PluginClientset, []string) pluginsinterface.PluginInterface) { pluginMutex.Lock() defer pluginMutex.Unlock() pluginBuilders[name] = pc } +// GetPluginBuilder get plugin builder func GetPluginBuilder(name string) (PluginBuilder, bool) { pluginMutex.Lock() defer pluginMutex.Unlock() diff --git a/pkg/controllers/job/plugins/interface/interface.go b/pkg/controllers/job/plugins/interface/interface.go index 8481e70327..8cbef8eb9c 100644 --- a/pkg/controllers/job/plugins/interface/interface.go +++ b/pkg/controllers/job/plugins/interface/interface.go @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package _interface +package pluginsinterface import ( "k8s.io/api/core/v1" @@ -23,10 +23,12 @@ import ( vkv1 "volcano.sh/volcano/pkg/apis/batch/v1alpha1" ) +// PluginClientset clientset type PluginClientset struct { KubeClients kubernetes.Interface } +// PluginInterface interface type PluginInterface interface { // The unique name of Plugin. Name() string diff --git a/pkg/controllers/job/plugins/ssh/ssh.go b/pkg/controllers/job/plugins/ssh/ssh.go index dd3775c4d1..f4d6e6c1fa 100644 --- a/pkg/controllers/job/plugins/ssh/ssh.go +++ b/pkg/controllers/job/plugins/ssh/ssh.go @@ -47,6 +47,7 @@ type sshPlugin struct { noRoot bool } +// New construct plugin func New(client vkinterface.PluginClientset, arguments []string) vkinterface.PluginInterface { sshPlugin := sshPlugin{pluginArguments: arguments, Clientset: client} diff --git a/pkg/controllers/job/plugins/ssh/types.go b/pkg/controllers/job/plugins/ssh/types.go index 2fc2a508b4..50e69cdd9f 100644 --- a/pkg/controllers/job/plugins/ssh/types.go +++ b/pkg/controllers/job/plugins/ssh/types.go @@ -17,11 +17,21 @@ limitations under the License. package ssh const ( - SSHPrivateKey = "id_rsa" - SSHPublicKey = "id_rsa.pub" + // SSHPrivateKey private key + SSHPrivateKey = "id_rsa" + + // SSHPublicKey public key + SSHPublicKey = "id_rsa.pub" + + // SSHAuthorizedKeys authkey SSHAuthorizedKeys = "authorized_keys" - SSHConfig = "config" + // SSHConfig ssh conf + SSHConfig = "config" + + // SSHAbsolutePath ssh abs path SSHAbsolutePath = "/root/.ssh" + + // SSHRelativePath ssh rel path SSHRelativePath = ".ssh" ) diff --git a/pkg/controllers/job/plugins/svc/svc.go b/pkg/controllers/job/plugins/svc/svc.go index 2bbcbcebb0..07ded32926 100644 --- a/pkg/controllers/job/plugins/svc/svc.go +++ b/pkg/controllers/job/plugins/svc/svc.go @@ -40,6 +40,7 @@ type servicePlugin struct { Clientset vkinterface.PluginClientset } +// New construct plugin func New(client vkinterface.PluginClientset, arguments []string) vkinterface.PluginInterface { servicePlugin := servicePlugin{pluginArguments: arguments, Clientset: client} @@ -156,11 +157,11 @@ func (sp *servicePlugin) createServiceIfNotExist(job *vkv1.Job) error { } if _, e := sp.Clientset.KubeClients.CoreV1().Services(job.Namespace).Create(svc); e != nil { - glog.V(3).Infof("Failed to create Service for Job <%s/%s>: %v", job.Namespace, job.Name, err) + glog.V(3).Infof("Failed to create Service for Job <%s/%s>: %v", job.Namespace, job.Name, e) return e - } else { - job.Status.ControlledResources["plugin-"+sp.Name()] = sp.Name() } + job.Status.ControlledResources["plugin-"+sp.Name()] = sp.Name() + } return nil diff --git a/pkg/controllers/job/plugins/svc/types.go b/pkg/controllers/job/plugins/svc/types.go index 0f91bb52a1..83e34484dd 100644 --- a/pkg/controllers/job/plugins/svc/types.go +++ b/pkg/controllers/job/plugins/svc/types.go @@ -17,7 +17,9 @@ limitations under the License. package svc const ( + // ConfigMapTaskHostFmt host format in config map ConfigMapTaskHostFmt = "%s.host" + // ConfigMapMountPath mount path ConfigMapMountPath = "/etc/volcano" ) From 5ecbd0794866e7d43e4a9def494298f665866d4c Mon Sep 17 00:00:00 2001 From: SrinivasChilveri Date: Wed, 12 Jun 2019 11:16:59 +0530 Subject: [PATCH 63/68] updated with proper decsriptions --- pkg/apis/batch/v1alpha1/job.go | 14 +++++++------- pkg/apis/batch/v1alpha1/labels.go | 10 +++++----- pkg/apis/bus/v1alpha1/types.go | 4 ++-- pkg/apis/helpers/helpers.go | 10 +++++----- pkg/cli/job/list.go | 7 ++++--- pkg/cli/job/resume.go | 4 ++-- pkg/cli/job/run.go | 2 +- pkg/cli/job/view.go | 4 ++-- pkg/controllers/job/helpers/helpers.go | 10 +++++----- pkg/controllers/job/plugins/env/env.go | 2 +- pkg/controllers/job/plugins/env/types.go | 4 ++-- pkg/controllers/job/plugins/factory.go | 4 ++-- pkg/controllers/job/plugins/ssh/ssh.go | 2 +- pkg/controllers/job/plugins/svc/svc.go | 2 +- pkg/controllers/job/plugins/svc/types.go | 2 +- 15 files changed, 41 insertions(+), 40 deletions(-) diff --git a/pkg/apis/batch/v1alpha1/job.go b/pkg/apis/batch/v1alpha1/job.go index 2ce3a28c00..e304ad195d 100644 --- a/pkg/apis/batch/v1alpha1/job.go +++ b/pkg/apis/batch/v1alpha1/job.go @@ -24,7 +24,7 @@ import ( // +genclient // +k8s:deepcopy-gen:interfaces=k8s.io/apimachinery/pkg/runtime.Object -// Job volcano job struct +// Job defines the volcano job type Job struct { metav1.TypeMeta `json:",inline"` @@ -104,13 +104,13 @@ type VolumeSpec struct { type JobEvent string const ( - // CommandIssued command issued + // CommandIssued command issued event is generated if a command is raised by user CommandIssued JobEvent = "CommandIssued" - // PluginError plugin error + // PluginError plugin error event is generated if error happens PluginError JobEvent = "PluginError" - // PVCError pvc error + // PVCError pvc error event is generated if error happens during IO creation PVCError JobEvent = "PVCError" - // PodGroupError pod grp error + // PodGroupError pod grp error event is generated if error happens during pod grp creation PodGroupError JobEvent = "PodGroupError" ) @@ -205,7 +205,7 @@ type TaskSpec struct { Policies []LifecyclePolicy `json:"policies,omitempty" protobuf:"bytes,4,opt,name=policies"` } -// JobPhase phase of the job +// JobPhase defines the phase of the job type JobPhase string const ( @@ -294,7 +294,7 @@ type JobStatus struct { // +k8s:deepcopy-gen:interfaces=k8s.io/apimachinery/pkg/runtime.Object -// JobList list of jobs +// JobList defines the list of jobs type JobList struct { metav1.TypeMeta `json:",inline"` metav1.ListMeta `json:"metadata,omitempty" protobuf:"bytes,1,opt,name=metadata"` diff --git a/pkg/apis/batch/v1alpha1/labels.go b/pkg/apis/batch/v1alpha1/labels.go index be1a505533..78e01d8a7a 100644 --- a/pkg/apis/batch/v1alpha1/labels.go +++ b/pkg/apis/batch/v1alpha1/labels.go @@ -17,16 +17,16 @@ limitations under the License. package v1alpha1 const ( - // TaskSpecKey task spec key + // TaskSpecKey task spec key used in pod annotation TaskSpecKey = "volcano.sh/task-spec" - // JobNameKey job name key + // JobNameKey job name key used in pod annotation / labels JobNameKey = "volcano.sh/job-name" // JobNamespaceKey job namespace key JobNamespaceKey = "volcano.sh/job-namespace" - // DefaultTaskSpec dfeault task spec + // DefaultTaskSpec default task spec value DefaultTaskSpec = "default" - // JobVersion job version key + // JobVersion job version key used in pod annotation JobVersion = "volcano.sh/job-version" - // JobTypeKey job type key + // JobTypeKey job type key used in labels JobTypeKey = "volcano.sh/job-type" ) diff --git a/pkg/apis/bus/v1alpha1/types.go b/pkg/apis/bus/v1alpha1/types.go index b725af492f..d7c1543492 100644 --- a/pkg/apis/bus/v1alpha1/types.go +++ b/pkg/apis/bus/v1alpha1/types.go @@ -7,7 +7,7 @@ import ( // +genclient // +k8s:deepcopy-gen:interfaces=k8s.io/apimachinery/pkg/runtime.Object -// Command command object +// Command defines command structure type Command struct { metav1.TypeMeta `json:",inline"` metav1.ObjectMeta `json:"metadata,omitempty" protobuf:"bytes,1,opt,name=metadata"` @@ -29,7 +29,7 @@ type Command struct { // +k8s:deepcopy-gen:interfaces=k8s.io/apimachinery/pkg/runtime.Object -// CommandList list of commands +// CommandList defines list of commands type CommandList struct { metav1.TypeMeta `json:",inline"` metav1.ListMeta `json:"metadata,omitempty" protobuf:"bytes,1,opt,name=metadata"` diff --git a/pkg/apis/helpers/helpers.go b/pkg/apis/helpers/helpers.go index 7f0f74fdcd..21476f3854 100644 --- a/pkg/apis/helpers/helpers.go +++ b/pkg/apis/helpers/helpers.go @@ -32,13 +32,13 @@ import ( vkcorev1 "volcano.sh/volcano/pkg/apis/bus/v1alpha1" ) -// JobKind job keind +// JobKind creates job GroupVersionKind var JobKind = vkbatchv1.SchemeGroupVersion.WithKind("Job") -// CommandKind command kid +// CommandKind creates command GroupVersionKind var CommandKind = vkcorev1.SchemeGroupVersion.WithKind("Command") -// GetController get the controller uid +// GetController returns the controller uid func GetController(obj interface{}) types.UID { accessor, err := meta.Accessor(obj) if err != nil { @@ -68,7 +68,7 @@ func ControlledBy(obj interface{}, gvk schema.GroupVersionKind) bool { return false } -// CreateConfigMapIfNotExist create config map if not there +// CreateConfigMapIfNotExist creates config map resource if not present func CreateConfigMapIfNotExist(job *vkv1.Job, kubeClients kubernetes.Interface, data map[string]string, cmName string) error { // If ConfigMap does not exist, create one for Job. cmOld, err := kubeClients.CoreV1().ConfigMaps(job.Namespace).Get(cmName, metav1.GetOptions{}) @@ -108,7 +108,7 @@ func CreateConfigMapIfNotExist(job *vkv1.Job, kubeClients kubernetes.Interface, return nil } -// DeleteConfigmap delete the config map +// DeleteConfigmap deletes the config map resource func DeleteConfigmap(job *vkv1.Job, kubeClients kubernetes.Interface, cmName string) error { if _, err := kubeClients.CoreV1().ConfigMaps(job.Namespace).Get(cmName, metav1.GetOptions{}); err != nil { if !apierrors.IsNotFound(err) { diff --git a/pkg/cli/job/list.go b/pkg/cli/job/list.go index 7baf415514..41dff254b6 100644 --- a/pkg/cli/job/list.go +++ b/pkg/cli/job/list.go @@ -37,7 +37,8 @@ type listFlags struct { } const ( - // Name name + + // Name name etc below key words are used in job print format Name string = "Name" // Creation create Creation string = "Creation" @@ -77,7 +78,7 @@ func InitListFlags(cmd *cobra.Command) { cmd.Flags().StringVarP(&listJobFlags.SchedulerName, "scheduler", "S", "", "list job with specified scheduler name") } -// ListJobs list all jobs +// ListJobs lists all jobs details func ListJobs() error { config, err := buildConfig(listJobFlags.Master, listJobFlags.Kubeconfig) if err != nil { @@ -99,7 +100,7 @@ func ListJobs() error { return nil } -// PrintJobs print all jobs +// PrintJobs prints all jobs details func PrintJobs(jobs *v1alpha1.JobList, writer io.Writer) { maxNameLen := getMaxNameLen(jobs) _, err := fmt.Fprintf(writer, fmt.Sprintf("%%-%ds%%-25s%%-12s%%-12s%%-12s%%-6s%%-10s%%-10s%%-12s%%-10s%%-12s\n", maxNameLen), diff --git a/pkg/cli/job/resume.go b/pkg/cli/job/resume.go index 025c1ccf44..1db9fbe74e 100644 --- a/pkg/cli/job/resume.go +++ b/pkg/cli/job/resume.go @@ -32,7 +32,7 @@ type resumeFlags struct { var resumeJobFlags = &resumeFlags{} -// InitResumeFlags init resume flags +// InitResumeFlags init resume command flags func InitResumeFlags(cmd *cobra.Command) { initFlags(cmd, &resumeJobFlags.commonFlags) @@ -40,7 +40,7 @@ func InitResumeFlags(cmd *cobra.Command) { cmd.Flags().StringVarP(&resumeJobFlags.JobName, "name", "n", "", "the name of job") } -// ResumeJob resume the job +// ResumeJob resumes the job func ResumeJob() error { config, err := buildConfig(resumeJobFlags.Master, resumeJobFlags.Kubeconfig) if err != nil { diff --git a/pkg/cli/job/run.go b/pkg/cli/job/run.go index 16efc6889d..1270aa7bff 100644 --- a/pkg/cli/job/run.go +++ b/pkg/cli/job/run.go @@ -58,7 +58,7 @@ func InitRunFlags(cmd *cobra.Command) { var jobName = "job.volcano.sh" -// RunJob run the job command +// RunJob creates the job func RunJob() error { config, err := buildConfig(launchJobFlags.Master, launchJobFlags.Kubeconfig) if err != nil { diff --git a/pkg/cli/job/view.go b/pkg/cli/job/view.go index 1316ad5866..4f7af6b696 100644 --- a/pkg/cli/job/view.go +++ b/pkg/cli/job/view.go @@ -47,7 +47,7 @@ func InitViewFlags(cmd *cobra.Command) { cmd.Flags().StringVarP(&viewJobFlags.JobName, "name", "n", "", "the name of job") } -// ViewJob view the job +// ViewJob gives full details of the job func ViewJob() error { config, err := buildConfig(viewJobFlags.Master, viewJobFlags.Kubeconfig) if err != nil { @@ -72,7 +72,7 @@ func ViewJob() error { return nil } -// PrintJob print the job +// PrintJob prints the job details func PrintJob(job *v1alpha1.Job, writer io.Writer) { replicas := int32(0) for _, ts := range job.Spec.Tasks { diff --git a/pkg/controllers/job/helpers/helpers.go b/pkg/controllers/job/helpers/helpers.go index 2df8081905..430a846cc1 100644 --- a/pkg/controllers/job/helpers/helpers.go +++ b/pkg/controllers/job/helpers/helpers.go @@ -25,13 +25,13 @@ import ( ) const ( - // PodNameFmt pod nameformat + // PodNameFmt pod name format PodNameFmt = "%s-%s-%d" - // VolumeClaimFmt volumeclaim name format + // VolumeClaimFmt volume claim name format VolumeClaimFmt = "%s-volume-%s" ) -// GetTaskIndex get task Index +// GetTaskIndex returns task Index func GetTaskIndex(pod *v1.Pod) string { num := strings.Split(pod.Name, "-") if len(num) >= 3 { @@ -41,7 +41,7 @@ func GetTaskIndex(pod *v1.Pod) string { return "" } -// MakePodName construct pod name +// MakePodName creates pod name func MakePodName(jobName string, taskName string, index int) string { return fmt.Sprintf(PodNameFmt, jobName, taskName, index) } @@ -57,7 +57,7 @@ func genRandomStr(l int) string { return string(result) } -// MakeVolumeClaimName construct volume claim name +// MakeVolumeClaimName creates volume claim name func MakeVolumeClaimName(jobName string) string { return fmt.Sprintf(VolumeClaimFmt, jobName, genRandomStr(12)) } diff --git a/pkg/controllers/job/plugins/env/env.go b/pkg/controllers/job/plugins/env/env.go index c247b2d3ac..7254d43e85 100644 --- a/pkg/controllers/job/plugins/env/env.go +++ b/pkg/controllers/job/plugins/env/env.go @@ -31,7 +31,7 @@ type envPlugin struct { Clientset vkinterface.PluginClientset } -// New construct plugin +// New creates env plugin func New(client vkinterface.PluginClientset, arguments []string) vkinterface.PluginInterface { envPlugin := envPlugin{pluginArguments: arguments, Clientset: client} diff --git a/pkg/controllers/job/plugins/env/types.go b/pkg/controllers/job/plugins/env/types.go index 84d4f2614d..5f751b5535 100644 --- a/pkg/controllers/job/plugins/env/types.go +++ b/pkg/controllers/job/plugins/env/types.go @@ -17,12 +17,12 @@ limitations under the License. package env const ( - // ConfigMapTaskHostFmt host format in config map + // ConfigMapTaskHostFmt key in config map ConfigMapTaskHostFmt = "%s.host" // ConfigMapMountPath mount path ConfigMapMountPath = "/etc/volcano" - // TaskVkIndex index + // TaskVkIndex used as key in container env TaskVkIndex = "VK_TASK_INDEX" ) diff --git a/pkg/controllers/job/plugins/factory.go b/pkg/controllers/job/plugins/factory.go index f227a67a06..ed87c2489d 100644 --- a/pkg/controllers/job/plugins/factory.go +++ b/pkg/controllers/job/plugins/factory.go @@ -36,7 +36,7 @@ var pluginMutex sync.Mutex // Plugin management var pluginBuilders = map[string]PluginBuilder{} -// PluginBuilder func type +// PluginBuilder func prototype type PluginBuilder func(pluginsinterface.PluginClientset, []string) pluginsinterface.PluginInterface // RegisterPluginBuilder register plugin builders @@ -47,7 +47,7 @@ func RegisterPluginBuilder(name string, pc func(pluginsinterface.PluginClientset pluginBuilders[name] = pc } -// GetPluginBuilder get plugin builder +// GetPluginBuilder returns plugin builder for a given plugin name func GetPluginBuilder(name string) (PluginBuilder, bool) { pluginMutex.Lock() defer pluginMutex.Unlock() diff --git a/pkg/controllers/job/plugins/ssh/ssh.go b/pkg/controllers/job/plugins/ssh/ssh.go index f4d6e6c1fa..a628762779 100644 --- a/pkg/controllers/job/plugins/ssh/ssh.go +++ b/pkg/controllers/job/plugins/ssh/ssh.go @@ -47,7 +47,7 @@ type sshPlugin struct { noRoot bool } -// New construct plugin +// New creates ssh plugin func New(client vkinterface.PluginClientset, arguments []string) vkinterface.PluginInterface { sshPlugin := sshPlugin{pluginArguments: arguments, Clientset: client} diff --git a/pkg/controllers/job/plugins/svc/svc.go b/pkg/controllers/job/plugins/svc/svc.go index 07ded32926..7856e1b0dd 100644 --- a/pkg/controllers/job/plugins/svc/svc.go +++ b/pkg/controllers/job/plugins/svc/svc.go @@ -40,7 +40,7 @@ type servicePlugin struct { Clientset vkinterface.PluginClientset } -// New construct plugin +// New creates service plugin func New(client vkinterface.PluginClientset, arguments []string) vkinterface.PluginInterface { servicePlugin := servicePlugin{pluginArguments: arguments, Clientset: client} diff --git a/pkg/controllers/job/plugins/svc/types.go b/pkg/controllers/job/plugins/svc/types.go index 83e34484dd..526500468a 100644 --- a/pkg/controllers/job/plugins/svc/types.go +++ b/pkg/controllers/job/plugins/svc/types.go @@ -17,7 +17,7 @@ limitations under the License. package svc const ( - // ConfigMapTaskHostFmt host format in config map + // ConfigMapTaskHostFmt key in config map ConfigMapTaskHostFmt = "%s.host" // ConfigMapMountPath mount path From ce5600f320d84c13a0dd79a3e544b04773516770 Mon Sep 17 00:00:00 2001 From: Thandayuthapani Date: Wed, 12 Jun 2019 16:11:54 +0530 Subject: [PATCH 64/68] Fix E2E because of Event Change --- test/e2e/util.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/e2e/util.go b/test/e2e/util.go index 2680568b7e..03af6c8beb 100644 --- a/test/e2e/util.go +++ b/test/e2e/util.go @@ -423,7 +423,7 @@ func jobUnschedulable(ctx *context, job *vkv1.Job, now time.Time) error { for _, event := range events.Items { target := event.InvolvedObject if strings.HasPrefix(target.Name, pg.Name) && target.Namespace == pg.Namespace { - if event.Reason == string("Unschedulable") && event.LastTimestamp.After(now) { + if event.Reason == string("Unschedulable") || event.Reason == string("FailedScheduling") && event.LastTimestamp.After(now) { return true, nil } } From 2155b6042af3030969c2869bb8f31d50098b6f08 Mon Sep 17 00:00:00 2001 From: fangyuan Date: Wed, 12 Jun 2019 22:24:51 +0800 Subject: [PATCH 65/68] Fix some typos in documents --- README.md | 2 +- docs/design/drf - fairshare.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 677d5c2563..6417c26465 100644 --- a/README.md +++ b/README.md @@ -33,7 +33,7 @@ Some examples of the mechanisms and features that Volcano adds to Kubernetes are 2. Fair-share scheduling 3. Queue scheduling 4. Preemption and reclaims - 5. Reservartions and backfills + 5. Reservations and backfills 6. Topology-based scheduling 3. Runtime extensions, e.g: 1. Support for specialized continer runtimes like Singularity, diff --git a/docs/design/drf - fairshare.md b/docs/design/drf - fairshare.md index 4b69448f97..038b60b5b4 100644 --- a/docs/design/drf - fairshare.md +++ b/docs/design/drf - fairshare.md @@ -175,5 +175,5 @@ All these plugin would choose some victims respective, and the intersection of t | queue | namespace | requested | queue assigned | namespace assigned | | ----- | --------- | --------- | -------------- | ------------------ | | q1 w1 | ns1 w2 | | 4 cpu | | - | q2 w3 | na1 w2 | 5 cpu | 12 cpu | 3 cpu | + | q2 w3 | ns1 w2 | 5 cpu | 12 cpu | 3 cpu | | | ns2 w6 | 20 cpu | | 9 cpu | \ No newline at end of file From c644aa5d5f19dfe187b185898c03713f5782a3bb Mon Sep 17 00:00:00 2001 From: fangyuan Date: Wed, 12 Jun 2019 23:31:34 +0800 Subject: [PATCH 66/68] fix typos in codes --- pkg/cli/job/delete.go | 3 ++- pkg/cli/job/resume.go | 3 ++- pkg/cli/job/suspend.go | 2 +- pkg/cli/job/view.go | 2 +- pkg/cli/queue/get.go | 2 +- 5 files changed, 7 insertions(+), 5 deletions(-) diff --git a/pkg/cli/job/delete.go b/pkg/cli/job/delete.go index e6f697b500..05789818ec 100644 --- a/pkg/cli/job/delete.go +++ b/pkg/cli/job/delete.go @@ -18,6 +18,7 @@ package job import ( "fmt" + "github.com/spf13/cobra" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" @@ -48,7 +49,7 @@ func DeleteJob() error { } if deleteJobFlags.JobName == "" { - err := fmt.Errorf("job name is mandaorty to delete a particular job") + err := fmt.Errorf("job name is mandatory to delete a particular job") return err } diff --git a/pkg/cli/job/resume.go b/pkg/cli/job/resume.go index d73c1d6dcd..8779bd70e1 100644 --- a/pkg/cli/job/resume.go +++ b/pkg/cli/job/resume.go @@ -17,6 +17,7 @@ package job import ( "fmt" + "github.com/spf13/cobra" "volcano.sh/volcano/pkg/apis/batch/v1alpha1" @@ -44,7 +45,7 @@ func ResumeJob() error { return err } if resumeJobFlags.JobName == "" { - err := fmt.Errorf("job name is mandaorty to resume a particular job") + err := fmt.Errorf("job name is mandatory to resume a particular job") return err } diff --git a/pkg/cli/job/suspend.go b/pkg/cli/job/suspend.go index 504cdd2381..fb0ebf7fd6 100644 --- a/pkg/cli/job/suspend.go +++ b/pkg/cli/job/suspend.go @@ -45,7 +45,7 @@ func SuspendJob() error { } if suspendJobFlags.JobName == "" { - err := fmt.Errorf("job name is mandaorty to suspend a particular job") + err := fmt.Errorf("job name is mandatory to suspend a particular job") return err } diff --git a/pkg/cli/job/view.go b/pkg/cli/job/view.go index 65c11a37d5..a4d9209fc4 100644 --- a/pkg/cli/job/view.go +++ b/pkg/cli/job/view.go @@ -36,7 +36,7 @@ func ViewJob() error { return err } if viewJobFlags.JobName == "" { - err := fmt.Errorf("job name (specified by --name or -n) is mandaorty to view a particular job") + err := fmt.Errorf("job name (specified by --name or -n) is mandatory to view a particular job") return err } diff --git a/pkg/cli/queue/get.go b/pkg/cli/queue/get.go index bc5a256bcf..4b3c946e42 100644 --- a/pkg/cli/queue/get.go +++ b/pkg/cli/queue/get.go @@ -52,7 +52,7 @@ func GetQueue() error { } if getQueueFlags.Name == "" { - err := fmt.Errorf("name is mandaorty to get the particular queue details") + err := fmt.Errorf("name is mandatory to get the particular queue details") return err } From d4d4a36b899b30e41a7e55a7677f9425a077189c Mon Sep 17 00:00:00 2001 From: Rajadeepan D Ramesh Date: Mon, 17 Jun 2019 07:49:12 +0530 Subject: [PATCH 67/68] Added/Updated license for copied scripts --- hack/update-gencode.sh | 2 +- hack/verify-gofmt.sh | 14 ++++++++++++++ hack/verify-golint.sh | 14 ++++++++++++++ 3 files changed, 29 insertions(+), 1 deletion(-) diff --git a/hack/update-gencode.sh b/hack/update-gencode.sh index 0aaad3a1b3..bc727d1afe 100755 --- a/hack/update-gencode.sh +++ b/hack/update-gencode.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2019 The Volcano Authors. +# Copyright 2014 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. diff --git a/hack/verify-gofmt.sh b/hack/verify-gofmt.sh index 1c83594c81..460f0ca93f 100755 --- a/hack/verify-gofmt.sh +++ b/hack/verify-gofmt.sh @@ -1,5 +1,19 @@ #!/bin/bash +# Copyright 2014 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. + set -o errexit set -o nounset set -o pipefail diff --git a/hack/verify-golint.sh b/hack/verify-golint.sh index 188bf8cf3d..b6fc290d66 100755 --- a/hack/verify-golint.sh +++ b/hack/verify-golint.sh @@ -1,5 +1,19 @@ #!/bin/bash +# Copyright 2014 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. + set -o errexit set -o nounset set -o pipefail From 06e89cee55f845556a59d627c28722e3e3e1ea65 Mon Sep 17 00:00:00 2001 From: TommyLike Date: Mon, 17 Jun 2019 10:43:36 +0800 Subject: [PATCH 68/68] Fix gcp auth issue --- pkg/cli/queue/util.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pkg/cli/queue/util.go b/pkg/cli/queue/util.go index 5081c3fb95..f6617f287d 100644 --- a/pkg/cli/queue/util.go +++ b/pkg/cli/queue/util.go @@ -21,6 +21,8 @@ import ( "k8s.io/client-go/rest" "k8s.io/client-go/tools/clientcmd" + // Initialize client auth plugin. + _ "k8s.io/client-go/plugin/pkg/client/auth/gcp" ) func homeDir() string {