You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@yunikorn.apache.org by pb...@apache.org on 2023/09/11 15:22:23 UTC

[yunikorn-core] branch yunikorn-1.3.1 created (now 65d329fc)

This is an automated email from the ASF dual-hosted git repository.

pbacsko pushed a change to branch yunikorn-1.3.1
in repository https://gitbox.apache.org/repos/asf/yunikorn-core.git


      at 65d329fc [YUNIKORN-1942] Null Batch API Response after buffer size change (#635)

This branch includes the following new commits:

     new 06667819 [YUNIKORN-1947] Update SI reference (#633)
     new 407b025e [YUNIKORN-1945] Add missing application state transition event (#629)
     new 65d329fc [YUNIKORN-1942] Null Batch API Response after buffer size change (#635)

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: issues-help@yunikorn.apache.org


[yunikorn-core] 03/03: [YUNIKORN-1942] Null Batch API Response after buffer size change (#635)

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pbacsko pushed a commit to branch yunikorn-1.3.1
in repository https://gitbox.apache.org/repos/asf/yunikorn-core.git

commit 65d329fc4b73d35dcd66c019968e38e8619fb736
Author: Peter Bacsko <pb...@cloudera.com>
AuthorDate: Fri Sep 8 12:56:20 2023 +1000

    [YUNIKORN-1942] Null Batch API Response after buffer size change (#635)
    
    When the ring buffer size is changed the first ID in the buffer needs to
    be updated. Normally the first ID is 0 but after resize this could be
    any random number. That breaks id2pos and causes a nil return in the
    REST call.
    
    Closes: #635
    
    Signed-off-by: Wilfred Spiegelenburg <wi...@apache.org>
    (cherry picked from commit ea422103e51844b598988d42e39e97ca2842df82)
---
 pkg/events/event_ringbuffer.go      | 79 +++++++++++++------------------------
 pkg/events/event_ringbuffer_test.go | 14 +++++--
 pkg/webservice/routes.go            |  2 +-
 3 files changed, 39 insertions(+), 56 deletions(-)

diff --git a/pkg/events/event_ringbuffer.go b/pkg/events/event_ringbuffer.go
index 71ab8cfa..479ea70f 100644
--- a/pkg/events/event_ringbuffer.go
+++ b/pkg/events/event_ringbuffer.go
@@ -35,19 +35,21 @@ type eventRange struct {
 
 // eventRingBuffer A specialized circular buffer to store event objects.
 //
-// Unlike to regular circular buffers, existing entries are never directly removed and new entries can be added if the buffer is full.
+// Unlike regular circular buffers, existing entries are never directly removed
+// and new entries can be added if the buffer is full.
 // In this case, the oldest entry is overwritten and can be collected by the GC.
-// Each event has an ID, however, this mapping is not stored directly. If needed, we calculate the id
-// of the event based on slice positions.
+// Each event has an ID; however, this mapping is not stored directly.
+// If needed, we calculate the id of the event based on slice positions.
 //
-// Retrieving the records can be achieved with GetEventsFromID and GetRecentEntries.
+// Retrieving the records can be achieved with GetEventsFromID.
 type eventRingBuffer struct {
-	events   []*si.EventRecord
-	capacity uint64 // capacity of the buffer
-	head     uint64 // position of the next element (no tail since we don't remove elements)
-	full     bool   // indicates whether the buffer if full - once it is, it stays full unless buffer is resized
-	id       uint64 // unique id of an event record
-	lowestId uint64 // lowest id of an event record available in the buffer at any given time
+	events       []*si.EventRecord
+	capacity     uint64 // capacity of the buffer
+	head         uint64 // position of the next element (no tail since we don't remove elements)
+	full         bool   // indicates whether the buffer if full - once it is, it stays full unless the buffer is resized
+	id           uint64 // unique id of an event record
+	lowestId     uint64 // lowest id of an event record available in the buffer at any given time
+	resizeOffset uint64 // used to aid the calculation of id->pos after resize (see id2pos)
 
 	sync.RWMutex
 }
@@ -62,16 +64,13 @@ func (e *eventRingBuffer) Add(event *si.EventRecord) {
 	if !e.full {
 		e.full = e.head == e.capacity-1
 	} else {
-		// lowest event id updates when new event added to a full buffer
-		log.Log(log.Events).Debug("event buffer full, oldest event will be lost",
-			zap.String("id", strconv.FormatUint(e.lowestId, 10)))
 		e.lowestId++
 	}
 	e.head = (e.head + 1) % e.capacity
 	e.id++
 }
 
-// GetEventsFromID returns "count" number of event records from id if possible. The id can be determined from
+// GetEventsFromID returns "count" number of event records from "id" if possible. The id can be determined from
 // the first call of the method - if it returns nothing because the id is not in the buffer, the lowest valid
 // identifier is returned which can be used to get the first batch.
 // If the caller does not want to pose limit on the number of events returned, "count" must be set to a high
@@ -162,36 +161,18 @@ func (e *eventRingBuffer) getEntriesFromRanges(r1, r2 *eventRange) []*si.EventRe
 }
 
 // id2pos translates the unique event ID to an index in the event slice.
-// If the event is present the position will be returned and the found flag will be true.
-// In the case that the event ID is not present the position returned is 0 and the flag false.
+// If the event is present, the position will be returned and the found flag will be true.
+// If the event ID is not present, the position returned is 0 and the flag is false.
 func (e *eventRingBuffer) id2pos(id uint64) (uint64, bool) {
-	pos := id % e.capacity
-	var calculatedID uint64 // calculated ID based on index values
-	if pos > e.head {
-		diff := pos - e.head
-		calculatedID = e.getLowestID() + diff
-	} else {
-		pId := e.id - 1
-		idAtZero := pId - (pId % e.capacity) // unique id at slice position 0
-		calculatedID = idAtZero + pos
-	}
-
-	if !e.full {
-		if e.head == 0 {
-			// empty
-			return 0, false
-		}
-		if pos >= e.head {
-			// "pos" is not in the [0..head-1] range
-			return 0, false
-		}
+	// id out of range?
+	if id < e.lowestId || id >= e.id {
+		return 0, false
 	}
 
-	if calculatedID != id {
-		return calculatedID, false
-	}
-
-	return pos, true
+	// resizeOffset tells how many elements were "shifted out" after resizing the buffer
+	// eg a buffer with 10 elements is full, then gets resized to 6
+	// the first element at index 0 is no longer 0 or the multiples of 10, but 4, 16, 22, etc.
+	return (id - e.resizeOffset) % e.capacity, true
 }
 
 // getLowestID returns the current lowest available id in the buffer.
@@ -206,7 +187,7 @@ func newEventRingBuffer(capacity uint64) *eventRingBuffer {
 	}
 }
 
-// called from Resize(), This functuin updates the lowest event id available in the buffer
+// called from Resize(), this function updates the lowest event id available in the buffer
 func (e *eventRingBuffer) updateLowestId(beginSize, endSize uint64) {
 	// if buffer size is increasing, lowestId stays the same
 	if beginSize < endSize {
@@ -214,7 +195,7 @@ func (e *eventRingBuffer) updateLowestId(beginSize, endSize uint64) {
 	}
 
 	// bufferSize is shrinking
-	// if number of events is < newSize no change
+	// if the number of events is < newSize, then no change
 	if (e.id - e.getLowestID()) <= endSize {
 		return
 	}
@@ -223,22 +204,18 @@ func (e *eventRingBuffer) updateLowestId(beginSize, endSize uint64) {
 	e.lowestId = e.id - endSize
 }
 
-// resize the existing ring buffer
+// Resize resizes the existing ring buffer
 // this method will be called upon configuration reload
 func (e *eventRingBuffer) Resize(newSize uint64) {
 	e.Lock()
 	defer e.Unlock()
 
 	if newSize == e.capacity {
-		return // Nothing to do if the size is the same
+		return
 	}
 
 	initialSize := e.capacity
-
-	// Create a new buffer with the desired size
 	newEvents := make([]*si.EventRecord, newSize)
-
-	// Determine the number of events to copy
 	var numEventsToCopy uint64
 	if e.id-e.getLowestID() > newSize {
 		numEventsToCopy = newSize
@@ -270,11 +247,9 @@ func (e *eventRingBuffer) Resize(newSize uint64) {
 		copy(newEvents[e.capacity-startIndex:], e.events[:endIndex+1])
 	}
 
-	// Update the buffer's state
 	e.capacity = newSize
 	e.events = newEvents
 	e.head = numEventsToCopy % newSize
-
-	// Update e.full based on whether the buffer is full after resizing
+	e.resizeOffset = e.lowestId
 	e.full = numEventsToCopy == e.capacity
 }
diff --git a/pkg/events/event_ringbuffer_test.go b/pkg/events/event_ringbuffer_test.go
index b155f850..6b390e69 100644
--- a/pkg/events/event_ringbuffer_test.go
+++ b/pkg/events/event_ringbuffer_test.go
@@ -192,6 +192,7 @@ func TestResize(t *testing.T) {
 	assert.Equal(t, uint64(6), ringBuffer.capacity)
 	assert.Equal(t, lastEventIdBeforeResize, ringBuffer.getLastEventID())
 	assert.Equal(t, 6, len(ringBuffer.events))
+	assert.Equal(t, uint64(0), ringBuffer.resizeOffset)
 
 	// Test case 2: Resize to a smaller size
 	lastEventIdBeforeResize = ringBuffer.GetLastEventID()
@@ -199,6 +200,7 @@ func TestResize(t *testing.T) {
 	assert.Equal(t, uint64(2), ringBuffer.capacity)
 	assert.Equal(t, lastEventIdBeforeResize, ringBuffer.getLastEventID())
 	assert.Equal(t, 2, len(ringBuffer.events))
+	assert.Equal(t, uint64(2), ringBuffer.resizeOffset)
 
 	// Test case 3: Resize to a larger size
 	lastEventIdBeforeResize = ringBuffer.GetLastEventID()
@@ -206,6 +208,7 @@ func TestResize(t *testing.T) {
 	assert.Equal(t, uint64(20), ringBuffer.capacity)
 	assert.Equal(t, lastEventIdBeforeResize, ringBuffer.getLastEventID())
 	assert.Equal(t, 20, len(ringBuffer.events))
+	assert.Equal(t, uint64(2), ringBuffer.resizeOffset)
 
 	// Test case 4: Resize when head is at the last element
 	ringBuffer = newEventRingBuffer(5)
@@ -215,6 +218,7 @@ func TestResize(t *testing.T) {
 	assert.Equal(t, uint64(2), ringBuffer.capacity)
 	assert.Equal(t, lastEventIdBeforeResize, ringBuffer.getLastEventID())
 	assert.Equal(t, 2, len(ringBuffer.events))
+	assert.Equal(t, uint64(2), ringBuffer.resizeOffset)
 
 	// Test case 5: Resize to events length when head is at the last element
 	ringBuffer = newEventRingBuffer(5)
@@ -225,7 +229,8 @@ func TestResize(t *testing.T) {
 	assert.Equal(t, uint64(4), ringBuffer.capacity)
 	assert.Equal(t, lastEventIdBeforeResize, ringBuffer.getLastEventID())
 	assert.Equal(t, 4, len(ringBuffer.events))
-	assert.Equal(t, true, ringBuffer.full)
+	assert.Equal(t, uint64(0), ringBuffer.resizeOffset)
+	assert.Assert(t, ringBuffer.full)
 
 	// Test case 6: Resize when the buffer is full
 	ringBuffer = newEventRingBuffer(10)
@@ -237,6 +242,7 @@ func TestResize(t *testing.T) {
 	assert.Equal(t, 6, len(ringBuffer.events))
 	assert.Equal(t, uint64(0), ringBuffer.head)
 	assert.Equal(t, true, ringBuffer.full)
+	assert.Equal(t, uint64(4), ringBuffer.resizeOffset)
 
 	// Test case 7: Resize when the buffer is overflown (head is wrapped and position > 0)
 	ringBuffer = newEventRingBuffer(10)
@@ -248,7 +254,8 @@ func TestResize(t *testing.T) {
 	assert.Equal(t, lastEventIdBeforeResize, ringBuffer.getLastEventID())
 	assert.Equal(t, 8, len(ringBuffer.events))
 	assert.Equal(t, uint64(0), ringBuffer.head)
-	assert.Equal(t, true, ringBuffer.full)
+	assert.Equal(t, uint64(7), ringBuffer.resizeOffset)
+	assert.Assert(t, ringBuffer.full)
 
 	// Test case 8: Test event full : Resize to lower size, followed by resize to a large size
 	ringBuffer = newEventRingBuffer(10)
@@ -258,6 +265,7 @@ func TestResize(t *testing.T) {
 	assert.Equal(t, true, ringBuffer.full)
 	ringBuffer.Resize(6)
 	assert.Equal(t, false, ringBuffer.full)
+	assert.Equal(t, uint64(7), ringBuffer.resizeOffset)
 
 	// Test case 9: Test resize to same size
 	lastEventIdBeforeResize = ringBuffer.GetLastEventID()
@@ -266,7 +274,7 @@ func TestResize(t *testing.T) {
 	assert.Equal(t, lastEventIdBeforeResize, ringBuffer.getLastEventID())
 	assert.Equal(t, 6, len(ringBuffer.events))
 	assert.Equal(t, false, ringBuffer.full)
-
+	assert.Equal(t, uint64(7), ringBuffer.resizeOffset)
 }
 
 func populate(buffer *eventRingBuffer, count int) {
diff --git a/pkg/webservice/routes.go b/pkg/webservice/routes.go
index 632a6dbc..bbe1629c 100644
--- a/pkg/webservice/routes.go
+++ b/pkg/webservice/routes.go
@@ -185,7 +185,7 @@ var webRoutes = routes{
 	route{
 		"Scheduler",
 		"GET",
-		"/ws/v1/events/batch/",
+		"/ws/v1/events/batch",
 		getEvents,
 	},
 	// endpoint to retrieve CPU, Memory profiling data,


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: issues-help@yunikorn.apache.org


[yunikorn-core] 02/03: [YUNIKORN-1945] Add missing application state transition event (#629)

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pbacsko pushed a commit to branch yunikorn-1.3.1
in repository https://gitbox.apache.org/repos/asf/yunikorn-core.git

commit 407b025efa7cfe9d007622870e3ba541b978534b
Author: Peter Bacsko <pb...@cloudera.com>
AuthorDate: Thu Aug 31 19:46:55 2023 +0200

    [YUNIKORN-1945] Add missing application state transition event (#629)
    
    Closes: #629
    
    Signed-off-by: Peter Bacsko <pb...@cloudera.com>
    (cherry picked from commit 2feb6ecc13849130c99f90a78e053fc6b319e725)
---
 pkg/scheduler/objects/application.go            |  3 ---
 pkg/scheduler/objects/application_state.go      | 27 ++++++++++++++++++-------
 pkg/scheduler/objects/application_state_test.go | 17 ++++++++++++++--
 3 files changed, 35 insertions(+), 12 deletions(-)

diff --git a/pkg/scheduler/objects/application.go b/pkg/scheduler/objects/application.go
index 3f58c1a7..56d6a6b8 100644
--- a/pkg/scheduler/objects/application.go
+++ b/pkg/scheduler/objects/application.go
@@ -308,9 +308,6 @@ func (sa *Application) HandleApplicationEventWithInfo(event applicationEvent, ev
 	if err != nil && err.Error() == noTransition {
 		return nil
 	}
-	if event == RejectApplication {
-		sa.appEvents.sendRejectApplicationEvent(eventInfo)
-	}
 	return err
 }
 
diff --git a/pkg/scheduler/objects/application_state.go b/pkg/scheduler/objects/application_state.go
index 98961884..35dbc6b3 100644
--- a/pkg/scheduler/objects/application_state.go
+++ b/pkg/scheduler/objects/application_state.go
@@ -74,6 +74,19 @@ const (
 	Resuming
 )
 
+var stateEvents = map[string]si.EventRecord_ChangeDetail{
+	Accepted.String():   si.EventRecord_APP_ACCEPTED,
+	Starting.String():   si.EventRecord_APP_STARTING,
+	Running.String():    si.EventRecord_APP_RUNNING,
+	Rejected.String():   si.EventRecord_APP_REJECT,
+	Completing.String(): si.EventRecord_APP_COMPLETING,
+	Completed.String():  si.EventRecord_APP_COMPLETED,
+	Failing.String():    si.EventRecord_APP_FAILING,
+	Failed.String():     si.EventRecord_APP_FAILED,
+	Resuming.String():   si.EventRecord_APP_RESUMING,
+	Expired.String():    si.EventRecord_APP_EXPIRED,
+}
+
 func (as applicationState) String() string {
 	return [...]string{"New", "Accepted", "Starting", "Running", "Rejected", "Completing", "Completed", "Failing", "Failed", "Expired", "Resuming"}[as]
 }
@@ -142,6 +155,13 @@ func NewAppState() *fsm.FSM {
 				} else {
 					app.OnStateChange(event, "")
 				}
+				eventDetails, ok := stateEvents[event.Dst]
+				if !ok {
+					log.Log(log.SchedFSM).Error("event details not found",
+						zap.String("state", event.Dst))
+					return
+				}
+				app.appEvents.sendStateChangeEvent(eventDetails)
 			},
 			"leave_state": func(_ context.Context, event *fsm.Event) {
 				event.Args[0].(*Application).clearStateTimer() //nolint:errcheck
@@ -153,7 +173,6 @@ func NewAppState() *fsm.FSM {
 				app.setStateTimer(app.startTimeout, app.stateMachine.Current(), RunApplication)
 				metrics.GetQueueMetrics(app.queuePath).IncQueueApplicationsRunning()
 				metrics.GetSchedulerMetrics().IncTotalApplicationsRunning()
-				app.appEvents.sendStateChangeEvent(si.EventRecord_APP_STARTING)
 			},
 			fmt.Sprintf("enter_%s", Resuming.String()): func(_ context.Context, event *fsm.Event) {
 				app := event.Args[0].(*Application) //nolint:errcheck
@@ -168,13 +187,11 @@ func NewAppState() *fsm.FSM {
 					metrics.GetSchedulerMetrics().DecTotalApplicationsRunning()
 				}
 				app.setStateTimer(completingTimeout, app.stateMachine.Current(), CompleteApplication)
-				app.appEvents.sendStateChangeEvent(si.EventRecord_APP_COMPLETING)
 			},
 			fmt.Sprintf("leave_%s", New.String()): func(_ context.Context, event *fsm.Event) {
 				app := event.Args[0].(*Application) //nolint:errcheck
 				metrics.GetQueueMetrics(app.queuePath).IncQueueApplicationsAccepted()
 				metrics.GetSchedulerMetrics().IncTotalApplicationsAccepted()
-				app.appEvents.sendStateChangeEvent(si.EventRecord_APP_ACCEPTED)
 			},
 			fmt.Sprintf("enter_%s", Rejected.String()): func(_ context.Context, event *fsm.Event) {
 				app := event.Args[0].(*Application) //nolint:errcheck
@@ -199,7 +216,6 @@ func NewAppState() *fsm.FSM {
 				// account for going back into running state
 				if event.Dst == Running.String() {
 					app.queue.incRunningApps(app.ApplicationID)
-					app.appEvents.sendStateChangeEvent(si.EventRecord_APP_RUNNING)
 				}
 			},
 			fmt.Sprintf("enter_%s", Completed.String()): func(_ context.Context, event *fsm.Event) {
@@ -207,7 +223,6 @@ func NewAppState() *fsm.FSM {
 				metrics.GetSchedulerMetrics().IncTotalApplicationsCompleted()
 				metrics.GetQueueMetrics(app.queuePath).IncQueueApplicationsCompleted()
 				app.setStateTimer(terminatedTimeout, app.stateMachine.Current(), ExpireApplication)
-				app.appEvents.sendStateChangeEvent(si.EventRecord_APP_COMPLETED)
 				app.executeTerminatedCallback()
 				app.clearPlaceholderTimer()
 				app.cleanupAsks()
@@ -216,12 +231,10 @@ func NewAppState() *fsm.FSM {
 				app := event.Args[0].(*Application) //nolint:errcheck
 				metrics.GetQueueMetrics(app.queuePath).IncQueueApplicationsFailed()
 				metrics.GetSchedulerMetrics().IncTotalApplicationsFailed()
-				app.appEvents.sendStateChangeEvent(si.EventRecord_APP_FAILING)
 			},
 			fmt.Sprintf("enter_%s", Failed.String()): func(_ context.Context, event *fsm.Event) {
 				app := event.Args[0].(*Application) //nolint:errcheck
 				app.setStateTimer(terminatedTimeout, app.stateMachine.Current(), ExpireApplication)
-				app.appEvents.sendStateChangeEvent(si.EventRecord_APP_FAILED)
 				app.executeTerminatedCallback()
 				app.cleanupAsks()
 			},
diff --git a/pkg/scheduler/objects/application_state_test.go b/pkg/scheduler/objects/application_state_test.go
index cdd4b278..2f1bf564 100644
--- a/pkg/scheduler/objects/application_state_test.go
+++ b/pkg/scheduler/objects/application_state_test.go
@@ -272,21 +272,34 @@ func TestAppStateTransitionEvents(t *testing.T) {
 	assert.NilError(t, err, "no error expected failing to failed")
 	assert.Assert(t, appInfo.IsFailed(), "App should be in Failed state")
 
+	// failed to expired
+	err = appInfo.HandleApplicationEvent(ExpireApplication)
+	assert.NilError(t, err, "no error expected failed to expired")
+	assert.Assert(t, appInfo.IsExpired(), "App should be in Expired state")
+
+	// accepted to resuming
+	appInfo.stateMachine.SetState(Accepted.String())
+	err = appInfo.HandleApplicationEvent(ResumeApplication)
+	assert.NilError(t, err, "no error expected accepted to resuming")
+	assert.Assert(t, appInfo.IsResuming(), "App should be in Resuming state")
+
 	// Verify application events
 	err = common.WaitFor(10*time.Millisecond, time.Second, func() bool {
 		fmt.Printf("checking event length: %d\n", eventSystem.Store.CountStoredEvents())
-		return eventSystem.Store.CountStoredEvents() == 6
+		return eventSystem.Store.CountStoredEvents() == 8
 	})
 	assert.NilError(t, err, "the event should have been processed")
 	records := eventSystem.Store.CollectEvents()
 	if records == nil {
 		t.Fatal("collecting eventChannel should return something")
 	}
-	assert.Equal(t, 6, len(records), "expecting 6 events")
+	assert.Equal(t, 8, len(records), "expecting 8 events")
 	isNewApplicationEvent(t, appInfo, records[0])
 	isStateChangeEvent(t, appInfo, si.EventRecord_APP_ACCEPTED, records[1])
 	isStateChangeEvent(t, appInfo, si.EventRecord_APP_COMPLETING, records[2])
 	isStateChangeEvent(t, appInfo, si.EventRecord_APP_RUNNING, records[3])
 	isStateChangeEvent(t, appInfo, si.EventRecord_APP_FAILING, records[4])
 	isStateChangeEvent(t, appInfo, si.EventRecord_APP_FAILED, records[5])
+	isStateChangeEvent(t, appInfo, si.EventRecord_APP_EXPIRED, records[6])
+	isStateChangeEvent(t, appInfo, si.EventRecord_APP_RESUMING, records[7])
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: issues-help@yunikorn.apache.org


[yunikorn-core] 01/03: [YUNIKORN-1947] Update SI reference (#633)

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pbacsko pushed a commit to branch yunikorn-1.3.1
in repository https://gitbox.apache.org/repos/asf/yunikorn-core.git

commit 06667819f397da45b1ad9ee99ef7e767803e7753
Author: Peter Bacsko <pb...@cloudera.com>
AuthorDate: Thu Aug 31 19:54:09 2023 +0530

    [YUNIKORN-1947] Update SI reference (#633)
    
    Closes: #633
    
    Signed-off-by: Manikandan R <ma...@gmail.com>
    (cherry picked from commit dede7c10593e44c299bd2eee84df380c57fd844b)
---
 go.mod | 2 +-
 go.sum | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/go.mod b/go.mod
index 456ca0f7..54e43e0a 100644
--- a/go.mod
+++ b/go.mod
@@ -22,7 +22,7 @@ module github.com/apache/yunikorn-core
 go 1.20
 
 require (
-	github.com/apache/yunikorn-scheduler-interface v0.0.0-20230803134744-355d6a16e4f0
+	github.com/apache/yunikorn-scheduler-interface v0.0.0-20230831133811-e7622cf54e95
 	github.com/google/btree v1.1.2
 	github.com/google/go-cmp v0.5.9
 	github.com/google/uuid v1.3.0
diff --git a/go.sum b/go.sum
index 33bfdea4..643263af 100644
--- a/go.sum
+++ b/go.sum
@@ -40,8 +40,8 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy
 github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
 github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
 github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho=
-github.com/apache/yunikorn-scheduler-interface v0.0.0-20230803134744-355d6a16e4f0 h1:DAyG6TAEbBJUYw7qEz5sysOM0lrR2d3NjHB4bHBdk8A=
-github.com/apache/yunikorn-scheduler-interface v0.0.0-20230803134744-355d6a16e4f0/go.mod h1:/n67iTTOytyVor6wETVjleqzsp/NxCUmPslHcTvJ+Nw=
+github.com/apache/yunikorn-scheduler-interface v0.0.0-20230831133811-e7622cf54e95 h1:ssLPAWkGdrBJwfPEPiv0np99IgynCmmR402TM0FxgPg=
+github.com/apache/yunikorn-scheduler-interface v0.0.0-20230831133811-e7622cf54e95/go.mod h1:/n67iTTOytyVor6wETVjleqzsp/NxCUmPslHcTvJ+Nw=
 github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8=
 github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
 github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: issues-help@yunikorn.apache.org