From 9c3a6a06c8b2faabde17db215bb94fafe1278d1c Mon Sep 17 00:00:00 2001 From: Craig Condit Date: Mon, 20 Nov 2023 12:17:57 -0600 Subject: [PATCH] [WIP] [YUNIKORN-2180] Clean up scheduler state initialization - Moved all state initialization code into context - Unified primary and secondary scheduler caches - Properly handle orphaned pods (those referring to unknown nodes) - TODO: Missing tests --- pkg/cache/amprotocol.go | 39 - pkg/cache/amprotocol_mock.go | 147 ---- pkg/cache/application.go | 18 +- pkg/cache/application_state.go | 11 +- pkg/cache/application_test.go | 44 +- pkg/cache/appmgmt.go | 221 ------ pkg/cache/appmgmt_recovery.go | 125 ---- pkg/cache/appmgmt_recovery_test.go | 326 --------- pkg/cache/appmgmt_test.go | 614 ---------------- pkg/cache/context.go | 807 +++++++++++++++++---- pkg/cache/context_recovery.go | 215 ------ pkg/cache/context_recovery_test.go | 215 ------ pkg/cache/context_test.go | 285 ++++++-- pkg/cache/external/scheduler_cache.go | 254 +++++-- pkg/cache/external/scheduler_cache_test.go | 121 +-- pkg/cache/metadata.go | 8 +- pkg/cache/metadata_test.go | 14 +- pkg/cache/node.go | 202 ------ pkg/cache/node_events.go | 47 ++ pkg/cache/node_graphviz_test.go | 49 -- pkg/cache/node_state.go | 159 ---- pkg/cache/node_test.go | 92 --- pkg/cache/nodes.go | 239 ------ pkg/cache/nodes_test.go | 503 ------------- pkg/cache/placeholder_manager_test.go | 2 +- pkg/cache/podevent_handler.go | 184 ----- pkg/cache/podevent_handler_test.go | 118 --- pkg/cache/task.go | 6 +- pkg/common/si_helper.go | 3 +- pkg/common/utils/utils.go | 5 +- pkg/common/utils/utils_test.go | 12 +- pkg/dispatcher/dispatch_test.go | 25 +- pkg/dispatcher/dispatcher.go | 41 +- pkg/plugin/support/nodeinfo_lister_test.go | 4 +- pkg/plugin/support/shared_lister_test.go | 2 +- pkg/shim/scheduler.go | 66 +- pkg/shim/scheduler_mock_test.go | 81 +-- pkg/shim/scheduler_test.go | 81 ++- 38 files changed, 1386 insertions(+), 3999 deletions(-) delete mode 100644 pkg/cache/amprotocol_mock.go delete mode 100644 pkg/cache/appmgmt.go delete mode 100644 pkg/cache/appmgmt_recovery.go delete mode 100644 pkg/cache/appmgmt_recovery_test.go delete mode 100644 pkg/cache/appmgmt_test.go delete mode 100644 pkg/cache/context_recovery.go delete mode 100644 pkg/cache/context_recovery_test.go delete mode 100644 pkg/cache/node.go create mode 100644 pkg/cache/node_events.go delete mode 100644 pkg/cache/node_graphviz_test.go delete mode 100644 pkg/cache/node_state.go delete mode 100644 pkg/cache/node_test.go delete mode 100644 pkg/cache/nodes.go delete mode 100644 pkg/cache/nodes_test.go delete mode 100644 pkg/cache/podevent_handler.go delete mode 100644 pkg/cache/podevent_handler_test.go diff --git a/pkg/cache/amprotocol.go b/pkg/cache/amprotocol.go index d81edc1b6..5f2dd31c6 100644 --- a/pkg/cache/amprotocol.go +++ b/pkg/cache/amprotocol.go @@ -24,45 +24,6 @@ import ( metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" ) -// app management protocol defines all the APIs needed for app management, -// this is the protocol between scheduler cache and app management plugins -type ApplicationManagementProtocol interface { - // returns app that already existed in the cache, - // or nil, false if app with the given appID is not found - GetApplication(appID string) *Application - - // add app to the context, app manager needs to provide all - // necessary app metadata through this call. If this a existing app - // for recovery, the AddApplicationRequest#Recovery must be true. - AddApplication(request *AddApplicationRequest) *Application - - // remove application from the context - // returns an error if for some reason the app cannot be removed, - // e.g the given app is not found in current context. - RemoveApplication(appID string) error - - // add task to the context, if add is successful, - AddTask(request *AddTaskRequest) *Task - - // remove task from the app - // return an error if for some reason the task cannot be removed - // e.g app that owns this task is not found in context. - RemoveTask(appID, taskID string) - - // notify the context that an app is completed, - // this will trigger some consequent operations for the given app - NotifyApplicationComplete(appID string) - - // notify the context that an app has failed, - // this will trigger some consequent operations for the given app - NotifyApplicationFail(appID string) - - // notify the context that an task is completed, - // this will trigger some consequent operations for a given task, - // e.g release the allocations that assigned for this task. - NotifyTaskComplete(appID, taskID string) -} - type AddApplicationRequest struct { Metadata ApplicationMetadata } diff --git a/pkg/cache/amprotocol_mock.go b/pkg/cache/amprotocol_mock.go deleted file mode 100644 index 4dc5e0078..000000000 --- a/pkg/cache/amprotocol_mock.go +++ /dev/null @@ -1,147 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "fmt" - - "go.uber.org/zap" - - "github.com/apache/yunikorn-k8shim/pkg/common/test" - "github.com/apache/yunikorn-k8shim/pkg/log" -) - -// implements ApplicationManagementProtocol -type MockedAMProtocol struct { - applications map[string]*Application - addTaskFn func(request *AddTaskRequest) -} - -func NewMockedAMProtocol() *MockedAMProtocol { - return &MockedAMProtocol{ - applications: make(map[string]*Application)} -} - -func (m *MockedAMProtocol) GetApplication(appID string) *Application { - if app, ok := m.applications[appID]; ok { - return app - } - return nil -} - -func (m *MockedAMProtocol) AddApplication(request *AddApplicationRequest) *Application { - if app := m.GetApplication(request.Metadata.ApplicationID); app != nil { - return app - } - - app := NewApplication( - request.Metadata.ApplicationID, - request.Metadata.QueueName, - request.Metadata.User, - request.Metadata.Groups, - request.Metadata.Tags, - test.NewSchedulerAPIMock()) - app.setPlaceholderOwnerReferences(request.Metadata.OwnerReferences) - - // add into cache - m.applications[app.GetApplicationID()] = app - - return app -} - -func (m *MockedAMProtocol) RemoveApplication(appID string) error { - if app := m.GetApplication(appID); app != nil { - delete(m.applications, appID) - return nil - } - return fmt.Errorf("application doesn't exist") -} - -func (m *MockedAMProtocol) AddTask(request *AddTaskRequest) *Task { - if m.addTaskFn != nil { - m.addTaskFn(request) - } - if app, ok := m.applications[request.Metadata.ApplicationID]; ok { - existingTask, err := app.GetTask(request.Metadata.TaskID) - if err != nil { - var originator bool - - // Is this task the originator of the application? - // If yes, then make it as "first pod/owner/driver" of the application and set the task as originator - if app.GetOriginatingTask() == nil { - for _, ownerReference := range app.getPlaceholderOwnerReferences() { - referenceID := string(ownerReference.UID) - if request.Metadata.TaskID == referenceID { - originator = true - break - } - } - } - task := NewFromTaskMeta(request.Metadata.TaskID, app, nil, request.Metadata, originator) - app.addTask(task) - log.Log(log.Test).Info("task added", - zap.String("appID", app.applicationID), - zap.String("taskID", task.taskID), - zap.String("taskState", task.GetTaskState())) - if originator { - if app.GetOriginatingTask() != nil { - log.Log(log.Test).Error("Inconsistent state - found another originator task for an application", - zap.String("taskId", task.GetTaskID())) - } - app.setOriginatingTask(task) - log.Log(log.Test).Info("app request originating pod added", - zap.String("appID", app.applicationID), - zap.String("original task", task.GetTaskID())) - } - return task - } - return existingTask - } - return nil -} - -func (m *MockedAMProtocol) RemoveTask(appID, taskID string) { - if app, ok := m.applications[appID]; ok { - app.removeTask(taskID) - } -} - -func (m *MockedAMProtocol) NotifyApplicationComplete(appID string) { - if app := m.GetApplication(appID); app != nil { - app.SetState(ApplicationStates().Completed) - } -} - -func (m *MockedAMProtocol) NotifyApplicationFail(appID string) { - if app := m.GetApplication(appID); app != nil { - app.SetState(ApplicationStates().Failed) - } -} - -func (m *MockedAMProtocol) NotifyTaskComplete(appID, taskID string) { - if app := m.GetApplication(appID); app != nil { - if task, err := app.GetTask(taskID); err == nil { - task.sm.SetState(TaskStates().Completed) - } - } -} - -func (m *MockedAMProtocol) UseAddTaskFn(fn func(request *AddTaskRequest)) { - m.addTaskFn = fn -} diff --git a/pkg/cache/application.go b/pkg/cache/application.go index 57ee897a4..9b0c91eb3 100644 --- a/pkg/cache/application.go +++ b/pkg/cache/application.go @@ -326,8 +326,12 @@ func (app *Application) SetState(state string) { app.sm.SetState(state) } +func (app *Application) TriggerAppSubmission() error { + return app.handle(NewSubmitApplicationEvent(app.applicationID)) +} + func (app *Application) TriggerAppRecovery() error { - return app.handle(NewSimpleApplicationEvent(app.applicationID, RecoverApplication)) + return app.handle(NewRecoverApplicationEvent(app.applicationID)) } // Schedule is called in every scheduling interval, @@ -339,12 +343,6 @@ func (app *Application) TriggerAppRecovery() error { // return true if the app needs scheduling or false if not func (app *Application) Schedule() bool { switch app.GetApplicationState() { - case ApplicationStates().New: - ev := NewSubmitApplicationEvent(app.GetApplicationID()) - if err := app.handle(ev); err != nil { - log.Log(log.ShimCacheApplication).Warn("failed to handle SUBMIT app event", - zap.Error(err)) - } case ApplicationStates().Accepted: // once the app is accepted by the scheduler core, // the next step is to send requests for scheduling @@ -404,7 +402,7 @@ func (app *Application) scheduleTasks(taskScheduleCondition func(t *Task) bool) } } -func (app *Application) handleSubmitApplicationEvent() { +func (app *Application) handleSubmitApplicationEvent() error { log.Log(log.ShimCacheApplication).Info("handle app submission", zap.Stringer("app", app), zap.String("clusterID", conf.GetSchedulerConf().ClusterID)) @@ -433,9 +431,10 @@ func (app *Application) handleSubmitApplicationEvent() { log.Log(log.ShimCacheApplication).Warn("failed to submit app", zap.Error(err)) dispatcher.Dispatch(NewFailApplicationEvent(app.applicationID, err.Error())) } + return err } -func (app *Application) handleRecoverApplicationEvent() { +func (app *Application) handleRecoverApplicationEvent() error { log.Log(log.ShimCacheApplication).Info("handle app recovering", zap.Stringer("app", app), zap.String("clusterID", conf.GetSchedulerConf().ClusterID)) @@ -464,6 +463,7 @@ func (app *Application) handleRecoverApplicationEvent() { log.Log(log.ShimCacheApplication).Warn("failed to recover app", zap.Error(err)) dispatcher.Dispatch(NewFailApplicationEvent(app.applicationID, err.Error())) } + return err } func (app *Application) skipReservationStage() bool { diff --git a/pkg/cache/application_state.go b/pkg/cache/application_state.go index 4afe33d28..10602bac4 100644 --- a/pkg/cache/application_state.go +++ b/pkg/cache/application_state.go @@ -166,6 +166,13 @@ func NewSubmitApplicationEvent(appID string) SubmitApplicationEvent { } } +func NewRecoverApplicationEvent(appID string) SubmitApplicationEvent { + return SubmitApplicationEvent{ + applicationID: appID, + event: RecoverApplication, + } +} + func (se SubmitApplicationEvent) GetEvent() string { return se.event.String() } @@ -517,11 +524,11 @@ func newAppState() *fsm.FSM { //nolint:funlen }, SubmitApplication.String(): func(_ context.Context, event *fsm.Event) { app := event.Args[0].(*Application) //nolint:errcheck - app.handleSubmitApplicationEvent() + event.Err = app.handleSubmitApplicationEvent() }, RecoverApplication.String(): func(_ context.Context, event *fsm.Event) { app := event.Args[0].(*Application) //nolint:errcheck - app.handleRecoverApplicationEvent() + event.Err = app.handleRecoverApplicationEvent() }, RejectApplication.String(): func(_ context.Context, event *fsm.Event) { app := event.Args[0].(*Application) //nolint:errcheck diff --git a/pkg/cache/application_test.go b/pkg/cache/application_test.go index 8b2f6d4bb..d70ce8a91 100644 --- a/pkg/cache/application_test.go +++ b/pkg/cache/application_test.go @@ -112,7 +112,7 @@ func TestRunApplication(t *testing.T) { func TestFailApplication(t *testing.T) { context := initContextForTest() - dispatcher.RegisterEventHandler(dispatcher.EventTypeApp, context.ApplicationEventHandler()) + dispatcher.RegisterEventHandler("TestAppHandler", dispatcher.EventTypeApp, context.ApplicationEventHandler()) dispatcher.Start() defer dispatcher.Stop() @@ -211,7 +211,7 @@ func TestFailApplication(t *testing.T) { func TestSetUnallocatedPodsToFailedWhenFailApplication(t *testing.T) { context := initContextForTest() - dispatcher.RegisterEventHandler(dispatcher.EventTypeApp, context.ApplicationEventHandler()) + dispatcher.RegisterEventHandler("TestAppHandler", dispatcher.EventTypeApp, context.ApplicationEventHandler()) dispatcher.Start() defer dispatcher.Stop() @@ -320,7 +320,7 @@ func TestSetUnallocatedPodsToFailedWhenFailApplication(t *testing.T) { func TestSetUnallocatedPodsToFailedWhenRejectApplication(t *testing.T) { context := initContextForTest() - dispatcher.RegisterEventHandler(dispatcher.EventTypeApp, context.ApplicationEventHandler()) + dispatcher.RegisterEventHandler("TestAppHandler", dispatcher.EventTypeApp, context.ApplicationEventHandler()) dispatcher.Start() defer dispatcher.Stop() @@ -532,7 +532,7 @@ func TestGetNonTerminatedTaskAlias(t *testing.T) { context := initContextForTest() appID := "app00001" app := NewApplication(appID, "root.a", "testuser", testGroups, map[string]string{}, newMockSchedulerAPI()) - context.addApplication(app) + context.addApplicationToContext(app) // app doesn't have any task res := app.getNonTerminatedTaskAlias() assert.Equal(t, len(res), 0) @@ -685,7 +685,7 @@ func (t *threadSafePodsMap) count() int { func TestTryReserve(t *testing.T) { context := initContextForTest() - dispatcher.RegisterEventHandler(dispatcher.EventTypeApp, context.ApplicationEventHandler()) + dispatcher.RegisterEventHandler("TestAppHandler", dispatcher.EventTypeApp, context.ApplicationEventHandler()) dispatcher.Start() defer dispatcher.Stop() @@ -703,7 +703,7 @@ func TestTryReserve(t *testing.T) { // create a new app app := NewApplication("app00001", "root.abc", "test-user", testGroups, map[string]string{}, mockedAPIProvider.GetAPIs().SchedulerAPI) - context.addApplication(app) + context.addApplicationToContext(app) // set taskGroups app.setTaskGroups([]TaskGroup{ @@ -751,7 +751,7 @@ func TestTryReserve(t *testing.T) { func TestTryReservePostRestart(t *testing.T) { context := initContextForTest() - dispatcher.RegisterEventHandler(dispatcher.EventTypeApp, context.ApplicationEventHandler()) + dispatcher.RegisterEventHandler("TestAppHandler", dispatcher.EventTypeApp, context.ApplicationEventHandler()) dispatcher.Start() defer dispatcher.Stop() @@ -769,7 +769,7 @@ func TestTryReservePostRestart(t *testing.T) { // create a new app app := NewApplication("app00001", "root.abc", "test-user", testGroups, map[string]string{}, mockedAPIProvider.GetAPIs().SchedulerAPI) - context.addApplication(app) + context.addApplicationToContext(app) // set taskGroups app.setTaskGroups([]TaskGroup{ @@ -1035,7 +1035,7 @@ func TestReleaseAppAllocationInFailingState(t *testing.T) { func TestResumingStateTransitions(t *testing.T) { context := initContextForTest() - dispatcher.RegisterEventHandler(dispatcher.EventTypeApp, context.ApplicationEventHandler()) + dispatcher.RegisterEventHandler("TestAppHandler", dispatcher.EventTypeApp, context.ApplicationEventHandler()) dispatcher.Start() defer dispatcher.Stop() @@ -1063,7 +1063,7 @@ func TestResumingStateTransitions(t *testing.T) { app.addTask(task2) UUID := "testUUID001" task1.allocationUUID = UUID - context.addApplication(app) + context.addApplicationToContext(app) // Set app state to "reserving" app.SetState(ApplicationStates().Reserving) @@ -1124,9 +1124,6 @@ func TestPlaceholderTimeoutEvents(t *testing.T) { t.Fatal("the EventRecorder is expected to be of type FakeRecorder") } - amprotocol := NewMockedAMProtocol() - am := NewAMService(amprotocol, client.NewMockedAPIProvider(false)) - am.podEventHandler.recoveryRunning = false pod1 := v1.Pod{ TypeMeta: apis.TypeMeta{ Kind: "Pod", @@ -1147,8 +1144,8 @@ func TestPlaceholderTimeoutEvents(t *testing.T) { }, } - // add a pending pod through the AM service - am.AddPod(&pod1) + // add a pending pod + context.AddPod(&pod1) pod := &v1.Pod{ TypeMeta: apis.TypeMeta{ @@ -1169,17 +1166,16 @@ func TestPlaceholderTimeoutEvents(t *testing.T) { Phase: v1.PodPending, }, } - app := amprotocol.GetApplication("app00001") + app := context.GetApplication("app00001") assert.Assert(t, app != nil) assert.Equal(t, app.GetApplicationID(), "app00001") - assert.Equal(t, app.GetApplicationState(), ApplicationStates().New) + assert.Equal(t, app.GetApplicationState(), ApplicationStates().Recovering) assert.Equal(t, app.GetQueue(), "root.a") assert.Equal(t, len(app.GetNewTasks()), 1) appID := "app00001" UUID := "UID-POD-00002" - context.addApplication(app) task1 := context.AddTask(&AddTaskRequest{ Metadata: TaskMetadata{ ApplicationID: "app00001", @@ -1196,14 +1192,10 @@ func TestPlaceholderTimeoutEvents(t *testing.T) { task1.allocationUUID = UUID - // app must be running states - err := app.handle(NewReleaseAppAllocationEvent(appID, si.TerminationType_TIMEOUT, UUID)) - assert.Error(t, err, "event ReleaseAppAllocation inappropriate in current state New") - // set app states to running, let event can be trigger app.SetState(ApplicationStates().Running) assertAppState(t, app, ApplicationStates().Running, 3*time.Second) - err = app.handle(NewReleaseAppAllocationEvent(appID, si.TerminationType_TIMEOUT, UUID)) + err := app.handle(NewReleaseAppAllocationEvent(appID, si.TerminationType_TIMEOUT, UUID)) assert.NilError(t, err) // after handle release event the states of app must be running assertAppState(t, app, ApplicationStates().Running, 3*time.Second) @@ -1228,12 +1220,12 @@ func TestPlaceholderTimeoutEvents(t *testing.T) { func TestApplication_onReservationStateChange(t *testing.T) { context := initContextForTest() - dispatcher.RegisterEventHandler(dispatcher.EventTypeApp, context.ApplicationEventHandler()) + dispatcher.RegisterEventHandler("TestAppHandler", dispatcher.EventTypeApp, context.ApplicationEventHandler()) dispatcher.Start() defer dispatcher.Stop() app := NewApplication(appID, "root.a", "testuser", testGroups, map[string]string{}, newMockSchedulerAPI()) - context.addApplication(app) + context.addApplicationToContext(app) app.sm.SetState("Accepted") app.onReservationStateChange() @@ -1301,7 +1293,7 @@ func TestApplication_onReservationStateChange(t *testing.T) { assertAppState(t, app, ApplicationStates().Running, 1*time.Second) } -func (ctx *Context) addApplication(app *Application) { +func (ctx *Context) addApplicationToContext(app *Application) { ctx.lock.Lock() defer ctx.lock.Unlock() ctx.applications[app.applicationID] = app diff --git a/pkg/cache/appmgmt.go b/pkg/cache/appmgmt.go deleted file mode 100644 index 25a67fdaa..000000000 --- a/pkg/cache/appmgmt.go +++ /dev/null @@ -1,221 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "strconv" - "sync/atomic" - - "go.uber.org/zap" - v1 "k8s.io/api/core/v1" - "k8s.io/apimachinery/pkg/labels" - k8sCache "k8s.io/client-go/tools/cache" - - "github.com/apache/yunikorn-k8shim/pkg/client" - "github.com/apache/yunikorn-k8shim/pkg/common" - "github.com/apache/yunikorn-k8shim/pkg/common/constants" - "github.com/apache/yunikorn-k8shim/pkg/common/utils" - "github.com/apache/yunikorn-k8shim/pkg/log" - siCommon "github.com/apache/yunikorn-scheduler-interface/lib/go/common" - "github.com/apache/yunikorn-scheduler-interface/lib/go/si" -) - -// AppManagementService is a central service that interacts with -// one or more K8s operators for app scheduling. -type AppManagementService struct { - apiProvider client.APIProvider - amProtocol ApplicationManagementProtocol - podEventHandler *PodEventHandler - cancelRecovery atomic.Bool -} - -func NewAMService(amProtocol ApplicationManagementProtocol, apiProvider client.APIProvider) *AppManagementService { - podEventHandler := NewPodEventHandler(amProtocol, true) - - log.Log(log.ShimCacheAppMgmt).Info("Initializing new AppMgmt service") - return &AppManagementService{ - apiProvider: apiProvider, - amProtocol: amProtocol, - podEventHandler: podEventHandler, - } -} - -func (svc *AppManagementService) Start() error { - svc.apiProvider.AddEventHandler( - &client.ResourceEventHandlers{ - Type: client.PodInformerHandlers, - FilterFn: svc.filterPods, - AddFn: svc.AddPod, - UpdateFn: svc.updatePod, - DeleteFn: svc.deletePod, - }) - return nil -} - -func (svc *AppManagementService) ListPods() ([]*v1.Pod, error) { - log.Log(log.ShimCacheAppMgmt).Info("Retrieving pod list") - // list all pods on this cluster - appPods, err := svc.apiProvider.GetAPIs().PodInformer.Lister().List(labels.NewSelector()) - if err != nil { - return nil, err - } - log.Log(log.ShimCacheAppMgmt).Info("Pod list retrieved from api server", zap.Int("nr of pods", len(appPods))) - // get existing apps - existingApps := make(map[string]struct{}) - podsRecovered := 0 - podsWithoutMetaData := 0 - pods := make([]*v1.Pod, 0) - for _, pod := range appPods { - log.Log(log.ShimCacheAppMgmt).Debug("Looking at pod for recovery candidates", zap.String("podNamespace", pod.Namespace), zap.String("podName", pod.Name)) - // general filter passes, and pod is assigned - // this means the pod is already scheduled by scheduler for an existing app - if utils.GetApplicationIDFromPod(pod) != "" && utils.IsAssignedPod(pod) { - if meta, ok := getAppMetadata(pod, true); ok { - podsRecovered++ - pods = append(pods, pod) - log.Log(log.ShimCacheAppMgmt).Debug("Adding appID as recovery candidate", zap.String("appID", meta.ApplicationID)) - existingApps[meta.ApplicationID] = struct{}{} - } else { - podsWithoutMetaData++ - } - } - } - log.Log(log.ShimCacheAppMgmt).Info("Application recovery statistics", - zap.Int("nr of recoverable apps", len(existingApps)), - zap.Int("nr of total pods", len(appPods)), - zap.Int("nr of pods without application metadata", podsWithoutMetaData), - zap.Int("nr of pods to be recovered", podsRecovered)) - - return pods, nil -} - -func (svc *AppManagementService) GetExistingAllocation(pod *v1.Pod) *si.Allocation { - if meta, valid := getAppMetadata(pod, false); valid { - // when submit a task, we use pod UID as the allocationKey, - // to keep consistent, during recovery, the pod UID is also used - // for an Allocation. - placeholder := utils.GetPlaceholderFlagFromPodSpec(pod) - taskGroupName := utils.GetTaskGroupFromPodSpec(pod) - - creationTime := pod.CreationTimestamp.Unix() - meta.Tags[siCommon.CreationTime] = strconv.FormatInt(creationTime, 10) - - return &si.Allocation{ - AllocationKey: string(pod.UID), - AllocationTags: meta.Tags, - UUID: string(pod.UID), - ResourcePerAlloc: common.GetPodResource(pod), - NodeID: pod.Spec.NodeName, - ApplicationID: meta.ApplicationID, - Placeholder: placeholder, - TaskGroupName: taskGroupName, - PartitionName: constants.DefaultPartition, - } - } - return nil -} - -// filter pods by scheduler name and state -func (svc *AppManagementService) filterPods(obj interface{}) bool { - switch object := obj.(type) { - case *v1.Pod: - pod := object - return utils.GetApplicationIDFromPod(pod) != "" - default: - return false - } -} - -// AddPod Add application and task using pod metadata -// Visibility: Public only for testing -func (svc *AppManagementService) AddPod(obj interface{}) { - pod, err := utils.Convert2Pod(obj) - if err != nil { - log.Log(log.ShimCacheAppMgmt).Error("failed to add pod", zap.Error(err)) - return - } - - log.Log(log.ShimCacheAppMgmt).Debug("pod added", - zap.String("Name", pod.Name), - zap.String("Namespace", pod.Namespace)) - - svc.podEventHandler.HandleEvent(AddPod, Informers, pod) -} - -// when pod resource is modified, we need to act accordingly -// e.g vertical scale out the pod, this requires the scheduler to be aware of this -func (svc *AppManagementService) updatePod(old, new interface{}) { - oldPod, err := utils.Convert2Pod(old) - if err != nil { - log.Log(log.ShimCacheAppMgmt).Error("expecting a pod object", zap.Error(err)) - return - } - - newPod, err := utils.Convert2Pod(new) - if err != nil { - log.Log(log.ShimCacheAppMgmt).Error("expecting a pod object", zap.Error(err)) - return - } - - // triggered when pod status' phase changes - if oldPod.Status.Phase != newPod.Status.Phase { - // pod succeed or failed means all containers in the pod have been terminated, - // and these container won't be restarted. In this case, we can safely release - // the resources for this allocation. And mark the task is done. - if utils.IsPodTerminated(newPod) { - log.Log(log.ShimCacheAppMgmt).Info("task completes", - zap.String("namespace", newPod.Namespace), - zap.String("podName", newPod.Name), - zap.String("podUID", string(newPod.UID)), - zap.String("podStatus", string(newPod.Status.Phase))) - svc.podEventHandler.HandleEvent(UpdatePod, Informers, newPod) - } - } -} - -// this function is called when a pod is deleted from api-server. -// when a pod is completed, the equivalent task's state will also be completed -// optionally, we run a completionHandler per workload, in order to determine -// if a application is completed along with this pod's completion -func (svc *AppManagementService) deletePod(obj interface{}) { - // when a pod is deleted, we need to check its role. - // for spark, if driver pod is deleted, then we consider the app is completed - var pod *v1.Pod - switch t := obj.(type) { - case *v1.Pod: - pod = t - case k8sCache.DeletedFinalStateUnknown: - var err error - pod, err = utils.Convert2Pod(t.Obj) - if err != nil { - log.Log(log.ShimCacheAppMgmt).Error(err.Error()) - return - } - default: - log.Log(log.ShimCacheAppMgmt).Error("cannot convert to pod") - return - } - - log.Log(log.ShimCacheAppMgmt).Info("delete pod", - zap.String("namespace", pod.Namespace), - zap.String("podName", pod.Name), - zap.String("podUID", string(pod.UID))) - - svc.podEventHandler.HandleEvent(DeletePod, Informers, pod) -} diff --git a/pkg/cache/appmgmt_recovery.go b/pkg/cache/appmgmt_recovery.go deleted file mode 100644 index aaeccc290..000000000 --- a/pkg/cache/appmgmt_recovery.go +++ /dev/null @@ -1,125 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "errors" - "sort" - "time" - - "go.uber.org/zap" - - "github.com/apache/yunikorn-k8shim/pkg/common/utils" - "github.com/apache/yunikorn-k8shim/pkg/log" -) - -// WaitForRecovery initiates and waits for the app management service to finish recovery. If recovery -// is canceled (used by testing code) or an error occurs, an error will be returned. In production, this -// method will block until recovery completes. -func (svc *AppManagementService) WaitForRecovery() error { - apps, err := svc.recoverApps() - if err != nil { - return err - } - if !svc.waitForAppRecovery(apps) { - return errors.New("recovery aborted") - } - return nil -} - -func (svc *AppManagementService) recoverApps() (map[string]*Application, error) { - log.Log(log.ShimCacheAppMgmt).Info("Starting app recovery") - recoveringApps := make(map[string]*Application) - pods, err := svc.ListPods() - if err != nil { - log.Log(log.ShimCacheAppMgmt).Error("failed to list apps", zap.Error(err)) - return recoveringApps, err - } - - sort.Slice(pods, func(i, j int) bool { - return pods[i].CreationTimestamp.Unix() < pods[j].CreationTimestamp.Unix() - }) - - // Track terminated pods that we have already seen in order to - // skip redundant handling of async events in RecoveryDone - // This filter is used for terminated pods to remain consistent - // with pod filters in the informer - terminatedYkPods := make(map[string]bool) - for _, pod := range pods { - if utils.GetApplicationIDFromPod(pod) != "" { - if !utils.IsPodTerminated(pod) { - app := svc.podEventHandler.HandleEvent(AddPod, Recovery, pod) - recoveringApps[app.GetApplicationID()] = app - continue - } - terminatedYkPods[string(pod.UID)] = true - } - } - log.Log(log.ShimCacheAppMgmt).Info("Recovery finished") - svc.podEventHandler.RecoveryDone(terminatedYkPods) - - return recoveringApps, nil -} - -// waitForAppRecovery blocks until either all applications have been processed (returning true) -// or cancelWaitForAppRecovery is called (returning false) -func (svc *AppManagementService) waitForAppRecovery(recoveringApps map[string]*Application) bool { - svc.cancelRecovery.Store(false) // reset cancellation token - recoveryStartTime := time.Now() - counter := 0 - for { - // check for cancellation token - if svc.cancelRecovery.Load() { - log.Log(log.ShimCacheAppMgmt).Info("Waiting for recovery canceled.") - svc.cancelRecovery.Store(false) - return false - } - - svc.removeRecoveredApps(recoveringApps) - if len(recoveringApps) == 0 { - log.Log(log.ShimCacheAppMgmt).Info("Application recovery complete.") - return true - } - counter++ - if counter%10 == 0 { - log.Log(log.ShimCacheAppMgmt).Info("Waiting for application recovery", - zap.Duration("timeElapsed", time.Since(recoveryStartTime).Round(time.Second)), - zap.Int("appsRemaining", len(recoveringApps))) - } - time.Sleep(1 * time.Second) - } -} - -// cancelWaitForAppRecovery is used by testing code to ensure that waitForAppRecovery does not block forever -func (svc *AppManagementService) cancelWaitForAppRecovery() { - svc.cancelRecovery.Store(true) -} - -// removeRecoveredApps is used to walk the currently recovering apps list and remove those that have finished recovering -func (svc *AppManagementService) removeRecoveredApps(recoveringApps map[string]*Application) { - for _, app := range recoveringApps { - state := app.GetApplicationState() - if state != ApplicationStates().New && state != ApplicationStates().Recovering { - log.Log(log.ShimCacheAppMgmt).Info("Recovered application", - zap.String("appId", app.GetApplicationID()), - zap.String("state", state)) - delete(recoveringApps, app.GetApplicationID()) - } - } -} diff --git a/pkg/cache/appmgmt_recovery_test.go b/pkg/cache/appmgmt_recovery_test.go deleted file mode 100644 index a0a94ed30..000000000 --- a/pkg/cache/appmgmt_recovery_test.go +++ /dev/null @@ -1,326 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "testing" - "time" - - "gotest.tools/v3/assert" - v1 "k8s.io/api/core/v1" - apis "k8s.io/apimachinery/pkg/apis/meta/v1" - "k8s.io/apimachinery/pkg/types" - - "github.com/apache/yunikorn-k8shim/pkg/client" - "github.com/apache/yunikorn-k8shim/pkg/common/constants" - "github.com/apache/yunikorn-k8shim/pkg/dispatcher" - "github.com/apache/yunikorn-scheduler-interface/lib/go/si" -) - -func TestAppManagerRecoveryState(t *testing.T) { - t.Skip("broken") - // conf.GetSchedulerConf().OperatorPlugins = "mocked-app-manager" - amProtocol := NewMockedAMProtocol() - apiProvider := client.NewMockedAPIProvider(false) - amService := NewAMService(amProtocol, apiProvider) - // amService.register(&mockedAppManager{}) - - apps, err := amService.recoverApps() - assert.NilError(t, err) - assert.Equal(t, len(apps), 2) - - for appId, app := range apps { - assert.Assert(t, appId == "app01" || appId == "app02") - assert.Equal(t, app.GetApplicationState(), ApplicationStates().Recovering) - } -} - -func TestAppManagerRecoveryTimeout(t *testing.T) { - t.Skip("broken") - // conf.GetSchedulerConf().OperatorPlugins = "mocked-app-manager" - amProtocol := NewMockedAMProtocol() - apiProvider := client.NewMockedAPIProvider(false) - amService := NewAMService(amProtocol, apiProvider) - // amService.register(&mockedAppManager{}) - - apps, err := amService.recoverApps() - assert.NilError(t, err) - assert.Equal(t, len(apps), 2) - - go func() { - time.Sleep(3 * time.Second) - amService.cancelWaitForAppRecovery() - }() - ok := amService.waitForAppRecovery(apps) - assert.Assert(t, !ok, "expected timeout") -} - -func TestAppManagerRecoveryExitCondition(t *testing.T) { - t.Skip("broken") - // conf.GetSchedulerConf().OperatorPlugins = "mocked-app-manager" - amProtocol := NewMockedAMProtocol() - apiProvider := client.NewMockedAPIProvider(false) - amService := NewAMService(amProtocol, apiProvider) - // amService.register(&mockedAppManager{}) - - apps, err := amService.recoverApps() - assert.NilError(t, err) - assert.Equal(t, len(apps), 2) - - // simulate app recovery succeed - for _, app := range apps { - app.SetState(ApplicationStates().Accepted) - } - - go func() { - time.Sleep(3 * time.Second) - amService.cancelWaitForAppRecovery() - }() - ok := amService.waitForAppRecovery(apps) - assert.Assert(t, ok, "timeout waiting for recovery") -} - -func TestAppManagerRecoveryFailureExitCondition(t *testing.T) { - t.Skip("broken") - // conf.GetSchedulerConf().OperatorPlugins = "mocked-app-manager" - amProtocol := NewMockedAMProtocol() - apiProvider := client.NewMockedAPIProvider(false) - amService := NewAMService(amProtocol, apiProvider) - // amService.register(&mockedAppManager{}) - - apps, err := amService.recoverApps() - assert.NilError(t, err) - assert.Equal(t, len(apps), 2) - - // simulate app rejected - for _, app := range apps { - app.SetState(ApplicationStates().Rejected) - } - - go func() { - time.Sleep(3 * time.Second) - amService.cancelWaitForAppRecovery() - }() - ok := amService.waitForAppRecovery(apps) - assert.Assert(t, ok, "timeout waiting for recovery") -} - -// test app state transition during recovery -func TestAppStatesDuringRecovery(t *testing.T) { - t.Skip("broken") - // conf.GetSchedulerConf().OperatorPlugins = "mocked-app-manager" - apiProvider := client.NewMockedAPIProvider(false) - ctx := NewContext(apiProvider) - cb := NewAsyncRMCallback(ctx) - - dispatcher.RegisterEventHandler(dispatcher.EventTypeApp, ctx.ApplicationEventHandler()) - dispatcher.Start() - defer dispatcher.Stop() - - amService := NewAMService(ctx, apiProvider) - _ = &mockedAppManager{} - // amService.register(&mockedAppManager{}) - - apps, err := amService.recoverApps() - assert.NilError(t, err) - assert.Equal(t, len(apps), 2) - - // when the recovery starts, all apps should be under Recovering state - app01 := ctx.GetApplication("app01") - app02 := ctx.GetApplication("app02") - - // waitForAppRecovery call should be blocked - // because the scheduler is still doing recovery - go func() { - time.Sleep(3 * time.Second) - amService.cancelWaitForAppRecovery() - }() - ok := amService.waitForAppRecovery(apps) - assert.Assert(t, !ok, "expected timeout") - assert.Equal(t, app01.GetApplicationState(), ApplicationStates().Recovering) - assert.Equal(t, app02.GetApplicationState(), ApplicationStates().Recovering) - - // mock the responses, simulate app01 has been accepted - err = cb.UpdateApplication(&si.ApplicationResponse{ - Accepted: []*si.AcceptedApplication{ - { - ApplicationID: "app01", - }, - }, - }) - assert.NilError(t, err, "failed to handle UpdateResponse") - - // since app02 is still under recovery - // waitForRecovery should timeout because the scheduler is still under recovery - go func() { - time.Sleep(3 * time.Second) - amService.cancelWaitForAppRecovery() - }() - ok = amService.waitForAppRecovery(apps) - assert.Assert(t, !ok, "expected timeout") - assert.Equal(t, app01.GetApplicationState(), ApplicationStates().Accepted) - assert.Equal(t, app02.GetApplicationState(), ApplicationStates().Recovering) - - // mock the responses, simulate app02 has been accepted - err = cb.UpdateApplication(&si.ApplicationResponse{ - Accepted: []*si.AcceptedApplication{ - { - ApplicationID: "app02", - }, - }, - }) - assert.NilError(t, err, "failed to handle UpdateResponse") - - // the app recovery has finished, - // this should not timeout anymore - go func() { - time.Sleep(3 * time.Second) - amService.cancelWaitForAppRecovery() - }() - ok = amService.waitForAppRecovery(apps) - assert.Assert(t, ok, "unexpected timeout") - assert.Equal(t, app01.GetApplicationState(), ApplicationStates().Accepted) - assert.Equal(t, app02.GetApplicationState(), ApplicationStates().Accepted) -} - -func TestPodRecovery(t *testing.T) { - t.Skip("broken") - // conf.GetSchedulerConf().OperatorPlugins = "mocked-app-manager" - amProtocol := NewMockedAMProtocol() - apiProvider := client.NewMockedAPIProvider(false) - taskRequests := make([]*AddTaskRequest, 0) - amProtocol.UseAddTaskFn(func(request *AddTaskRequest) { - taskRequests = append(taskRequests, request) - }) - amService := NewAMService(amProtocol, apiProvider) - _ = &mockedAppManager{} - // amService.register(&mockedAppManager{}) - - apps, err := amService.recoverApps() - assert.NilError(t, err) - assert.Equal(t, 4, len(taskRequests)) - assert.Equal(t, 2, len(apps)) - - expected := map[string]map[string]bool{ - "app01": { - "task01": true, - "task02": true, - }, - "app02": { - "task04": true, - "task05": true, - }, - } - - for _, tr := range taskRequests { - check, ok := expected[tr.Metadata.ApplicationID] - assert.Assert(t, ok, "app should not be recovered: "+tr.Metadata.ApplicationID) - assert.Assert(t, check[tr.Metadata.TaskID], "task should not be recovered: "+tr.Metadata.TaskID) - } -} - -func TestPodsSortedDuringRecovery(t *testing.T) { - t.Skip("broken") - // conf.GetSchedulerConf().OperatorPlugins = "mocked-app-manager" - amProtocol := NewMockedAMProtocol() - taskRequests := make([]*AddTaskRequest, 0) - amProtocol.UseAddTaskFn(func(request *AddTaskRequest) { - taskRequests = append(taskRequests, request) - }) - apiProvider := client.NewMockedAPIProvider(false) - amService := NewAMService(amProtocol, apiProvider) - _ = &mockedAppManager{} - // amService.register(&mockedAppManager{}) - - _, err := amService.recoverApps() - assert.NilError(t, err) - - assert.Equal(t, 4, len(taskRequests)) - var previous int64 - previous = -1 - for _, req := range taskRequests { - current := req.Metadata.Pod.CreationTimestamp.Unix() - assert.Assert(t, current > previous, "Pods were not processed in sorted order") - previous = current - } -} - -type mockedAppManager struct { -} - -func (ma *mockedAppManager) Name() string { - return "mocked-app-manager" -} - -func (ma *mockedAppManager) ServiceInit() error { - return nil -} - -func (ma *mockedAppManager) Start() error { - return nil -} - -func (ma *mockedAppManager) Stop() { - // noop -} - -func (ma *mockedAppManager) ListPods() ([]*v1.Pod, error) { - pods := make([]*v1.Pod, 8) - pods[0] = ma.newPod("pod1", "task01", "app01", time.Unix(100, 0), v1.PodRunning) - pods[1] = ma.newPod("pod2", "task02", "app01", time.Unix(500, 0), v1.PodPending) - pods[2] = ma.newPod("pod3", "task03", "app01", time.Unix(200, 0), v1.PodSucceeded) - pods[3] = ma.newPod("pod4", "task04", "app02", time.Unix(400, 0), v1.PodRunning) - pods[4] = ma.newPod("pod5", "task05", "app02", time.Unix(300, 0), v1.PodPending) - pods[5] = ma.newPod("pod6", "task06", "app02", time.Unix(600, 0), v1.PodFailed) - - // these pods and apps should never be recovered - pods[6] = ma.newPod("pod7", "task07", "app03", time.Unix(300, 0), v1.PodFailed) - pods[7] = ma.newPod("pod8", "task08", "app04", time.Unix(300, 0), v1.PodSucceeded) - - return pods, nil -} - -func (ma *mockedAppManager) GetExistingAllocation(pod *v1.Pod) *si.Allocation { - return nil -} - -func (ma *mockedAppManager) newPod(name, podUID, appID string, creationTimeStamp time.Time, phase v1.PodPhase) *v1.Pod { - return &v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: name, - Namespace: "yk", - UID: types.UID(podUID), - Annotations: map[string]string{ - constants.AnnotationApplicationID: appID, - }, - CreationTimestamp: apis.NewTime(creationTimeStamp), - }, - Spec: v1.PodSpec{ - NodeName: "fake-node", - SchedulerName: constants.SchedulerName, - }, - Status: v1.PodStatus{ - Phase: phase, - }, - } -} diff --git a/pkg/cache/appmgmt_test.go b/pkg/cache/appmgmt_test.go deleted file mode 100644 index bff387030..000000000 --- a/pkg/cache/appmgmt_test.go +++ /dev/null @@ -1,614 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "testing" - - "gotest.tools/v3/assert" - v1 "k8s.io/api/core/v1" - apis "k8s.io/apimachinery/pkg/apis/meta/v1" - - "github.com/apache/yunikorn-k8shim/pkg/client" - "github.com/apache/yunikorn-k8shim/pkg/common/constants" - "github.com/apache/yunikorn-k8shim/pkg/common/test" - "github.com/apache/yunikorn-k8shim/pkg/common/utils" -) - -func TestAMSvcAddPod(t *testing.T) { - amProtocol := NewMockedAMProtocol() - am := NewAMService(amProtocol, client.NewMockedAPIProvider(false)) - am.podEventHandler.recoveryRunning = false - - pod := v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: "pod00001", - Namespace: "default", - UID: "UID-POD-00001", - Labels: map[string]string{ - "applicationId": "app00001", - "queue": "root.a", - }, - }, - Spec: v1.PodSpec{SchedulerName: constants.SchedulerName}, - Status: v1.PodStatus{ - Phase: v1.PodPending, - }, - } - - // add a pending pod through the AM service - am.AddPod(&pod) - - managedApp := amProtocol.GetApplication("app00001") - assert.Assert(t, managedApp != nil) - app, valid := anyToApplication(managedApp) - assert.Equal(t, valid, true) - assert.Equal(t, app.GetApplicationID(), "app00001") - assert.Equal(t, app.GetApplicationState(), ApplicationStates().New) - assert.Equal(t, app.GetQueue(), "root.a") - assert.Equal(t, len(app.GetNewTasks()), 1) - - task, err := app.GetTask("UID-POD-00001") - assert.Assert(t, err == nil) - assert.Equal(t, task.GetTaskState(), TaskStates().New) - - // add another pod for same application - pod1 := v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: "pod00002", - Namespace: "default", - UID: "UID-POD-00002", - Labels: map[string]string{ - "applicationId": "app00001", - "queue": "root.a", - }, - }, - Spec: v1.PodSpec{SchedulerName: constants.SchedulerName}, - Status: v1.PodStatus{ - Phase: v1.PodPending, - }, - } - - am.AddPod(&pod1) - assert.Equal(t, len(app.GetNewTasks()), 2) - - // add another pod from another app - pod2 := v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: "pod00004", - Namespace: "default", - UID: "UID-POD-00004", - Labels: map[string]string{ - "applicationId": "app00002", - "queue": "root.a", - }, - }, - Spec: v1.PodSpec{SchedulerName: constants.SchedulerName}, - Status: v1.PodStatus{ - Phase: v1.PodPending, - }, - } - - am.AddPod(&pod2) - app02 := amProtocol.GetApplication("app00002") - assert.Assert(t, app02 != nil) - app, valid = anyToApplication(app02) - assert.Equal(t, valid, true) - assert.Equal(t, len(app.GetNewTasks()), 1) - assert.Equal(t, app.GetApplicationID(), "app00002") - assert.Equal(t, app.GetNewTasks()[0].GetTaskPod().Name, "pod00004") -} - -func TestAMSvcOriginatorPod(t *testing.T) { - amProtocol := NewMockedAMProtocol() - am := NewAMService(amProtocol, client.NewMockedAPIProvider(false)) - am.podEventHandler.recoveryRunning = false - - pod := v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: "pod00001", - Namespace: "default", - UID: "UID-POD-00001", - Labels: map[string]string{ - "applicationId": "app00001", - "queue": "root.a", - }, - }, - Spec: v1.PodSpec{SchedulerName: constants.SchedulerName}, - Status: v1.PodStatus{ - Phase: v1.PodPending, - }, - } - - // add pod 2 as owner for pod 1 - owner := apis.OwnerReference{ - APIVersion: "v1", - UID: "UID-POD-00002", - Kind: "Pod", - } - - refer := []apis.OwnerReference{ - owner, - } - pod.SetOwnerReferences(refer) - - // add a pending pod through the AM service - am.AddPod(&pod) - - managedApp := amProtocol.GetApplication("app00001") - assert.Assert(t, managedApp != nil) - app, valid := anyToApplication(managedApp) - assert.Equal(t, valid, true) - assert.Equal(t, len(app.GetNewTasks()), 1) - - task, err := app.GetTask("UID-POD-00001") - assert.Assert(t, err == nil) - assert.Equal(t, task.GetTaskState(), TaskStates().New) - - // add another pod, pod 2 (owner) for same application - pod1 := v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: "pod00002", - Namespace: "default", - UID: "UID-POD-00002", - Labels: map[string]string{ - "applicationId": "app00001", - "queue": "root.a", - }, - }, - Spec: v1.PodSpec{SchedulerName: constants.SchedulerName}, - Status: v1.PodStatus{ - Phase: v1.PodPending, - }, - } - am.AddPod(&pod1) - assert.Equal(t, len(app.GetNewTasks()), 2) - task, err = app.GetTask("UID-POD-00001") - assert.Assert(t, err == nil) - - // app originator task should be pod 1 - // even the pod 2 is the ownerreference for pod 1 - // And pod 1 is first added to the AM service - assert.Equal(t, app.GetOriginatingTask().GetTaskID(), task.GetTaskID()) -} - -func TestAMSvcUpdatePodWhenSucceed(t *testing.T) { - amProtocol := NewMockedAMProtocol() - am := NewAMService(amProtocol, client.NewMockedAPIProvider(false)) - am.podEventHandler.recoveryRunning = false - - pod := v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: "pod00001", - Namespace: "default", - UID: "UID-POD-00001", - Labels: map[string]string{ - "applicationId": "app00001", - "queue": "root.a", - }, - }, - Spec: v1.PodSpec{SchedulerName: constants.SchedulerName}, - Status: v1.PodStatus{ - Phase: v1.PodPending, - }, - } - - // add a pending pod through the AM service - am.AddPod(&pod) - - managedApp := amProtocol.GetApplication("app00001") - assert.Assert(t, managedApp != nil) - app, valid := anyToApplication(managedApp) - assert.Equal(t, valid, true) - assert.Equal(t, app.GetApplicationID(), "app00001") - assert.Equal(t, app.GetApplicationState(), ApplicationStates().New) - assert.Equal(t, app.GetQueue(), "root.a") - assert.Equal(t, len(app.GetNewTasks()), 1) - - task, err := app.GetTask("UID-POD-00001") - assert.Assert(t, err == nil) - assert.Equal(t, task.GetTaskState(), TaskStates().New) - - // try update the pod - - newPod := v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: "pod00001", - Namespace: "default", - UID: "UID-POD-00001", - Labels: map[string]string{ - "applicationId": "app00001", - "queue": "root.a", - }, - }, - Spec: v1.PodSpec{SchedulerName: constants.SchedulerName}, - Status: v1.PodStatus{ - Phase: v1.PodSucceeded, - }, - } - - am.updatePod(&pod, &newPod) - - // this is to verify NotifyTaskComplete is called - assert.Equal(t, task.GetTaskState(), TaskStates().Completed) -} - -func TestAMSvcUpdatePodWhenFailed(t *testing.T) { - amProtocol := NewMockedAMProtocol() - am := NewAMService(amProtocol, client.NewMockedAPIProvider(false)) - am.podEventHandler.recoveryRunning = false - - pod := v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: "pod00001", - Namespace: "default", - UID: "UID-POD-00001", - Labels: map[string]string{ - "applicationId": "app00001", - "queue": "root.a", - }, - }, - Spec: v1.PodSpec{SchedulerName: constants.SchedulerName}, - Status: v1.PodStatus{ - Phase: v1.PodPending, - }, - } - - // add a pending pod through the AM service - am.AddPod(&pod) - - // try update the pod to Failed status - newPod := v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: "pod00001", - Namespace: "default", - UID: "UID-POD-00001", - Labels: map[string]string{ - "applicationId": "app00001", - "queue": "root.a", - }, - }, - Spec: v1.PodSpec{SchedulerName: constants.SchedulerName}, - Status: v1.PodStatus{ - Phase: v1.PodSucceeded, - }, - } - - am.updatePod(&pod, &newPod) - - managedApp := amProtocol.GetApplication("app00001") - assert.Assert(t, managedApp != nil) - app, valid := anyToApplication(managedApp) - assert.Equal(t, valid, true) - task, err := app.GetTask("UID-POD-00001") - assert.Assert(t, err == nil) - // this is to verify NotifyTaskComplete is called - assert.Equal(t, task.GetTaskState(), TaskStates().Completed) -} - -func TestAMSvcDeletePod(t *testing.T) { - amProtocol := NewMockedAMProtocol() - am := NewAMService(amProtocol, client.NewMockedAPIProvider(false)) - am.podEventHandler.recoveryRunning = false - - pod := v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: "pod00001", - Namespace: "default", - UID: "UID-POD-00001", - Labels: map[string]string{ - "applicationId": "app00001", - "queue": "root.a", - }, - }, - Spec: v1.PodSpec{SchedulerName: constants.SchedulerName}, - Status: v1.PodStatus{ - Phase: v1.PodPending, - }, - } - - // add a pending pod through the AM service - am.AddPod(&pod) - - managedApp := amProtocol.GetApplication("app00001") - assert.Assert(t, managedApp != nil) - app, valid := anyToApplication(managedApp) - assert.Equal(t, valid, true) - assert.Equal(t, app.GetApplicationID(), "app00001") - assert.Equal(t, app.GetApplicationState(), ApplicationStates().New) - assert.Equal(t, app.GetQueue(), "root.a") - assert.Equal(t, len(app.GetNewTasks()), 1) - - task, err := app.GetTask("UID-POD-00001") - assert.Assert(t, err == nil) - assert.Equal(t, task.GetTaskState(), TaskStates().New) - - // try delete the pod - am.deletePod(&pod) - - // this is to verify NotifyTaskComplete is called - assert.Equal(t, task.GetTaskState(), TaskStates().Completed) -} - -func anyToApplication(something interface{}) (*Application, bool) { - if app, valid := something.(*Application); valid { - return app, true - } - return nil, false -} - -func TestAMSvcGetExistingAllocation(t *testing.T) { - amProtocol := NewMockedAMProtocol() - am := NewAMService(amProtocol, client.NewMockedAPIProvider(false)) - am.podEventHandler.recoveryRunning = true - - pod := &v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: "pod00001", - Namespace: "default", - UID: "UID-POD-00001", - Labels: map[string]string{ - "applicationId": "app00001", - "queue": "root.a", - }, - }, - Spec: v1.PodSpec{ - SchedulerName: constants.SchedulerName, - NodeName: "allocated-node", - }, - Status: v1.PodStatus{ - Phase: v1.PodPending, - }, - } - - // verifies the existing allocation is correctly returned - alloc := am.GetExistingAllocation(pod) - assert.Equal(t, alloc.ApplicationID, "app00001") - assert.Equal(t, alloc.AllocationKey, string(pod.UID)) - assert.Equal(t, alloc.UUID, string(pod.UID)) - assert.Equal(t, alloc.NodeID, "allocated-node") -} - -type Template struct { - podName string - namespace string - label map[string]string - annotation map[string]string - scheduler bool - node bool -} - -// nolint: funlen -func TestAMSvcListApplication(t *testing.T) { - // mock the pod lister for this test - mockedAPIProvider := client.NewMockedAPIProvider(false) - mockedPodLister := test.NewPodListerMock() - mockedAPIProvider.SetPodLister(mockedPodLister) - appName := []string{"app00001", "app00002", "app00003", "app00004", "app00005"} - var queue01 = "root.queue01" - var ns01 = "namespace01" - var ns02 = "namespace02" - type testcase struct { - description string - applicationID string - input *v1.Pod - expectedOutput bool - } - podCase := []Template{ - // Application 1 - { - podName: "app01pod01", - namespace: ns01, - label: map[string]string{ - constants.LabelApplicationID: appName[0], - constants.LabelQueueName: queue01, - }, - annotation: nil, - scheduler: true, - node: true, - }, - { - podName: "app01pod02", - namespace: ns01, - label: map[string]string{ - constants.LabelApplicationID: appName[0], - constants.LabelQueueName: queue01, - }, - annotation: nil, - scheduler: true, - node: false, - }, - // Application 2 - { - podName: "app02pod01", - namespace: ns02, - label: map[string]string{ - constants.SparkLabelAppID: appName[1], - }, - annotation: nil, - scheduler: true, - node: true, - }, - // Application 3 - { - podName: "app03pod01", - namespace: ns01, - label: nil, - annotation: map[string]string{ - constants.AnnotationApplicationID: appName[2], - }, - scheduler: true, - node: true, - }, - // Application 4 - { - podName: "app04pod01", - namespace: ns01, - label: nil, - annotation: nil, - scheduler: true, - node: true, - }, - // Application 5 - { - podName: "app05pod01", - namespace: ns01, - label: map[string]string{ - constants.SparkLabelAppID: appName[4], - }, - annotation: nil, - scheduler: false, - node: true, - }, - } - listAppTestCase := []testcase{ - // Application 1 - { - description: "running in queue01 and namespace01, with labels, schedulerName, nodeName", - applicationID: appName[0], - input: podCase[0].InjectPod(), - expectedOutput: true, - }, - { - description: "running in queue01 and namespace01, with labels, schedulerName", - applicationID: appName[0], - input: podCase[1].InjectPod(), - expectedOutput: true, - }, - // Application 2 - { - description: "running in default queue and namespace02, with spark labels, schedulerName, and nodeName", - applicationID: appName[1], - input: podCase[2].InjectPod(), - expectedOutput: true, - }, - // Application 3 - { - description: "running in default queue and namespace01, with annotation, schedulerName, and nodeName", - applicationID: appName[2], - input: podCase[3].InjectPod(), - expectedOutput: true, - }, - // Application 4 - { - description: "running in default queue and namespace01, without label and annotation", - applicationID: "yunikorn-namespace01-autogen", - input: podCase[4].InjectPod(), - expectedOutput: true, - }, - // Application 5 - { - description: "running in default queue and namespace01, with label and nodeName", - applicationID: appName[4], - input: podCase[5].InjectPod(), - expectedOutput: false, - }, - } - expectOutput := make(map[string]bool) - descriptionMap := make(map[string]string) - for index := range listAppTestCase { - mockedPodLister.AddPod(listAppTestCase[index].input) - expectOutput[listAppTestCase[index].applicationID] = listAppTestCase[index].expectedOutput - descriptionMap[listAppTestCase[index].applicationID] = listAppTestCase[index].description - } - // init the app manager and run listApp - amProtocol := NewMockedAMProtocol() - am := NewAMService(amProtocol, mockedAPIProvider) - am.podEventHandler.recoveryRunning = true - - pods, err := am.ListPods() - assert.NilError(t, err) - assert.Equal(t, len(pods), 4) - for _, pod := range pods { - name := utils.GetApplicationIDFromPod(pod) - expected := expectOutput[name] - description := descriptionMap[name] - assert.Assert(t, expected, description) - } -} - -func (temp Template) InjectPod() *v1.Pod { - tempPod := &v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: temp.podName, - Namespace: temp.namespace, - }, - } - if temp.label != nil { - tempPod.Labels = temp.label - } - if temp.annotation != nil { - tempPod.Annotations = temp.annotation - } - if temp.scheduler { - tempPod.Spec.SchedulerName = constants.SchedulerName - } - if temp.node { - tempPod.Spec.NodeName = "some-node" - } - return tempPod -} diff --git a/pkg/cache/context.go b/pkg/cache/context.go index 9060aec73..762d853c9 100644 --- a/pkg/cache/context.go +++ b/pkg/cache/context.go @@ -22,13 +22,17 @@ import ( "context" "encoding/json" "fmt" + "sort" "strconv" "strings" "sync" + "sync/atomic" "go.uber.org/zap" v1 "k8s.io/api/core/v1" schedulingv1 "k8s.io/api/scheduling/v1" + "k8s.io/apimachinery/pkg/labels" + "k8s.io/apimachinery/pkg/types" "k8s.io/client-go/tools/cache" "k8s.io/kubernetes/pkg/scheduler/framework" "k8s.io/kubernetes/pkg/scheduler/framework/plugins/volumebinding" @@ -48,10 +52,11 @@ import ( "github.com/apache/yunikorn-scheduler-interface/lib/go/si" ) +const registerNodeContextHandler = "RegisterNodeContextHandler" + // context maintains scheduling state, like apps and apps' tasks. type Context struct { applications map[string]*Application // apps - nodes *schedulerNodes // nodes schedulerCache *schedulercache.SchedulerCache // external cache apiProvider client.APIProvider // apis to interact with api-server, scheduler-core, etc predManager predicates.PredicateManager // K8s predicates @@ -59,6 +64,7 @@ type Context struct { namespace string // yunikorn namespace configMaps []*v1.ConfigMap // cached yunikorn configmaps lock *sync.RWMutex // lock + txnID atomic.Uint64 // transaction ID counter } // NewContext create a new context for the scheduler using a default (empty) configuration @@ -85,9 +91,6 @@ func NewContextWithBootstrapConfigMaps(apis client.APIProvider, bootstrapConfigM // create the cache ctx.schedulerCache = schedulercache.NewSchedulerCache(apis.GetAPIs()) - // init the controllers and plugins (need the cache) - ctx.nodes = newSchedulerNodes(apis.GetAPIs().SchedulerAPI, ctx.schedulerCache) - // create the predicate manager sharedLister := support.NewSharedLister(ctx.schedulerCache) clientSet := apis.GetAPIs().KubeClient.GetClientSet() @@ -98,20 +101,6 @@ func NewContextWithBootstrapConfigMaps(apis client.APIProvider, bootstrapConfigM } func (ctx *Context) AddSchedulingEventHandlers() { - ctx.apiProvider.AddEventHandler(&client.ResourceEventHandlers{ - Type: client.NodeInformerHandlers, - AddFn: ctx.addNode, - UpdateFn: ctx.updateNode, - DeleteFn: ctx.deleteNode, - }) - - ctx.apiProvider.AddEventHandler(&client.ResourceEventHandlers{ - Type: client.PodInformerHandlers, - AddFn: ctx.addPod, - UpdateFn: ctx.updatePod, - DeleteFn: ctx.deletePod, - }) - ctx.apiProvider.AddEventHandler(&client.ResourceEventHandlers{ Type: client.ConfigMapInformerHandlers, FilterFn: ctx.filterConfigMaps, @@ -126,6 +115,18 @@ func (ctx *Context) AddSchedulingEventHandlers() { UpdateFn: ctx.updatePriorityClass, DeleteFn: ctx.deletePriorityClass, }) + ctx.apiProvider.AddEventHandler(&client.ResourceEventHandlers{ + Type: client.NodeInformerHandlers, + AddFn: ctx.addNode, + UpdateFn: ctx.updateNode, + DeleteFn: ctx.deleteNode, + }) + ctx.apiProvider.AddEventHandler(&client.ResourceEventHandlers{ + Type: client.PodInformerHandlers, + AddFn: ctx.AddPod, + UpdateFn: ctx.UpdatePod, + DeleteFn: ctx.DeletePod, + }) } func (ctx *Context) IsPluginMode() bool { @@ -133,57 +134,82 @@ func (ctx *Context) IsPluginMode() bool { } func (ctx *Context) addNode(obj interface{}) { + ctx.updateNode(nil, obj) +} + +func (ctx *Context) updateNode(_, obj interface{}) { ctx.lock.Lock() defer ctx.lock.Unlock() - node, err := convertToNode(obj) if err != nil { log.Log(log.ShimContext).Error("node conversion failed", zap.Error(err)) return } - - // add node to secondary scheduler cache - log.Log(log.ShimContext).Warn("adding node to cache", zap.String("NodeName", node.Name)) - ctx.schedulerCache.AddNode(node) - - // add node to internal cache - ctx.nodes.addNode(node) - - // post the event - events.GetRecorder().Eventf(node.DeepCopy(), nil, v1.EventTypeNormal, "NodeAccepted", "NodeAccepted", - fmt.Sprintf("node %s is accepted by the scheduler", node.Name)) + ctx.updateNodeInternal(node, true) } -func (ctx *Context) updateNode(oldObj, newObj interface{}) { - ctx.lock.Lock() - defer ctx.lock.Unlock() +func (ctx *Context) updateNodeInternal(node *v1.Node, register bool) { + // update scheduler cache + if prevNode, adoptedPods := ctx.schedulerCache.UpdateNode(node); prevNode == nil { + // newly added node + + // if requested, register this node with the scheduler core. this is optional to allow for bulk registration + // during scheduler initialization. + if register { + if err := ctx.registerNode(node); err != nil { + // remove from secondary cache and return + log.Log(log.ShimContext).Error("node registration failed", zap.Error(err)) + ctx.schedulerCache.RemoveNode(node) + return + } + } - // we only trigger update when resource changes - oldNode, err := convertToNode(oldObj) - if err != nil { - log.Log(log.ShimContext).Error("old node conversion failed", - zap.Error(err)) - return - } + // iterate newly adopted pods and register them with the scheduler + for _, pod := range adoptedPods { + applicationID := utils.GetApplicationIDFromPod(pod) + if applicationID == "" { + ctx.updateForeignPod(pod) + } else { + ctx.updateYuniKornPod(pod) + } + } - newNode, err := convertToNode(newObj) - if err != nil { - log.Log(log.ShimContext).Error("new node conversion failed", - zap.Error(err)) - return + // if node was registered in-line, enable it in the core + if err := ctx.enableNode(node); err != nil { + log.Log(log.ShimContext).Warn("Failed to enable node", zap.Error(err)) + } + } else { + // existing node + prevCapacity := common.GetNodeResource(&prevNode.Status) + newCapacity := common.GetNodeResource(&node.Status) + prevReady := hasReadyCondition(prevNode) + newReady := hasReadyCondition(node) + + if !common.Equals(prevCapacity, newCapacity) { + // update capacity + if capacity, occupied, ok := ctx.schedulerCache.UpdateCapacity(node.Name, newCapacity); ok { + if err := ctx.updateNodeResources(node, capacity, occupied, newReady); err != nil { + log.Log(log.ShimContext).Warn("Failed to update node capacity", zap.Error(err)) + } + } else { + log.Log(log.ShimContext).Warn("Failed to update cached node capacity", zap.String("nodeName", node.Name)) + } + } else if newReady != prevReady { + // update readiness + if capacity, occupied, ok := ctx.schedulerCache.SnapshotResources(node.Name); ok { + if err := ctx.updateNodeResources(node, capacity, occupied, newReady); err != nil { + log.Log(log.ShimContext).Warn("Failed to update node readiness", zap.Error(err)) + } + } else { + log.Log(log.ShimContext).Warn("Failed to snapshot cached node capacity", zap.String("nodeName", node.Name)) + } + } } - - // update secondary cache - ctx.schedulerCache.UpdateNode(newNode) - - // update primary cache - ctx.nodes.updateNode(oldNode, newNode) } func (ctx *Context) deleteNode(obj interface{}) { ctx.lock.Lock() defer ctx.lock.Unlock() - var node *v1.Node switch t := obj.(type) { case *v1.Node: @@ -199,33 +225,51 @@ func (ctx *Context) deleteNode(obj interface{}) { log.Log(log.ShimContext).Error("cannot convert to *v1.Node", zap.Any("object", t)) return } + ctx.deleteNodeInternal(node) +} + +func (ctx *Context) addNodesWithoutRegistering(nodes []*v1.Node) { + ctx.lock.Lock() + defer ctx.lock.Unlock() + + for _, node := range nodes { + ctx.updateNodeInternal(node, false) + } +} + +func (ctx *Context) deleteNodeInternal(node *v1.Node) { + // remove node from scheduler cache + prevNode, orphanedPods := ctx.schedulerCache.RemoveNode(node) + if prevNode == nil { + // nothing to do if node wasn't there + return + } - // delete node from secondary cache - log.Log(log.ShimContext).Debug("delete node from cache", zap.String("nodeName", node.Name)) - ctx.schedulerCache.RemoveNode(node) + // log the number of orphaned pods, but we shouldn't need to do any processing of them as the core will send + // back remove events for each of them + log.Log(log.ShimContext).Info("Removing node", + zap.String("nodeName", node.Name), + zap.Int("assignedPods", len(orphanedPods))) - // delete node from primary cache - ctx.nodes.deleteNode(node) + // decommission node + log.Log(log.ShimContext).Info("Decommissioning node", zap.String("nodeName", node.Name)) + if err := ctx.decommissionNode(node); err != nil { + log.Log(log.ShimContext).Warn("Unable to decommission node", zap.Error(err)) + } // post the event events.GetRecorder().Eventf(node.DeepCopy(), nil, v1.EventTypeNormal, "NodeDeleted", "NodeDeleted", fmt.Sprintf("node %s is deleted from the scheduler", node.Name)) } -func (ctx *Context) addPod(obj interface{}) { - pod, err := utils.Convert2Pod(obj) - if err != nil { - log.Log(log.ShimContext).Error("failed to add pod", zap.Error(err)) - return - } - if utils.GetApplicationIDFromPod(pod) == "" { - ctx.updateForeignPod(pod) - } else { - ctx.updateYuniKornPod(pod) - } +func (ctx *Context) AddPod(obj interface{}) { + ctx.UpdatePod(nil, obj) } -func (ctx *Context) updatePod(_, newObj interface{}) { +func (ctx *Context) UpdatePod(_, newObj interface{}) { + ctx.lock.Lock() + defer ctx.lock.Unlock() + pod, err := utils.Convert2Pod(newObj) if err != nil { log.Log(log.ShimContext).Error("failed to update pod", zap.Error(err)) @@ -239,22 +283,81 @@ func (ctx *Context) updatePod(_, newObj interface{}) { } func (ctx *Context) updateYuniKornPod(pod *v1.Pod) { - ctx.lock.Lock() - defer ctx.lock.Unlock() - // treat terminated pods like a remove if utils.IsPodTerminated(pod) { + if taskMeta, ok := getTaskMetadata(pod); ok { + if app := ctx.getApplication(taskMeta.ApplicationID); app != nil { + ctx.notifyTaskComplete(taskMeta.ApplicationID, taskMeta.TaskID) + } + } + log.Log(log.ShimContext).Debug("Request to update terminated pod, removing from cache", zap.String("podName", pod.Name)) ctx.schedulerCache.RemovePod(pod) return } - ctx.schedulerCache.UpdatePod(pod) + + if ctx.schedulerCache.UpdatePod(pod) { + // pod was accepted; ensure the application and task objects have been created + ctx.ensureAppAndTaskCreated(pod) + } } -func (ctx *Context) updateForeignPod(pod *v1.Pod) { - ctx.lock.Lock() - defer ctx.lock.Unlock() +func (ctx *Context) ensureAppAndTaskCreated(pod *v1.Pod) { + appExists := false + + // get app metadata + appMeta, ok := getAppMetadata(pod) + if !ok { + log.Log(log.ShimContext).Warn("BUG: Unable to retrieve application metadata from YuniKorn-managed Pod", + zap.String("namespace", pod.Namespace), + zap.String("name", pod.Name)) + return + } + + // add app if it doesn't already exist + app := ctx.getApplication(appMeta.ApplicationID) + if app == nil { + app = ctx.addApplication(&AddApplicationRequest{ + Metadata: appMeta, + }) + } else { + appExists = true + } + + // get task metadata + taskMeta, ok := getTaskMetadata(pod) + if !ok { + log.Log(log.ShimContext).Warn("BUG: Unable to retrieve task metadata from YuniKorn-managed Pod", + zap.String("namespace", pod.Namespace), + zap.String("name", pod.Name)) + return + } + + // add task if it doesn't already exist + if _, taskErr := app.GetTask(string(pod.UID)); taskErr != nil { + ctx.addTask(&AddTaskRequest{ + Metadata: taskMeta, + }) + } + // if app was newly created, submit it to the core + if !appExists { + // the only real distinction between app submission and app recovery is whether the task is assigned to a node + if utils.IsAssignedPod(pod) { + err := app.TriggerAppSubmission() + if err != nil { + log.Log(log.ShimContext).Error("app failed submission", zap.Error(err)) + } + } else { + err := app.TriggerAppRecovery() + if err != nil { + log.Log(log.ShimContext).Error("app failed recovery", zap.Error(err)) + } + } + } +} + +func (ctx *Context) updateForeignPod(pod *v1.Pod) { podStatusBefore := "" oldPod, ok := ctx.schedulerCache.GetPod(string(pod.UID)) if ok { @@ -265,37 +368,53 @@ func (ctx *Context) updateForeignPod(pod *v1.Pod) { // 1. pod was previously assigned // 2. pod is now assigned // 3. pod is not in terminated state + // 4. pod references a known node if oldPod == nil && utils.IsAssignedPod(pod) && !utils.IsPodTerminated(pod) { - log.Log(log.ShimContext).Debug("pod is assigned to a node, trigger occupied resource update", - zap.String("namespace", pod.Namespace), - zap.String("podName", pod.Name), - zap.String("podStatusBefore", podStatusBefore), - zap.String("podStatusCurrent", string(pod.Status.Phase))) - podResource := common.GetPodResource(pod) - ctx.nodes.updateNodeOccupiedResources(pod.Spec.NodeName, podResource, AddOccupiedResource) - ctx.schedulerCache.AddPod(pod) + if ctx.schedulerCache.UpdatePod(pod) { + // pod was accepted by a real node + log.Log(log.ShimContext).Debug("pod is assigned to a node, trigger occupied resource update", + zap.String("namespace", pod.Namespace), + zap.String("podName", pod.Name), + zap.String("podStatusBefore", podStatusBefore), + zap.String("podStatusCurrent", string(pod.Status.Phase))) + ctx.updateNodeOccupiedResources(pod.Spec.NodeName, pod.Namespace, pod.Name, common.GetPodResource(pod), schedulercache.AddOccupiedResource) + } else { + // pod is orphaned (references an unknown node) + log.Log(log.ShimContext).Info("skipping occupied resource update for assigned orphaned pod", + zap.String("namespace", pod.Namespace), + zap.String("podName", pod.Name), + zap.String("nodeName", pod.Spec.NodeName)) + } return } // conditions for release: // 1. pod was previously assigned // 2. pod is now in a terminated state + // 3. pod references a known node if oldPod != nil && utils.IsPodTerminated(pod) { - log.Log(log.ShimContext).Debug("pod terminated, trigger occupied resource update", - zap.String("namespace", pod.Namespace), - zap.String("podName", pod.Name), - zap.String("podStatusBefore", podStatusBefore), - zap.String("podStatusCurrent", string(pod.Status.Phase))) - // this means pod is terminated - // we need sub the occupied resource and re-sync with the scheduler-core - podResource := common.GetPodResource(pod) - ctx.nodes.updateNodeOccupiedResources(pod.Spec.NodeName, podResource, SubOccupiedResource) - ctx.schedulerCache.RemovePod(pod) + if !ctx.schedulerCache.IsPodOrphaned(string(pod.UID)) { + log.Log(log.ShimContext).Debug("pod terminated, trigger occupied resource update", + zap.String("namespace", pod.Namespace), + zap.String("podName", pod.Name), + zap.String("podStatusBefore", podStatusBefore), + zap.String("podStatusCurrent", string(pod.Status.Phase))) + // this means pod is terminated + // we need sub the occupied resource and re-sync with the scheduler-core + ctx.updateNodeOccupiedResources(pod.Spec.NodeName, pod.Namespace, pod.Name, common.GetPodResource(pod), schedulercache.SubOccupiedResource) + ctx.schedulerCache.RemovePod(pod) + } else { + // pod is orphaned (references an unknown node) + log.Log(log.ShimContext).Info("skipping occupied resource update for terminated orphaned pod", + zap.String("namespace", pod.Namespace), + zap.String("podName", pod.Name), + zap.String("nodeName", pod.Spec.NodeName)) + } return } } -func (ctx *Context) deletePod(obj interface{}) { +func (ctx *Context) DeletePod(obj interface{}) { var pod *v1.Pod switch t := obj.(type) { case *v1.Pod: @@ -323,6 +442,12 @@ func (ctx *Context) deleteYuniKornPod(pod *v1.Pod) { ctx.lock.Lock() defer ctx.lock.Unlock() + if taskMeta, ok := getTaskMetadata(pod); ok { + if app := ctx.getApplication(taskMeta.ApplicationID); app != nil { + ctx.notifyTaskComplete(taskMeta.ApplicationID, taskMeta.TaskID) + } + } + log.Log(log.ShimContext).Debug("removing pod from cache", zap.String("podName", pod.Name)) ctx.schedulerCache.RemovePod(pod) } @@ -342,20 +467,42 @@ func (ctx *Context) deleteForeignPod(pod *v1.Pod) { // conditions for release: // 1. pod is already assigned to a node - if oldPod != nil { - log.Log(log.ShimContext).Debug("foreign pod deleted, triggering occupied resource update", - zap.String("namespace", pod.Namespace), - zap.String("podName", pod.Name), - zap.String("podStatusBefore", string(oldPod.Status.Phase)), - zap.String("podStatusCurrent", string(pod.Status.Phase))) - // this means pod is terminated - // we need sub the occupied resource and re-sync with the scheduler-core - podResource := common.GetPodResource(pod) - ctx.nodes.updateNodeOccupiedResources(pod.Spec.NodeName, podResource, SubOccupiedResource) + // 2. pod was not in a terminal state before + // 3. pod references a known node + if oldPod != nil && !utils.IsPodTerminated(oldPod) { + if !ctx.schedulerCache.IsPodOrphaned(string(oldPod.UID)) { + log.Log(log.ShimContext).Debug("foreign pod deleted, triggering occupied resource update", + zap.String("namespace", pod.Namespace), + zap.String("podName", pod.Name), + zap.String("podStatusBefore", string(oldPod.Status.Phase)), + zap.String("podStatusCurrent", string(pod.Status.Phase))) + // this means pod is terminated + // we need sub the occupied resource and re-sync with the scheduler-core + ctx.updateNodeOccupiedResources(pod.Spec.NodeName, pod.Namespace, pod.Name, common.GetPodResource(pod), schedulercache.SubOccupiedResource) + } else { + // pod is orphaned (references an unknown node) + log.Log(log.ShimContext).Info("skipping occupied resource update for removed orphaned pod", + zap.String("namespace", pod.Namespace), + zap.String("podName", pod.Name), + zap.String("nodeName", pod.Spec.NodeName)) + } ctx.schedulerCache.RemovePod(pod) } } +func (ctx *Context) updateNodeOccupiedResources(nodeName string, namespace string, podName string, resource *si.Resource, opt schedulercache.UpdateType) { + if common.IsZero(resource) { + return + } + if node, capacity, occupied, ok := ctx.schedulerCache.UpdateOccupiedResource(nodeName, namespace, podName, resource, opt); ok { + if err := ctx.updateNodeResources(node, capacity, occupied, hasReadyCondition(node)); err != nil { + log.Log(log.ShimContext).Warn("scheduler rejected update to node occupied resources", zap.Error(err)) + } + } else { + log.Log(log.ShimContext).Warn("unable to update occupied resources for node", zap.String("nodeName", nodeName)) + } +} + // filter configMap for the scheduler func (ctx *Context) filterConfigMaps(obj interface{}) bool { switch obj := obj.(type) { @@ -435,27 +582,21 @@ func (ctx *Context) filterPriorityClasses(obj interface{}) bool { } func (ctx *Context) addPriorityClass(obj interface{}) { - ctx.lock.Lock() - defer ctx.lock.Unlock() - - log.Log(log.ShimContext).Debug("priority class added") - priorityClass := utils.Convert2PriorityClass(obj) - if priorityClass != nil { - ctx.schedulerCache.AddPriorityClass(priorityClass) - } + ctx.updatePriorityClass(nil, obj) } func (ctx *Context) updatePriorityClass(_, newObj interface{}) { ctx.lock.Lock() defer ctx.lock.Unlock() - - log.Log(log.ShimContext).Debug("priority class updated") - priorityClass := utils.Convert2PriorityClass(newObj) - if priorityClass != nil { - ctx.schedulerCache.UpdatePriorityClass(priorityClass) + if priorityClass := utils.Convert2PriorityClass(newObj); priorityClass != nil { + ctx.updatePriorityClassInternal(priorityClass) } } +func (ctx *Context) updatePriorityClassInternal(priorityClass *schedulingv1.PriorityClass) { + ctx.schedulerCache.UpdatePriorityClass(priorityClass) +} + func (ctx *Context) deletePriorityClass(obj interface{}) { ctx.lock.Lock() defer ctx.lock.Unlock() @@ -779,10 +920,16 @@ func (ctx *Context) NotifyApplicationFail(appID string) { } func (ctx *Context) NotifyTaskComplete(appID, taskID string) { + ctx.lock.Lock() + defer ctx.lock.Unlock() + ctx.notifyTaskComplete(appID, taskID) +} + +func (ctx *Context) notifyTaskComplete(appID, taskID string) { log.Log(log.ShimContext).Debug("NotifyTaskComplete", zap.String("appID", appID), zap.String("taskID", taskID)) - if app := ctx.GetApplication(appID); app != nil { + if app := ctx.getApplication(appID); app != nil { log.Log(log.ShimContext).Debug("release allocation", zap.String("appID", appID), zap.String("taskID", taskID)) @@ -846,14 +993,18 @@ func (ctx *Context) getNamespaceObject(namespace string) *v1.Namespace { } func (ctx *Context) AddApplication(request *AddApplicationRequest) *Application { + ctx.lock.Lock() + defer ctx.lock.Unlock() + + return ctx.addApplication(request) +} + +func (ctx *Context) addApplication(request *AddApplicationRequest) *Application { log.Log(log.ShimContext).Debug("AddApplication", zap.Any("Request", request)) - if app := ctx.GetApplication(request.Metadata.ApplicationID); app != nil { + if app := ctx.getApplication(request.Metadata.ApplicationID); app != nil { return app } - ctx.lock.Lock() - defer ctx.lock.Unlock() - if ns, ok := request.Metadata.Tags[constants.AppTagNamespace]; ok { log.Log(log.ShimContext).Debug("app namespace info", zap.String("appID", request.Metadata.ApplicationID), @@ -950,10 +1101,16 @@ func (ctx *Context) RemoveApplicationInternal(appID string) { // this implements ApplicationManagementProtocol func (ctx *Context) AddTask(request *AddTaskRequest) *Task { + ctx.lock.Lock() + defer ctx.lock.Unlock() + return ctx.addTask(request) +} + +func (ctx *Context) addTask(request *AddTaskRequest) *Task { log.Log(log.ShimContext).Debug("AddTask", zap.String("appID", request.Metadata.ApplicationID), zap.String("taskID", request.Metadata.TaskID)) - if app := ctx.GetApplication(request.Metadata.ApplicationID); app != nil { + if app := ctx.getApplication(request.Metadata.ApplicationID); app != nil { existingTask, err := app.GetTask(request.Metadata.TaskID) if err != nil { var originator bool @@ -1070,7 +1227,7 @@ func (ctx *Context) PublishEvents(eventRecords []*si.EventRecord) { continue } events.GetRecorder().Eventf(node.DeepCopy(), nil, - v1.EventTypeNormal, "", "", record.Message) + v1.EventTypeNormal, "Informational", "Informational", record.Message) } } } @@ -1180,14 +1337,6 @@ func (ctx *Context) TaskEventHandler() func(obj interface{}) { } } -func (ctx *Context) SchedulerNodeEventHandler() func(obj interface{}) { - if ctx != nil && ctx.nodes != nil { - return ctx.nodes.schedulerNodeEventHandler() - } - // this is not required in some tests - return nil -} - func (ctx *Context) LoadConfigMaps() ([]*v1.ConfigMap, error) { kubeClient := ctx.apiProvider.GetAPIs().KubeClient @@ -1233,3 +1382,395 @@ func isPublishableNodeEvent(event *si.EventRecord) bool { func (ctx *Context) GetSchedulerCache() *schedulercache.SchedulerCache { return ctx.schedulerCache } + +// InitializeState is used to initialize the state of the scheduler context using the Kubernetes informers. +// This registers priority classes, nodes, and pods and ensures the scheduler core is synchronized. +func (ctx *Context) InitializeState() error { + // Step 1: Register priority classes. This is first so that we can rely on the information they + // provide to properly register tasks with correct priority and preemption metadata. + priorityClasses, err := ctx.registerPriorityClasses() + if err != nil { + log.Log(log.ShimContext).Error("failed to register priority classes", zap.Error(err)) + return err + } + + // Step 2: Register nodes. Nodes are registered with the scheduler core in an initially disabled state. + // This allows the existing allocations for each node to be processed before activating the node. + nodes, err := ctx.loadNodes() + if err != nil { + log.Log(log.ShimContext).Error("failed to load nodes", zap.Error(err)) + return err + } + acceptedNodes, err := ctx.registerNodes(nodes) + if err != nil { + log.Log(log.ShimContext).Error("failed to register nodes", zap.Error(err)) + return err + } + ctx.addNodesWithoutRegistering(acceptedNodes) + + // Step 3: Register pods. Pods are handled in creation order to provide consistency with previous scheduler runs. + // If pods are associated with existing nodes, they are treated as allocations (rather than asks). + pods, err := ctx.registerPods() + if err != nil { + log.Log(log.ShimContext).Error("failed to register pods", zap.Error(err)) + return err + } + + // Step 4: Enable nodes. At this point all allocations and asks have been processed, so it is safe to allow the + // core to begin scheduling. + err = ctx.enableNodes(acceptedNodes) + if err != nil { + log.Log(log.ShimContext).Error("failed to enable nodes", zap.Error(err)) + return err + } + + // Step 5: Start scheduling event handlers. At this point, initialization is mostly complete, and any existing + // objects will show up as newly added objects. Since the add/update event handlers are idempotent, this is fine. + ctx.AddSchedulingEventHandlers() + + // Step 6: Finalize priority classes. Between the start of initialization and when the informer event handlers are + // registered, it is possible that a priority class object was deleted. Process them again and remove + // any that no longer exist. + err = ctx.finalizePriorityClasses(priorityClasses) + if err != nil { + log.Log(log.ShimContext).Error("failed to finalize priority classes", zap.Error(err)) + return err + } + + // Step 7: Finalize nodes. Between the start of initialization and when the informer event handlers are registered, + // it is possible that a node object was deleted. Process them again and remove any that no longer exist. + err = ctx.finalizeNodes(nodes) + if err != nil { + log.Log(log.ShimContext).Error("failed to finalize nodes", zap.Error(err)) + return err + } + + // Step 8: Finalize pods. Between the start of initialization and when the informer event handlers are registered, + // it is possible that a pod object was deleted. Process them again and remove any that no longer exist. + err = ctx.finalizePods(pods) + if err != nil { + log.Log(log.ShimContext).Error("failed to finalize pods", zap.Error(err)) + return err + } + + return nil +} + +func (ctx *Context) registerPriorityClasses() ([]*schedulingv1.PriorityClass, error) { + // list all priority classes via the informer + priorityClasses, err := ctx.apiProvider.GetAPIs().PriorityClassInformer.Lister().List(labels.Everything()) + if err != nil { + log.Log(log.ShimContext).Error("Failed to read priority classes from informer", zap.Error(err)) + return nil, err + } + for _, priorityClass := range priorityClasses { + ctx.schedulerCache.UpdatePriorityClass(priorityClass) + } + return priorityClasses, nil +} + +func (ctx *Context) finalizePriorityClasses(existingPriorityClasses []*schedulingv1.PriorityClass) error { + // list all priority classes via the informer + priorityClasses, err := ctx.apiProvider.GetAPIs().PriorityClassInformer.Lister().List(labels.Everything()) + if err != nil { + log.Log(log.ShimContext).Error("Failed to read priority classes from informer", zap.Error(err)) + return err + } + + // convert the priority class list into a map + pcMap := make(map[string]*schedulingv1.PriorityClass) + for _, priorityClass := range priorityClasses { + pcMap[priorityClass.Name] = priorityClass + } + + // find any existing priority classes that no longer exist + for _, priorityClass := range existingPriorityClasses { + if _, ok := pcMap[priorityClass.Name]; !ok { + // priority class no longer exists, delete it + log.Log(log.ShimContext).Info("Removing priority class which went away during initialization", + zap.String("name", priorityClass.Name)) + ctx.deletePriorityClass(priorityClass) + } + } + + return nil +} + +func (ctx *Context) loadNodes() ([]*v1.Node, error) { + // list all nodes via the informer + nodes, err := ctx.apiProvider.GetAPIs().NodeInformer.Lister().List(labels.Everything()) + if err != nil { + log.Log(log.ShimContext).Error("Failed to read nodes from informer", zap.Error(err)) + return nil, err + } + return nodes, err +} + +func (ctx *Context) registerNode(node *v1.Node) error { + acceptedNodes, err := ctx.registerNodes([]*v1.Node{node}) + if err != nil { + return err + } + if len(acceptedNodes) != 1 { + return fmt.Errorf("node rejected: %s", node.Name) + } + return nil +} + +func (ctx *Context) registerNodes(nodes []*v1.Node) ([]*v1.Node, error) { + nodesToRegister := make([]*si.NodeInfo, 0) + pendingNodes := make(map[string]*v1.Node) + acceptedNodes := make([]*v1.Node, 0) + rejectedNodes := make([]*v1.Node, 0) + + // Generate a NodeInfo object for each node and add to the registration request + for _, node := range nodes { + log.Log(log.ShimContext).Info("Registering node", zap.String("name", node.Name)) + nodeStatus := node.Status + nodesToRegister = append(nodesToRegister, &si.NodeInfo{ + NodeID: node.Name, + Action: si.NodeInfo_CREATE_DRAIN, + Attributes: map[string]string{ + constants.DefaultNodeAttributeHostNameKey: node.Name, + constants.DefaultNodeAttributeRackNameKey: constants.DefaultRackName, + siCommon.NodeReadyAttribute: strconv.FormatBool(hasReadyCondition(node)), + }, + SchedulableResource: common.GetNodeResource(&nodeStatus), + OccupiedResource: common.NewResourceBuilder().Build(), + ExistingAllocations: make([]*si.Allocation, 0), + }) + pendingNodes[node.Name] = node + } + + var wg sync.WaitGroup + + // initialize wait group with the number of responses we expect + wg.Add(len(pendingNodes)) + + // register with the dispatcher so that we can track our response + handlerID := fmt.Sprintf("%s-%d", registerNodeContextHandler, ctx.txnID.Load()) + dispatcher.RegisterEventHandler(handlerID, dispatcher.EventTypeNode, func(event interface{}) { + nodeEvent, ok := event.(CachedSchedulerNodeEvent) + if !ok { + return + } + node, ok := pendingNodes[nodeEvent.NodeID] + if !ok { + return + } + delete(pendingNodes, nodeEvent.NodeID) + + switch nodeEvent.Event { + case NodeAccepted: + log.Log(log.ShimContext).Info("Node registration accepted", zap.String("name", nodeEvent.NodeID)) + acceptedNodes = append(acceptedNodes, node) + case NodeRejected: + log.Log(log.ShimContext).Warn("Node registration rejected", zap.String("name", nodeEvent.NodeID)) + rejectedNodes = append(rejectedNodes, node) + default: + log.Log(log.ShimContext).Error("BUG: Unexpected node event", zap.Stringer("eventType", nodeEvent.Event)) + } + wg.Done() + }) + defer dispatcher.UnregisterEventHandler(handlerID, dispatcher.EventTypeNode) + + if err := ctx.apiProvider.GetAPIs().SchedulerAPI.UpdateNode(&si.NodeRequest{ + Nodes: nodesToRegister, + RmID: schedulerconf.GetSchedulerConf().ClusterID, + }); err != nil { + log.Log(log.ShimContext).Error("Failed to register nodes", zap.Error(err)) + return nil, err + } + + // wait for all responses to accumulate + wg.Wait() + + for _, node := range acceptedNodes { + // post a successful event to the node + events.GetRecorder().Eventf(node.DeepCopy(), nil, v1.EventTypeNormal, "NodeAccepted", "NodeAccepted", + fmt.Sprintf("node %s is accepted by the scheduler", node.Name)) + } + for _, node := range rejectedNodes { + // post a failure event to the node + events.GetRecorder().Eventf(node.DeepCopy(), nil, v1.EventTypeWarning, "NodeRejected", "NodeRejected", + fmt.Sprintf("node %s is rejected by the scheduler", node.Name)) + } + + return acceptedNodes, nil +} + +func (ctx *Context) decommissionNode(node *v1.Node) error { + request := common.CreateUpdateRequestForDeleteOrRestoreNode(node.Name, si.NodeInfo_DECOMISSION) + return ctx.apiProvider.GetAPIs().SchedulerAPI.UpdateNode(request) +} + +func (ctx *Context) updateNodeResources(node *v1.Node, capacity *si.Resource, occupied *si.Resource, ready bool) error { + request := common.CreateUpdateRequestForUpdatedNode(node.Name, capacity, occupied, ready) + return ctx.apiProvider.GetAPIs().SchedulerAPI.UpdateNode(request) +} + +func (ctx *Context) enableNode(node *v1.Node) error { + return ctx.enableNodes([]*v1.Node{node}) +} + +func (ctx *Context) enableNodes(nodes []*v1.Node) error { + nodesToEnable := make([]*si.NodeInfo, 0) + + // Generate a NodeInfo object for each node and add to the enablement request + for _, node := range nodes { + log.Log(log.ShimContext).Info("Enabling node", zap.String("name", node.Name)) + nodesToEnable = append(nodesToEnable, &si.NodeInfo{ + NodeID: node.Name, + Action: si.NodeInfo_DRAIN_TO_SCHEDULABLE, + Attributes: map[string]string{ + siCommon.NodeReadyAttribute: strconv.FormatBool(hasReadyCondition(node)), + }, + }) + } + + // enable scheduling on all nodes + if err := ctx.apiProvider.GetAPIs().SchedulerAPI.UpdateNode(&si.NodeRequest{ + Nodes: nodesToEnable, + RmID: schedulerconf.GetSchedulerConf().ClusterID, + }); err != nil { + log.Log(log.ShimContext).Error("Failed to enable nodes", zap.Error(err)) + return err + } + return nil +} + +func (ctx *Context) finalizeNodes(existingNodes []*v1.Node) error { + // list all nodes via the informer + nodes, err := ctx.apiProvider.GetAPIs().NodeInformer.Lister().List(labels.Everything()) + if err != nil { + log.Log(log.ShimContext).Error("Failed to read nodes from informer", zap.Error(err)) + return err + } + + // convert the node list into a map + nodeMap := make(map[string]*v1.Node) + for _, node := range nodes { + nodeMap[node.Name] = node + } + + ctx.lock.Lock() + defer ctx.lock.Unlock() + + // find any existing nodes that no longer exist + for _, node := range existingNodes { + if _, ok := nodeMap[node.Name]; !ok { + // node no longer exists, delete it + log.Log(log.ShimContext).Info("Removing node which went away during initialization", + zap.String("name", node.Name)) + ctx.deleteNodeInternal(node) + } + } + + return nil +} + +func (ctx *Context) registerPods() ([]*v1.Pod, error) { + log.Log(log.ShimContext).Info("Starting node registration...") + + // list all pods via the informer + pods, err := ctx.apiProvider.GetAPIs().PodInformer.Lister().List(labels.Everything()) + if err != nil { + log.Log(log.ShimContext).Error("Failed to read pods from informer", zap.Error(err)) + return nil, err + } + + // sort pods by creation time so that overall queue ordering is consistent with prior runs + sort.Slice(pods, func(i, j int) bool { + return pods[i].CreationTimestamp.Unix() < pods[j].CreationTimestamp.Unix() + }) + + // add all pods to the context + for _, pod := range pods { + // skip terminated pods + if utils.IsPodTerminated(pod) { + continue + } + ctx.AddPod(pod) + } + + return pods, nil +} + +func (ctx *Context) finalizePods(existingPods []*v1.Pod) error { + // list all pods via the informer + pods, err := ctx.apiProvider.GetAPIs().PodInformer.Lister().List(labels.Everything()) + if err != nil { + log.Log(log.ShimContext).Error("Failed to read pods from informer", zap.Error(err)) + return err + } + + // convert the pod list into a map + podMap := make(map[types.UID]*v1.Pod) + for _, pod := range pods { + podMap[pod.UID] = pod + } + + // find any existing nodes that no longer exist + for _, pod := range existingPods { + if _, ok := podMap[pod.UID]; !ok { + // node no longer exists, delete it + log.Log(log.ShimContext).Info("Removing pod which went away during initialization", + zap.String("namespace", pod.Namespace), + zap.String("name", pod.Name), + zap.String("uid", string(pod.UID))) + ctx.DeletePod(pod) + } + } + + return nil +} + +// for a given pod, return an allocation if found +func getExistingAllocation(pod *v1.Pod) *si.Allocation { + // skip terminated pods + if utils.IsPodTerminated(pod) { + return nil + } + + if meta, valid := getAppMetadata(pod); valid { + // when submit a task, we use pod UID as the allocationKey, + // to keep consistent, during recovery, the pod UID is also used + // for an Allocation. + placeholder := utils.GetPlaceholderFlagFromPodSpec(pod) + taskGroupName := utils.GetTaskGroupFromPodSpec(pod) + + creationTime := pod.CreationTimestamp.Unix() + meta.Tags[siCommon.CreationTime] = strconv.FormatInt(creationTime, 10) + + return &si.Allocation{ + AllocationKey: string(pod.UID), + AllocationTags: meta.Tags, + UUID: string(pod.UID), + ResourcePerAlloc: common.GetPodResource(pod), + NodeID: pod.Spec.NodeName, + ApplicationID: meta.ApplicationID, + Placeholder: placeholder, + TaskGroupName: taskGroupName, + PartitionName: constants.DefaultPartition, + } + } + return nil +} + +func convertToNode(obj interface{}) (*v1.Node, error) { + if node, ok := obj.(*v1.Node); ok { + return node, nil + } + return nil, fmt.Errorf("cannot convert to *v1.Node: %v", obj) +} + +func hasReadyCondition(node *v1.Node) bool { + if node != nil { + for _, condition := range node.Status.Conditions { + if condition.Type == v1.NodeReady && condition.Status == v1.ConditionTrue { + return true + } + } + } + return false +} diff --git a/pkg/cache/context_recovery.go b/pkg/cache/context_recovery.go deleted file mode 100644 index 996a24938..000000000 --- a/pkg/cache/context_recovery.go +++ /dev/null @@ -1,215 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "fmt" - "time" - - "go.uber.org/zap" - corev1 "k8s.io/api/core/v1" - "k8s.io/apimachinery/pkg/labels" - - "github.com/apache/yunikorn-k8shim/pkg/client" - "github.com/apache/yunikorn-k8shim/pkg/common" - "github.com/apache/yunikorn-k8shim/pkg/common/utils" - "github.com/apache/yunikorn-k8shim/pkg/dispatcher" - "github.com/apache/yunikorn-k8shim/pkg/log" - "github.com/apache/yunikorn-scheduler-interface/lib/go/si" -) - -func (ctx *Context) WaitForRecovery(mgr *AppManagementService, maxTimeout time.Duration) error { - if err := ctx.recover(mgr, maxTimeout); err != nil { - log.Log(log.ShimContext).Error("nodes recovery failed", zap.Error(err)) - return err - } - - return nil -} - -// for a given pod, return an allocation if found -func getExistingAllocation(mgr *AppManagementService, pod *corev1.Pod) *si.Allocation { - // only collect pod that needs recovery - if !utils.IsPodTerminated(pod) { - if alloc := mgr.GetExistingAllocation(pod); alloc != nil { - return alloc - } - } - return nil -} - -// Recover nodes and the placed allocations on these nodes. -// In this process, shim sends all nodes along with existing allocations on these nodes to the -// scheduler core, scheduler-core recovers its state and accept a node only it is able to recover -// node state plus the allocations. If a node is recovered successfully, its state is marked as -// healthy. Only healthy nodes can be used for scheduling. -// -//nolint:funlen -func (ctx *Context) recover(mgr *AppManagementService, due time.Duration) error { - allNodes, err := waitAndListNodes(ctx.apiProvider) - if err != nil { - return err - } - - // add all known nodes to cache, waiting for recover - for _, node := range allNodes { - ctx.nodes.addAndReportNode(node, false) - } - - pods, err := ctx.apiProvider.GetAPIs().PodInformer.Lister().List(labels.Everything()) - if err != nil { - return err - } - - nodeOccupiedResources := make(map[string]*si.Resource) - for _, pod := range pods { - // only handle assigned pods - if !utils.IsAssignedPod(pod) { - log.Log(log.ShimContext).Info("Skipping unassigned pod", - zap.String("podUID", string(pod.UID)), - zap.String("podName", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name))) - continue - } - // yunikorn scheduled pods add to existing allocations - ykPod := utils.GetApplicationIDFromPod(pod) != "" - switch { - case ykPod: - if existingAlloc := getExistingAllocation(mgr, pod); existingAlloc != nil { - log.Log(log.ShimContext).Debug("Adding resources for existing pod", - zap.String("appID", existingAlloc.ApplicationID), - zap.String("podUID", string(pod.UID)), - zap.String("podName", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)), - zap.String("nodeName", existingAlloc.NodeID), - zap.Stringer("resources", common.GetPodResource(pod))) - existingAlloc.AllocationTags = common.CreateTagsForTask(pod) - if err = ctx.nodes.addExistingAllocation(existingAlloc); err != nil { - log.Log(log.ShimContext).Warn("Failed to add existing allocation", zap.Error(err)) - } - } else { - log.Log(log.ShimContext).Warn("No allocation found for existing pod", - zap.String("podUID", string(pod.UID)), - zap.String("podName", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)), - zap.String("nodeName", pod.Spec.NodeName), - zap.Stringer("resources", common.GetPodResource(pod))) - } - case !utils.IsPodTerminated(pod): - // pod is not terminated (succeed or failed) state, - // and it has a node assigned, that means the scheduler - // has already allocated the pod onto a node - // we should report this occupied resource to scheduler-core - occupiedResource := nodeOccupiedResources[pod.Spec.NodeName] - if occupiedResource == nil { - occupiedResource = common.NewResourceBuilder().Build() - } - podResource := common.GetPodResource(pod) - log.Log(log.ShimContext).Debug("Adding resources for occupied pod", - zap.String("podUID", string(pod.UID)), - zap.String("podName", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name)), - zap.String("nodeName", pod.Spec.NodeName), - zap.Stringer("resources", podResource)) - occupiedResource = common.Add(occupiedResource, podResource) - nodeOccupiedResources[pod.Spec.NodeName] = occupiedResource - ctx.nodes.cache.AddPod(pod) - default: - log.Log(log.ShimContext).Debug("Skipping terminated pod", - zap.String("podUID", string(pod.UID)), - zap.String("podName", fmt.Sprintf("%s/%s", pod.Namespace, pod.Name))) - } - } - - // why we need to calculate the occupied resources here? why not add an event-handler - // in node_coordinator#addPod? - // this is because the occupied resources must be calculated and counted before the - // scheduling started. If we do both updating existing occupied resources along with - // new pods scheduling, due to the fact that we cannot predicate the ordering of K8s - // events, it could be dangerous because we might schedule pods onto some node that - // doesn't have enough capacity (occupied resources not yet reported). - for nodeName, occupiedResource := range nodeOccupiedResources { - if cachedNode := ctx.nodes.getNode(nodeName); cachedNode != nil { - cachedNode.updateOccupiedResource(occupiedResource, AddOccupiedResource) - } - } - - // start new nodes - for _, node := range ctx.nodes.nodesMap { - log.Log(log.ShimContext).Info("node state", - zap.String("nodeName", node.name), - zap.String("nodeState", node.getNodeState())) - if node.getNodeState() == SchedulerNodeStates().New { - dispatcher.Dispatch(CachedSchedulerNodeEvent{ - NodeID: node.name, - Event: RecoverNode, - }) - } - } - - // wait for nodes to be recovered - if err = utils.WaitForCondition(func() bool { - nodesRecovered := 0 - for _, node := range ctx.nodes.nodesMap { - log.Log(log.ShimContext).Info("node state", - zap.String("nodeName", node.name), - zap.String("nodeState", node.getNodeState())) - switch node.getNodeState() { - case SchedulerNodeStates().Healthy: - nodesRecovered++ - case SchedulerNodeStates().Draining: - nodesRecovered++ - case SchedulerNodeStates().Rejected: - nodesRecovered++ - } - } - - if nodesRecovered == len(allNodes) { - log.Log(log.ShimContext).Info("nodes recovery is successful", - zap.Int("recoveredNodes", nodesRecovered)) - return true - } - log.Log(log.ShimContext).Info("still waiting for recovering nodes", - zap.Int("totalNodes", len(allNodes)), - zap.Int("recoveredNodes", nodesRecovered)) - return false - }, time.Second, due); err != nil { - return fmt.Errorf("timeout waiting for app recovery in %s", due.String()) - } - - return nil -} - -func waitAndListNodes(apiProvider client.APIProvider) ([]*corev1.Node, error) { - var allNodes []*corev1.Node - var listErr error - - // need to wait for sync - // because the shared indexer doesn't sync its cache periodically - apiProvider.WaitForSync() - - // list all nodes in the cluster, - // retry for sometime if there is some errors - err := utils.WaitForCondition(func() bool { - allNodes, listErr = apiProvider.GetAPIs(). - NodeInformer.Lister().List(labels.Everything()) - return listErr == nil - }, time.Second, time.Minute) - if err != nil { - return nil, err - } - - return allNodes, nil -} diff --git a/pkg/cache/context_recovery_test.go b/pkg/cache/context_recovery_test.go deleted file mode 100644 index 87d752302..000000000 --- a/pkg/cache/context_recovery_test.go +++ /dev/null @@ -1,215 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "reflect" - "strconv" - "testing" - "time" - - "gotest.tools/v3/assert" - v1 "k8s.io/api/core/v1" - "k8s.io/apimachinery/pkg/api/resource" - apis "k8s.io/apimachinery/pkg/apis/meta/v1" - "k8s.io/apimachinery/pkg/types" - - "github.com/apache/yunikorn-k8shim/pkg/client" - "github.com/apache/yunikorn-k8shim/pkg/common/test" - "github.com/apache/yunikorn-k8shim/pkg/common/utils" - "github.com/apache/yunikorn-k8shim/pkg/dispatcher" -) - -type K8sResource struct { - ResourceName v1.ResourceName - Value int64 -} - -func NewK8sResourceList(resources ...K8sResource) map[v1.ResourceName]resource.Quantity { - resourceList := make(map[v1.ResourceName]resource.Quantity) - for _, r := range resources { - resourceList[r.ResourceName] = *resource.NewQuantity(r.Value, resource.DecimalSI) - } - return resourceList -} - -func TestNodeRecoveringState(t *testing.T) { - t.Skip("broken") - apiProvider4test := client.NewMockedAPIProvider(false) - context := NewContext(apiProvider4test) - dispatcher.RegisterEventHandler(dispatcher.EventTypeNode, context.nodes.schedulerNodeEventHandler()) - dispatcher.Start() - defer dispatcher.Stop() - - var node1 = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Capacity: NewK8sResourceList( - K8sResource{ - ResourceName: v1.ResourceMemory, - Value: 1024, - }, K8sResource{ - ResourceName: v1.ResourceCPU, - Value: 10, - }), - }, - } - - var node2 = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0002", - Namespace: "default", - UID: "uid_0002", - }, - Status: v1.NodeStatus{ - Capacity: NewK8sResourceList( - K8sResource{ - ResourceName: v1.ResourceMemory, - Value: 1024, - }, K8sResource{ - ResourceName: v1.ResourceCPU, - Value: 10, - }), - }, - } - - nodeLister := test.NewNodeListerMock() - nodeLister.AddNode(&node1) - nodeLister.AddNode(&node2) - apiProvider4test.SetNodeLister(nodeLister) - - /* - mockedAppMgr := test.NewMockedRecoverableAppManager() - if err := context.recover([]interfaces.Recoverable{mockedAppMgr}, 3*time.Second); err == nil { - t.Fatalf("expecting timeout here!") - } else { - t.Logf("context stays waiting for recovery, error: %v", err) - } - */ - - sn1 := context.nodes.getNode("host0001") - sn2 := context.nodes.getNode("host0002") - - assert.Assert(t, sn1 != nil) - assert.Assert(t, sn2 != nil) - - assert.Equal(t, sn1.getNodeState(), SchedulerNodeStates().Recovering) - assert.Equal(t, sn2.getNodeState(), SchedulerNodeStates().Recovering) -} - -func TestNodesRecovery(t *testing.T) { - t.Skip("broken") - apiProvide4test := client.NewMockedAPIProvider(false) - context := NewContext(apiProvide4test) - dispatcher.RegisterEventHandler(dispatcher.EventTypeNode, context.nodes.schedulerNodeEventHandler()) - dispatcher.Start() - defer dispatcher.Stop() - - numNodes := 3 - nodes := make([]*v1.Node, numNodes) - expectedStates := make([]string, numNodes) - for i := 0; i < numNodes; i++ { - nodes[i] = &v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host000%d" + strconv.Itoa(i), - Namespace: "default", - UID: types.UID("uid_000" + strconv.Itoa(i)), - }, - Status: v1.NodeStatus{ - Capacity: NewK8sResourceList( - K8sResource{ - ResourceName: v1.ResourceMemory, - Value: 1024, - }, K8sResource{ - ResourceName: v1.ResourceCPU, - Value: 10, - }), - }, - } - expectedStates[i] = SchedulerNodeStates().Recovering - } - - nodeLister := test.NewNodeListerMock() - for _, node := range nodes { - nodeLister.AddNode(node) - } - apiProvide4test.SetNodeLister(nodeLister) - - /* - mockedAppRecover := test.NewMockedRecoverableAppManager() - if err := context.recover([]interfaces.Recoverable{mockedAppRecover}, 1*time.Second); err == nil { - t.Fatalf("expecting timeout here!") - } - */ - - // verify all nodes were added into context - schedulerNodes := make([]*SchedulerNode, len(nodes)) - for i, node := range nodes { - schedulerNodes[i] = context.nodes.getNode(node.Name) - assert.Assert(t, schedulerNodes[i] != nil) - } - - // dispatch NodeAccepted event for the first node, its expected state should be Healthy - dispatcher.Dispatch(CachedSchedulerNodeEvent{ - NodeID: schedulerNodes[0].name, - Event: NodeAccepted, - }) - expectedStates[0] = SchedulerNodeStates().Healthy - err := utils.WaitForCondition(func() bool { - return reflect.DeepEqual(getNodeStates(schedulerNodes), expectedStates) - }, 100*time.Millisecond, 3*time.Second) - assert.NilError(t, err, "unexpected node states, actual: %v, expected: %v", getNodeStates(schedulerNodes), expectedStates) - - // dispatch NodeRejected event for the second node, its expected state should be Rejected - dispatcher.Dispatch(CachedSchedulerNodeEvent{ - NodeID: schedulerNodes[1].name, - Event: NodeRejected, - }) - expectedStates[1] = SchedulerNodeStates().Rejected - err = utils.WaitForCondition(func() bool { - return reflect.DeepEqual(getNodeStates(schedulerNodes), expectedStates) - }, 100*time.Millisecond, 3*time.Second) - assert.NilError(t, err, "unexpected node states, actual: %v, expected: %v", getNodeStates(schedulerNodes), expectedStates) - - // dispatch DrainNode event for the third node, its expected state should be Draining - schedulerNodes[2].schedulable = false - dispatcher.Dispatch(CachedSchedulerNodeEvent{ - NodeID: schedulerNodes[2].name, - Event: NodeAccepted, - }) - expectedStates[2] = SchedulerNodeStates().Draining - /* - err = context.recover([]interfaces.Recoverable{mockedAppRecover}, 3*time.Second) - assert.NilError(t, err, "recovery should be successful, however got error") - assert.DeepEqual(t, getNodeStates(schedulerNodes), expectedStates) - - */ -} - -func getNodeStates(schedulerNodes []*SchedulerNode) []string { - nodeStates := make([]string, len(schedulerNodes)) - for i, sn := range schedulerNodes { - nodeStates[i] = sn.getNodeState() - } - return nodeStates -} diff --git a/pkg/cache/context_test.go b/pkg/cache/context_test.go index 1769c950a..b4583f2f1 100644 --- a/pkg/cache/context_test.go +++ b/pkg/cache/context_test.go @@ -35,6 +35,7 @@ import ( k8sEvents "k8s.io/client-go/tools/events" "github.com/apache/yunikorn-core/pkg/common" + schedulercache "github.com/apache/yunikorn-k8shim/pkg/cache/external" "github.com/apache/yunikorn-k8shim/pkg/client" "github.com/apache/yunikorn-k8shim/pkg/common/constants" "github.com/apache/yunikorn-k8shim/pkg/common/events" @@ -90,7 +91,20 @@ func newPodHelper(name, namespace, podUID, nodeName string, appID string, podPha } func TestAddNodes(t *testing.T) { - ctx := initContextForTest() + ctx, apiProvider := initContextAndAPIProviderForTest() + dispatcher.Start() + defer dispatcher.UnregisterAllEventHandlers() + defer dispatcher.Stop() + + apiProvider.MockSchedulerAPIUpdateNodeFn(func(request *si.NodeRequest) error { + for _, node := range request.Nodes { + dispatcher.Dispatch(CachedSchedulerNodeEvent{ + NodeID: node.NodeID, + Event: NodeAccepted, + }) + } + return nil + }) node := v1.Node{ ObjectMeta: apis.ObjectMeta{ @@ -101,12 +115,25 @@ func TestAddNodes(t *testing.T) { } ctx.addNode(&node) + assert.Equal(t, true, ctx.schedulerCache.GetNode("host0001") != nil) - assert.Equal(t, true, ctx.nodes.getNode("host0001") != nil) } func TestUpdateNodes(t *testing.T) { - ctx := initContextForTest() + ctx, apiProvider := initContextAndAPIProviderForTest() + dispatcher.Start() + defer dispatcher.UnregisterAllEventHandlers() + defer dispatcher.Stop() + + apiProvider.MockSchedulerAPIUpdateNodeFn(func(request *si.NodeRequest) error { + for _, node := range request.Nodes { + dispatcher.Dispatch(CachedSchedulerNodeEvent{ + NodeID: node.NodeID, + Event: NodeAccepted, + }) + } + return nil + }) oldNodeResource := make(map[v1.ResourceName]resource.Quantity) oldNodeResource[v1.ResourceName("memory")] = *resource.NewQuantity(1024*1000*1000, resource.DecimalSI) @@ -139,12 +166,28 @@ func TestUpdateNodes(t *testing.T) { ctx.addNode(&oldNode) ctx.updateNode(&oldNode, &newNode) - assert.Equal(t, int64(2048*1000*1000), ctx.nodes.getNode("host0001").capacity.Resources[siCommon.Memory].Value) - assert.Equal(t, int64(4000), ctx.nodes.getNode("host0001").capacity.Resources[siCommon.CPU].Value) + _, capacity, _, ok := ctx.schedulerCache.UpdateOccupiedResource( + "host0001", "n/a", "n/a", nil, schedulercache.AddOccupiedResource) + assert.Assert(t, ok, "unable to retrieve node capacity") + assert.Equal(t, int64(2048*1000*1000), capacity.Resources[siCommon.Memory].Value) + assert.Equal(t, int64(4000), capacity.Resources[siCommon.CPU].Value) } func TestDeleteNodes(t *testing.T) { - ctx := initContextForTest() + ctx, apiProvider := initContextAndAPIProviderForTest() + dispatcher.Start() + defer dispatcher.UnregisterAllEventHandlers() + defer dispatcher.Stop() + + apiProvider.MockSchedulerAPIUpdateNodeFn(func(request *si.NodeRequest) error { + for _, node := range request.Nodes { + dispatcher.Dispatch(CachedSchedulerNodeEvent{ + NodeID: node.NodeID, + Event: NodeAccepted, + }) + } + return nil + }) node := v1.Node{ ObjectMeta: apis.ObjectMeta{ @@ -156,19 +199,15 @@ func TestDeleteNodes(t *testing.T) { ctx.addNode(&node) assert.Equal(t, true, ctx.schedulerCache.GetNode("host0001") != nil) - assert.Equal(t, true, ctx.nodes.getNode("host0001") != nil) ctx.deleteNode(&node) assert.Equal(t, true, ctx.schedulerCache.GetNode("host0001") == nil) - assert.Equal(t, true, ctx.nodes.getNode("host0001") == nil) ctx.addNode(&node) assert.Equal(t, true, ctx.schedulerCache.GetNode("host0001") != nil) - assert.Equal(t, true, ctx.nodes.getNode("host0001") != nil) ctx.deleteNode(cache.DeletedFinalStateUnknown{Key: "UID-00001", Obj: &node}) assert.Equal(t, true, ctx.schedulerCache.GetNode("host0001") == nil) - assert.Equal(t, true, ctx.nodes.getNode("host0001") == nil) } func TestAddApplications(t *testing.T) { @@ -372,9 +411,9 @@ func TestAddPod(t *testing.T) { }, } - context.addPod(nil) // no-op, but should not crash - context.addPod(pod1) // should be added - context.addPod(pod2) // should skip as pod is terminated + context.AddPod(nil) // no-op, but should not crash + context.AddPod(pod1) // should be added + context.AddPod(pod2) // should skip as pod is terminated _, ok := context.schedulerCache.GetPod("UID-00001") assert.Check(t, ok, "active pod was not added") @@ -433,22 +472,22 @@ func TestUpdatePod(t *testing.T) { }, } - context.addPod(pod1) + context.AddPod(pod1) _, ok := context.schedulerCache.GetPod("UID-00001") assert.Assert(t, ok, "pod1 is not present after adding") // these should not fail, but are no-ops - context.updatePod(nil, nil) - context.updatePod(nil, pod1) - context.updatePod(pod1, nil) + context.UpdatePod(nil, nil) + context.UpdatePod(nil, pod1) + context.UpdatePod(pod1, nil) // ensure a terminated pod is removed - context.updatePod(pod1, pod3) + context.UpdatePod(pod1, pod3) _, ok = context.schedulerCache.GetPod("UID-00001") assert.Check(t, !ok, "pod still found after termination") // ensure a non-terminated pod is updated - context.updatePod(pod1, pod2) + context.UpdatePod(pod1, pod2) found, ok := context.schedulerCache.GetPod("UID-00001") if assert.Check(t, ok, "pod not found after update") { assert.Check(t, found.GetAnnotations()["test.state"] == "updated", "pod state not updated") @@ -487,45 +526,53 @@ func TestDeletePod(t *testing.T) { Spec: v1.PodSpec{SchedulerName: "yunikorn"}, } - context.addPod(pod1) - context.addPod(pod2) + context.AddPod(pod1) + context.AddPod(pod2) _, ok := context.schedulerCache.GetPod("UID-00001") assert.Assert(t, ok, "pod1 is not present after adding") _, ok = context.schedulerCache.GetPod("UID-00002") assert.Assert(t, ok, "pod2 is not present after adding") // these should not fail, but here for completeness - context.deletePod(nil) - context.deletePod(cache.DeletedFinalStateUnknown{Key: "UID-00000", Obj: nil}) + context.DeletePod(nil) + context.DeletePod(cache.DeletedFinalStateUnknown{Key: "UID-00000", Obj: nil}) - context.deletePod(pod1) + context.DeletePod(pod1) _, ok = context.schedulerCache.GetPod("UID-00001") assert.Check(t, !ok, "pod1 is still present") - context.deletePod(cache.DeletedFinalStateUnknown{Key: "UID-00002", Obj: pod2}) + context.DeletePod(cache.DeletedFinalStateUnknown{Key: "UID-00002", Obj: pod2}) _, ok = context.schedulerCache.GetPod("UID-00002") assert.Check(t, !ok, "pod2 is still present") } //nolint:funlen func TestAddUpdatePodForeign(t *testing.T) { - mockedSchedulerApi := newMockSchedulerAPI() - context := initContextForTest() - context.nodes = newSchedulerNodes(mockedSchedulerApi, NewTestSchedulerCache()) - host1 := nodeForTest(Host1, "10G", "10") - context.nodes.addNode(host1) + context, apiProvider := initContextAndAPIProviderForTest() + dispatcher.Start() + defer dispatcher.UnregisterAllEventHandlers() + defer dispatcher.Stop() executed := false expectAdd := false expectRemove := false tc := "" - mockedSchedulerApi.UpdateNodeFn = func(request *si.NodeRequest) error { - executed = true + validatorFunc := func(request *si.NodeRequest) error { assert.Equal(t, len(request.Nodes), 1, "%s: wrong node count", tc) updatedNode := request.Nodes[0] assert.Equal(t, updatedNode.NodeID, Host1, "%s: wrong nodeID", tc) - assert.Equal(t, updatedNode.Action, si.NodeInfo_UPDATE, "%s: wrong action", tc) + switch updatedNode.Action { + case si.NodeInfo_CREATE_DRAIN: + return nil + case si.NodeInfo_DRAIN_TO_SCHEDULABLE: + return nil + case si.NodeInfo_UPDATE: + executed = true + default: + assert.Equal(t, false, "Unexpected action: %d", updatedNode.Action) + return nil + } assert.Equal(t, updatedNode.SchedulableResource.Resources[siCommon.Memory].Value, int64(10000*1000*1000), "%s: wrong schedulable memory", tc) assert.Equal(t, updatedNode.SchedulableResource.Resources[siCommon.CPU].Value, int64(10000), "%s: wrong schedulable cpu", tc) if expectAdd { @@ -539,6 +586,21 @@ func TestAddUpdatePodForeign(t *testing.T) { return nil } + apiProvider.MockSchedulerAPIUpdateNodeFn(func(request *si.NodeRequest) error { + for _, node := range request.Nodes { + if node.Action == si.NodeInfo_CREATE_DRAIN { + dispatcher.Dispatch(CachedSchedulerNodeEvent{ + NodeID: node.NodeID, + Event: NodeAccepted, + }) + } + } + return validatorFunc(request) + }) + + host1 := nodeForTest(Host1, "10G", "10") + context.updateNode(nil, host1) + // pod is not assigned to any node pod1 := foreignPod("pod1", "1G", "500m") pod1.Status.Phase = v1.PodPending @@ -549,7 +611,7 @@ func TestAddUpdatePodForeign(t *testing.T) { executed = false expectAdd = false expectRemove = false - context.addPod(pod1) + context.AddPod(pod1) assert.Assert(t, !executed, "unexpected update") _, ok := context.schedulerCache.GetPod(string(pod1.UID)) assert.Assert(t, !ok, "unassigned pod found in cache") @@ -557,11 +619,9 @@ func TestAddUpdatePodForeign(t *testing.T) { // validate update tc = "update-pod1" executed = false - expectAdd = false expectRemove = false - context.updatePod(nil, pod1) + context.UpdatePod(nil, pod1) assert.Assert(t, !executed, "unexpected update") - _, ok = context.schedulerCache.GetPod(string(pod1.UID)) assert.Assert(t, !ok, "unassigned pod found in cache") // pod is assigned to a node but still in pending state, should update @@ -574,8 +634,8 @@ func TestAddUpdatePodForeign(t *testing.T) { executed = false expectAdd = true expectRemove = false - context.addPod(pod2) - assert.Assert(t, executed, "update not executed") + context.AddPod(pod2) + assert.Assert(t, executed, "updated expected") _, ok = context.schedulerCache.GetPod(string(pod2.UID)) assert.Assert(t, ok, "pod not found in cache") @@ -584,7 +644,7 @@ func TestAddUpdatePodForeign(t *testing.T) { executed = false expectAdd = false expectRemove = false - context.updatePod(nil, pod2) + context.UpdatePod(nil, pod2) assert.Assert(t, !executed, "unexpected update") _, ok = context.schedulerCache.GetPod(string(pod2.UID)) assert.Assert(t, ok, "pod not found in cache") @@ -594,13 +654,14 @@ func TestAddUpdatePodForeign(t *testing.T) { executed = false expectAdd = false expectRemove = true - context.deletePod(pod2) + context.DeletePod(pod2) + assert.Assert(t, executed, "expected update") tc = "update-pod2-nocache" executed = false expectAdd = true expectRemove = false - context.updatePod(nil, pod2) - assert.Assert(t, executed, "update not executed") + context.UpdatePod(nil, pod2) + assert.Assert(t, executed, "expected update") _, ok = context.schedulerCache.GetPod(string(pod2.UID)) assert.Assert(t, ok, "pod not found in cache") @@ -613,8 +674,8 @@ func TestAddUpdatePodForeign(t *testing.T) { executed = false expectAdd = false expectRemove = true - context.addPod(pod3) - assert.Assert(t, executed, "update not executed") + context.AddPod(pod3) + assert.Assert(t, executed, "expected update") _, ok = context.schedulerCache.GetPod(string(pod3.UID)) assert.Assert(t, !ok, "failed pod found in cache") @@ -623,33 +684,43 @@ func TestAddUpdatePodForeign(t *testing.T) { executed = false expectAdd = true expectRemove = false - context.addPod(pod2) + context.AddPod(pod2) tc = "update-pod3" executed = false expectAdd = false expectRemove = true - context.updatePod(nil, pod3) - assert.Assert(t, executed, "update not executed") + context.UpdatePod(nil, pod3) + assert.Assert(t, executed, "expected update") _, ok = context.schedulerCache.GetPod(string(pod3.UID)) assert.Assert(t, !ok, "failed pod found in cache") } func TestDeletePodForeign(t *testing.T) { - mockedSchedulerApi := newMockSchedulerAPI() - context := initContextForTest() - context.nodes = newSchedulerNodes(mockedSchedulerApi, NewTestSchedulerCache()) - host1 := nodeForTest(Host1, "10G", "10") - context.nodes.addNode(host1) + context, apiProvider := initContextAndAPIProviderForTest() + dispatcher.Start() + defer dispatcher.UnregisterAllEventHandlers() + defer dispatcher.Stop() executed := false expectAdd := false expectRemove := false tc := "" - mockedSchedulerApi.UpdateNodeFn = func(request *si.NodeRequest) error { + validatorFunc := func(request *si.NodeRequest) error { executed = true assert.Equal(t, len(request.Nodes), 1, "%s: wrong node count", tc) updatedNode := request.Nodes[0] + switch updatedNode.Action { + case si.NodeInfo_CREATE_DRAIN: + return nil + case si.NodeInfo_DRAIN_TO_SCHEDULABLE: + return nil + case si.NodeInfo_UPDATE: + executed = true + default: + assert.Equal(t, false, "Unexpected action: %d", updatedNode.Action) + return nil + } assert.Equal(t, updatedNode.NodeID, Host1, "%s: wrong nodeID", tc) assert.Equal(t, updatedNode.Action, si.NodeInfo_UPDATE, "%s: wrong action", tc) assert.Equal(t, updatedNode.SchedulableResource.Resources[siCommon.Memory].Value, int64(10000*1000*1000), "%s: wrong schedulable memory", tc) @@ -665,6 +736,21 @@ func TestDeletePodForeign(t *testing.T) { return nil } + apiProvider.MockSchedulerAPIUpdateNodeFn(func(request *si.NodeRequest) error { + for _, node := range request.Nodes { + if node.Action == si.NodeInfo_CREATE_DRAIN { + dispatcher.Dispatch(CachedSchedulerNodeEvent{ + NodeID: node.NodeID, + Event: NodeAccepted, + }) + } + } + return validatorFunc(request) + }) + + host1 := nodeForTest(Host1, "10G", "10") + context.updateNode(nil, host1) + // add existing pod pod1 := foreignPod("pod1", "1G", "500m") pod1.Status.Phase = v1.PodRunning @@ -675,12 +761,12 @@ func TestDeletePodForeign(t *testing.T) { executed = false expectAdd = true expectRemove = false - context.addPod(pod1) + context.AddPod(pod1) tc = "delete-pod1" executed = false expectAdd = false expectRemove = true - context.deletePod(pod1) + context.DeletePod(pod1) assert.Assert(t, executed, "update not executed") _, ok := context.schedulerCache.GetPod(string(pod1.UID)) assert.Assert(t, !ok, "deleted pod found in cache") @@ -690,7 +776,7 @@ func TestDeletePodForeign(t *testing.T) { executed = false expectAdd = false expectRemove = false - context.deletePod(pod1) + context.DeletePod(pod1) assert.Assert(t, !executed, "unexpected update") _, ok = context.schedulerCache.GetPod(string(pod1.UID)) assert.Assert(t, !ok, "deleted pod found in cache") @@ -875,8 +961,8 @@ func TestRecoverTask(t *testing.T) { func TestTaskReleaseAfterRecovery(t *testing.T) { context := initContextForTest() - dispatcher.RegisterEventHandler(dispatcher.EventTypeApp, context.ApplicationEventHandler()) - dispatcher.RegisterEventHandler(dispatcher.EventTypeTask, context.TaskEventHandler()) + dispatcher.RegisterEventHandler("TestAppHandler", dispatcher.EventTypeApp, context.ApplicationEventHandler()) + dispatcher.RegisterEventHandler("TestTaskHandler", dispatcher.EventTypeTask, context.TaskEventHandler()) dispatcher.Start() defer dispatcher.Stop() @@ -1097,7 +1183,21 @@ func TestNodeEventPublishedCorrectly(t *testing.T) { if !ok { t.Fatal("the EventRecorder is expected to be of type FakeRecorder") } - context := initContextForTest() + + context, apiProvider := initContextAndAPIProviderForTest() + dispatcher.Start() + defer dispatcher.UnregisterAllEventHandlers() + defer dispatcher.Stop() + + apiProvider.MockSchedulerAPIUpdateNodeFn(func(request *si.NodeRequest) error { + for _, node := range request.Nodes { + dispatcher.Dispatch(CachedSchedulerNodeEvent{ + NodeID: node.NodeID, + Event: NodeAccepted, + }) + } + return nil + }) node := v1.Node{ ObjectMeta: apis.ObjectMeta{ @@ -1144,7 +1244,21 @@ func TestFilteredEventsNotPublished(t *testing.T) { if !ok { t.Fatal("the EventRecorder is expected to be of type FakeRecorder") } - context := initContextForTest() + + context, apiProvider := initContextAndAPIProviderForTest() + dispatcher.Start() + defer dispatcher.UnregisterAllEventHandlers() + defer dispatcher.Stop() + + apiProvider.MockSchedulerAPIUpdateNodeFn(func(request *si.NodeRequest) error { + for _, node := range request.Nodes { + dispatcher.Dispatch(CachedSchedulerNodeEvent{ + NodeID: node.NodeID, + Event: NodeAccepted, + }) + } + return nil + }) node := v1.Node{ ObjectMeta: apis.ObjectMeta{ @@ -1461,7 +1575,20 @@ func TestPendingPodAllocations(t *testing.T) { utils.SetPluginMode(true) defer utils.SetPluginMode(false) - context := initContextForTest() + context, apiProvider := initContextAndAPIProviderForTest() + dispatcher.Start() + defer dispatcher.UnregisterAllEventHandlers() + defer dispatcher.Stop() + + apiProvider.MockSchedulerAPIUpdateNodeFn(func(request *si.NodeRequest) error { + for _, node := range request.Nodes { + dispatcher.Dispatch(CachedSchedulerNodeEvent{ + NodeID: node.NodeID, + Event: NodeAccepted, + }) + } + return nil + }) node1 := v1.Node{ ObjectMeta: apis.ObjectMeta{ @@ -1577,7 +1704,7 @@ func TestGetStateDump(t *testing.T) { }, Spec: v1.PodSpec{SchedulerName: "yunikorn"}, } - context.addPod(pod1) + context.AddPod(pod1) stateDumpStr, err := context.GetStateDump() assert.NilError(t, err, "error during state dump") @@ -1742,6 +1869,38 @@ func TestCtxUpdatePodCondition(t *testing.T) { assert.Equal(t, true, updated) } +func TestGetExistingAllocation(t *testing.T) { + pod := &v1.Pod{ + TypeMeta: apis.TypeMeta{ + Kind: "Pod", + APIVersion: "v1", + }, + ObjectMeta: apis.ObjectMeta{ + Name: "pod00001", + Namespace: "default", + UID: "UID-POD-00001", + Labels: map[string]string{ + "applicationId": "app00001", + "queue": "root.a", + }, + }, + Spec: v1.PodSpec{ + SchedulerName: constants.SchedulerName, + NodeName: "allocated-node", + }, + Status: v1.PodStatus{ + Phase: v1.PodPending, + }, + } + + // verifies the existing allocation is correctly returned + alloc := getExistingAllocation(pod) + assert.Equal(t, alloc.ApplicationID, "app00001") + assert.Equal(t, alloc.AllocationKey, string(pod.UID)) + assert.Equal(t, alloc.UUID, string(pod.UID)) + assert.Equal(t, alloc.NodeID, "allocated-node") +} + func waitForNodeAcceptedEvent(recorder *k8sEvents.FakeRecorder) error { // fetch the "node accepted" event err := utils.WaitForCondition(func() bool { diff --git a/pkg/cache/external/scheduler_cache.go b/pkg/cache/external/scheduler_cache.go index aaf063f51..16265ae38 100644 --- a/pkg/cache/external/scheduler_cache.go +++ b/pkg/cache/external/scheduler_cache.go @@ -27,13 +27,21 @@ import ( v1 "k8s.io/api/core/v1" schedulingv1 "k8s.io/api/scheduling/v1" storageV1 "k8s.io/api/storage/v1" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/labels" "k8s.io/kubernetes/pkg/scheduler/framework" "github.com/apache/yunikorn-k8shim/pkg/client" + "github.com/apache/yunikorn-k8shim/pkg/common" "github.com/apache/yunikorn-k8shim/pkg/common/utils" "github.com/apache/yunikorn-k8shim/pkg/log" + "github.com/apache/yunikorn-scheduler-interface/lib/go/si" +) + +type UpdateType int + +const ( + AddOccupiedResource UpdateType = iota + SubOccupiedResource ) // SchedulerCache maintains some critical information about nodes and pods used for scheduling. @@ -49,12 +57,15 @@ import ( // is called in the plugin to signify completion of the allocation, it is removed. type SchedulerCache struct { nodesMap map[string]*framework.NodeInfo // node name to NodeInfo map + nodeCapacity map[string]*si.Resource // node name to node resource capacity + nodeOccupied map[string]*si.Resource // node name to node occupied resources podsMap map[string]*v1.Pod pcMap map[string]*schedulingv1.PriorityClass - assignedPods map[string]string // map of pods to the node they are currently assigned to - assumedPods map[string]bool // map of assumed pods, value indicates if pod volumes are all bound - pendingAllocations map[string]string // map of pod to node ID, presence indicates a pending allocation for scheduler - inProgressAllocations map[string]string // map of pod to node ID, presence indicates an in-process allocation for scheduler + assignedPods map[string]string // map of pods to the node they are currently assigned to + assumedPods map[string]bool // map of assumed pods, value indicates if pod volumes are all bound + orphanedPods map[string]*v1.Pod // map of orphaned pods, keyed by pod UID + pendingAllocations map[string]string // map of pod to node ID, presence indicates a pending allocation for scheduler + inProgressAllocations map[string]string // map of pod to node ID, presence indicates an in-process allocation for scheduler pvcRefCounts map[string]map[string]int lock sync.RWMutex clients *client.Clients // client APIs @@ -68,10 +79,13 @@ type SchedulerCache struct { func NewSchedulerCache(clients *client.Clients) *SchedulerCache { cache := &SchedulerCache{ nodesMap: make(map[string]*framework.NodeInfo), + nodeCapacity: make(map[string]*si.Resource), + nodeOccupied: make(map[string]*si.Resource), podsMap: make(map[string]*v1.Pod), pcMap: make(map[string]*schedulingv1.PriorityClass), assignedPods: make(map[string]string), assumedPods: make(map[string]bool), + orphanedPods: make(map[string]*v1.Pod), pendingAllocations: make(map[string]string), inProgressAllocations: make(map[string]string), pvcRefCounts: make(map[string]map[string]int), @@ -145,62 +159,73 @@ func (cache *SchedulerCache) UnlockForReads() { func (cache *SchedulerCache) GetNode(name string) *framework.NodeInfo { cache.lock.RLock() defer cache.lock.RUnlock() - if n, ok := cache.nodesMap[name]; ok { return n } return nil } -func (cache *SchedulerCache) AddNode(node *v1.Node) { - cache.lock.Lock() - defer cache.lock.Unlock() - cache.dumpState("AddNode.Pre") - defer cache.dumpState("AddNode.Post") - - cache.updateNode(node) -} - -func (cache *SchedulerCache) UpdateNode(newNode *v1.Node) { +// UpdateNode updates the given node in the cache and returns the previous node if it exists +func (cache *SchedulerCache) UpdateNode(node *v1.Node) (*v1.Node, []*v1.Pod) { cache.lock.Lock() defer cache.lock.Unlock() cache.dumpState("UpdateNode.Pre") defer cache.dumpState("UpdateNode.Post") - - cache.updateNode(newNode) + return cache.updateNode(node) } -func (cache *SchedulerCache) updateNode(node *v1.Node) { +func (cache *SchedulerCache) updateNode(node *v1.Node) (*v1.Node, []*v1.Pod) { + var prevNode *v1.Node + adopted := make([]*v1.Pod, 0) + nodeInfo, ok := cache.nodesMap[node.Name] if !ok { log.Log(log.ShimCacheExternal).Debug("Adding node to cache", zap.String("nodeName", node.Name)) nodeInfo = framework.NewNodeInfo() cache.nodesMap[node.Name] = nodeInfo + cache.nodeCapacity[node.Name] = common.GetNodeResource(&node.Status) + cache.nodeOccupied[node.Name] = common.NewResourceBuilder().Build() cache.nodesInfo = nil + nodeInfo.SetNode(node) + + // look for orphaned pods to adopt + for _, pod := range cache.orphanedPods { + if pod.Spec.NodeName == node.Name { + if cache.updatePod(pod) { + adopted = append(adopted, pod) + } + } + } } else { log.Log(log.ShimCacheExternal).Debug("Updating node in cache", zap.String("nodeName", node.Name)) + prevNode = nodeInfo.Node() + nodeInfo.SetNode(node) } - nodeInfo.SetNode(node) + cache.nodesInfoPodsWithAffinity = nil cache.nodesInfoPodsWithReqAntiAffinity = nil cache.updatePVCRefCounts(nodeInfo, false) + + return prevNode, adopted } -func (cache *SchedulerCache) RemoveNode(node *v1.Node) { +func (cache *SchedulerCache) RemoveNode(node *v1.Node) (*v1.Node, []*v1.Pod) { cache.lock.Lock() defer cache.lock.Unlock() cache.dumpState("RemoveNode.Pre") defer cache.dumpState("RemoveNode.Post") - cache.removeNode(node) + return cache.removeNode(node) } -func (cache *SchedulerCache) removeNode(node *v1.Node) { +func (cache *SchedulerCache) removeNode(node *v1.Node) (*v1.Node, []*v1.Pod) { + orphans := make([]*v1.Pod, 0) nodeInfo, ok := cache.nodesMap[node.Name] if !ok { log.Log(log.ShimCacheExternal).Debug("Attempted to remove non-existent node", zap.String("nodeName", node.Name)) - return + return nil, nil } + result := nodeInfo.Node() for _, pod := range nodeInfo.Pods { key := string(pod.Pod.UID) @@ -208,33 +233,122 @@ func (cache *SchedulerCache) removeNode(node *v1.Node) { delete(cache.assumedPods, key) delete(cache.pendingAllocations, key) delete(cache.inProgressAllocations, key) + cache.orphanedPods[key] = pod.Pod + orphans = append(orphans, pod.Pod) } log.Log(log.ShimCacheExternal).Debug("Removing node from cache", zap.String("nodeName", node.Name)) delete(cache.nodesMap, node.Name) + delete(cache.nodeOccupied, node.Name) + delete(cache.nodeCapacity, node.Name) cache.nodesInfo = nil cache.nodesInfoPodsWithAffinity = nil cache.nodesInfoPodsWithReqAntiAffinity = nil cache.updatePVCRefCounts(nodeInfo, true) + + return result, orphans } -func (cache *SchedulerCache) GetPriorityClass(name string) *schedulingv1.PriorityClass { +func (cache *SchedulerCache) SnapshotResources(nodeName string) (capacity *si.Resource, occupied *si.Resource, ok bool) { cache.lock.RLock() defer cache.lock.RUnlock() - if n, ok := cache.pcMap[name]; ok { - return n + occupied, ok = cache.nodeOccupied[nodeName] + if !ok { + log.Log(log.ShimCacheExternal).Warn("Unable to snapshot resources for node (missing occupied)", zap.String("nodeName", nodeName)) + return nil, nil, false } - return nil + + _, ok = cache.nodeCapacity[nodeName] + if !ok { + log.Log(log.ShimCacheExternal).Warn("Unable to snapshot resources for node (missing capacity)", zap.String("nodeName", nodeName)) + return nil, nil, false + } + return capacity, occupied, true } -func (cache *SchedulerCache) AddPriorityClass(priorityClass *schedulingv1.PriorityClass) { +func (cache *SchedulerCache) UpdateCapacity(nodeName string, resource *si.Resource) (capacity *si.Resource, occupied *si.Resource, ok bool) { cache.lock.Lock() defer cache.lock.Unlock() - cache.dumpState("AddPriorityClass.Pre") - defer cache.dumpState("AddPriorityClass.Post") - cache.updatePriorityClass(priorityClass) + occupied, ok = cache.nodeOccupied[nodeName] + if !ok { + log.Log(log.ShimCacheExternal).Warn("Unable to update capacity for node (missing occupied)", zap.String("nodeName", nodeName)) + return nil, nil, false + } + + _, ok = cache.nodeCapacity[nodeName] + if !ok { + log.Log(log.ShimCacheExternal).Warn("Unable to update capacity for node (missing capacity)", zap.String("nodeName", nodeName)) + return nil, nil, false + } + cache.nodeCapacity[nodeName] = resource + return capacity, occupied, true +} + +func (cache *SchedulerCache) UpdateOccupiedResource(nodeName string, namespace string, podName string, resource *si.Resource, opt UpdateType) (node *v1.Node, capacity *si.Resource, occupied *si.Resource, ok bool) { + cache.lock.Lock() + defer cache.lock.Unlock() + + nodeInfo, ok := cache.nodesMap[nodeName] + if !ok { + log.Log(log.ShimCacheExternal).Warn("Unable to update occupied resources for node (missing node)", + zap.String("nodeName", nodeName), + zap.String("namespace", namespace), + zap.String("podName", podName)) + return nil, nil, nil, false + } + node = nodeInfo.Node() + + occupied, ok = cache.nodeOccupied[nodeName] + if !ok { + log.Log(log.ShimCacheExternal).Warn("Unable to update occupied resources for node (missing occupied)", + zap.String("nodeName", nodeName), + zap.String("namespace", namespace), + zap.String("podName", podName)) + return nil, nil, nil, false + } + + capacity, ok = cache.nodeCapacity[nodeName] + if !ok { + log.Log(log.ShimCacheExternal).Warn("Unable to update occupied resources for node (missing capacity)", + zap.String("nodeName", nodeName), + zap.String("namespace", namespace), + zap.String("podName", podName)) + return nil, nil, nil, false + } + + switch opt { + case AddOccupiedResource: + log.Log(log.ShimCacheExternal).Info("Adding occupied resources to node", + zap.String("nodeID", nodeName), + zap.String("namespace", namespace), + zap.String("podName", podName), + zap.Stringer("occupied", resource)) + occupied = common.Add(occupied, resource) + cache.nodeOccupied[nodeName] = occupied + case SubOccupiedResource: + log.Log(log.ShimCacheExternal).Info("Subtracting occupied resources from node", + zap.String("nodeID", nodeName), + zap.String("namespace", namespace), + zap.String("podName", podName), + zap.Stringer("occupied", resource)) + occupied = common.Sub(occupied, resource) + cache.nodeOccupied[nodeName] = occupied + default: + // noop + } + return node, capacity, occupied, true +} + +func (cache *SchedulerCache) GetPriorityClass(name string) *schedulingv1.PriorityClass { + cache.lock.RLock() + defer cache.lock.RUnlock() + + if n, ok := cache.pcMap[name]; ok { + return n + } + return nil } func (cache *SchedulerCache) UpdatePriorityClass(priorityClass *schedulingv1.PriorityClass) { @@ -341,31 +455,24 @@ func (cache *SchedulerCache) ArePodVolumesAllBound(podKey string) bool { return cache.assumedPods[podKey] } -// AddPod adds a pod to the scheduler cache -func (cache *SchedulerCache) AddPod(pod *v1.Pod) { - cache.lock.Lock() - defer cache.lock.Unlock() - cache.dumpState("AddPod.Pre") - defer cache.dumpState("AddPod.Post") - cache.updatePod(pod) -} - // UpdatePod updates a pod in the cache -func (cache *SchedulerCache) UpdatePod(newPod *v1.Pod) { +func (cache *SchedulerCache) UpdatePod(newPod *v1.Pod) bool { cache.lock.Lock() defer cache.lock.Unlock() cache.dumpState("UpdatePod.Pre") defer cache.dumpState("UpdatePod.Post") - cache.updatePod(newPod) + return cache.updatePod(newPod) } -func (cache *SchedulerCache) updatePod(pod *v1.Pod) { +func (cache *SchedulerCache) updatePod(pod *v1.Pod) bool { key := string(pod.UID) + result := true currState, ok := cache.podsMap[key] if ok { // remove current version of pod delete(cache.podsMap, key) + delete(cache.orphanedPods, key) nodeName, ok := cache.assignedPods[key] if ok { nodeInfo, ok := cache.nodesMap[nodeName] @@ -403,21 +510,20 @@ func (cache *SchedulerCache) updatePod(pod *v1.Pod) { // assign to node nodeInfo, ok := cache.nodesMap[pod.Spec.NodeName] if !ok { - // node doesn't exist, create a synthetic one for now - nodeInfo = framework.NewNodeInfo() - cache.nodesMap[pod.Spec.NodeName] = nodeInfo - // work around a crash bug in NodeInfo.RemoveNode() when Node is unset - nodeInfo.SetNode(&v1.Node{ObjectMeta: metav1.ObjectMeta{Name: pod.Spec.NodeName}}) - } - nodeInfo.AddPod(pod) - cache.assignedPods[key] = pod.Spec.NodeName - if podWithAffinity(pod) { - cache.nodesInfoPodsWithAffinity = nil - } - if podWithRequiredAntiAffinity(pod) { - cache.nodesInfoPodsWithReqAntiAffinity = nil + // node doesn't exist, so this is an orphaned pod + cache.orphanedPods[key] = pod + result = false + } else { + nodeInfo.AddPod(pod) + cache.assignedPods[key] = pod.Spec.NodeName + if podWithAffinity(pod) { + cache.nodesInfoPodsWithAffinity = nil + } + if podWithRequiredAntiAffinity(pod) { + cache.nodesInfoPodsWithReqAntiAffinity = nil + } + cache.updatePVCRefCounts(nodeInfo, false) } - cache.updatePVCRefCounts(nodeInfo, false) } // if pod is not in a terminal state, add it back into cache @@ -429,9 +535,12 @@ func (cache *SchedulerCache) updatePod(pod *v1.Pod) { delete(cache.podsMap, key) delete(cache.assignedPods, key) delete(cache.assumedPods, key) + delete(cache.orphanedPods, key) delete(cache.pendingAllocations, key) delete(cache.inProgressAllocations, key) } + + return result } // RemovePod removes a pod from the cache @@ -462,6 +571,7 @@ func (cache *SchedulerCache) removePod(pod *v1.Pod) { delete(cache.podsMap, key) delete(cache.assignedPods, key) delete(cache.assumedPods, key) + delete(cache.orphanedPods, key) delete(cache.pendingAllocations, key) delete(cache.inProgressAllocations, key) cache.nodesInfoPodsWithAffinity = nil @@ -474,6 +584,13 @@ func (cache *SchedulerCache) GetPod(uid string) (*v1.Pod, bool) { return cache.GetPodNoLock(uid) } +func (cache *SchedulerCache) IsPodOrphaned(uid string) bool { + cache.lock.RLock() + defer cache.lock.RUnlock() + _, ok := cache.orphanedPods[uid] + return ok +} + func (cache *SchedulerCache) GetPodNoLock(uid string) (*v1.Pod, bool) { if pod, ok := cache.podsMap[uid]; ok { return pod, true @@ -526,6 +643,27 @@ func (cache *SchedulerCache) forgetPod(pod *v1.Pod) { delete(cache.inProgressAllocations, key) } +// AdoptPodsForNode finds pods on the orphan list that reference this node, assigns them, and returns the list +func (cache *SchedulerCache) AdoptPodsForNode(node *v1.Node) []*v1.Pod { + cache.lock.Lock() + defer cache.lock.Unlock() + + result := make([]*v1.Pod, 0) + ni, ok := cache.nodesMap[node.Name] + if !ok { + log.Log(log.ShimCacheExternal).Warn("Tried to adopt pods for nonexistent node", zap.String("nodeName", node.Name)) + return result + } + for id, pod := range cache.orphanedPods { + if pod.Spec.NodeName == node.Name { + delete(cache.orphanedPods, id) + ni.AddPod(pod) + result = append(result, pod) + } + } + return result +} + // Implement k8s.io/client-go/listers/core/v1#PodLister interface func (cache *SchedulerCache) List(selector labels.Selector) ([]*v1.Pod, error) { cache.lock.RLock() diff --git a/pkg/cache/external/scheduler_cache_test.go b/pkg/cache/external/scheduler_cache_test.go index 992d31fbb..1788af35b 100644 --- a/pkg/cache/external/scheduler_cache_test.go +++ b/pkg/cache/external/scheduler_cache_test.go @@ -233,7 +233,7 @@ func TestUpdateNode(t *testing.T) { } // first add the old node - cache.AddNode(oldNode) + cache.UpdateNode(oldNode) // make sure the node is added to the cache nodeInCache := cache.GetNode(host1) @@ -250,7 +250,7 @@ func TestUpdateNode(t *testing.T) { assert.Equal(t, nodeInCache.Node().Name, host1) assert.Equal(t, nodeInCache.Node().Spec.Unschedulable, false) - cache.removeNode(newNode) + cache.RemoveNode(newNode) assert.Equal(t, 0, len(cache.nodesInfo), "nodesInfo list size") } @@ -267,7 +267,7 @@ func TestGetNodesInfo(t *testing.T) { Unschedulable: true, }, } - cache.AddNode(node) + cache.UpdateNode(node) assert.Assert(t, cache.nodesInfo == nil) nodesInfo := cache.GetNodesInfo() expectHost(t, host1, nodesInfo) @@ -283,7 +283,7 @@ func TestGetNodesInfo(t *testing.T) { Unschedulable: false, }, } - cache.updateNode(updatedNode) + cache.UpdateNode(updatedNode) expectHost(t, host1, nodesInfo) // add new @@ -294,13 +294,13 @@ func TestGetNodesInfo(t *testing.T) { UID: nodeUID2, }, } - cache.AddNode(newNode) + cache.UpdateNode(newNode) assert.Assert(t, cache.nodesInfo == nil, "nodesInfo list was not invalidated") nodesInfo = cache.GetNodesInfo() expectHost1AndHost2(t, nodesInfo) // remove - cache.removeNode(node) + cache.RemoveNode(node) assert.Assert(t, cache.nodesInfo == nil, "nodesInfo list was not invalidated") nodesInfo = cache.GetNodesInfo() expectHost(t, host2, nodesInfo) @@ -319,7 +319,7 @@ func TestGetNodesInfoPodsWithAffinity(t *testing.T) { Unschedulable: true, }, } - cache.AddNode(node) + cache.UpdateNode(node) assert.Assert(t, cache.nodesInfoPodsWithAffinity == nil) cache.AssumePod(&v1.Pod{ TypeMeta: apis.TypeMeta{ @@ -351,7 +351,7 @@ func TestGetNodesInfoPodsWithAffinity(t *testing.T) { Unschedulable: false, }, } - cache.updateNode(newNode) + cache.UpdateNode(newNode) assert.Assert(t, cache.nodesInfoPodsWithAffinity == nil, "nodesInfo list was not invalidated") cache.AssumePod(&v1.Pod{ TypeMeta: apis.TypeMeta{ @@ -374,7 +374,7 @@ func TestGetNodesInfoPodsWithAffinity(t *testing.T) { expectHost1AndHost2(t, nodesInfo) // remove node - cache.removeNode(newNode) + cache.RemoveNode(newNode) assert.Assert(t, cache.nodesInfoPodsWithAffinity == nil, "nodesInfo list was not invalidated") nodesInfo = cache.GetNodesInfoPodsWithAffinity() expectHost(t, host1, nodesInfo) @@ -390,7 +390,7 @@ func TestGetNodesInfoPodsWithAffinity(t *testing.T) { Unschedulable: false, }, } - cache.updateNode(updatedNode) + cache.UpdateNode(updatedNode) assert.Assert(t, cache.nodesInfoPodsWithAffinity == nil, "node list was not invalidated") nodesInfo = cache.GetNodesInfoPodsWithAffinity() expectHost(t, host1, nodesInfo) @@ -478,7 +478,7 @@ func TestGetNodesInfoPodsWithReqAntiAffinity(t *testing.T) { Unschedulable: true, }, } - cache.AddNode(node) + cache.UpdateNode(node) assert.Assert(t, cache.nodesInfoPodsWithReqAntiAffinity == nil) cache.AssumePod(&v1.Pod{ TypeMeta: apis.TypeMeta{ @@ -512,7 +512,7 @@ func TestGetNodesInfoPodsWithReqAntiAffinity(t *testing.T) { Unschedulable: false, }, } - cache.updateNode(newNode) + cache.UpdateNode(newNode) assert.Assert(t, cache.nodesInfoPodsWithReqAntiAffinity == nil, "nodesInfo list was not invalidated") cache.AssumePod(&v1.Pod{ TypeMeta: apis.TypeMeta{ @@ -537,7 +537,7 @@ func TestGetNodesInfoPodsWithReqAntiAffinity(t *testing.T) { expectHost1AndHost2(t, nodesInfo) // remove node - cache.removeNode(newNode) + cache.RemoveNode(newNode) assert.Assert(t, cache.nodesInfoPodsWithReqAntiAffinity == nil, "nodesInfo list was not invalidated") nodesInfo = cache.GetNodesInfoPodsWithReqAntiAffinity() expectHost(t, host1, nodesInfo) @@ -553,7 +553,7 @@ func TestGetNodesInfoPodsWithReqAntiAffinity(t *testing.T) { Unschedulable: false, }, } - cache.updateNode(updatedNode) + cache.UpdateNode(updatedNode) assert.Assert(t, cache.nodesInfoPodsWithReqAntiAffinity == nil, "node list was not invalidated") nodesInfo = cache.GetNodesInfoPodsWithReqAntiAffinity() expectHost(t, host1, nodesInfo) @@ -665,9 +665,9 @@ func add2Cache(cache *SchedulerCache, objects ...interface{}) error { for _, obj := range objects { switch podOrNode := obj.(type) { case *v1.Node: - cache.AddNode(podOrNode) + cache.UpdateNode(podOrNode) case *v1.Pod: - cache.AddPod(podOrNode) + cache.UpdatePod(podOrNode) default: return fmt.Errorf("unknown object type") } @@ -682,7 +682,7 @@ func TestGetNodesInfoMap(t *testing.T) { assert.Equal(t, len(ref), 0) for i := 0; i < 10; i++ { - cache.AddNode(&v1.Node{ + cache.UpdateNode(&v1.Node{ ObjectMeta: apis.ObjectMeta{ Name: fmt.Sprintf("node-%d", i), Labels: map[string]string{ @@ -707,43 +707,6 @@ func TestGetNodesInfoMap(t *testing.T) { } } -func TestAddPod(t *testing.T) { - cache := NewSchedulerCache(client.NewMockedAPIProvider(false).GetAPIs()) - - pod1 := &v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: podName1, - UID: podUID1, - }, - Spec: v1.PodSpec{}, - } - - // add - cache.AddPod(pod1) - - _, ok := cache.GetPod(podUID1) - assert.Equal(t, len(cache.podsMap), 1, "wrong pod count after add of pod1") - assert.Check(t, ok, "pod1 not found") - - // re-add - cache.AddPod(pod1) - _, ok = cache.GetPod(podUID1) - assert.Equal(t, len(cache.podsMap), 1, "wrong pod count after re-add of pod1") - assert.Check(t, ok, "pod1 not found") - - // assumed pod with node should still be assumed if re-add - pod1Copy := pod1.DeepCopy() - pod1Copy.Spec.NodeName = "test-node-add" - cache.AssumePod(pod1Copy, true) - assert.Check(t, cache.isAssumedPod(podUID1), "pod is not assumed") - cache.AddPod(pod1) - assert.Check(t, cache.isAssumedPod(podUID1), "pod is not assumed after re-add") -} - func TestUpdatePod(t *testing.T) { cache := NewSchedulerCache(client.NewMockedAPIProvider(false).GetAPIs()) resourceList := make(map[v1.ResourceName]resource.Quantity) @@ -776,8 +739,8 @@ func TestUpdatePod(t *testing.T) { }, } - cache.AddNode(node1) - cache.AddNode(node2) + cache.UpdateNode(node1) + cache.UpdateNode(node2) podTemplate := &v1.Pod{ TypeMeta: apis.TypeMeta{ @@ -793,7 +756,7 @@ func TestUpdatePod(t *testing.T) { pod1 := podTemplate.DeepCopy() pod1.ObjectMeta.Name = podName1 pod1.ObjectMeta.UID = podUID1 - cache.AddPod(pod1) + cache.UpdatePod(pod1) assert.Equal(t, len(cache.podsMap), 1, "wrong pod count after add of pod1") _, ok := cache.GetPod(podUID1) assert.Check(t, ok, "pod1 not found") @@ -818,7 +781,7 @@ func TestUpdatePod(t *testing.T) { // assumed pod should still be assumed if node changes pod1.Spec.NodeName = node1.Name - cache.AddPod(pod1) + cache.UpdatePod(pod1) cache.AssumePod(pod1, true) assert.Check(t, cache.isAssumedPod(podUID1), "pod is not assumed") pod1Copy = pod1.DeepCopy() @@ -831,7 +794,7 @@ func TestUpdatePod(t *testing.T) { pod3.ObjectMeta.Name = "pod00003" pod3.ObjectMeta.UID = "Pod-UID-00003" pod3.Spec.NodeName = "orig-node" - cache.AddPod(pod3) + cache.UpdatePod(pod3) pod3Copy := pod3.DeepCopy() pod3Copy.Spec.NodeName = "new-node" cache.UpdatePod(pod3Copy) @@ -857,7 +820,7 @@ func TestRemovePod(t *testing.T) { } // add pod1 - cache.AddPod(pod1) + cache.UpdatePod(pod1) assert.Equal(t, len(cache.podsMap), 1, "wrong pod count after add of pod1") _, ok := cache.GetPod(podUID1) assert.Check(t, ok, "pod1 not found") @@ -870,7 +833,7 @@ func TestRemovePod(t *testing.T) { // again, with assigned node pod1.Spec.NodeName = "test-node-remove" - cache.AddPod(pod1) + cache.UpdatePod(pod1) assert.Equal(t, len(cache.podsMap), 1, "wrong pod count after add of pod1 with node") _, ok = cache.GetPod(podUID1) assert.Check(t, ok, "pod1 not found") @@ -883,7 +846,7 @@ func TestRemovePod(t *testing.T) { // removal of pod added to synthetic node should succeed pod1.Spec.NodeName = "unknown-node" - cache.AddPod(pod1) + cache.UpdatePod(pod1) assert.Equal(t, len(cache.podsMap), 1, "wrong pod count after add of pod1 with synthetic node") cache.RemovePod(pod1) assert.Equal(t, len(cache.podsMap), 0, "wrong pod count after remove of pod1 with synthetic node") @@ -897,22 +860,6 @@ func TestRemovePod(t *testing.T) { cache.RemovePod(pod1) } -func TestAddPriorityClass(t *testing.T) { - cache := NewSchedulerCache(client.NewMockedAPIProvider(false).GetAPIs()) - pc := &schedulingv1.PriorityClass{ - ObjectMeta: apis.ObjectMeta{ - Name: "class001", - UID: "Class-UID-00001", - }, - Value: 10, - } - - cache.AddPriorityClass(pc) - result := cache.GetPriorityClass("class001") - assert.Assert(t, result != nil) - assert.Equal(t, result.Value, int32(10)) -} - func TestUpdatePriorityClass(t *testing.T) { cache := NewSchedulerCache(client.NewMockedAPIProvider(false).GetAPIs()) pc := &schedulingv1.PriorityClass{ @@ -930,7 +877,7 @@ func TestUpdatePriorityClass(t *testing.T) { Value: 20, } - cache.AddPriorityClass(pc) + cache.UpdatePriorityClass(pc) cache.UpdatePriorityClass(pc2) result := cache.GetPriorityClass("class001") @@ -948,7 +895,7 @@ func TestRemovePriorityClass(t *testing.T) { Value: 10, } - cache.AddPriorityClass(pc) + cache.UpdatePriorityClass(pc) result := cache.GetPriorityClass("class001") assert.Assert(t, result != nil) assert.Equal(t, result.Value, int32(10)) @@ -1011,9 +958,9 @@ func TestGetSchedulerCacheDao(t *testing.T) { Value: 10, } - cache.AddNode(node) - cache.AddPod(pod) - cache.AddPriorityClass(pc) + cache.UpdateNode(node) + cache.UpdatePod(pod) + cache.UpdatePriorityClass(pc) // test with data dao = cache.GetSchedulerCacheDao() @@ -1092,8 +1039,8 @@ func TestUpdatePVCRefCounts(t *testing.T) { }, } - cache.AddNode(node1) - cache.AddNode(node2) + cache.UpdateNode(node1) + cache.UpdateNode(node2) podTemplate := &v1.Pod{ TypeMeta: apis.TypeMeta{ @@ -1117,7 +1064,7 @@ func TestUpdatePVCRefCounts(t *testing.T) { VolumeSource: v1.VolumeSource{PersistentVolumeClaim: &v1.PersistentVolumeClaimVolumeSource{ClaimName: pvcName1}}, }, } - cache.AddPod(pod1) + cache.UpdatePod(pod1) assert.Check(t, cache.IsPVCUsedByPods(framework.GetNamespacedName(pod1.Namespace, pvcName1)), "pvc1 is not in pvcRefCounts") // add a pod without assigned node can't update pvcRefCounts @@ -1130,7 +1077,7 @@ func TestUpdatePVCRefCounts(t *testing.T) { VolumeSource: v1.VolumeSource{PersistentVolumeClaim: &v1.PersistentVolumeClaimVolumeSource{ClaimName: pvcName2}}, }, } - cache.AddPod(pod2) + cache.UpdatePod(pod2) assert.Check(t, !cache.IsPVCUsedByPods(framework.GetNamespacedName(pod2.Namespace, pvcName2)), "pvc2 is in pvcRefCounts") // assign a node to pod2 diff --git a/pkg/cache/metadata.go b/pkg/cache/metadata.go index f4ff387c8..b3b038733 100644 --- a/pkg/cache/metadata.go +++ b/pkg/cache/metadata.go @@ -60,7 +60,7 @@ func getTaskMetadata(pod *v1.Pod) (TaskMetadata, bool) { }, true } -func getAppMetadata(pod *v1.Pod, recovery bool) (ApplicationMetadata, bool) { +func getAppMetadata(pod *v1.Pod) (ApplicationMetadata, bool) { appID := utils.GetApplicationIDFromPod(pod) if appID == "" { log.Log(log.ShimCacheApplication).Debug("unable to get application for pod", @@ -113,11 +113,7 @@ func getAppMetadata(pod *v1.Pod, recovery bool) (ApplicationMetadata, bool) { ownerReferences := getOwnerReference(pod) schedulingPolicyParams := GetSchedulingPolicyParam(pod) tags[constants.AnnotationSchedulingPolicyParam] = pod.Annotations[constants.AnnotationSchedulingPolicyParam] - - var creationTime int64 - if recovery { - creationTime = pod.CreationTimestamp.Unix() - } + creationTime := pod.CreationTimestamp.Unix() return ApplicationMetadata{ ApplicationID: appID, diff --git a/pkg/cache/metadata_test.go b/pkg/cache/metadata_test.go index c85400378..f727cfaa8 100644 --- a/pkg/cache/metadata_test.go +++ b/pkg/cache/metadata_test.go @@ -133,7 +133,7 @@ func TestGetAppMetadata(t *testing.T) { //nolint:funlen }, } - app, ok := getAppMetadata(&pod, false) + app, ok := getAppMetadata(&pod) assert.Equal(t, ok, true) assert.Equal(t, app.ApplicationID, "app00001") assert.Equal(t, app.QueueName, "root.a") @@ -175,7 +175,7 @@ func TestGetAppMetadata(t *testing.T) { //nolint:funlen }, } - app, ok = getAppMetadata(&pod, false) + app, ok = getAppMetadata(&pod) assert.Equal(t, ok, true) assert.Equal(t, app.ApplicationID, "app00002") assert.Equal(t, app.QueueName, "root.b") @@ -208,7 +208,7 @@ func TestGetAppMetadata(t *testing.T) { //nolint:funlen }, } - app, ok = getAppMetadata(&pod, false) + app, ok = getAppMetadata(&pod) assert.Equal(t, ok, true) assert.Equal(t, app.SchedulingPolicyParameters.GetGangSchedulingStyle(), "Soft") @@ -238,7 +238,7 @@ func TestGetAppMetadata(t *testing.T) { //nolint:funlen }, } - app, ok = getAppMetadata(&pod, false) + app, ok = getAppMetadata(&pod) assert.Equal(t, ok, true) assert.Equal(t, app.SchedulingPolicyParameters.GetGangSchedulingStyle(), "Soft") @@ -261,19 +261,19 @@ func TestGetAppMetadata(t *testing.T) { //nolint:funlen } utils.SetPluginMode(false) - app, ok = getAppMetadata(&pod, false) + app, ok = getAppMetadata(&pod) conf.GetSchedulerConf().GenerateUniqueAppIds = true assert.Equal(t, ok, true) assert.Equal(t, app.ApplicationID, "yunikorn-app-namespace-01-autogen") utils.SetPluginMode(false) conf.GetSchedulerConf().GenerateUniqueAppIds = true - app, ok = getAppMetadata(&pod, false) + app, ok = getAppMetadata(&pod) assert.Equal(t, ok, true) assert.Equal(t, app.ApplicationID, "app-namespace-01-UID-POD-00001") utils.SetPluginMode(true) - app, ok = getAppMetadata(&pod, false) + app, ok = getAppMetadata(&pod) assert.Equal(t, ok, false) } diff --git a/pkg/cache/node.go b/pkg/cache/node.go deleted file mode 100644 index 3f69fcb5b..000000000 --- a/pkg/cache/node.go +++ /dev/null @@ -1,202 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "context" - "sync" - - "github.com/looplab/fsm" - "go.uber.org/zap" - - "github.com/apache/yunikorn-k8shim/pkg/common" - "github.com/apache/yunikorn-k8shim/pkg/common/events" - "github.com/apache/yunikorn-k8shim/pkg/dispatcher" - "github.com/apache/yunikorn-k8shim/pkg/log" - "github.com/apache/yunikorn-scheduler-interface/lib/go/api" - "github.com/apache/yunikorn-scheduler-interface/lib/go/si" -) - -// stores info about what scheduler cares about a node -type SchedulerNode struct { - name string - uid string - labels map[string]string - schedulable bool - schedulerAPI api.SchedulerAPI - fsm *fsm.FSM - - // mutable values need locking - capacity *si.Resource - occupied *si.Resource - ready bool - existingAllocations []*si.Allocation - - lock *sync.RWMutex -} - -func newSchedulerNode(nodeName string, nodeUID string, nodeLabels map[string]string, - nodeResource *si.Resource, schedulerAPI api.SchedulerAPI, schedulable bool, ready bool) *SchedulerNode { - schedulerNode := &SchedulerNode{ - name: nodeName, - uid: nodeUID, - labels: nodeLabels, - capacity: nodeResource, - occupied: common.NewResourceBuilder().Build(), - schedulerAPI: schedulerAPI, - schedulable: schedulable, - lock: &sync.RWMutex{}, - ready: ready, - fsm: newSchedulerNodeState(), - } - return schedulerNode -} - -func (n *SchedulerNode) snapshotState() (capacity *si.Resource, occupied *si.Resource, ready bool) { - n.lock.RLock() - defer n.lock.RUnlock() - return n.capacity, n.occupied, n.ready -} - -func (n *SchedulerNode) addExistingAllocation(allocation *si.Allocation) { - n.lock.Lock() - defer n.lock.Unlock() - log.Log(log.ShimCacheNode).Info("add existing allocation", - zap.String("nodeID", n.name), - zap.Any("allocation", allocation)) - n.existingAllocations = append(n.existingAllocations, allocation) -} - -func (n *SchedulerNode) updateOccupiedResource(resource *si.Resource, opt updateType) (capacity *si.Resource, occupied *si.Resource, ready bool) { - n.lock.Lock() - defer n.lock.Unlock() - switch opt { - case AddOccupiedResource: - log.Log(log.ShimCacheNode).Info("add node occupied resource", - zap.String("nodeID", n.name), - zap.Stringer("occupied", resource)) - n.occupied = common.Add(n.occupied, resource) - case SubOccupiedResource: - log.Log(log.ShimCacheNode).Info("subtract node occupied resource", - zap.String("nodeID", n.name), - zap.Stringer("occupied", resource)) - n.occupied = common.Sub(n.occupied, resource) - default: - // noop - } - return n.capacity, n.occupied, n.ready -} - -func (n *SchedulerNode) setCapacity(capacity *si.Resource) { - n.lock.Lock() - defer n.lock.Unlock() - log.Log(log.ShimCacheNode).Debug("set node capacity", - zap.String("nodeID", n.name), - zap.Stringer("capacity", capacity)) - n.capacity = capacity -} - -func (n *SchedulerNode) setReadyStatus(ready bool) { - n.lock.Lock() - defer n.lock.Unlock() - log.Log(log.ShimCacheNode).Debug("set node ready status", - zap.String("nodeID", n.name), - zap.Bool("ready", ready)) - n.ready = ready -} - -func (n *SchedulerNode) getNodeState() string { - // fsm has its own internal lock, we don't need to hold node's lock here - return n.fsm.Current() -} - -func (n *SchedulerNode) postNodeAccepted() { - // when node is accepted, it means the node is already registered to the scheduler, - // this doesn't mean this node is ready for scheduling, there is a step away. - // we need to check the K8s node state, if it is not schedulable, then we should notify - // the scheduler to not schedule new pods onto it. - if n.schedulable { - dispatcher.Dispatch(CachedSchedulerNodeEvent{ - NodeID: n.name, - Event: NodeReady, - }) - } else { - dispatcher.Dispatch(CachedSchedulerNodeEvent{ - NodeID: n.name, - Event: DrainNode, - }) - } -} - -func (n *SchedulerNode) handleNodeRecovery() { - log.Log(log.ShimCacheNode).Info("node recovering", - zap.String("nodeID", n.name), - zap.Bool("schedulable", n.schedulable)) - - nodeRequest := common.CreateUpdateRequestForNewNode(n.name, n.labels, n.capacity, n.occupied, n.existingAllocations, n.ready) - - // send node request to scheduler-core - if err := n.schedulerAPI.UpdateNode(nodeRequest); err != nil { - log.Log(log.ShimCacheNode).Error("failed to send UpdateNode request", - zap.Any("request", nodeRequest)) - } -} - -func (n *SchedulerNode) handleDrainNode() { - log.Log(log.ShimCacheNode).Info("node enters draining mode", - zap.String("nodeID", n.name)) - - nodeRequest := common.CreateUpdateRequestForDeleteOrRestoreNode(n.name, si.NodeInfo_DRAIN_NODE) - - // send request to scheduler-core - if err := n.schedulerAPI.UpdateNode(nodeRequest); err != nil { - log.Log(log.ShimCacheNode).Error("failed to send UpdateNode request", - zap.Any("request", nodeRequest)) - } -} - -func (n *SchedulerNode) handleRestoreNode() { - log.Log(log.ShimCacheNode).Info("restore node from draining mode", - zap.String("nodeID", n.name)) - - nodeRequest := common.CreateUpdateRequestForDeleteOrRestoreNode(n.name, si.NodeInfo_DRAIN_TO_SCHEDULABLE) - - // send request to scheduler-core - if err := n.schedulerAPI.UpdateNode(nodeRequest); err != nil { - log.Log(log.ShimCacheNode).Error("failed to send UpdateNode request", - zap.Any("request", nodeRequest)) - } -} - -func (n *SchedulerNode) handle(ev events.SchedulerNodeEvent) error { - n.lock.Lock() - defer n.lock.Unlock() - err := n.fsm.Event(context.Background(), ev.GetEvent(), n) - // handle the same state transition not nil error (limit of fsm). - if err != nil && err.Error() != "no transition" { - return err - } - return nil -} - -func (n *SchedulerNode) canHandle(ev events.SchedulerNodeEvent) bool { - n.lock.RLock() - defer n.lock.RUnlock() - return n.fsm.Can(ev.GetEvent()) -} diff --git a/pkg/cache/node_events.go b/pkg/cache/node_events.go new file mode 100644 index 000000000..545691ca4 --- /dev/null +++ b/pkg/cache/node_events.go @@ -0,0 +1,47 @@ +/* + Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +package cache + +type SchedulerNodeEventType int + +const ( + NodeAccepted SchedulerNodeEventType = iota + NodeRejected +) + +func (ae SchedulerNodeEventType) String() string { + return [...]string{"NodeAccepted", "NodeRejected"}[ae] +} + +type CachedSchedulerNodeEvent struct { + NodeID string + Event SchedulerNodeEventType +} + +func (sn CachedSchedulerNodeEvent) GetEvent() string { + return sn.Event.String() +} + +func (sn CachedSchedulerNodeEvent) GetNodeID() string { + return sn.NodeID +} + +func (sn CachedSchedulerNodeEvent) GetArgs() []interface{} { + return nil +} diff --git a/pkg/cache/node_graphviz_test.go b/pkg/cache/node_graphviz_test.go deleted file mode 100644 index a31e9df4f..000000000 --- a/pkg/cache/node_graphviz_test.go +++ /dev/null @@ -1,49 +0,0 @@ -//go:build graphviz -// +build graphviz - -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "os" - "testing" - - "github.com/looplab/fsm" - "gotest.tools/v3/assert" - - "github.com/apache/yunikorn-k8shim/pkg/common" - "github.com/apache/yunikorn-k8shim/pkg/common/test" - siCommon "github.com/apache/yunikorn-scheduler-interface/lib/go/common" -) - -func TestNodeFsmGraph(t *testing.T) { - api := test.NewSchedulerAPIMock() - r1 := common.NewResourceBuilder(). - AddResource(siCommon.Memory, 1). - AddResource(siCommon.CPU, 1). - Build() - node := newSchedulerNode("host001", "UID001", map[string]string{}, r1, api, false, false) - graph := fsm.Visualize(node.fsm) - - err := os.MkdirAll("../../build/fsm", 0755) - assert.NilError(t, err, "Creating output dir failed") - os.WriteFile("../../build/fsm/k8shim-node-state.dot", []byte(graph), 0644) - assert.NilError(t, err, "Writing graph failed") -} diff --git a/pkg/cache/node_state.go b/pkg/cache/node_state.go deleted file mode 100644 index 94191ab08..000000000 --- a/pkg/cache/node_state.go +++ /dev/null @@ -1,159 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "context" - "sync" - - "github.com/looplab/fsm" - "go.uber.org/zap" - - "github.com/apache/yunikorn-k8shim/pkg/common/events" - "github.com/apache/yunikorn-k8shim/pkg/log" -) - -var nodeStatesOnce sync.Once - -// ---------------------------------------------- -// SchedulerNode events -// ---------------------------------------------- -type SchedulerNodeEventType int - -const ( - RecoverNode SchedulerNodeEventType = iota - NodeAccepted - NodeRejected - DrainNode - RestoreNode - NodeReady -) - -func (ae SchedulerNodeEventType) String() string { - return [...]string{"RecoverNode", "NodeAccepted", "NodeRejected", "DrainNode", "RestoreNode", "NodeReady"}[ae] -} - -type CachedSchedulerNodeEvent struct { - NodeID string - Event SchedulerNodeEventType -} - -func (sn CachedSchedulerNodeEvent) GetEvent() string { - return sn.Event.String() -} - -func (sn CachedSchedulerNodeEvent) GetNodeID() string { - return sn.NodeID -} - -func (sn CachedSchedulerNodeEvent) GetArgs() []interface{} { - return nil -} - -// ---------------------------------- -// SchedulerNode states -// ---------------------------------- -var storeSchedulerNodeStates *NStates - -type NStates struct { - New string - Recovering string - Accepted string - Healthy string - Rejected string - Draining string -} - -func SchedulerNodeStates() *NStates { - nodeStatesOnce.Do(func() { - storeSchedulerNodeStates = &NStates{ - New: "New", - Recovering: "Recovering", - Accepted: "Accepted", - Healthy: "Healthy", - Rejected: "Rejected", - Draining: "Draining", - } - }) - return storeSchedulerNodeStates -} - -func newSchedulerNodeState() *fsm.FSM { - states := SchedulerNodeStates() - return fsm.NewFSM( - states.New, fsm.Events{ - { - Name: RecoverNode.String(), - Src: []string{states.New}, - Dst: states.Recovering, - }, - { - Name: NodeAccepted.String(), - Src: []string{states.Recovering}, - Dst: states.Accepted, - }, - { - Name: NodeReady.String(), - Src: []string{states.Accepted}, - Dst: states.Healthy, - }, - { - Name: NodeRejected.String(), - Src: []string{states.New, states.Recovering}, - Dst: states.Rejected, - }, - { - Name: DrainNode.String(), - Src: []string{states.Healthy, states.Accepted}, - Dst: states.Draining, - }, - { - Name: RestoreNode.String(), - Src: []string{states.Draining}, - Dst: states.Healthy, - }, - }, - fsm.Callbacks{ - events.EnterState: func(_ context.Context, event *fsm.Event) { - node := event.Args[0].(*SchedulerNode) //nolint:errcheck - log.Log(log.ShimFSM).Debug("shim node state transition", - zap.String("nodeID", node.name), - zap.String("source", event.Src), - zap.String("destination", event.Dst), - zap.String("event", event.Event)) - }, - states.Accepted: func(_ context.Context, event *fsm.Event) { - node := event.Args[0].(*SchedulerNode) //nolint:errcheck - node.postNodeAccepted() - }, - states.Recovering: func(_ context.Context, event *fsm.Event) { - node := event.Args[0].(*SchedulerNode) //nolint:errcheck - node.handleNodeRecovery() - }, - DrainNode.String(): func(_ context.Context, event *fsm.Event) { - node := event.Args[0].(*SchedulerNode) //nolint:errcheck - node.handleDrainNode() - }, - RestoreNode.String(): func(_ context.Context, event *fsm.Event) { - node := event.Args[0].(*SchedulerNode) //nolint:errcheck - node.handleRestoreNode() - }, - }, - ) -} diff --git a/pkg/cache/node_test.go b/pkg/cache/node_test.go deleted file mode 100644 index e244c2cbd..000000000 --- a/pkg/cache/node_test.go +++ /dev/null @@ -1,92 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "testing" - - "github.com/google/go-cmp/cmp/cmpopts" - "gotest.tools/v3/assert" - - "github.com/apache/yunikorn-k8shim/pkg/common" - "github.com/apache/yunikorn-k8shim/pkg/common/constants" - "github.com/apache/yunikorn-k8shim/pkg/common/test" - siCommon "github.com/apache/yunikorn-scheduler-interface/lib/go/common" - "github.com/apache/yunikorn-scheduler-interface/lib/go/si" -) - -func TestAddExistingAllocation(t *testing.T) { - node := NewTestSchedulerNode() - alloc01 := si.Allocation{ - AllocationKey: "pod001", - AllocationTags: nil, - UUID: "podUID001", - ResourcePerAlloc: nil, - Priority: 0, - NodeID: "host001", - ApplicationID: "", - PartitionName: constants.DefaultPartition, - } - node.addExistingAllocation(&alloc01) - assert.Equal(t, len(node.existingAllocations), 1) - alloc02 := node.existingAllocations[0] - assert.Equal(t, alloc02.AllocationKey, alloc01.AllocationKey) - assert.Equal(t, alloc02.UUID, alloc01.UUID) - assert.Equal(t, alloc02.NodeID, alloc01.NodeID) - assert.Equal(t, alloc02.PartitionName, alloc01.PartitionName) -} - -func TestUpdateOccupiedResource(t *testing.T) { - node := NewTestSchedulerNode() - r1 := common.NewResourceBuilder(). - AddResource(siCommon.Memory, 5). - AddResource(siCommon.CPU, 5). - Build() - r2 := common.NewResourceBuilder(). - AddResource(siCommon.Memory, 1). - AddResource(siCommon.CPU, 1). - Build() - r3 := common.NewResourceBuilder(). - AddResource(siCommon.Memory, 4). - AddResource(siCommon.CPU, 4). - Build() - - capacity, occupied, ready := node.updateOccupiedResource(r1, AddOccupiedResource) - assert.DeepEqual(t, capacity, r2, cmpopts.IgnoreUnexported(si.Resource{}, si.Quantity{})) - assert.DeepEqual(t, occupied, r1, cmpopts.IgnoreUnexported(si.Resource{}, si.Quantity{})) - assert.Assert(t, ready) - - capacity, occupied, ready = node.updateOccupiedResource(r2, SubOccupiedResource) - assert.DeepEqual(t, capacity, r2, cmpopts.IgnoreUnexported(si.Resource{}, si.Quantity{})) - assert.DeepEqual(t, occupied, r3, cmpopts.IgnoreUnexported(si.Resource{}, si.Quantity{})) - assert.Assert(t, ready) -} - -func NewTestSchedulerNode() *SchedulerNode { - api := test.NewSchedulerAPIMock() - r1 := common.NewResourceBuilder(). - AddResource(siCommon.Memory, 1). - AddResource(siCommon.CPU, 1). - Build() - node := newSchedulerNode("host001", "UID001", map[string]string{ - "key1": "label1", - "key2": "label2", - }, r1, api, false, true) - return node -} diff --git a/pkg/cache/nodes.go b/pkg/cache/nodes.go deleted file mode 100644 index b92bf037f..000000000 --- a/pkg/cache/nodes.go +++ /dev/null @@ -1,239 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "fmt" - "sync" - - "go.uber.org/zap" - v1 "k8s.io/api/core/v1" - - "github.com/apache/yunikorn-k8shim/pkg/cache/external" - "github.com/apache/yunikorn-k8shim/pkg/common" - "github.com/apache/yunikorn-k8shim/pkg/common/events" - "github.com/apache/yunikorn-k8shim/pkg/dispatcher" - "github.com/apache/yunikorn-k8shim/pkg/log" - "github.com/apache/yunikorn-scheduler-interface/lib/go/api" - "github.com/apache/yunikorn-scheduler-interface/lib/go/si" -) - -type updateType int - -const ( - AddOccupiedResource updateType = iota - SubOccupiedResource -) - -// scheduler nodes maintain cluster nodes and their status for the scheduler -type schedulerNodes struct { - proxy api.SchedulerAPI - nodesMap map[string]*SchedulerNode - cache *external.SchedulerCache - lock *sync.RWMutex -} - -func newSchedulerNodes(schedulerAPI api.SchedulerAPI, cache *external.SchedulerCache) *schedulerNodes { - return &schedulerNodes{ - proxy: schedulerAPI, - nodesMap: make(map[string]*SchedulerNode), - cache: cache, - lock: &sync.RWMutex{}, - } -} - -func (nc *schedulerNodes) getNode(name string) *SchedulerNode { - nc.lock.RLock() - defer nc.lock.RUnlock() - if node, ok := nc.nodesMap[name]; ok { - return node - } - return nil -} - -func convertToNode(obj interface{}) (*v1.Node, error) { - if node, ok := obj.(*v1.Node); ok { - return node, nil - } - return nil, fmt.Errorf("cannot convert to *v1.Node: %v", obj) -} - -func equals(n1 *v1.Node, n2 *v1.Node) bool { - n1Resource := common.GetNodeResource(&n1.Status) - n2Resource := common.GetNodeResource(&n2.Status) - return common.Equals(n1Resource, n2Resource) -} - -func (nc *schedulerNodes) addExistingAllocation(allocation *si.Allocation) error { - nc.lock.Lock() - defer nc.lock.Unlock() - if schedulerNode, ok := nc.nodesMap[allocation.NodeID]; ok { - schedulerNode.addExistingAllocation(allocation) - return nil - } - return fmt.Errorf("orphan allocation %v", allocation) -} - -func (nc *schedulerNodes) addNode(node *v1.Node) { - nc.addAndReportNode(node, true) -} - -func (nc *schedulerNodes) addAndReportNode(node *v1.Node, reportNode bool) { - nc.lock.Lock() - defer nc.lock.Unlock() - - // add node to nodes map - if _, ok := nc.nodesMap[node.Name]; !ok { - log.Log(log.ShimCacheNode).Info("adding node to context", - zap.String("nodeName", node.Name), - zap.Any("nodeLabels", node.Labels), - zap.Bool("schedulable", !node.Spec.Unschedulable)) - - ready := hasReadyCondition(node) - newNode := newSchedulerNode(node.Name, string(node.UID), node.Labels, - common.GetNodeResource(&node.Status), nc.proxy, !node.Spec.Unschedulable, ready) - nc.nodesMap[node.Name] = newNode - } - - // once node is added to scheduler, first thing is to recover its state - // node might already be in healthy state, previously recovered during recovery process, - // do not trigger recover again in this case. - if reportNode { - if node, ok := nc.nodesMap[node.Name]; ok { - triggerEvent(node, SchedulerNodeStates().New, RecoverNode) - } - } -} - -func (nc *schedulerNodes) updateNodeOccupiedResources(name string, resource *si.Resource, opt updateType) { - if common.IsZero(resource) { - return - } - - if schedulerNode := nc.getNode(name); schedulerNode != nil { - capacity, occupied, ready := schedulerNode.updateOccupiedResource(resource, opt) - request := common.CreateUpdateRequestForUpdatedNode(name, capacity, occupied, ready) - log.Log(log.ShimCacheNode).Info("report occupied resources updates", - zap.String("node", schedulerNode.name), - zap.Any("request", request)) - if err := nc.proxy.UpdateNode(request); err != nil { - log.Log(log.ShimCacheNode).Info("hitting error while handling UpdateNode", zap.Error(err)) - } - } -} - -func (nc *schedulerNodes) updateNode(oldNode, newNode *v1.Node) { - // before updating a node, check if it exists in the cache or not - // if we receive a update node event but the node doesn't exist, - // we need to add it instead of updating it. - cachedNode := nc.getNode(newNode.Name) - if cachedNode == nil { - nc.addNode(newNode) - return - } - - nc.lock.Lock() - defer nc.lock.Unlock() - - // cordon or restore node - if (!oldNode.Spec.Unschedulable) && newNode.Spec.Unschedulable { - triggerEvent(cachedNode, SchedulerNodeStates().Healthy, DrainNode) - } else if oldNode.Spec.Unschedulable && !newNode.Spec.Unschedulable { - triggerEvent(cachedNode, SchedulerNodeStates().Draining, RestoreNode) - } - - ready := hasReadyCondition(newNode) - capacityUpdated := equals(oldNode, newNode) - readyUpdated := cachedNode.ready == ready - - if capacityUpdated && readyUpdated { - return - } - - // Has node resource updated? - if !capacityUpdated { - cachedNode.setCapacity(common.GetNodeResource(&newNode.Status)) - } - - // Has node ready status flag updated? - if !readyUpdated { - cachedNode.setReadyStatus(ready) - } - - log.Log(log.ShimCacheNode).Info("Node's ready status flag", zap.String("Node name", newNode.Name), - zap.Bool("ready", ready)) - - capacity, occupied, ready := cachedNode.snapshotState() - request := common.CreateUpdateRequestForUpdatedNode(newNode.Name, capacity, occupied, ready) - log.Log(log.ShimCacheNode).Info("report updated nodes to scheduler", zap.Any("request", request)) - if err := nc.proxy.UpdateNode(request); err != nil { - log.Log(log.ShimCacheNode).Info("hitting error while handling UpdateNode", zap.Error(err)) - } -} - -func (nc *schedulerNodes) deleteNode(node *v1.Node) { - nc.lock.Lock() - defer nc.lock.Unlock() - - delete(nc.nodesMap, node.Name) - - request := common.CreateUpdateRequestForDeleteOrRestoreNode(node.Name, si.NodeInfo_DECOMISSION) - log.Log(log.ShimCacheNode).Info("report updated nodes to scheduler", zap.Any("request", request.String())) - if err := nc.proxy.UpdateNode(request); err != nil { - log.Log(log.ShimCacheNode).Error("hitting error while handling UpdateNode", zap.Error(err)) - } -} - -func (nc *schedulerNodes) schedulerNodeEventHandler() func(obj interface{}) { - return func(obj interface{}) { - if event, ok := obj.(events.SchedulerNodeEvent); ok { - if node := nc.getNode(event.GetNodeID()); node != nil { - if node.canHandle(event) { - if err := node.handle(event); err != nil { - log.Log(log.ShimCacheNode).Error("failed to handle scheduler node event", - zap.String("event", event.GetEvent()), - zap.Error(err)) - } - } - } - } - } -} - -func hasReadyCondition(node *v1.Node) bool { - if node != nil { - for _, condition := range node.Status.Conditions { - if condition.Type == v1.NodeReady && condition.Status == v1.ConditionTrue { - return true - } - } - } - return false -} - -func triggerEvent(node *SchedulerNode, currentState string, eventType SchedulerNodeEventType) { - log.Log(log.ShimCacheNode).Info("scheduler node event ", zap.String("name", node.name), - zap.String("current state ", currentState), zap.Stringer("transition to ", eventType)) - if node.getNodeState() == currentState { - dispatcher.Dispatch(CachedSchedulerNodeEvent{ - NodeID: node.name, - Event: eventType, - }) - } -} diff --git a/pkg/cache/nodes_test.go b/pkg/cache/nodes_test.go deleted file mode 100644 index b65dfe827..000000000 --- a/pkg/cache/nodes_test.go +++ /dev/null @@ -1,503 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "strconv" - "testing" - "time" - - "gotest.tools/v3/assert" - v1 "k8s.io/api/core/v1" - "k8s.io/apimachinery/pkg/api/resource" - apis "k8s.io/apimachinery/pkg/apis/meta/v1" - - "github.com/apache/yunikorn-k8shim/pkg/cache/external" - "github.com/apache/yunikorn-k8shim/pkg/client" - "github.com/apache/yunikorn-k8shim/pkg/common/test" - "github.com/apache/yunikorn-k8shim/pkg/common/utils" - "github.com/apache/yunikorn-k8shim/pkg/dispatcher" - siCommon "github.com/apache/yunikorn-scheduler-interface/lib/go/common" - "github.com/apache/yunikorn-scheduler-interface/lib/go/si" -) - -func TestAddNode(t *testing.T) { - api := test.NewSchedulerAPIMock() - - // register fn doesn't nothing than checking input - api.UpdateNodeFunction(getUpdateNodeFunction(t, "host0001", 1024*1000*1000, 10000, false)) - - nodes := newSchedulerNodes(api, NewTestSchedulerCache()) - dispatcher.RegisterEventHandler(dispatcher.EventTypeNode, nodes.schedulerNodeEventHandler()) - dispatcher.Start() - defer dispatcher.Stop() - - resourceList := make(map[v1.ResourceName]resource.Quantity) - resourceList[v1.ResourceName("memory")] = *resource.NewQuantity(1024*1000*1000, resource.DecimalSI) - resourceList[v1.ResourceName("cpu")] = *resource.NewQuantity(10, resource.DecimalSI) - var newNode = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Allocatable: resourceList, - }, - } - - nodes.addNode(&newNode) - - // values are verified in injected fn - // verify register is not called, update is called and just called once - err := utils.WaitForCondition(func() bool { - return api.GetRegisterCount() == 0 - }, time.Second, 5*time.Second) - assert.NilError(t, err) - - err = utils.WaitForCondition(func() bool { - return api.GetUpdateNodeCount() == 1 - }, time.Second, 5*time.Second) - assert.NilError(t, err) -} - -func TestUpdateNode(t *testing.T) { - api := test.NewSchedulerAPIMock() - - nodes := newSchedulerNodes(api, NewTestSchedulerCache()) - dispatcher.RegisterEventHandler(dispatcher.EventTypeNode, nodes.schedulerNodeEventHandler()) - dispatcher.Start() - defer dispatcher.Stop() - - resourceList := make(map[v1.ResourceName]resource.Quantity) - resourceList[v1.ResourceName("memory")] = *resource.NewQuantity(1024*1000*1000, resource.DecimalSI) - resourceList[v1.ResourceName("cpu")] = *resource.NewQuantity(10, resource.DecimalSI) - - var oldNode = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Allocatable: resourceList, - }, - } - - var newNode = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Allocatable: resourceList, - }, - } - - // this function validates the new node can be added - // this verifies the shim sends the si.UpdateRequest to core with the new node info - api.UpdateNodeFunction(getUpdateNodeFunction(t, "host0001", 1024*1000*1000, 10000, false)) - - // add the node first - nodes.addNode(&oldNode) - - // wait for node being added - assert.NilError(t, utils.WaitForCondition(func() bool { - return api.GetUpdateNodeCount() == 1 - }, time.Second, 5*time.Second)) - assert.Assert(t, nodes.getNode("host0001") != nil) - assert.Equal(t, nodes.getNode("host0001").name, "host0001") - - // reset all counters to make the verification easier - api.ResetAllCounters() - - // if node resource stays same, update update should be ignored - ignoreNodeUpdateFn := func(request *si.NodeRequest) error { - if request.Nodes != nil && len(request.Nodes) > 0 { - t.Fatalf("expecting no update nodes sent to scheduler as node resource has no change") - } - - return nil - } - api.UpdateNodeFunction(ignoreNodeUpdateFn) - nodes.updateNode(&oldNode, &newNode) - assert.Equal(t, api.GetRegisterCount(), int32(0)) - assert.Equal(t, api.GetUpdateNodeCount(), int32(0)) - - // change new node's resource, afterwards the update request should be sent to the scheduler - newResourceList := make(map[v1.ResourceName]resource.Quantity) - newResourceList[v1.ResourceName("memory")] = *resource.NewQuantity(2048*1000*1000, resource.DecimalSI) - newResourceList[v1.ResourceName("cpu")] = *resource.NewQuantity(10, resource.DecimalSI) - newNode = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Allocatable: newResourceList, - }, - } - - api.UpdateNodeFunction(getUpdateNodeFunction(t, "host0001", 2048*1000*1000, 10000, false)) - - nodes.updateNode(&oldNode, &newNode) - assert.Equal(t, api.GetRegisterCount(), int32(0)) - assert.Equal(t, api.GetUpdateNodeCount(), int32(1)) - - condition := v1.NodeCondition{Type: v1.NodeReady, Status: v1.ConditionTrue} - var conditions []v1.NodeCondition - conditions = append(conditions, condition) - - newNode1 := v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Allocatable: newResourceList, - Conditions: conditions, - }, - } - - api.UpdateNodeFunction(getUpdateNodeFunction(t, "host0001", 2048*1000*1000, 10000, true)) - - nodes.updateNode(&oldNode, &newNode1) - assert.Equal(t, api.GetRegisterCount(), int32(0)) - assert.Equal(t, api.GetUpdateNodeCount(), int32(2)) -} - -func TestUpdateWithoutNodeAdded(t *testing.T) { - api := test.NewSchedulerAPIMock() - - nodes := newSchedulerNodes(api, NewTestSchedulerCache()) - dispatcher.RegisterEventHandler(dispatcher.EventTypeNode, nodes.schedulerNodeEventHandler()) - dispatcher.Start() - defer dispatcher.Stop() - - resourceList := make(map[v1.ResourceName]resource.Quantity) - resourceList[v1.ResourceName("memory")] = *resource.NewQuantity(1024*1000*1000, resource.DecimalSI) - resourceList[v1.ResourceName("cpu")] = *resource.NewQuantity(10, resource.DecimalSI) - - var oldNode = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Allocatable: resourceList, - }, - } - - var newNode = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Allocatable: resourceList, - }, - } - - api.UpdateNodeFunction(getUpdateNodeFunction(t, "host0001", 1024*1000*1000, 10000, false)) - - // directly trigger an update - // if the node was not seeing in the cache, we should see the node be added - nodes.updateNode(&oldNode, &newNode) - - // wait for node being added - assert.NilError(t, utils.WaitForCondition(func() bool { - return api.GetUpdateNodeCount() == 1 - }, time.Second, 5*time.Second)) - assert.Assert(t, nodes.getNode("host0001") != nil) - assert.Equal(t, nodes.getNode("host0001").name, "host0001") - assert.Equal(t, api.GetUpdateNodeCount(), int32(1)) - - // change new node's resource, afterwards the update request should be sent to the scheduler - newResourceList := make(map[v1.ResourceName]resource.Quantity) - newResourceList[v1.ResourceName("memory")] = *resource.NewQuantity(2048*1000*1000, resource.DecimalSI) - newResourceList[v1.ResourceName("cpu")] = *resource.NewQuantity(10, resource.DecimalSI) - newNode = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Allocatable: newResourceList, - }, - } - - api.UpdateNodeFunction(getUpdateNodeFunction(t, "host0001", 2048*1000*1000, 10000, false)) - - nodes.updateNode(&oldNode, &newNode) - assert.Equal(t, api.GetRegisterCount(), int32(0)) - assert.Equal(t, api.GetUpdateNodeCount(), int32(2)) -} - -func TestDeleteNode(t *testing.T) { - api := test.NewSchedulerAPIMock() - nodes := newSchedulerNodes(api, NewTestSchedulerCache()) - dispatcher.RegisterEventHandler(dispatcher.EventTypeNode, nodes.schedulerNodeEventHandler()) - dispatcher.Start() - defer dispatcher.Stop() - - resourceList := make(map[v1.ResourceName]resource.Quantity) - resourceList[v1.ResourceName("memory")] = *resource.NewQuantity(1024*1000*1000, resource.DecimalSI) - resourceList[v1.ResourceName("cpu")] = *resource.NewQuantity(10, resource.DecimalSI) - - var node = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Allocatable: resourceList, - }, - } - - ignoreNodeUpdateFn := func(request *si.NodeRequest) error { - // fake update - return nil - } - api.UpdateNodeFunction(ignoreNodeUpdateFn) - - // add node to the cache - nodes.addNode(&node) - err := utils.WaitForCondition(func() bool { - return api.GetRegisterCount() == 0 - }, 1*time.Second, 5*time.Second) - assert.NilError(t, err) - err = utils.WaitForCondition(func() bool { - return api.GetUpdateNodeCount() == 1 - }, 100*time.Millisecond, 1000*time.Millisecond) - assert.NilError(t, err) - - // delete node should trigger another update - nodes.deleteNode(&node) - err = utils.WaitForCondition(func() bool { - return api.GetUpdateNodeCount() == 2 - }, 100*time.Millisecond, 1000*time.Millisecond) - assert.NilError(t, err) - - // ensure the node is removed from cache - assert.Assert(t, nodes.getNode("host0001") == nil) - - // add the node back, hostName is same but UID is different - var nodeNew = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_002", - }, - Status: v1.NodeStatus{ - Allocatable: resourceList, - }, - } - nodes.addNode(&nodeNew) - err = utils.WaitForCondition(func() bool { - return api.GetUpdateNodeCount() == 3 - }, 100*time.Millisecond, 1000*time.Millisecond) - assert.NilError(t, err) - - assert.Assert(t, nodes.getNode("host0001") != nil) - assert.Equal(t, nodes.getNode("host0001").name, "host0001") - assert.Equal(t, nodes.getNode("host0001").uid, "uid_002") - - // remove the node again, and then try update - nodes.deleteNode(&nodeNew) - err = utils.WaitForCondition(func() bool { - return api.GetUpdateNodeCount() == 4 - }, 100*time.Millisecond, 1000*time.Millisecond) - assert.NilError(t, err) - - // instead of a add, do a update - // this could happen when a node is removed and added back, - // or a new node is created with the same hostname - var nodeNew2 = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_003", - }, - Status: v1.NodeStatus{ - Allocatable: resourceList, - }, - } - - // update the node, this will trigger a update to add the node - nodes.updateNode(&nodeNew, &nodeNew2) - err = utils.WaitForCondition(func() bool { - return api.GetUpdateNodeCount() == 5 - }, 100*time.Millisecond, 1000*time.Millisecond) - assert.NilError(t, err) - - assert.Assert(t, nodes.getNode("host0001") != nil) - assert.Equal(t, nodes.getNode("host0001").name, "host0001") - assert.Equal(t, nodes.getNode("host0001").uid, "uid_003") -} - -// A wrapper around the scheduler cache which does not initialise the lister and volumebinder -func NewTestSchedulerCache() *external.SchedulerCache { - return external.NewSchedulerCache(client.NewMockedAPIProvider(false).GetAPIs()) -} - -func TestCordonNode(t *testing.T) { - api := test.NewSchedulerAPIMock() - - // register fn doesn't nothing than checking input - inputCheckerUpdateFn := func(request *si.NodeRequest) error { - if request.Nodes == nil { - t.Fatalf("updated nodes should not be nil") - } - - if len(request.Nodes) != 1 { - t.Fatalf("expecting 1 updated node") - } - - if request.Nodes[0].Action != si.NodeInfo_DRAIN_NODE { - t.Fatalf("expecting NodeInfo_DRAIN_NODE but get %s", - request.Nodes[0].Action.String()) - } - return nil - } - - nodes := newSchedulerNodes(api, NewTestSchedulerCache()) - dispatcher.RegisterEventHandler(dispatcher.EventTypeNode, nodes.schedulerNodeEventHandler()) - dispatcher.Start() - defer dispatcher.Stop() - - resourceList := make(map[v1.ResourceName]resource.Quantity) - resourceList[v1.ResourceName("memory")] = *resource.NewQuantity(1024*1000*1000, resource.DecimalSI) - resourceList[v1.ResourceName("cpu")] = *resource.NewQuantity(10, resource.DecimalSI) - - var oldNode = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Allocatable: resourceList, - }, - Spec: v1.NodeSpec{ - Unschedulable: false, - }, - } - - var newNode = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Allocatable: resourceList, - }, - Spec: v1.NodeSpec{ - Unschedulable: true, - }, - } - - api.UpdateNodeFunction(inputCheckerUpdateFn) - nodes.addAndReportNode(&oldNode, false) - nodes.getNode("host0001").fsm.SetState(SchedulerNodeStates().Healthy) - api.UpdateNodeFunction(inputCheckerUpdateFn) - nodes.updateNode(&oldNode, &newNode) - - // wait until node reaches Draining state - err := utils.WaitForCondition(func() bool { - return nodes.getNode("host0001").getNodeState() == SchedulerNodeStates().Draining - }, 1*time.Second, 5*time.Second) - assert.NilError(t, err) - - // restore the node - var newNode2 = v1.Node{ - ObjectMeta: apis.ObjectMeta{ - Name: "host0001", - Namespace: "default", - UID: "uid_0001", - }, - Status: v1.NodeStatus{ - Allocatable: resourceList, - }, - Spec: v1.NodeSpec{ - Unschedulable: false, - }, - } - - // register fn doesn't nothing than checking input - inputCheckerUpdateFn2 := func(request *si.NodeRequest) error { - if request.Nodes == nil { - t.Fatalf("updated nodes should not be nil") - } - - if len(request.Nodes) != 1 { - t.Fatalf("expecting 1 updated node") - } - - if request.Nodes[0].Action != si.NodeInfo_DRAIN_TO_SCHEDULABLE { - t.Fatalf("expecting UpdateNodeInfo_DRAIN_NODE but get %s", - request.Nodes[0].Action.String()) - } - return nil - } - - api.UpdateNodeFunction(inputCheckerUpdateFn2) - nodes.updateNode(&newNode, &newNode2) - - // wait until node reaches Draining state - err = utils.WaitForCondition(func() bool { - return nodes.getNode("host0001").getNodeState() == SchedulerNodeStates().Healthy - }, 1*time.Second, 5*time.Second) - assert.NilError(t, err) -} - -func getUpdateNodeFunction(t *testing.T, expectedNodeID string, expectedMem int32, - expectedCores int32, expectedReady bool) func(request *si.NodeRequest) error { - updateFn := func(request *si.NodeRequest) error { - if request.Nodes == nil || len(request.Nodes) != 1 { - t.Fatalf("unexpected new nodes info from the request") - } - - info := request.Nodes[0] - if info.NodeID != expectedNodeID { - t.Fatalf("unexpected node name %s", info.NodeID) - } - - if memory := info.SchedulableResource.Resources[siCommon.Memory].Value; memory != int64(expectedMem) { - t.Fatalf("unexpected node memory %d", memory) - } - - if cpu := info.SchedulableResource.Resources[siCommon.CPU].Value; cpu != int64(expectedCores) { - t.Fatalf("unexpected node CPU %d", cpu) - } - - if ready := info.Attributes[siCommon.NodeReadyAttribute]; ready != strconv.FormatBool(expectedReady) { - t.Fatalf("unexpected node ready flag %s", ready) - } - return nil - } - return updateFn -} diff --git a/pkg/cache/placeholder_manager_test.go b/pkg/cache/placeholder_manager_test.go index ba2ac8877..799d559d2 100644 --- a/pkg/cache/placeholder_manager_test.go +++ b/pkg/cache/placeholder_manager_test.go @@ -280,7 +280,7 @@ func TestCleanUp(t *testing.T) { placeholderMgr := NewPlaceholderManager(mockedAPIProvider.GetAPIs()) placeholderMgr.cleanUp(app) - // check both pod-01 and pod-02 in deletePod list and pod-03 isn't contain + // check both pod-01 and pod-02 in DeletePod list and pod-03 isn't contain assert.Assert(t, is.Contains(deletePod, "pod-01")) assert.Assert(t, is.Contains(deletePod, "pod-02")) exist := false diff --git a/pkg/cache/podevent_handler.go b/pkg/cache/podevent_handler.go deleted file mode 100644 index f494bd24a..000000000 --- a/pkg/cache/podevent_handler.go +++ /dev/null @@ -1,184 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "sync" - - "github.com/apache/yunikorn-k8shim/pkg/log" - - "go.uber.org/zap" - v1 "k8s.io/api/core/v1" -) - -type PodEventHandler struct { - recoveryRunning bool - amProtocol ApplicationManagementProtocol - asyncEvents []*podAsyncEvent - sync.Mutex -} - -const ( - AddPod = iota - UpdatePod - DeletePod -) - -const ( - Recovery = iota - Informers -) - -type EventType int -type EventSource int - -type podAsyncEvent struct { - eventType EventType - pod *v1.Pod -} - -func (p *PodEventHandler) HandleEvent(eventType EventType, source EventSource, pod *v1.Pod) *Application { - if p.handleEventFromInformers(eventType, source, pod) { - return nil - } - - return p.internalHandle(eventType, source, pod) -} - -func (p *PodEventHandler) handleEventFromInformers(eventType EventType, source EventSource, pod *v1.Pod) bool { - p.Lock() - defer p.Unlock() - - if p.recoveryRunning && source == Informers { - log.Log(log.ShimCacheAppMgmt).Debug("Storing async event", zap.Int("eventType", int(eventType)), - zap.String("pod", pod.GetName())) - p.asyncEvents = append(p.asyncEvents, &podAsyncEvent{eventType, pod}) - return true - } - return false -} - -func (p *PodEventHandler) internalHandle(eventType EventType, source EventSource, pod *v1.Pod) *Application { - switch eventType { - case AddPod: - return p.addPod(pod, source) - case UpdatePod: - return p.updatePod(pod) - case DeletePod: - return p.deletePod(pod) - default: - log.Log(log.ShimCacheAppMgmt).Error("Unknown pod eventType", zap.Int("eventType", int(eventType))) - return nil - } -} - -func (p *PodEventHandler) RecoveryDone(terminatedPods map[string]bool) { - p.Lock() - defer p.Unlock() - - noOfEvents := len(p.asyncEvents) - if noOfEvents > 0 { - log.Log(log.ShimCacheAppMgmt).Info("Processing async events that arrived during recovery", - zap.Int("no. of events", noOfEvents)) - for _, event := range p.asyncEvents { - // ignore all events for pods that have already been determined to - // be terminated, as these will not have been recovered and are - // therefore not claiming any scheduler resources that we care about - if terminatedPods[string(event.pod.UID)] { - continue - } - p.internalHandle(event.eventType, Informers, event.pod) - } - } else { - log.Log(log.ShimCacheAppMgmt).Info("No async pod events to process") - } - - p.recoveryRunning = false - p.asyncEvents = nil -} - -func (p *PodEventHandler) addPod(pod *v1.Pod, eventSource EventSource) *Application { - recovery := eventSource == Recovery - var app *Application - var appExists bool - - // add app - if appMeta, ok := getAppMetadata(pod, recovery); ok { - // check if app already exist - app = p.amProtocol.GetApplication(appMeta.ApplicationID) - if app == nil { - app = p.amProtocol.AddApplication(&AddApplicationRequest{ - Metadata: appMeta, - }) - } else { - appExists = true - } - } - - // add task - if taskMeta, ok := getTaskMetadata(pod); ok { - if _, taskErr := app.GetTask(string(pod.UID)); taskErr != nil { - p.amProtocol.AddTask(&AddTaskRequest{ - Metadata: taskMeta, - }) - } - } - - // only trigger recovery once - if appExists = true, it means we already - // called TriggerAppRecovery() - if recovery && !appExists { - err := app.TriggerAppRecovery() - if err != nil { - log.Log(log.ShimCacheAppMgmt).Error("failed to recover app", zap.Error(err)) - } - } - - return app -} - -func (p *PodEventHandler) updatePod(pod *v1.Pod) *Application { - if taskMeta, ok := getTaskMetadata(pod); ok { - if app := p.amProtocol.GetApplication(taskMeta.ApplicationID); app != nil { - p.amProtocol.NotifyTaskComplete(taskMeta.ApplicationID, taskMeta.TaskID) - return app - } - } - return nil -} - -func (p *PodEventHandler) deletePod(pod *v1.Pod) *Application { - if taskMeta, ok := getTaskMetadata(pod); ok { - if app := p.amProtocol.GetApplication(taskMeta.ApplicationID); app != nil { - p.amProtocol.NotifyTaskComplete(taskMeta.ApplicationID, taskMeta.TaskID) - return app - } - } - return nil -} - -func NewPodEventHandler(amProtocol ApplicationManagementProtocol, recoveryRunning bool) *PodEventHandler { - asyncEvents := make([]*podAsyncEvent, 0) - podEventHandler := &PodEventHandler{ - recoveryRunning: recoveryRunning, - asyncEvents: asyncEvents, - amProtocol: amProtocol, - } - - return podEventHandler -} diff --git a/pkg/cache/podevent_handler_test.go b/pkg/cache/podevent_handler_test.go deleted file mode 100644 index decb94dd4..000000000 --- a/pkg/cache/podevent_handler_test.go +++ /dev/null @@ -1,118 +0,0 @@ -/* - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -package cache - -import ( - "testing" - - "gotest.tools/v3/assert" - v1 "k8s.io/api/core/v1" - apis "k8s.io/apimachinery/pkg/apis/meta/v1" - "k8s.io/apimachinery/pkg/types" - - "github.com/apache/yunikorn-k8shim/pkg/common/constants" -) - -func TestHandleAsyncEventDuringRecovery(t *testing.T) { - amProtocol := NewMockedAMProtocol() - podEventHandler := NewPodEventHandler(amProtocol, true) - pod1 := newPod("pod1") - pod2 := newPod("pod2") - pod3 := newPod("pod3") - - app1 := podEventHandler.HandleEvent(AddPod, Informers, pod1) - app2 := podEventHandler.HandleEvent(UpdatePod, Informers, pod2) - app3 := podEventHandler.HandleEvent(AddPod, Recovery, pod3) - - assert.Equal(t, len(podEventHandler.asyncEvents), 2) - assert.Equal(t, podEventHandler.asyncEvents[0].pod, pod1) - assert.Equal(t, int(podEventHandler.asyncEvents[0].eventType), AddPod) - assert.Equal(t, podEventHandler.asyncEvents[1].pod, pod2) - assert.Equal(t, int(podEventHandler.asyncEvents[1].eventType), UpdatePod) - assert.Assert(t, app1 == nil) - assert.Assert(t, app2 == nil) - assert.Equal(t, ApplicationStates().Recovering, app3.GetApplicationState()) -} - -func TestHandleAsyncEventWhenNotRecovering(t *testing.T) { - amProtocol := NewMockedAMProtocol() - podEventHandler := NewPodEventHandler(amProtocol, false) - - pod1 := newPod("pod1") - pod2 := newPod("pod2") - - app1 := podEventHandler.HandleEvent(AddPod, Informers, pod1) - app2 := podEventHandler.HandleEvent(UpdatePod, Informers, pod2) - app3 := podEventHandler.HandleEvent(DeletePod, Informers, pod2) - - assert.Equal(t, len(podEventHandler.asyncEvents), 0) - assert.Assert(t, app1 != nil) - assert.Assert(t, app2 != nil) - assert.Assert(t, app3 != nil) -} - -func TestRecoveryDone(t *testing.T) { - amProtocol := NewMockedAMProtocol() - podEventHandler := NewPodEventHandler(amProtocol, true) - - pod1 := newPod("pod1") - pod2 := newPod("pod2") - - podEventHandler.HandleEvent(AddPod, Informers, pod1) - podEventHandler.HandleEvent(AddPod, Informers, pod2) - podEventHandler.HandleEvent(DeletePod, Informers, pod1) - - seenPods := map[string]bool{ - string(pod2.UID): true, // should not be added - } - podEventHandler.RecoveryDone(seenPods) - - assert.Equal(t, len(podEventHandler.asyncEvents), 0) - app := amProtocol.GetApplication(appID) - - task, err := app.GetTask("pod1") - assert.NilError(t, err) - assert.Equal(t, TaskStates().Completed, task.GetTaskState()) - - _, err = app.GetTask("pod2") - assert.ErrorContains(t, err, "task pod2 doesn't exist in application") - - assert.Equal(t, false, podEventHandler.recoveryRunning) -} - -func newPod(name string) *v1.Pod { - return &v1.Pod{ - TypeMeta: apis.TypeMeta{ - Kind: "Pod", - APIVersion: "v1", - }, - ObjectMeta: apis.ObjectMeta{ - Name: name, - Namespace: "default", - UID: types.UID(name), - Labels: map[string]string{ - "queue": "root.a", - "applicationId": appID, - }, - }, - Spec: v1.PodSpec{ - SchedulerName: constants.SchedulerName, - }, - } -} diff --git a/pkg/cache/task.go b/pkg/cache/task.go index 8c58695b3..ab62eea70 100644 --- a/pkg/cache/task.go +++ b/pkg/cache/task.go @@ -198,17 +198,17 @@ func (task *Task) isTerminated() bool { // task object initialization // normally when task is added, the task state is New -// but during recovery, we need to init the task state according to +// but during scheduler init after restart, we need to init the task state according to // the task pod status. if the pod is already terminated, // we should mark the task as completed according. func (task *Task) initialize() { task.lock.Lock() defer task.lock.Unlock() - // task needs recovery means the task has already been + // task already bound means the task has already been // scheduled by us with an allocation, instead of starting // from New, directly set the task to Bound. - if utils.NeedRecovery(task.pod) { + if utils.PodAlreadyBound(task.pod) { task.allocationUUID = string(task.pod.UID) task.nodeName = task.pod.Spec.NodeName task.sm.SetState(TaskStates().Bound) diff --git a/pkg/common/si_helper.go b/pkg/common/si_helper.go index a0638aa66..aa0a4fe44 100644 --- a/pkg/common/si_helper.go +++ b/pkg/common/si_helper.go @@ -174,8 +174,7 @@ func CreateUpdateRequestForNewNode(nodeID string, nodeLabels map[string]string, // CreateUpdateRequestForUpdatedNode builds a NodeRequest for any node updates like capacity, // ready status flag etc -func CreateUpdateRequestForUpdatedNode(nodeID string, capacity *si.Resource, occupied *si.Resource, - ready bool) *si.NodeRequest { +func CreateUpdateRequestForUpdatedNode(nodeID string, capacity *si.Resource, occupied *si.Resource, ready bool) *si.NodeRequest { nodeInfo := &si.NodeInfo{ NodeID: nodeID, Attributes: map[string]string{ diff --git a/pkg/common/utils/utils.go b/pkg/common/utils/utils.go index 837cd1d1b..c95860608 100644 --- a/pkg/common/utils/utils.go +++ b/pkg/common/utils/utils.go @@ -76,8 +76,9 @@ func Convert2PriorityClass(obj interface{}) *schedulingv1.PriorityClass { return nil } -func NeedRecovery(pod *v1.Pod) bool { - // pod requires recovery needs to satisfy both conditions +// PodAlreadyBound returns true if a newly initializing Pod is already assigned to a Node +func PodAlreadyBound(pod *v1.Pod) bool { + // pod already bound needs to satisfy conditions: // 1. Pod is scheduled by us // 2. pod is already assigned to a node // 3. pod is not in terminated state diff --git a/pkg/common/utils/utils_test.go b/pkg/common/utils/utils_test.go index 7f966ae13..f653ba221 100644 --- a/pkg/common/utils/utils_test.go +++ b/pkg/common/utils/utils_test.go @@ -810,12 +810,12 @@ func TestGetQueueNameFromPod(t *testing.T) { } } -func TestNeedRecovery(t *testing.T) { +func TestPodAlreadyBound(t *testing.T) { const fakeNodeID = "fake-node" testCases := []struct { - description string - pod *v1.Pod - expectedRecoveryFlag bool + description string + pod *v1.Pod + expectedBoundFlag bool }{ {"New pod pending for scheduling", &v1.Pod{ @@ -881,8 +881,8 @@ func TestNeedRecovery(t *testing.T) { for _, tc := range testCases { t.Run(tc.description, func(t *testing.T) { - recovery := NeedRecovery(tc.pod) - assert.Equal(t, recovery, tc.expectedRecoveryFlag, tc.description) + bound := PodAlreadyBound(tc.pod) + assert.Equal(t, bound, tc.expectedBoundFlag, tc.description) }) } } diff --git a/pkg/dispatcher/dispatch_test.go b/pkg/dispatcher/dispatch_test.go index 04a8e20be..7060d4763 100644 --- a/pkg/dispatcher/dispatch_test.go +++ b/pkg/dispatcher/dispatch_test.go @@ -58,10 +58,21 @@ func TestRegisterEventHandler(t *testing.T) { createDispatcher() defer createDispatcher() - RegisterEventHandler(EventTypeApp, func(obj interface{}) {}) - RegisterEventHandler(EventTypeTask, func(obj interface{}) {}) - RegisterEventHandler(EventTypeTask, func(obj interface{}) {}) + RegisterEventHandler("TestAppHandler", EventTypeApp, func(obj interface{}) {}) + RegisterEventHandler("TestTaskHandler", EventTypeTask, func(obj interface{}) {}) + RegisterEventHandler("TestTaskHandler2", EventTypeTask, func(obj interface{}) {}) assert.Equal(t, len(dispatcher.handlers), 2) + assert.Equal(t, len(dispatcher.handlers[EventTypeTask]), 2) + + UnregisterEventHandler("TestTaskHandler2", EventTypeTask) + assert.Equal(t, len(dispatcher.handlers), 2) + assert.Equal(t, len(dispatcher.handlers[EventTypeTask]), 1) + + UnregisterEventHandler("TestTaskHandler", EventTypeTask) + assert.Equal(t, len(dispatcher.handlers), 1) + + UnregisterEventHandler("TestAppHandler", EventTypeApp) + assert.Equal(t, len(dispatcher.handlers), 0) } type appEventsRecorder struct { @@ -101,7 +112,7 @@ func TestDispatcherStartStop(t *testing.T) { lock: &sync.RWMutex{}, } - RegisterEventHandler(EventTypeApp, func(obj interface{}) { + RegisterEventHandler("TestAppHandler", EventTypeApp, func(obj interface{}) { if event, ok := obj.(events.ApplicationEvent); ok { recorder.addApp(event.GetApplicationID()) } @@ -157,7 +168,7 @@ func TestEventWillNotBeLostWhenEventChannelIsFull(t *testing.T) { lock: &sync.RWMutex{}, } // pretend to be an time-consuming event-handler - RegisterEventHandler(EventTypeApp, func(obj interface{}) { + RegisterEventHandler("TestAppHandler", EventTypeApp, func(obj interface{}) { if event, ok := obj.(events.ApplicationEvent); ok { recorder.addApp(event.GetApplicationID()) time.Sleep(1 * time.Millisecond) @@ -204,7 +215,7 @@ func TestDispatchTimeout(t *testing.T) { DispatchTimeout = 500 * time.Millisecond // start the handler, but waiting on a flag - RegisterEventHandler(EventTypeApp, func(obj interface{}) { + RegisterEventHandler("TestAppHandler", EventTypeApp, func(obj interface{}) { if appEvent, ok := obj.(TestAppEvent); ok { fmt.Printf("handling %s\n", appEvent.appID) <-appEvent.flag @@ -262,7 +273,7 @@ func TestExceedAsyncDispatchLimit(t *testing.T) { dispatcher.eventChan = make(chan events.SchedulingEvent, 1) AsyncDispatchLimit = 1 // pretend to be an time-consuming event-handler - RegisterEventHandler(EventTypeApp, func(obj interface{}) { + RegisterEventHandler("TestAppHandler", EventTypeApp, func(obj interface{}) { if _, ok := obj.(events.ApplicationEvent); ok { time.Sleep(2 * time.Second) } diff --git a/pkg/dispatcher/dispatcher.go b/pkg/dispatcher/dispatcher.go index 719c5ccb5..7bfd17cfb 100644 --- a/pkg/dispatcher/dispatcher.go +++ b/pkg/dispatcher/dispatcher.go @@ -53,7 +53,7 @@ var ( type Dispatcher struct { eventChan chan events.SchedulingEvent stopChan chan struct{} - handlers map[EventType]func(interface{}) + handlers map[EventType]map[string]func(interface{}) running atomic.Value lock sync.RWMutex } @@ -62,7 +62,7 @@ func initDispatcher() { eventChannelCapacity := conf.GetSchedulerConf().EventChannelCapacity dispatcher = &Dispatcher{ eventChan: make(chan events.SchedulingEvent, eventChannelCapacity), - handlers: make(map[EventType]func(interface{})), + handlers: make(map[EventType]map[string]func(interface{})), stopChan: make(chan struct{}), running: atomic.Value{}, lock: sync.RWMutex{}, @@ -79,11 +79,33 @@ func initDispatcher() { zap.Float64("DispatchTimeoutInSeconds", DispatchTimeout.Seconds())) } -func RegisterEventHandler(eventType EventType, handlerFn func(interface{})) { +func RegisterEventHandler(handlerID string, eventType EventType, handlerFn func(interface{})) { eventDispatcher := getDispatcher() eventDispatcher.lock.Lock() defer eventDispatcher.lock.Unlock() - eventDispatcher.handlers[eventType] = handlerFn + if _, ok := eventDispatcher.handlers[eventType]; !ok { + eventDispatcher.handlers[eventType] = make(map[string]func(interface{})) + } + eventDispatcher.handlers[eventType][handlerID] = handlerFn +} + +func UnregisterEventHandler(handlerID string, eventType EventType) { + eventDispatcher := getDispatcher() + eventDispatcher.lock.Lock() + defer eventDispatcher.lock.Unlock() + if _, ok := eventDispatcher.handlers[eventType]; ok { + delete(eventDispatcher.handlers[eventType], handlerID) + if len(eventDispatcher.handlers[eventType]) == 0 { + delete(eventDispatcher.handlers, eventType) + } + } +} + +func UnregisterAllEventHandlers() { + eventDispatcher := getDispatcher() + eventDispatcher.lock.Lock() + defer eventDispatcher.lock.Unlock() + eventDispatcher.handlers = make(map[EventType]map[string]func(interface{})) } // a thread-safe way to get event handlers @@ -91,7 +113,16 @@ func getEventHandler(eventType EventType) func(interface{}) { eventDispatcher := getDispatcher() eventDispatcher.lock.RLock() defer eventDispatcher.lock.RUnlock() - return eventDispatcher.handlers[eventType] + + handlers := make([]func(interface{}), 0) + for _, handler := range eventDispatcher.handlers[eventType] { + handlers = append(handlers, handler) + } + return func(event interface{}) { + for _, handler := range handlers { + handler(event) + } + } } func getDispatcher() *Dispatcher { diff --git a/pkg/plugin/support/nodeinfo_lister_test.go b/pkg/plugin/support/nodeinfo_lister_test.go index 6629ac123..4036c6af1 100644 --- a/pkg/plugin/support/nodeinfo_lister_test.go +++ b/pkg/plugin/support/nodeinfo_lister_test.go @@ -80,14 +80,14 @@ func initLister(t *testing.T) *nodeInfoListerImpl { lister, ok := NewSharedLister(cache).NodeInfos().(*nodeInfoListerImpl) assert.Assert(t, ok, "wrong type for node lister") - cache.AddNode(&v1.Node{ + cache.UpdateNode(&v1.Node{ ObjectMeta: apis.ObjectMeta{ Name: "host0001", Namespace: "default", UID: "Node-UID-00001", }, }) - cache.AddNode(&v1.Node{ + cache.UpdateNode(&v1.Node{ ObjectMeta: apis.ObjectMeta{ Name: "host0002", Namespace: "default", diff --git a/pkg/plugin/support/shared_lister_test.go b/pkg/plugin/support/shared_lister_test.go index ec3b22aaa..9da0a2633 100644 --- a/pkg/plugin/support/shared_lister_test.go +++ b/pkg/plugin/support/shared_lister_test.go @@ -40,7 +40,7 @@ func TestNewSharedLister(t *testing.T) { UID: "Node-UID-00001", }, } - cache.AddNode(node) + cache.UpdateNode(node) nodeInfo, err := lister.NodeInfos().Get("host0001") assert.NilError(t, err, "err returned from Get call") diff --git a/pkg/shim/scheduler.go b/pkg/shim/scheduler.go index 0be8ade2d..76a2c717a 100644 --- a/pkg/shim/scheduler.go +++ b/pkg/shim/scheduler.go @@ -41,7 +41,6 @@ import ( type KubernetesShim struct { apiFactory client.APIProvider context *cache.Context - appManager *cache.AppManagementService phManager *cache.PlaceholderManager callback api.ResourceManagerCallback stopChan chan struct{} @@ -49,6 +48,11 @@ type KubernetesShim struct { outstandingAppsFound bool } +const ( + AppHandler string = "ShimAppHandler" + TaskHandler string = "ShimTaskHandler" +) + var ( // timeout for logging a message if no outstanding apps were found for scheduling outstandingAppLogTimeout = 2 * time.Minute @@ -63,8 +67,7 @@ func NewShimScheduler(scheduler api.SchedulerAPI, configs *conf.SchedulerConf, b apiFactory := client.NewAPIFactory(scheduler, informerFactory, configs, false) context := cache.NewContextWithBootstrapConfigMaps(apiFactory, bootstrapConfigMaps) rmCallback := cache.NewAsyncRMCallback(context) - appManager := cache.NewAMService(context, apiFactory) - return newShimSchedulerInternal(context, apiFactory, appManager, rmCallback) + return newShimSchedulerInternal(context, apiFactory, rmCallback) } func NewShimSchedulerForPlugin(scheduler api.SchedulerAPI, informerFactory informers.SharedInformerFactory, configs *conf.SchedulerConf, bootstrapConfigMaps []*v1.ConfigMap) *KubernetesShim { @@ -72,17 +75,14 @@ func NewShimSchedulerForPlugin(scheduler api.SchedulerAPI, informerFactory infor context := cache.NewContextWithBootstrapConfigMaps(apiFactory, bootstrapConfigMaps) utils.SetPluginMode(true) rmCallback := cache.NewAsyncRMCallback(context) - appManager := cache.NewAMService(context, apiFactory) - return newShimSchedulerInternal(context, apiFactory, appManager, rmCallback) + return newShimSchedulerInternal(context, apiFactory, rmCallback) } // this is visible for testing -func newShimSchedulerInternal(ctx *cache.Context, apiFactory client.APIProvider, - am *cache.AppManagementService, cb api.ResourceManagerCallback) *KubernetesShim { +func newShimSchedulerInternal(ctx *cache.Context, apiFactory client.APIProvider, cb api.ResourceManagerCallback) *KubernetesShim { ss := &KubernetesShim{ apiFactory: apiFactory, context: ctx, - appManager: am, phManager: cache.NewPlaceholderManager(apiFactory.GetAPIs()), callback: cb, stopChan: make(chan struct{}), @@ -90,9 +90,8 @@ func newShimSchedulerInternal(ctx *cache.Context, apiFactory client.APIProvider, outstandingAppsFound: false, } // init dispatcher - dispatcher.RegisterEventHandler(dispatcher.EventTypeApp, ctx.ApplicationEventHandler()) - dispatcher.RegisterEventHandler(dispatcher.EventTypeTask, ctx.TaskEventHandler()) - dispatcher.RegisterEventHandler(dispatcher.EventTypeNode, ctx.SchedulerNodeEventHandler()) + dispatcher.RegisterEventHandler(AppHandler, dispatcher.EventTypeApp, ctx.ApplicationEventHandler()) + dispatcher.RegisterEventHandler(TaskHandler, dispatcher.EventTypeTask, ctx.TaskEventHandler()) return ss } @@ -101,37 +100,17 @@ func (ss *KubernetesShim) GetContext() *cache.Context { return ss.context } -func (ss *KubernetesShim) recoverSchedulerState() error { - log.Log(log.ShimScheduler).Info("recovering scheduler states") - // step 1: recover all applications - // this step, we collect all the existing allocated pods from api-server, - // identify the scheduling identity (aka applicationInfo) from the pod, - // and then add these applications to the scheduler. - if err := ss.appManager.WaitForRecovery(); err != nil { - // failed - log.Log(log.ShimScheduler).Error("scheduler recovery failed", zap.Error(err)) - return err - } - - // step 2: recover existing allocations - // this step, we collect all existing allocations (allocated pods) from api-server, - // rerun the scheduling for these allocations in order to restore scheduler-state, - // the rerun is like a replay, not a actual scheduling procedure. - if err := ss.context.WaitForRecovery(ss.appManager, 5*time.Minute); err != nil { - // failed - log.Log(log.ShimScheduler).Error("scheduler recovery failed", zap.Error(err)) +func (ss *KubernetesShim) initSchedulerState() error { + log.Log(log.ShimScheduler).Info("initializing scheduler state") + if err := ss.context.InitializeState(); err != nil { + log.Log(log.ShimScheduler).Error("failed to initialize scheduler state", zap.Error(err)) return err } - - // success - log.Log(log.ShimScheduler).Info("scheduler recovery succeed") + log.Log(log.ShimScheduler).Info("scheduler state initialized") return nil } func (ss *KubernetesShim) doScheduling() { - // add event handlers to the context - ss.context.AddSchedulingEventHandlers() - // run main scheduling loop go wait.Until(ss.schedule, conf.GetSchedulerConf().GetSchedulingInterval(), ss.stopChan) // log a message if no outstanding requests were found for a while @@ -207,18 +186,9 @@ func (ss *KubernetesShim) Run() error { return err } - // run app managers - // the app manager launches the pod event handlers - // it needs to be started after the shim is registered with the core - if err := ss.appManager.Start(); err != nil { - log.Log(log.ShimScheduler).Error("failed to start app manager", zap.Error(err)) - ss.Stop() - return err - } - - // recover scheduler state - if err := ss.recoverSchedulerState(); err != nil { - log.Log(log.ShimScheduler).Error("failed to recover scheduler state", zap.Error(err)) + // initialize scheduler state + if err := ss.initSchedulerState(); err != nil { + log.Log(log.ShimScheduler).Error("failed to initialize scheduler state", zap.Error(err)) ss.Stop() return err } diff --git a/pkg/shim/scheduler_mock_test.go b/pkg/shim/scheduler_mock_test.go index 436d1ea62..d267e5eeb 100644 --- a/pkg/shim/scheduler_mock_test.go +++ b/pkg/shim/scheduler_mock_test.go @@ -30,13 +30,11 @@ import ( schedv1 "k8s.io/api/scheduling/v1" "k8s.io/apimachinery/pkg/api/resource" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - "k8s.io/apimachinery/pkg/types" "github.com/apache/yunikorn-core/pkg/entrypoint" "github.com/apache/yunikorn-k8shim/pkg/cache" "github.com/apache/yunikorn-k8shim/pkg/client" "github.com/apache/yunikorn-k8shim/pkg/common" - "github.com/apache/yunikorn-k8shim/pkg/common/constants" "github.com/apache/yunikorn-k8shim/pkg/common/events" "github.com/apache/yunikorn-k8shim/pkg/common/utils" "github.com/apache/yunikorn-k8shim/pkg/conf" @@ -69,8 +67,7 @@ func (fc *MockScheduler) init() { context := cache.NewContext(mockedAPIProvider) rmCallback := cache.NewAsyncRMCallback(context) - amSvc := cache.NewAMService(context, mockedAPIProvider) - ss := newShimSchedulerInternal(context, mockedAPIProvider, amSvc, rmCallback) + ss := newShimSchedulerInternal(context, mockedAPIProvider, rmCallback) fc.context = context fc.scheduler = ss @@ -102,7 +99,7 @@ func (fc *MockScheduler) addNode(nodeName string, nodeLabels map[string]string, cache := fc.context.GetSchedulerCache() zero := resource.Scale(0) // add node to the cache so that predicates can run properly - cache.AddNode(&v1.Node{ + cache.UpdateNode(&v1.Node{ ObjectMeta: metav1.ObjectMeta{ Name: nodeName, Labels: nodeLabels, @@ -131,81 +128,32 @@ func (fc *MockScheduler) addNode(nodeName string, nodeLabels map[string]string, return fc.apiProvider.GetAPIs().SchedulerAPI.UpdateNode(request) } -// Deprecated: this method only updates the core without the shim. Prefer MockScheduler.AddPod(*v1.Pod) instead. -func (fc *MockScheduler) addTask(appID string, taskID string, ask *si.Resource) { - schedCache := fc.context.GetSchedulerCache() - // add pod to the cache so that predicates can run properly - resources := make(map[v1.ResourceName]resource.Quantity) - for k, v := range ask.Resources { - resources[v1.ResourceName(k)] = *resource.NewQuantity(v.Value, resource.DecimalSI) - } - containers := make([]v1.Container, 0) - containers = append(containers, v1.Container{ - Name: "container-01", - Resources: v1.ResourceRequirements{ - Requests: resources, - }, - }) - pod := &v1.Pod{ - ObjectMeta: metav1.ObjectMeta{ - UID: types.UID(taskID), - Name: taskID, - Annotations: map[string]string{ - constants.AnnotationApplicationID: appID, - }, - Labels: map[string]string{ - constants.LabelApplicationID: appID, - }, - }, - Spec: v1.PodSpec{ - SchedulerName: constants.SchedulerName, - Containers: containers, - }, - } - schedCache.AddPod(pod) - - fc.context.AddTask(&cache.AddTaskRequest{ - Metadata: cache.TaskMetadata{ - ApplicationID: appID, - TaskID: taskID, - Pod: pod, - }, - }) -} - func (fc *MockScheduler) waitAndAssertApplicationState(t *testing.T, appID, expectedState string) { - app := fc.context.GetApplication(appID) - assert.Equal(t, app != nil, true) - assert.Equal(t, app.GetApplicationID(), appID) deadline := time.Now().Add(10 * time.Second) for { - if app.GetApplicationState() == expectedState { + app := fc.context.GetApplication(appID) + if app != nil { + assert.Equal(t, app.GetApplicationID(), appID) + } + if app != nil && app.GetApplicationState() == expectedState { break } + actual := "" + if app != nil { + actual = app.GetApplicationState() + } log.Log(log.Test).Info("waiting for app state", zap.String("expected", expectedState), - zap.String("actual", app.GetApplicationState())) + zap.String("actual", actual)) time.Sleep(time.Second) if time.Now().After(deadline) { t.Errorf("application %s doesn't reach expected state in given time, expecting: %s, actual: %s", - appID, expectedState, app.GetApplicationState()) + appID, expectedState, actual) + return } } } -// Deprecated: this method adds an application directly to the Context, and it skips relevant -// code paths. Prefer MockScheduler.AddPod(*v1.Pod) instead. -func (fc *MockScheduler) addApplication(appId string, queue string) { - fc.context.AddApplication(&cache.AddApplicationRequest{ - Metadata: cache.ApplicationMetadata{ - ApplicationID: appId, - QueueName: queue, - User: "test-user", - Tags: map[string]string{"app-type": "test-app"}, - }, - }) -} - func (fc *MockScheduler) removeApplication(appId string) error { return fc.context.RemoveApplication(appId) } @@ -229,6 +177,7 @@ func (fc *MockScheduler) waitAndAssertTaskState(t *testing.T, appID, taskID, exp if time.Now().After(deadline) { t.Errorf("task %s doesn't reach expected state in given time, expecting: %s, actual: %s", taskID, expectedState, task.GetTaskState()) + return } } } diff --git a/pkg/shim/scheduler_test.go b/pkg/shim/scheduler_test.go index d938872e5..bbb1d9ad2 100644 --- a/pkg/shim/scheduler_test.go +++ b/pkg/shim/scheduler_test.go @@ -24,10 +24,14 @@ import ( "gotest.tools/v3/assert" v1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/resource" + apis "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/types" "github.com/apache/yunikorn-k8shim/pkg/cache" "github.com/apache/yunikorn-k8shim/pkg/client" "github.com/apache/yunikorn-k8shim/pkg/common" + "github.com/apache/yunikorn-k8shim/pkg/common/constants" "github.com/apache/yunikorn-k8shim/pkg/common/test" "github.com/apache/yunikorn-scheduler-interface/lib/go/api" siCommon "github.com/apache/yunikorn-scheduler-interface/lib/go/common" @@ -71,13 +75,16 @@ partitions: assert.NilError(t, err, "add node failed") // create app and tasks - cluster.addApplication("app0001", "root.a") taskResource := common.NewResourceBuilder(). AddResource(siCommon.Memory, 10000000). AddResource(siCommon.CPU, 1). Build() - cluster.addTask("app0001", "task0001", taskResource) - cluster.addTask("app0001", "task0002", taskResource) + + task1 := createTestPod("root.a", "app0001", "task0001", taskResource) + task2 := createTestPod("root.a", "app0001", "task0002", taskResource) + + cluster.AddPod(task1) + cluster.AddPod(task2) // wait for scheduling app and tasks // verify app state @@ -123,16 +130,15 @@ partitions: err = cluster.addNode("test.host.02", nodeLabels, 100000000, 10, 10) assert.NilError(t, err) - // add app to context - appID := "app0001" - cluster.addApplication(appID, "root.non_exist_queue") - // create app and tasks + appID := "app0001" taskResource := common.NewResourceBuilder(). AddResource(siCommon.Memory, 10000000). AddResource(siCommon.CPU, 1). Build() - cluster.addTask(appID, "task0001", taskResource) + + task1 := createTestPod("root.non_exist_queue", appID, "task0001", taskResource) + cluster.AddPod(task1) // wait for scheduling app and tasks // verify app state @@ -144,9 +150,10 @@ partitions: err = cluster.removeApplication(appID) assert.Assert(t, err == nil) - // submit the app again - cluster.addApplication(appID, "root.a") - cluster.addTask(appID, "task0001", taskResource) + // submit again + task1 = createTestPod("root.a", appID, "task0001", taskResource) + cluster.AddPod(task1) + cluster.waitAndAssertApplicationState(t, appID, cache.ApplicationStates().Running) cluster.waitAndAssertTaskState(t, appID, "task0001", cache.TaskStates().Bound) } @@ -154,7 +161,6 @@ partitions: func TestSchedulerRegistrationFailed(t *testing.T) { var callback api.ResourceManagerCallback - mockedAMProtocol := cache.NewMockedAMProtocol() mockedAPIProvider := client.NewMockedAPIProvider(false) mockedAPIProvider.GetAPIs().SchedulerAPI = test.NewSchedulerAPIMock().RegisterFunction( func(request *si.RegisterResourceManagerRequest, @@ -163,8 +169,7 @@ func TestSchedulerRegistrationFailed(t *testing.T) { }) ctx := cache.NewContext(mockedAPIProvider) - shim := newShimSchedulerInternal(ctx, mockedAPIProvider, - cache.NewAMService(mockedAMProtocol, mockedAPIProvider), callback) + shim := newShimSchedulerInternal(ctx, mockedAPIProvider, callback) assert.Error(t, shim.Run(), "some error") shim.Stop() } @@ -217,13 +222,15 @@ partitions: assert.NilError(t, err, "add node failed") // create app and tasks - cluster.addApplication("app0001", "root.a") taskResource := common.NewResourceBuilder(). AddResource(siCommon.Memory, 50000000). AddResource(siCommon.CPU, 5). Build() - cluster.addTask("app0001", "task0001", taskResource) - cluster.addTask("app0001", "task0002", taskResource) + task1 := createTestPod("root.a", "app0001", "task0001", taskResource) + task2 := createTestPod("root.a", "app0001", "task0002", taskResource) + cluster.AddPod(task1) + cluster.AddPod(task2) + // wait for scheduling app and tasks // verify app state cluster.waitAndAssertApplicationState(t, "app0001", cache.ApplicationStates().Running) @@ -235,3 +242,43 @@ partitions: "[mycluster]default", "app0001", 1) assert.NilError(t, err, "number of allocations is not expected, error") } + +func createTestPod(queue string, appID string, taskID string, taskResource *si.Resource) *v1.Pod { + containers := make([]v1.Container, 0) + c1Resources := make(map[v1.ResourceName]resource.Quantity) + for k, v := range taskResource.Resources { + if k == siCommon.CPU { + c1Resources[v1.ResourceName(k)] = *resource.NewMilliQuantity(v.Value, resource.DecimalSI) + } else { + c1Resources[v1.ResourceName(k)] = *resource.NewQuantity(v.Value, resource.DecimalSI) + } + } + containers = append(containers, v1.Container{ + Name: "container-01", + Resources: v1.ResourceRequirements{ + Requests: c1Resources, + }, + }) + return &v1.Pod{ + TypeMeta: apis.TypeMeta{ + Kind: "Pod", + APIVersion: "v1", + }, + ObjectMeta: apis.ObjectMeta{ + Name: taskID, + Namespace: "default", + UID: types.UID(taskID), + Labels: map[string]string{ + constants.LabelApplicationID: appID, + constants.LabelQueueName: queue, + }, + }, + Spec: v1.PodSpec{ + SchedulerName: constants.SchedulerName, + Containers: containers, + }, + Status: v1.PodStatus{ + Phase: v1.PodPending, + }, + } +}