Since all CR watch actions are placed into corresponding controller, controllers/downstream.go is unnecessary. Signed-off-by: llhuii <liulinghui@huawei.com>tags/v0.3.1
| @@ -17,9 +17,12 @@ limitations under the License. | |||
| package dataset | |||
| import ( | |||
| "k8s.io/apimachinery/pkg/watch" | |||
| "k8s.io/client-go/kubernetes" | |||
| "k8s.io/client-go/tools/cache" | |||
| sednaclientset "github.com/kubeedge/sedna/pkg/client/clientset/versioned/typed/sedna/v1alpha1" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/config" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/runtime" | |||
| ) | |||
| @@ -33,9 +36,12 @@ const ( | |||
| // Controller handles all dataset objects including: syncing to edge and update from edge. | |||
| type Controller struct { | |||
| client sednaclientset.SednaV1alpha1Interface | |||
| kubeClient kubernetes.Interface | |||
| client sednaclientset.SednaV1alpha1Interface | |||
| cfg *config.ControllerConfig | |||
| sendToEdgeFunc runtime.DownstreamSendFunc | |||
| } | |||
| func (c *Controller) Run(stopCh <-chan struct{}) { | |||
| @@ -45,8 +51,24 @@ func (c *Controller) Run(stopCh <-chan struct{}) { | |||
| // New creates a dataset controller | |||
| func New(cc *runtime.ControllerContext) (runtime.FeatureControllerI, error) { | |||
| c := &Controller{ | |||
| client: cc.SednaClient.SednaV1alpha1(), | |||
| client: cc.SednaClient.SednaV1alpha1(), | |||
| kubeClient: cc.KubeClient, | |||
| } | |||
| informer := cc.SednaInformerFactory.Sedna().V1alpha1().Datasets().Informer() | |||
| informer.AddEventHandler(cache.ResourceEventHandlerFuncs{ | |||
| AddFunc: func(obj interface{}) { | |||
| c.syncToEdge(watch.Added, obj) | |||
| }, | |||
| UpdateFunc: func(old, cur interface{}) { | |||
| c.syncToEdge(watch.Added, cur) | |||
| }, | |||
| DeleteFunc: func(obj interface{}) { | |||
| c.syncToEdge(watch.Deleted, obj) | |||
| }, | |||
| }) | |||
| c.addUpstreamHandler(cc) | |||
| @@ -15,3 +15,43 @@ limitations under the License. | |||
| */ | |||
| package dataset | |||
| import ( | |||
| "fmt" | |||
| "k8s.io/apimachinery/pkg/watch" | |||
| sednav1 "github.com/kubeedge/sedna/pkg/apis/sedna/v1alpha1" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/runtime" | |||
| ) | |||
| // syncToEdge syncs the dataset resources | |||
| func (c *Controller) syncToEdge(eventType watch.EventType, obj interface{}) error { | |||
| dataset, ok := obj.(*sednav1.Dataset) | |||
| if !ok { | |||
| return nil | |||
| } | |||
| // Here only propagate to the nodes with non empty name | |||
| nodeName := dataset.Spec.NodeName | |||
| if len(nodeName) == 0 { | |||
| return fmt.Errorf("empty node name") | |||
| } | |||
| // Since t.Kind may be empty, | |||
| // we need to fix the kind here if missing. | |||
| // more details at https://github.com/kubernetes/kubernetes/issues/3030 | |||
| if len(dataset.Kind) == 0 { | |||
| dataset.Kind = KindName | |||
| } | |||
| runtime.InjectSecretAnnotations(c.kubeClient, dataset, dataset.Spec.CredentialName) | |||
| return c.sendToEdgeFunc(nodeName, eventType, dataset) | |||
| } | |||
| func (c *Controller) SetDownstreamSendFunc(f runtime.DownstreamSendFunc) error { | |||
| c.sendToEdgeFunc = f | |||
| return nil | |||
| } | |||
| @@ -1,369 +0,0 @@ | |||
| /* | |||
| Copyright 2021 The KubeEdge 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 controllers | |||
| import ( | |||
| "context" | |||
| "fmt" | |||
| "time" | |||
| metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" | |||
| "k8s.io/apimachinery/pkg/fields" | |||
| k8sruntime "k8s.io/apimachinery/pkg/runtime" | |||
| "k8s.io/apimachinery/pkg/watch" | |||
| "k8s.io/client-go/kubernetes" | |||
| "k8s.io/client-go/tools/cache" | |||
| "k8s.io/klog/v2" | |||
| sednav1 "github.com/kubeedge/sedna/pkg/apis/sedna/v1alpha1" | |||
| clientset "github.com/kubeedge/sedna/pkg/client/clientset/versioned/typed/sedna/v1alpha1" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/config" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/messagelayer" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/runtime" | |||
| ) | |||
| // DownstreamController watch kubernetes api server and send the controller resource change to edge | |||
| type DownstreamController struct { | |||
| // events from watch kubernetes api server | |||
| events chan watch.Event | |||
| cfg *config.ControllerConfig | |||
| client clientset.SednaV1alpha1Interface | |||
| kubeClient kubernetes.Interface | |||
| messageLayer messagelayer.MessageLayer | |||
| } | |||
| func (dc *DownstreamController) injectSecret(obj runtime.CommonInterface, secretName string) error { | |||
| if secretName == "" { | |||
| return nil | |||
| } | |||
| secret, err := dc.kubeClient.CoreV1().Secrets(obj.GetNamespace()).Get(context.TODO(), secretName, metav1.GetOptions{}) | |||
| if err != nil { | |||
| klog.Warningf("failed to get the secret %s: %+v", | |||
| secretName, err) | |||
| return err | |||
| } | |||
| runtime.InjectSecretObj(obj, secret) | |||
| return err | |||
| } | |||
| // syncDataset syncs the dataset resources | |||
| func (dc *DownstreamController) syncDataset(eventType watch.EventType, dataset *sednav1.Dataset) error { | |||
| // Here only propagate to the nodes with non empty name | |||
| nodeName := dataset.Spec.NodeName | |||
| if len(nodeName) == 0 { | |||
| return fmt.Errorf("empty node name") | |||
| } | |||
| dc.injectSecret(dataset, dataset.Spec.CredentialName) | |||
| return dc.messageLayer.SendResourceObject(nodeName, eventType, dataset) | |||
| } | |||
| // syncJointInferenceService syncs the joint-inference-service resources | |||
| func (dc *DownstreamController) syncJointInferenceService(eventType watch.EventType, joint *sednav1.JointInferenceService) error { | |||
| // Here only propagate to the nodes with non empty name | |||
| // FIXME: only the case that Spec.NodeName specified is support | |||
| nodeName := joint.Spec.EdgeWorker.Template.Spec.NodeName | |||
| if len(nodeName) == 0 { | |||
| return fmt.Errorf("empty node name") | |||
| } | |||
| return dc.messageLayer.SendResourceObject(nodeName, eventType, joint) | |||
| } | |||
| // syncFederatedLearningJob syncs the federated resources | |||
| func (dc *DownstreamController) syncFederatedLearningJob(eventType watch.EventType, job *sednav1.FederatedLearningJob) error { | |||
| // broadcast to all nodes specified in spec | |||
| nodeset := make(map[string]bool) | |||
| for _, trainingWorker := range job.Spec.TrainingWorkers { | |||
| // Here only propagate to the nodes with non empty name | |||
| if len(trainingWorker.Template.Spec.NodeName) > 0 { | |||
| nodeset[trainingWorker.Template.Spec.NodeName] = true | |||
| } | |||
| } | |||
| for nodeName := range nodeset { | |||
| dc.messageLayer.SendResourceObject(nodeName, eventType, job) | |||
| } | |||
| return nil | |||
| } | |||
| // syncModelWithName will sync the model to the specified node. | |||
| // Now called when creating the incrementaljob. | |||
| func (dc *DownstreamController) syncModelWithName(nodeName, modelName, namespace string) error { | |||
| model, err := dc.client.Models(namespace).Get(context.TODO(), modelName, metav1.GetOptions{}) | |||
| if err != nil { | |||
| // TODO: maybe use err.ErrStatus.Code == 404 | |||
| return fmt.Errorf("model(%s/%s) not found", namespace, modelName) | |||
| } | |||
| // Since model.Kind may be empty, | |||
| // we need to fix the kind here if missing. | |||
| // more details at https://github.com/kubernetes/kubernetes/issues/3030 | |||
| if len(model.Kind) == 0 { | |||
| model.Kind = "Model" | |||
| } | |||
| dc.injectSecret(model, model.Spec.CredentialName) | |||
| dc.messageLayer.SendResourceObject(nodeName, watch.Added, model) | |||
| return nil | |||
| } | |||
| // syncIncrementalJob syncs the incremental learning jobs | |||
| func (dc *DownstreamController) syncIncrementalJob(eventType watch.EventType, job *sednav1.IncrementalLearningJob) error { | |||
| jobConditions := job.Status.Conditions | |||
| if len(jobConditions) == 0 { | |||
| return nil | |||
| } | |||
| dataName := job.Spec.Dataset.Name | |||
| ds, err := dc.client.Datasets(job.Namespace).Get(context.TODO(), dataName, metav1.GetOptions{}) | |||
| if err != nil { | |||
| return fmt.Errorf("dataset(%s/%s) not found", job.Namespace, dataName) | |||
| } | |||
| // LC has dataset object on this node that may call dataset node | |||
| dsNodeName := ds.Spec.NodeName | |||
| var trainNodeName string | |||
| var evalNodeName string | |||
| ann := job.GetAnnotations() | |||
| if ann != nil { | |||
| trainNodeName = ann[runtime.AnnotationsKeyPrefix+string(sednav1.ILJobTrain)] | |||
| evalNodeName = ann[runtime.AnnotationsKeyPrefix+string(sednav1.ILJobEval)] | |||
| } | |||
| if eventType == watch.Deleted { | |||
| // delete jobs from all LCs | |||
| for _, v := range []string{dsNodeName, trainNodeName, evalNodeName} { | |||
| if v != "" { | |||
| dc.messageLayer.SendResourceObject(v, eventType, job) | |||
| } | |||
| } | |||
| return nil | |||
| } | |||
| latestCondition := jobConditions[len(jobConditions)-1] | |||
| currentType := latestCondition.Type | |||
| jobStage := latestCondition.Stage | |||
| syncModelWithName := func(modelName string) { | |||
| if err := dc.syncModelWithName(dsNodeName, modelName, job.Namespace); err != nil { | |||
| klog.Warningf("Error to sync model %s when sync incremental learning job %s to node %s: %v", | |||
| modelName, job.Name, dsNodeName, err) | |||
| } | |||
| } | |||
| syncJobWithNodeName := func(nodeName string) { | |||
| if err := dc.messageLayer.SendResourceObject(nodeName, eventType, job); err != nil { | |||
| klog.Warningf("Error to sync incremental learning job %s to node %s in stage %s: %v", | |||
| job.Name, nodeName, jobStage, err) | |||
| } | |||
| } | |||
| dc.injectSecret(job, job.Spec.CredentialName) | |||
| doJobStageEvent := func(modelName string, nodeName string) { | |||
| if currentType == sednav1.ILJobStageCondWaiting { | |||
| syncJobWithNodeName(dsNodeName) | |||
| syncModelWithName(modelName) | |||
| } else if currentType == sednav1.ILJobStageCondRunning { | |||
| if nodeName != "" { | |||
| syncJobWithNodeName(nodeName) | |||
| } | |||
| } else if currentType == sednav1.ILJobStageCondCompleted || currentType == sednav1.ILJobStageCondFailed { | |||
| if nodeName != dsNodeName { | |||
| // delete LC's job from nodeName that's different from dataset node when worker's status is completed or failed. | |||
| dc.messageLayer.SendResourceObject(nodeName, watch.Deleted, job) | |||
| } | |||
| } | |||
| } | |||
| switch jobStage { | |||
| case sednav1.ILJobTrain: | |||
| doJobStageEvent(job.Spec.InitialModel.Name, trainNodeName) | |||
| case sednav1.ILJobEval: | |||
| doJobStageEvent(job.Spec.DeploySpec.Model.Name, evalNodeName) | |||
| } | |||
| return nil | |||
| } | |||
| // syncLifelongLearningJob syncs the lifelonglearning jobs | |||
| func (dc *DownstreamController) syncLifelongLearningJob(eventType watch.EventType, job *sednav1.LifelongLearningJob) error { | |||
| // Here only propagate to the nodes with non empty name | |||
| // FIXME(llhuii): only the case that all workers having the same nodeName are support, | |||
| // will support Spec.NodeSelector and differenect nodeName. | |||
| nodeName := job.Spec.TrainSpec.Template.Spec.NodeName | |||
| if len(nodeName) == 0 { | |||
| return fmt.Errorf("empty node name") | |||
| } | |||
| dc.injectSecret(job, job.Spec.CredentialName) | |||
| dc.messageLayer.SendResourceObject(nodeName, eventType, job) | |||
| return nil | |||
| } | |||
| // sync defines the entrypoint of syncing all resources | |||
| func (dc *DownstreamController) sync(stopCh <-chan struct{}) { | |||
| for { | |||
| select { | |||
| case <-stopCh: | |||
| klog.Info("Stop controller downstream loop") | |||
| return | |||
| case e := <-dc.events: | |||
| var err error | |||
| var kind, namespace, name string | |||
| switch t := e.Object.(type) { | |||
| case (*sednav1.Dataset): | |||
| // Since t.Kind may be empty, | |||
| // we need to fix the kind here if missing. | |||
| // more details at https://github.com/kubernetes/kubernetes/issues/3030 | |||
| if len(t.Kind) == 0 { | |||
| t.Kind = "Dataset" | |||
| } | |||
| kind = t.Kind | |||
| namespace = t.Namespace | |||
| name = t.Name | |||
| err = dc.syncDataset(e.Type, t) | |||
| case (*sednav1.JointInferenceService): | |||
| // TODO: find a good way to avoid these duplicate codes | |||
| if len(t.Kind) == 0 { | |||
| t.Kind = "JointInferenceService" | |||
| } | |||
| kind = t.Kind | |||
| namespace = t.Namespace | |||
| name = t.Name | |||
| err = dc.syncJointInferenceService(e.Type, t) | |||
| case (*sednav1.FederatedLearningJob): | |||
| if len(t.Kind) == 0 { | |||
| t.Kind = "FederatedLearningJob" | |||
| } | |||
| kind = t.Kind | |||
| namespace = t.Namespace | |||
| name = t.Name | |||
| err = dc.syncFederatedLearningJob(e.Type, t) | |||
| case (*sednav1.IncrementalLearningJob): | |||
| if len(t.Kind) == 0 { | |||
| t.Kind = "IncrementalLearningJob" | |||
| } | |||
| kind = t.Kind | |||
| namespace = t.Namespace | |||
| name = t.Name | |||
| err = dc.syncIncrementalJob(e.Type, t) | |||
| case (*sednav1.LifelongLearningJob): | |||
| if len(t.Kind) == 0 { | |||
| t.Kind = "LifelongLearningJob" | |||
| } | |||
| kind = t.Kind | |||
| namespace = t.Namespace | |||
| name = t.Name | |||
| err = dc.syncLifelongLearningJob(e.Type, t) | |||
| default: | |||
| klog.Warningf("object type: %T unsupported", e) | |||
| continue | |||
| } | |||
| if err != nil { | |||
| klog.Warningf("Error to sync %s(%s/%s), err: %+v", kind, namespace, name, err) | |||
| } else { | |||
| klog.V(2).Infof("synced %s(%s/%s)", kind, namespace, name) | |||
| } | |||
| } | |||
| } | |||
| } | |||
| // watch function watches the crd resources which should by synced to nodes | |||
| func (dc *DownstreamController) watch(stopCh <-chan struct{}) { | |||
| rh := cache.ResourceEventHandlerFuncs{ | |||
| AddFunc: func(obj interface{}) { | |||
| eventObj := obj.(k8sruntime.Object) | |||
| dc.events <- watch.Event{Type: watch.Added, Object: eventObj} | |||
| }, | |||
| UpdateFunc: func(old, cur interface{}) { | |||
| // Since we don't support the spec update operation currently, | |||
| // so only status updates arrive here and NO propagation to edge. | |||
| // Update: | |||
| // We sync it to edge when using self-built websocket, and | |||
| // this sync isn't needed when we switch out self-built websocket. | |||
| dc.events <- watch.Event{Type: watch.Added, Object: cur.(k8sruntime.Object)} | |||
| }, | |||
| DeleteFunc: func(obj interface{}) { | |||
| eventObj := obj.(k8sruntime.Object) | |||
| dc.events <- watch.Event{Type: watch.Deleted, Object: eventObj} | |||
| }, | |||
| } | |||
| client := dc.client.RESTClient() | |||
| // make this option configurable | |||
| resyncPeriod := time.Second * 60 | |||
| namespace := dc.cfg.Namespace | |||
| // TODO: use the informer | |||
| for resourceName, object := range map[string]k8sruntime.Object{ | |||
| "datasets": &sednav1.Dataset{}, | |||
| "jointinferenceservices": &sednav1.JointInferenceService{}, | |||
| "federatedlearningjobs": &sednav1.FederatedLearningJob{}, | |||
| "incrementallearningjobs": &sednav1.IncrementalLearningJob{}, | |||
| "lifelonglearningjobs": &sednav1.LifelongLearningJob{}, | |||
| } { | |||
| lw := cache.NewListWatchFromClient(client, resourceName, namespace, fields.Everything()) | |||
| si := cache.NewSharedInformer(lw, object, resyncPeriod) | |||
| si.AddEventHandler(rh) | |||
| go si.Run(stopCh) | |||
| } | |||
| } | |||
| // Start starts the controller | |||
| func (dc *DownstreamController) Run(stopCh <-chan struct{}) { | |||
| // watch is an asynchronous call | |||
| dc.watch(stopCh) | |||
| // sync is a synchronous call | |||
| dc.sync(stopCh) | |||
| } | |||
| // NewDownstreamController creates a controller DownstreamController from config | |||
| func NewDownstreamController(cc *runtime.ControllerContext) (runtime.FeatureControllerI, error) { | |||
| // TODO: make bufferSize configurable | |||
| bufferSize := 10 | |||
| events := make(chan watch.Event, bufferSize) | |||
| dc := &DownstreamController{ | |||
| cfg: cc.Config, | |||
| events: events, | |||
| client: cc.SednaClient.SednaV1alpha1(), | |||
| kubeClient: cc.KubeClient, | |||
| messageLayer: messagelayer.NewContextMessageLayer(), | |||
| } | |||
| return dc, nil | |||
| } | |||
| @@ -0,0 +1,51 @@ | |||
| /* | |||
| Copyright 2021 The KubeEdge 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 federatedlearning | |||
| import ( | |||
| "k8s.io/apimachinery/pkg/watch" | |||
| sednav1 "github.com/kubeedge/sedna/pkg/apis/sedna/v1alpha1" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/runtime" | |||
| ) | |||
| func (c *Controller) syncToEdge(eventType watch.EventType, obj interface{}) error { | |||
| job, ok := obj.(*sednav1.FederatedLearningJob) | |||
| if !ok { | |||
| return nil | |||
| } | |||
| // broadcast to all nodes specified in spec | |||
| nodeset := make(map[string]bool) | |||
| for _, trainingWorker := range job.Spec.TrainingWorkers { | |||
| // Here only propagate to the nodes with non empty name | |||
| if len(trainingWorker.Template.Spec.NodeName) > 0 { | |||
| nodeset[trainingWorker.Template.Spec.NodeName] = true | |||
| } | |||
| } | |||
| for nodeName := range nodeset { | |||
| c.sendToEdgeFunc(nodeName, eventType, job) | |||
| } | |||
| return nil | |||
| } | |||
| func (c *Controller) SetDownstreamSendFunc(f runtime.DownstreamSendFunc) error { | |||
| c.sendToEdgeFunc = f | |||
| return nil | |||
| } | |||
| @@ -28,6 +28,7 @@ import ( | |||
| utilrand "k8s.io/apimachinery/pkg/util/rand" | |||
| utilruntime "k8s.io/apimachinery/pkg/util/runtime" | |||
| "k8s.io/apimachinery/pkg/util/wait" | |||
| "k8s.io/apimachinery/pkg/watch" | |||
| "k8s.io/client-go/kubernetes" | |||
| "k8s.io/client-go/kubernetes/scheme" | |||
| v1core "k8s.io/client-go/kubernetes/typed/core/v1" | |||
| @@ -85,6 +86,8 @@ type Controller struct { | |||
| recorder record.EventRecorder | |||
| cfg *config.ControllerConfig | |||
| sendToEdgeFunc runtime.DownstreamSendFunc | |||
| } | |||
| // Run starts the main goroutine responsible for watching and syncing jobs. | |||
| @@ -550,12 +553,15 @@ func New(cc *runtime.ControllerContext) (runtime.FeatureControllerI, error) { | |||
| jobInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ | |||
| AddFunc: func(obj interface{}) { | |||
| fc.enqueueController(obj, true) | |||
| fc.syncToEdge(watch.Added, obj) | |||
| }, | |||
| UpdateFunc: func(old, cur interface{}) { | |||
| fc.enqueueController(cur, true) | |||
| fc.syncToEdge(watch.Added, cur) | |||
| }, | |||
| DeleteFunc: func(obj interface{}) { | |||
| fc.enqueueController(obj, true) | |||
| fc.syncToEdge(watch.Deleted, obj) | |||
| }, | |||
| }) | |||
| fc.jobLister = jobInformer.Lister() | |||
| @@ -0,0 +1,142 @@ | |||
| /* | |||
| Copyright 2021 The KubeEdge 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 incrementallearning | |||
| import ( | |||
| "context" | |||
| "fmt" | |||
| metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" | |||
| "k8s.io/apimachinery/pkg/watch" | |||
| "k8s.io/klog/v2" | |||
| sednav1 "github.com/kubeedge/sedna/pkg/apis/sedna/v1alpha1" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/runtime" | |||
| ) | |||
| // syncModelWithName will sync the model to the specified node. | |||
| // Now called when creating the incrementaljob. | |||
| func (c *Controller) syncModelWithName(nodeName, modelName, namespace string) error { | |||
| model, err := c.client.Models(namespace).Get(context.TODO(), modelName, metav1.GetOptions{}) | |||
| if err != nil { | |||
| // TODO: maybe use err.ErrStatus.Code == 404 | |||
| return fmt.Errorf("model(%s/%s) not found", namespace, modelName) | |||
| } | |||
| // Since model.Kind may be empty, | |||
| // we need to fix the kind here if missing. | |||
| // more details at https://github.com/kubernetes/kubernetes/issues/3030 | |||
| if len(model.Kind) == 0 { | |||
| model.Kind = "Model" | |||
| } | |||
| runtime.InjectSecretAnnotations(c.kubeClient, model, model.Spec.CredentialName) | |||
| c.sendToEdgeFunc(nodeName, watch.Added, model) | |||
| return nil | |||
| } | |||
| func (c *Controller) syncToEdge(eventType watch.EventType, obj interface{}) error { | |||
| job, ok := obj.(*sednav1.IncrementalLearningJob) | |||
| if !ok { | |||
| return nil | |||
| } | |||
| job.Kind = KindName | |||
| jobConditions := job.Status.Conditions | |||
| if len(jobConditions) == 0 { | |||
| return nil | |||
| } | |||
| dataName := job.Spec.Dataset.Name | |||
| ds, err := c.client.Datasets(job.Namespace).Get(context.TODO(), dataName, metav1.GetOptions{}) | |||
| if err != nil { | |||
| return fmt.Errorf("dataset(%s/%s) not found", job.Namespace, dataName) | |||
| } | |||
| // LC has dataset object on this node that may call dataset node | |||
| dsNodeName := ds.Spec.NodeName | |||
| var trainNodeName string | |||
| var evalNodeName string | |||
| ann := job.GetAnnotations() | |||
| if ann != nil { | |||
| trainNodeName = ann[runtime.AnnotationsKeyPrefix+string(sednav1.ILJobTrain)] | |||
| evalNodeName = ann[runtime.AnnotationsKeyPrefix+string(sednav1.ILJobEval)] | |||
| } | |||
| if eventType == watch.Deleted { | |||
| // delete jobs from all LCs | |||
| for _, v := range []string{dsNodeName, trainNodeName, evalNodeName} { | |||
| if v != "" { | |||
| c.sendToEdgeFunc(v, eventType, job) | |||
| } | |||
| } | |||
| return nil | |||
| } | |||
| latestCondition := jobConditions[len(jobConditions)-1] | |||
| currentType := latestCondition.Type | |||
| jobStage := latestCondition.Stage | |||
| syncModelWithName := func(modelName string) { | |||
| if err := c.syncModelWithName(dsNodeName, modelName, job.Namespace); err != nil { | |||
| klog.Warningf("Error to sync model %s when sync incremental learning job %s to node %s: %v", | |||
| modelName, job.Name, dsNodeName, err) | |||
| } | |||
| } | |||
| syncJobWithNodeName := func(nodeName string) { | |||
| if err := c.sendToEdgeFunc(nodeName, eventType, job); err != nil { | |||
| klog.Warningf("Error to sync incremental learning job %s to node %s in stage %s: %v", | |||
| job.Name, nodeName, jobStage, err) | |||
| } | |||
| } | |||
| runtime.InjectSecretAnnotations(c.kubeClient, job, job.Spec.CredentialName) | |||
| doJobStageEvent := func(modelName string, nodeName string) { | |||
| if currentType == sednav1.ILJobStageCondWaiting { | |||
| syncJobWithNodeName(dsNodeName) | |||
| syncModelWithName(modelName) | |||
| } else if currentType == sednav1.ILJobStageCondRunning { | |||
| if nodeName != "" { | |||
| syncJobWithNodeName(nodeName) | |||
| } | |||
| } else if currentType == sednav1.ILJobStageCondCompleted || currentType == sednav1.ILJobStageCondFailed { | |||
| if nodeName != dsNodeName { | |||
| // delete LC's job from nodeName that's different from dataset node when worker's status is completed or failed. | |||
| c.sendToEdgeFunc(nodeName, watch.Deleted, job) | |||
| } | |||
| } | |||
| } | |||
| switch jobStage { | |||
| case sednav1.ILJobTrain: | |||
| doJobStageEvent(job.Spec.InitialModel.Name, trainNodeName) | |||
| case sednav1.ILJobEval: | |||
| doJobStageEvent(job.Spec.DeploySpec.Model.Name, evalNodeName) | |||
| } | |||
| return nil | |||
| } | |||
| func (c *Controller) SetDownstreamSendFunc(f runtime.DownstreamSendFunc) error { | |||
| c.sendToEdgeFunc = f | |||
| return nil | |||
| } | |||
| @@ -30,6 +30,7 @@ import ( | |||
| utilrand "k8s.io/apimachinery/pkg/util/rand" | |||
| utilruntime "k8s.io/apimachinery/pkg/util/runtime" | |||
| "k8s.io/apimachinery/pkg/util/wait" | |||
| "k8s.io/apimachinery/pkg/watch" | |||
| "k8s.io/client-go/kubernetes" | |||
| "k8s.io/client-go/kubernetes/scheme" | |||
| v1core "k8s.io/client-go/kubernetes/typed/core/v1" | |||
| @@ -83,6 +84,8 @@ type Controller struct { | |||
| recorder record.EventRecorder | |||
| cfg *config.ControllerConfig | |||
| sendToEdgeFunc runtime.DownstreamSendFunc | |||
| } | |||
| // Run starts the main goroutine responsible for watching and syncing jobs. | |||
| @@ -834,12 +837,15 @@ func New(cc *runtime.ControllerContext) (runtime.FeatureControllerI, error) { | |||
| jobInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ | |||
| AddFunc: func(obj interface{}) { | |||
| jc.enqueueController(obj, true) | |||
| jc.syncToEdge(watch.Added, obj) | |||
| }, | |||
| UpdateFunc: func(old, cur interface{}) { | |||
| jc.enqueueController(cur, true) | |||
| jc.syncToEdge(watch.Added, cur) | |||
| }, | |||
| DeleteFunc: func(obj interface{}) { | |||
| jc.enqueueController(obj, true) | |||
| jc.syncToEdge(watch.Deleted, obj) | |||
| }, | |||
| }) | |||
| jc.jobLister = jobInformer.Lister() | |||
| @@ -0,0 +1,51 @@ | |||
| /* | |||
| Copyright 2021 The KubeEdge 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 jointinference | |||
| import ( | |||
| "fmt" | |||
| "k8s.io/apimachinery/pkg/watch" | |||
| sednav1 "github.com/kubeedge/sedna/pkg/apis/sedna/v1alpha1" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/runtime" | |||
| ) | |||
| func (c *Controller) syncToEdge(eventType watch.EventType, obj interface{}) error { | |||
| joint, ok := obj.(*sednav1.JointInferenceService) | |||
| if !ok { | |||
| return nil | |||
| } | |||
| // Here only propagate to the nodes with non empty name | |||
| // FIXME: only the case that Spec.NodeName specified is support | |||
| nodeName := joint.Spec.EdgeWorker.Template.Spec.NodeName | |||
| if len(nodeName) == 0 { | |||
| return fmt.Errorf("empty node name") | |||
| } | |||
| if len(joint.Kind) == 0 { | |||
| joint.Kind = KindName | |||
| } | |||
| return c.sendToEdgeFunc(nodeName, eventType, joint) | |||
| } | |||
| func (c *Controller) SetDownstreamSendFunc(f runtime.DownstreamSendFunc) error { | |||
| c.sendToEdgeFunc = f | |||
| return nil | |||
| } | |||
| @@ -29,6 +29,7 @@ import ( | |||
| utilrand "k8s.io/apimachinery/pkg/util/rand" | |||
| utilruntime "k8s.io/apimachinery/pkg/util/runtime" | |||
| "k8s.io/apimachinery/pkg/util/wait" | |||
| "k8s.io/apimachinery/pkg/watch" | |||
| "k8s.io/client-go/kubernetes" | |||
| "k8s.io/client-go/kubernetes/scheme" | |||
| v1core "k8s.io/client-go/kubernetes/typed/core/v1" | |||
| @@ -85,6 +86,8 @@ type Controller struct { | |||
| recorder record.EventRecorder | |||
| cfg *config.ControllerConfig | |||
| sendToEdgeFunc runtime.DownstreamSendFunc | |||
| } | |||
| // Run starts the main goroutine responsible for watching and syncing services. | |||
| @@ -557,14 +560,17 @@ func New(cc *runtime.ControllerContext) (runtime.FeatureControllerI, error) { | |||
| serviceInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ | |||
| AddFunc: func(obj interface{}) { | |||
| jc.enqueueController(obj, true) | |||
| jc.syncToEdge(watch.Added, obj) | |||
| }, | |||
| UpdateFunc: func(old, cur interface{}) { | |||
| jc.enqueueController(cur, true) | |||
| jc.syncToEdge(watch.Added, cur) | |||
| }, | |||
| DeleteFunc: func(obj interface{}) { | |||
| jc.enqueueController(obj, true) | |||
| jc.syncToEdge(watch.Deleted, obj) | |||
| }, | |||
| }) | |||
| @@ -0,0 +1,49 @@ | |||
| /* | |||
| Copyright 2021 The KubeEdge 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 lifelonglearning | |||
| import ( | |||
| "fmt" | |||
| "k8s.io/apimachinery/pkg/watch" | |||
| sednav1 "github.com/kubeedge/sedna/pkg/apis/sedna/v1alpha1" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/runtime" | |||
| ) | |||
| func (c *Controller) syncToEdge(eventType watch.EventType, obj interface{}) error { | |||
| job, ok := obj.(*sednav1.LifelongLearningJob) | |||
| if !ok { | |||
| return nil | |||
| } | |||
| // Here only propagate to the nodes with non empty name | |||
| // FIXME(llhuii): only the case that all workers having the same nodeName are support, | |||
| // will support Spec.NodeSelector and differenect nodeName. | |||
| nodeName := job.Spec.TrainSpec.Template.Spec.NodeName | |||
| if len(nodeName) == 0 { | |||
| return fmt.Errorf("empty node name") | |||
| } | |||
| runtime.InjectSecretAnnotations(c.kubeClient, job, job.Spec.CredentialName) | |||
| return c.sendToEdgeFunc(nodeName, eventType, job) | |||
| } | |||
| func (c *Controller) SetDownstreamSendFunc(f runtime.DownstreamSendFunc) error { | |||
| c.sendToEdgeFunc = f | |||
| return nil | |||
| } | |||
| @@ -28,6 +28,7 @@ import ( | |||
| utilrand "k8s.io/apimachinery/pkg/util/rand" | |||
| utilruntime "k8s.io/apimachinery/pkg/util/runtime" | |||
| "k8s.io/apimachinery/pkg/util/wait" | |||
| "k8s.io/apimachinery/pkg/watch" | |||
| "k8s.io/client-go/kubernetes" | |||
| "k8s.io/client-go/kubernetes/scheme" | |||
| v1core "k8s.io/client-go/kubernetes/typed/core/v1" | |||
| @@ -80,6 +81,8 @@ type Controller struct { | |||
| recorder record.EventRecorder | |||
| cfg *config.ControllerConfig | |||
| sendToEdgeFunc runtime.DownstreamSendFunc | |||
| } | |||
| // Run starts the main goroutine responsible for watching and syncing jobs. | |||
| @@ -726,12 +729,15 @@ func New(cc *runtime.ControllerContext) (runtime.FeatureControllerI, error) { | |||
| jobInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ | |||
| AddFunc: func(obj interface{}) { | |||
| jc.enqueueController(obj, true) | |||
| jc.syncToEdge(watch.Added, obj) | |||
| }, | |||
| UpdateFunc: func(old, cur interface{}) { | |||
| jc.enqueueController(cur, true) | |||
| jc.syncToEdge(watch.Added, cur) | |||
| }, | |||
| DeleteFunc: func(obj interface{}) { | |||
| jc.enqueueController(obj, true) | |||
| jc.syncToEdge(watch.Deleted, obj) | |||
| }, | |||
| }) | |||
| jc.jobLister = jobInformer.Lister() | |||
| @@ -28,6 +28,7 @@ import ( | |||
| clientset "github.com/kubeedge/sedna/pkg/client/clientset/versioned" | |||
| sednainformers "github.com/kubeedge/sedna/pkg/client/informers/externalversions" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/config" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/messagelayer" | |||
| websocket "github.com/kubeedge/sedna/pkg/globalmanager/messagelayer/ws" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/runtime" | |||
| "github.com/kubeedge/sedna/pkg/globalmanager/utils" | |||
| @@ -93,22 +94,24 @@ func (m *Manager) Start() error { | |||
| } | |||
| uc, _ := NewUpstreamController(context) | |||
| dc, _ := NewDownstreamController(context) | |||
| context.UpstreamController = uc | |||
| downstreamSendFunc := messagelayer.NewContextMessageLayer().SendResourceObject | |||
| stopCh := make(chan struct{}) | |||
| kubeInformerFactory.Start(stopCh) | |||
| sednaInformerFactory.Start(stopCh) | |||
| go uc.Run(stopCh) | |||
| go dc.Run(stopCh) | |||
| for name, factory := range NewRegistry() { | |||
| f, err := factory(context) | |||
| if err != nil { | |||
| return fmt.Errorf("failed to initialize controller %s: %v", name, err) | |||
| } | |||
| f.SetDownstreamSendFunc(downstreamSendFunc) | |||
| go f.Run(stopCh) | |||
| klog.Infof("started controller %s", name) | |||
| } | |||
| @@ -17,10 +17,13 @@ limitations under the License. | |||
| package runtime | |||
| import ( | |||
| "context" | |||
| "encoding/json" | |||
| "fmt" | |||
| v1 "k8s.io/api/core/v1" | |||
| metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" | |||
| "k8s.io/client-go/kubernetes" | |||
| ) | |||
| const ( | |||
| @@ -106,10 +109,18 @@ func MergeSecretEnvs(nowE, newE []v1.EnvVar, overwrite bool) []v1.EnvVar { | |||
| return nowE | |||
| } | |||
| func InjectSecretObj(obj CommonInterface, secret *v1.Secret) { | |||
| if secret == nil { | |||
| func InjectSecretAnnotations(client kubernetes.Interface, obj CommonInterface, secretName string) (err error) { | |||
| if len(secretName) == 0 { | |||
| return | |||
| } | |||
| secret, err := client.CoreV1().Secrets(obj.GetNamespace()).Get(context.TODO(), secretName, metav1.GetOptions{}) | |||
| if err != nil { | |||
| return | |||
| } | |||
| return injectSecretObj(obj, secret) | |||
| } | |||
| func injectSecretObj(obj CommonInterface, secret *v1.Secret) (err error) { | |||
| secretData := secret.GetAnnotations() | |||
| @@ -127,4 +138,5 @@ func InjectSecretObj(obj CommonInterface, secret *v1.Secret) { | |||
| ann[SecretAnnotationKey] = string(b) | |||
| obj.SetAnnotations(ann) | |||
| return nil | |||
| } | |||
| @@ -21,6 +21,7 @@ import ( | |||
| metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" | |||
| k8sruntime "k8s.io/apimachinery/pkg/runtime" | |||
| "k8s.io/apimachinery/pkg/runtime/schema" | |||
| "k8s.io/apimachinery/pkg/watch" | |||
| kubeinformers "k8s.io/client-go/informers" | |||
| "k8s.io/client-go/kubernetes" | |||
| @@ -35,9 +36,15 @@ type CommonInterface interface { | |||
| k8sruntime.Object | |||
| } | |||
| // BaseControllerI defines the interface of an controller | |||
| type BaseControllerI interface { | |||
| Run(stopCh <-chan struct{}) | |||
| } | |||
| // FeatureControllerI defines the interface of an AI Feature controller | |||
| type FeatureControllerI interface { | |||
| Run(stopCh <-chan struct{}) | |||
| BaseControllerI | |||
| SetDownstreamSendFunc(f DownstreamSendFunc) error | |||
| } | |||
| type Model struct { | |||
| @@ -67,10 +74,12 @@ func (m *Model) GetURL() string { | |||
| type UpstreamUpdateHandler func(namespace, name, operation string, content []byte) error | |||
| type UpstreamControllerI interface { | |||
| FeatureControllerI | |||
| BaseControllerI | |||
| Add(kind string, updateHandler UpstreamUpdateHandler) error | |||
| } | |||
| type DownstreamSendFunc = func(nodeName string, eventType watch.EventType, obj interface{}) error | |||
| type ControllerContext struct { | |||
| Config *config.ControllerConfig | |||
| UpstreamController UpstreamControllerI | |||