You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@yunikorn.apache.org by GitBox <gi...@apache.org> on 2022/05/26 15:48:05 UTC

[GitHub] [yunikorn-k8shim] craigcondit commented on a diff in pull request #422: [YUNIKORN-186] clean up events in k8shim

craigcondit commented on code in PR #422:
URL: https://github.com/apache/yunikorn-k8shim/pull/422#discussion_r882800847


##########
pkg/cache/node_state.go:
##########
@@ -0,0 +1,154 @@
+/*
+ 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 (
+	"github.com/looplab/fsm"
+	"go.uber.org/zap"
+
+	"github.com/apache/yunikorn-k8shim/pkg/common/events"
+	"github.com/apache/yunikorn-k8shim/pkg/log"
+)
+
+//----------------------------------------------
+// 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 {
+	if storeSchedulerNodeStates == nil {

Review Comment:
   As before, use sync.Once().



##########
pkg/cache/application.go:
##########
@@ -696,14 +603,7 @@ func (app *Application) handleFailApplicationEvent(event *fsm.Event) {
 	}
 }
 
-func (app *Application) handleReleaseAppAllocationEvent(event *fsm.Event) {
-	eventArgs := make([]string, 2)
-	if err := events.GetEventArgsAsStrings(eventArgs, event.Args); err != nil {
-		log.Logger().Error("fail to parse event arg", zap.Error(err))
-		return
-	}
-	allocUUID := eventArgs[0]
-	terminationTypeStr := eventArgs[1]
+func (app *Application) handleReleaseAppAllocationEvent(allocUUID string, terminationTypeStr string) {

Review Comment:
   Can we use just `terminationType` here?



##########
pkg/cache/task_state.go:
##########
@@ -0,0 +1,432 @@
+/*
+ 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 (
+	"github.com/looplab/fsm"
+	"go.uber.org/zap"
+
+	"github.com/apache/yunikorn-k8shim/pkg/common/events"
+	"github.com/apache/yunikorn-k8shim/pkg/log"
+)
+
+//----------------------------------------------
+// Task events
+//----------------------------------------------
+type TaskEventType int
+
+const (
+	InitTask TaskEventType = iota
+	SubmitTask
+	TaskAllocated
+	TaskRejected
+	TaskBound
+	CompleteTask
+	TaskFail
+	KillTask
+	TaskKilled
+)
+
+func (ae TaskEventType) String() string {
+	return [...]string{"InitTask", "SubmitTask", "TaskAllocated", "TaskRejected", "TaskBound", "CompleteTask", "TaskFail", "KillTask", "TaskKilled"}[ae]
+}
+
+// ------------------------
+// Simple task Event simply moves task to next state, it has no arguments provided
+// ------------------------
+type SimpleTaskEvent struct {
+	applicationID string
+	taskID        string
+	event         TaskEventType
+}
+
+func NewSimpleTaskEvent(appID string, taskID string, taskType TaskEventType) SimpleTaskEvent {
+	return SimpleTaskEvent{
+		applicationID: appID,
+		taskID:        taskID,
+		event:         taskType,
+	}
+}
+
+func (st SimpleTaskEvent) GetEvent() string {
+	return st.event.String()
+}
+
+func (st SimpleTaskEvent) GetArgs() []interface{} {
+	return nil
+}
+
+func (st SimpleTaskEvent) GetTaskID() string {
+	return st.taskID
+}
+
+func (st SimpleTaskEvent) GetApplicationID() string {
+	return st.applicationID
+}
+
+// ------------------------
+// SubmitTask Event
+// ------------------------
+type SubmitTaskEvent struct {
+	applicationID string
+	taskID        string
+	event         TaskEventType
+}
+
+func NewSubmitTaskEvent(appID string, taskID string) SubmitTaskEvent {
+	return SubmitTaskEvent{
+		applicationID: appID,
+		taskID:        taskID,
+		event:         SubmitTask,
+	}
+}
+
+func (st SubmitTaskEvent) GetEvent() string {
+	return st.event.String()
+}
+
+func (st SubmitTaskEvent) GetArgs() []interface{} {
+	return nil
+}
+
+func (st SubmitTaskEvent) GetTaskID() string {
+	return st.taskID
+}
+
+func (st SubmitTaskEvent) GetApplicationID() string {
+	return st.applicationID
+}
+
+// ------------------------
+// Allocate Event
+// ------------------------
+type AllocatedTaskEvent struct {
+	applicationID  string
+	taskID         string
+	event          TaskEventType
+	nodeID         string
+	allocationUUID string
+}
+
+func NewAllocateTaskEvent(appID string, taskID string, allocUUID string, nid string) AllocatedTaskEvent {
+	return AllocatedTaskEvent{
+		applicationID:  appID,
+		taskID:         taskID,
+		event:          TaskAllocated,
+		allocationUUID: allocUUID,
+		nodeID:         nid,
+	}
+}
+
+func (ae AllocatedTaskEvent) GetEvent() string {
+	return ae.event.String()
+}
+
+func (ae AllocatedTaskEvent) GetArgs() []interface{} {
+	args := make([]interface{}, 2)
+	args[0] = ae.allocationUUID
+	args[1] = ae.nodeID
+	return args
+}
+
+func (ae AllocatedTaskEvent) GetTaskID() string {
+	return ae.taskID
+}
+
+func (ae AllocatedTaskEvent) GetApplicationID() string {
+	return ae.applicationID
+}
+
+// ------------------------
+// Bound Event
+// ------------------------
+type BindTaskEvent struct {
+	applicationID string
+	taskID        string
+	event         TaskEventType
+}
+
+func NewBindTaskEvent(appID string, taskID string) BindTaskEvent {
+	return BindTaskEvent{
+		applicationID: appID,
+		taskID:        taskID,
+		event:         TaskBound,
+	}
+}
+
+func (bt BindTaskEvent) GetEvent() string {
+	return bt.event.String()
+}
+
+func (bt BindTaskEvent) GetArgs() []interface{} {
+	return nil
+}
+
+func (bt BindTaskEvent) GetTaskID() string {
+	return bt.taskID
+}
+
+func (bt BindTaskEvent) GetApplicationID() string {
+	return bt.applicationID
+}
+
+// ------------------------
+// Fail Event
+// ------------------------
+type FailTaskEvent struct {
+	applicationID string
+	taskID        string
+	event         TaskEventType
+	message       string
+}
+
+func NewFailTaskEvent(appID string, taskID string, failedMessage string) FailTaskEvent {
+	return FailTaskEvent{
+		applicationID: appID,
+		taskID:        taskID,
+		event:         TaskFail,
+		message:       failedMessage,
+	}
+}
+
+func (fe FailTaskEvent) GetEvent() string {
+	return fe.event.String()
+}
+
+func (fe FailTaskEvent) GetArgs() []interface{} {
+	args := make([]interface{}, 1)
+	args[0] = fe.message
+	return args
+}
+
+func (fe FailTaskEvent) GetTaskID() string {
+	return fe.taskID
+}
+
+func (fe FailTaskEvent) GetApplicationID() string {
+	return fe.applicationID
+}
+
+// ------------------------
+// Reject Event
+// ------------------------
+type RejectTaskEvent struct {
+	applicationID string
+	taskID        string
+	event         TaskEventType
+	message       string
+}
+
+func NewRejectTaskEvent(appID string, taskID string, rejectedMessage string) RejectTaskEvent {
+	return RejectTaskEvent{
+		applicationID: appID,
+		taskID:        taskID,
+		event:         TaskRejected,
+		message:       rejectedMessage,
+	}
+}
+
+func (re RejectTaskEvent) GetEvent() string {
+	return re.event.String()
+}
+
+func (re RejectTaskEvent) GetArgs() []interface{} {
+	args := make([]interface{}, 1)
+	args[0] = re.message
+	return args
+}
+
+func (re RejectTaskEvent) GetTaskID() string {
+	return re.taskID
+}
+
+func (re RejectTaskEvent) GetApplicationID() string {
+	return re.applicationID
+}
+
+// ----------------------------------
+// task states
+// ----------------------------------
+var storeTaskStates *TStates
+
+type TStates struct {
+	New        string
+	Pending    string
+	Scheduling string
+	Allocated  string
+	Rejected   string
+	Bound      string
+	Killing    string
+	Killed     string
+	Failed     string
+	Completed  string
+	Any        []string // Any refers to all possible states
+	Terminated []string // Rejected, Killed, Failed, Completed
+}
+
+func TaskStates() *TStates {
+	if storeTaskStates == nil {

Review Comment:
   Use sync.Once() for initialization.



##########
pkg/cache/application.go:
##########
@@ -721,14 +621,7 @@ func (app *Application) handleReleaseAppAllocationEvent(event *fsm.Event) {
 	}
 }
 
-func (app *Application) handleReleaseAppAllocationAskEvent(event *fsm.Event) {
-	eventArgs := make([]string, 2)
-	if err := events.GetEventArgsAsStrings(eventArgs, event.Args); err != nil {
-		log.Logger().Error("fail to parse event arg", zap.Error(err))
-		return
-	}
-	taskID := eventArgs[0]
-	terminationTypeStr := eventArgs[1]
+func (app *Application) handleReleaseAppAllocationAskEvent(taskID string, terminationTypeStr string) {

Review Comment:
   Can we use just `terminationType` here?



##########
pkg/cache/application_state.go:
##########
@@ -0,0 +1,577 @@
+/*
+ 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 (
+	"github.com/looplab/fsm"
+	"go.uber.org/zap"
+
+	"github.com/apache/yunikorn-k8shim/pkg/common/events"
+	"github.com/apache/yunikorn-k8shim/pkg/log"
+	"github.com/apache/yunikorn-scheduler-interface/lib/go/si"
+)
+
+//----------------------------------------------
+// Application events
+//----------------------------------------------
+type ApplicationEventType int
+
+const (
+	SubmitApplication ApplicationEventType = iota
+	RecoverApplication
+	AcceptApplication
+	TryReserve
+	UpdateReservation
+	RunApplication
+	RejectApplication
+	CompleteApplication
+	FailApplication
+	KillApplication
+	KilledApplication
+	ReleaseAppAllocation
+	ReleaseAppAllocationAsk
+	AppStateChange
+	ResumingApplication
+	AppTaskCompleted
+)
+
+func (ae ApplicationEventType) String() string {
+	return [...]string{"SubmitApplication", "RecoverApplication", "AcceptApplication", "TryReserve", "UpdateReservation", "RunApplication", "RejectApplication", "CompleteApplication", "FailApplication", "KillApplication", "KilledApplication", "ReleaseAppAllocation", "ReleaseAppAllocationAsk", "AppStateChange", "ResumingApplication", "AppTaskCompleted"}[ae]
+}
+
+// ------------------------
+// SimpleApplicationEvent simply moves application states
+// ------------------------
+type SimpleApplicationEvent struct {
+	applicationID string
+	event         ApplicationEventType
+}
+
+func NewSimpleApplicationEvent(appID string, eventType ApplicationEventType) SimpleApplicationEvent {
+	return SimpleApplicationEvent{
+		applicationID: appID,
+		event:         eventType,
+	}
+}
+
+func (st SimpleApplicationEvent) GetEvent() string {
+	return st.event.String()
+}
+
+func (st SimpleApplicationEvent) GetArgs() []interface{} {
+	return nil
+}
+
+func (st SimpleApplicationEvent) GetApplicationID() string {
+	return st.applicationID
+}
+
+// ------------------------
+// ApplicationEvent is used for testing and rejected app's events
+// ------------------------
+type ApplicationEvent struct {
+	applicationID string
+	event         ApplicationEventType
+	message       string
+}
+
+func NewApplicationEvent(appID string, eventType ApplicationEventType, msg string) ApplicationEvent {
+	return ApplicationEvent{
+		applicationID: appID,
+		event:         eventType,
+		message:       msg,
+	}
+}
+
+func (st ApplicationEvent) GetEvent() string {
+	return st.event.String()
+}
+
+func (st ApplicationEvent) GetArgs() []interface{} {
+	args := make([]interface{}, 1)
+	args[0] = st.message
+	return args
+}
+
+func (st ApplicationEvent) GetApplicationID() string {
+	return st.applicationID
+}
+
+// ------------------------
+// ApplicationStatusChangeEvent updates the status in the application CRD
+// ------------------------
+type ApplicationStatusChangeEvent struct {
+	applicationID string
+	event         ApplicationEventType
+	state         string
+}
+
+func NewApplicationStatusChangeEvent(appID string, eventType ApplicationEventType, state string) ApplicationStatusChangeEvent {
+	return ApplicationStatusChangeEvent{
+		applicationID: appID,
+		event:         eventType,
+		state:         state,
+	}
+}
+
+func (st ApplicationStatusChangeEvent) GetEvent() string {
+	return st.event.String()
+}
+
+func (st ApplicationStatusChangeEvent) GetArgs() []interface{} {
+	return nil
+}
+
+func (st ApplicationStatusChangeEvent) GetApplicationID() string {
+	return st.applicationID
+}
+
+func (st ApplicationStatusChangeEvent) GetState() string {
+	return st.state
+}
+
+// ------------------------
+// SubmitTask application
+// ------------------------
+type SubmitApplicationEvent struct {
+	applicationID string
+	event         ApplicationEventType
+}
+
+func NewSubmitApplicationEvent(appID string) SubmitApplicationEvent {
+	return SubmitApplicationEvent{
+		applicationID: appID,
+		event:         SubmitApplication,
+	}
+}
+
+func (se SubmitApplicationEvent) GetEvent() string {
+	return se.event.String()
+}
+
+func (se SubmitApplicationEvent) GetArgs() []interface{} {
+	return nil
+}
+
+func (se SubmitApplicationEvent) GetApplicationID() string {
+	return se.applicationID
+}
+
+// ------------------------
+// Run application
+// ------------------------
+type RunApplicationEvent struct {
+	applicationID string
+	event         ApplicationEventType
+}
+
+func NewRunApplicationEvent(appID string) RunApplicationEvent {
+	return RunApplicationEvent{
+		applicationID: appID,
+		event:         RunApplication,
+	}
+}
+
+func (re RunApplicationEvent) GetEvent() string {
+	return re.event.String()
+}
+
+func (re RunApplicationEvent) GetArgs() []interface{} {
+	return nil
+}
+
+func (re RunApplicationEvent) GetApplicationID() string {
+	return re.applicationID
+}
+
+// ------------------------
+// Fail application
+// ------------------------
+type FailApplicationEvent struct {
+	applicationID string
+	event         ApplicationEventType
+	errorMessage  string
+}
+
+func NewFailApplicationEvent(appID, message string) FailApplicationEvent {
+	return FailApplicationEvent{
+		applicationID: appID,
+		event:         FailApplication,
+		errorMessage:  message,
+	}
+}
+
+func (fe FailApplicationEvent) GetEvent() string {
+	return fe.event.String()
+}
+
+func (fe FailApplicationEvent) GetArgs() []interface{} {
+	args := make([]interface{}, 1)
+	args[0] = fe.errorMessage
+	return args
+}
+
+func (fe FailApplicationEvent) GetApplicationID() string {
+	return fe.applicationID
+}
+
+// ------------------------
+// Reservation Update Event
+// ------------------------
+type UpdateApplicationReservationEvent struct {
+	applicationID string
+	event         ApplicationEventType
+}
+
+func NewUpdateApplicationReservationEvent(appID string) UpdateApplicationReservationEvent {
+	return UpdateApplicationReservationEvent{
+		applicationID: appID,
+		event:         UpdateReservation,
+	}
+}
+
+func (ue UpdateApplicationReservationEvent) GetEvent() string {
+	return ue.event.String()
+}
+
+func (ue UpdateApplicationReservationEvent) GetArgs() []interface{} {
+	return nil
+}
+
+func (ue UpdateApplicationReservationEvent) GetApplicationID() string {
+	return ue.applicationID
+}
+
+// ------------------------
+// Release application allocations
+// ------------------------
+type ReleaseAppAllocationEvent struct {
+	applicationID   string
+	allocationUUID  string
+	terminationType string
+	event           ApplicationEventType
+}
+
+func NewReleaseAppAllocationEvent(appID string, allocTermination si.TerminationType, uuid string) ReleaseAppAllocationEvent {
+	return ReleaseAppAllocationEvent{
+		applicationID:   appID,
+		allocationUUID:  uuid,
+		terminationType: si.TerminationType_name[int32(allocTermination)],
+		event:           ReleaseAppAllocation,
+	}
+}
+
+func (re ReleaseAppAllocationEvent) GetApplicationID() string {
+	return re.applicationID
+}
+
+func (re ReleaseAppAllocationEvent) GetArgs() []interface{} {
+	args := make([]interface{}, 2)
+	args[0] = re.allocationUUID
+	args[1] = re.terminationType
+	return args
+}
+
+func (re ReleaseAppAllocationEvent) GetEvent() string {
+	return re.event.String()
+}
+
+type ReleaseAppAllocationAskEvent struct {
+	applicationID   string
+	taskID          string
+	terminationType string
+	event           ApplicationEventType
+}
+
+func NewReleaseAppAllocationAskEvent(appID string, allocTermination si.TerminationType, taskID string) ReleaseAppAllocationAskEvent {
+	return ReleaseAppAllocationAskEvent{
+		applicationID:   appID,
+		taskID:          taskID,
+		terminationType: si.TerminationType_name[int32(allocTermination)],
+		event:           ReleaseAppAllocationAsk,
+	}
+}
+
+func (re ReleaseAppAllocationAskEvent) GetApplicationID() string {
+	return re.applicationID
+}
+
+func (re ReleaseAppAllocationAskEvent) GetArgs() []interface{} {
+	args := make([]interface{}, 2)
+	args[0] = re.taskID
+	args[1] = re.terminationType
+	return args
+}
+
+func (re ReleaseAppAllocationAskEvent) GetEvent() string {
+	return re.event.String()
+}
+
+// ------------------------
+// Resuming application
+// ------------------------
+type ResumingApplicationEvent struct {
+	applicationID string
+	event         ApplicationEventType
+}
+
+func NewResumingApplicationEvent(appID string) ResumingApplicationEvent {
+	return ResumingApplicationEvent{
+		applicationID: appID,
+		event:         ResumingApplication,
+	}
+}
+
+func (re ResumingApplicationEvent) GetEvent() string {
+	return re.event.String()
+}
+
+func (re ResumingApplicationEvent) GetArgs() []interface{} {
+	return nil
+}
+
+func (re ResumingApplicationEvent) GetApplicationID() string {
+	return re.applicationID
+}
+
+// ----------------------------------
+// Application states
+// ----------------------------------
+var storeApplicationStates *AStates
+
+type AStates struct {
+	New        string
+	Recovering string
+	Submitted  string
+	Accepted   string
+	Reserving  string
+	Running    string
+	Rejected   string
+	Completed  string
+	Killing    string
+	Killed     string
+	Failing    string
+	Failed     string
+	Resuming   string
+}
+
+func ApplicationStates() *AStates {
+	if storeApplicationStates == nil {

Review Comment:
   Use a sync.Once() construct here for initial instantiation.



##########
pkg/cache/context.go:
##########
@@ -910,13 +910,13 @@ func (ctx *Context) TaskEventHandler() func(obj interface{}) {
 				log.Logger().Error("failed to handle application event")
 				return
 			}
-
 			if task.canHandle(event) {
 				if err := task.handle(event); err != nil {
 					log.Logger().Error("failed to handle task event",
 						zap.String("applicationID", task.applicationID),
 						zap.String("taskID", task.taskID),
 						zap.String("event", string(event.GetEvent())),

Review Comment:
   Looks like this line should be removed?



##########
pkg/shim/scheduler_state.go:
##########
@@ -0,0 +1,178 @@
+/*
+ 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 shim
+
+import (
+	"github.com/looplab/fsm"
+	"go.uber.org/zap"
+
+	"github.com/apache/yunikorn-k8shim/pkg/common/events"
+	"github.com/apache/yunikorn-k8shim/pkg/log"
+)
+
+//----------------------------------------------
+// Scheduler events
+//----------------------------------------------
+type SchedulerEventType int
+
+const (
+	RegisterScheduler SchedulerEventType = iota
+	RegisterSchedulerSucceed
+	RegisterSchedulerFailed
+	RecoverScheduler
+	RecoverSchedulerSucceed
+	RecoverSchedulerFailed
+)
+
+func (ae SchedulerEventType) String() string {
+	return [...]string{"RegisterScheduler", "RegisterSchedulerSucceed", "RegisterSchedulerFailed", "RecoverScheduler", "RecoverSchedulerSucceed", "RecoverSchedulerFailed"}[ae]
+}
+
+type ShimSchedulerEvent struct { //nolint:golint
+	event SchedulerEventType
+}
+
+func (rs ShimSchedulerEvent) GetEvent() string {
+	return rs.event.String()
+}
+
+func (rs ShimSchedulerEvent) GetArgs() []interface{} {
+	return nil
+}
+
+// -------------------------------------------------------------------
+// event to trigger scheduler registration
+// --------------------------------------------------------------------
+type RegisterSchedulerEvent struct {
+	event SchedulerEventType
+}
+
+func newRegisterSchedulerEvent() RegisterSchedulerEvent {
+	return RegisterSchedulerEvent{
+		event: RegisterScheduler,
+	}
+}
+
+func (rs RegisterSchedulerEvent) GetEvent() string {
+	return rs.event.String()
+}
+
+func (rs RegisterSchedulerEvent) GetArgs() []interface{} {
+	return nil
+}
+
+// ----------------------------------
+// Scheduler states
+// ----------------------------------
+var storeScheduleStates *SStates
+
+type SStates struct {
+	New         string
+	Registered  string
+	Registering string
+	Recovering  string
+	Running     string
+	Draining    string
+	Stopped     string
+}
+
+func SchedulerStates() *SStates {
+	if storeScheduleStates == nil {

Review Comment:
   Use sync.Once()



##########
pkg/cache/nodes.go:
##########
@@ -234,9 +234,9 @@ func hasReadyCondition(node *v1.Node) bool {
 	return false
 }
 
-func triggerEvent(node *SchedulerNode, currentState string, eventType events.SchedulerNodeEventType) {
+func triggerEvent(node *SchedulerNode, currentState string, eventType SchedulerNodeEventType) {
 	log.Logger().Info("scheduler node event ", zap.String("name", node.name),
-		zap.String("current state ", currentState), zap.String("transition to ", string(eventType)))
+		zap.String("current state ", currentState), zap.String("transition to ", eventType.String()))

Review Comment:
   Should probably use zap.Stringer() here rather than explicit .String() call.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@yunikorn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org