You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@yunikorn.apache.org by "pbacsko (via GitHub)" <gi...@apache.org> on 2023/06/20 10:15:35 UTC

[GitHub] [yunikorn-core] pbacsko opened a new pull request, #574: [YUNIKORN-1803] Add wrapper for queue events

pbacsko opened a new pull request, #574:
URL: https://github.com/apache/yunikorn-core/pull/574

   ### What is this PR for?
   Add queue events to the core scheduler.
   
   ### What type of PR is it?
   * [ ] - Bug Fix
   * [ ] - Improvement
   * [x] - Feature
   * [ ] - Documentation
   * [ ] - Hot Fix
   * [ ] - Refactoring
   
   ### Todos
   * [ ] - Task
   
   ### What is the Jira issue?
   https://issues.apache.org/jira/browse/YUNIKORN-1803
   
   ### How should this be tested?
   
   ### Screenshots (if appropriate)
   
   ### Questions:
   * [ ] - The licenses files need update.
   * [ ] - There is breaking changes for older versions.
   * [ ] - It needs documentation.
   


-- 
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


[GitHub] [yunikorn-core] manirajv06 commented on a diff in pull request #574: [YUNIKORN-1803] Add queue events

Posted by "manirajv06 (via GitHub)" <gi...@apache.org>.
manirajv06 commented on code in PR #574:
URL: https://github.com/apache/yunikorn-core/pull/574#discussion_r1257834550


##########
pkg/scheduler/objects/queue_events.go:
##########
@@ -0,0 +1,127 @@
+/*
+ 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 objects
+
+import (
+	"github.com/apache/yunikorn-core/pkg/common"
+	"github.com/apache/yunikorn-core/pkg/events"
+	"github.com/apache/yunikorn-scheduler-interface/lib/go/si"
+)
+
+type queueEvents struct {
+	enabled     bool
+	eventSystem events.EventSystem
+	queue       *Queue
+}
+
+func (q *queueEvents) sendNewQueueEvent() {
+	if !q.enabled {
+		return
+	}

Review Comment:
   Do we really need `q.enabled` ? Instead, Can we use `eventSystem != nil` ?



##########
pkg/scheduler/objects/queue_events.go:
##########
@@ -0,0 +1,127 @@
+/*
+ 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 objects
+
+import (
+	"github.com/apache/yunikorn-core/pkg/common"
+	"github.com/apache/yunikorn-core/pkg/events"
+	"github.com/apache/yunikorn-scheduler-interface/lib/go/si"
+)
+
+type queueEvents struct {
+	enabled     bool
+	eventSystem events.EventSystem
+	queue       *Queue
+}
+
+func (q *queueEvents) sendNewQueueEvent() {
+	if !q.enabled {
+		return
+	}
+
+	var detail si.EventRecord_ChangeDetail
+	if q.queue.IsManaged() {
+		detail = si.EventRecord_DETAILS_NONE
+	} else {
+		detail = si.EventRecord_QUEUE_DYNAMIC
+	}

Review Comment:
   detail could be assigned with most frequently used value by default and change it only when necessary.



##########
pkg/scheduler/partition.go:
##########
@@ -196,6 +196,7 @@ func (pc *PartitionContext) addQueue(conf []configs.QueueConfig, parent *objects
 		if err != nil {
 			return err
 		}
+		thisQueue.SendNewQueueEvent()

Review Comment:
   I am not really sure about triggering new queue event from this place as there are some other places where this needs to be triggered. Instead of doing this in 'n' number of places, I think it is better to trigger this even after queue creation in corresponding constructors in `Queue` itself. For example, NewConfiguredQueue and NewDynamicQueue



##########
pkg/scheduler/objects/queue_events.go:
##########
@@ -0,0 +1,127 @@
+/*
+ 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 objects
+
+import (
+	"github.com/apache/yunikorn-core/pkg/common"
+	"github.com/apache/yunikorn-core/pkg/events"
+	"github.com/apache/yunikorn-scheduler-interface/lib/go/si"
+)
+
+type queueEvents struct {
+	enabled     bool
+	eventSystem events.EventSystem
+	queue       *Queue
+}
+
+func (q *queueEvents) sendNewQueueEvent() {
+	if !q.enabled {
+		return
+	}
+
+	var detail si.EventRecord_ChangeDetail
+	if q.queue.IsManaged() {
+		detail = si.EventRecord_DETAILS_NONE
+	} else {
+		detail = si.EventRecord_QUEUE_DYNAMIC
+	}
+	event := events.CreateQueueEventRecord(q.queue.QueuePath, common.Empty, common.Empty, si.EventRecord_ADD,
+		detail, nil)
+	q.eventSystem.AddEvent(event)
+}
+
+func (q *queueEvents) sendNewApplicationEvent(appID string) {
+	if !q.enabled {
+		return
+	}
+
+	event := events.CreateQueueEventRecord(q.queue.QueuePath, common.Empty, appID, si.EventRecord_ADD,
+		si.EventRecord_QUEUE_APP, nil)
+	q.eventSystem.AddEvent(event)
+}
+
+func (q *queueEvents) sendRemoveQueueEvent() {
+	if !q.enabled {
+		return
+	}
+
+	var detail si.EventRecord_ChangeDetail
+	if q.queue.IsManaged() {
+		detail = si.EventRecord_DETAILS_NONE
+	} else {
+		detail = si.EventRecord_QUEUE_DYNAMIC
+	}
+	event := events.CreateQueueEventRecord(q.queue.QueuePath, common.Empty, common.Empty, si.EventRecord_REMOVE,
+		detail, nil)
+	q.eventSystem.AddEvent(event)
+}
+
+func (q *queueEvents) sendRemoveApplicationEvent(appID string) {
+	if !q.enabled {
+		return
+	}
+
+	event := events.CreateQueueEventRecord(q.queue.QueuePath, common.Empty, appID, si.EventRecord_REMOVE,
+		si.EventRecord_QUEUE_APP, nil)
+	q.eventSystem.AddEvent(event)
+}
+func (q *queueEvents) sendMaxResourceChangedEvent() {
+	if !q.enabled {
+		return
+	}
+
+	event := events.CreateQueueEventRecord(q.queue.QueuePath, common.Empty, common.Empty, si.EventRecord_SET,
+		si.EventRecord_QUEUE_MAX, q.queue.maxResource)
+	q.eventSystem.AddEvent(event)
+}
+
+func (q *queueEvents) sendGuaranteedResourceChangedEvent() {
+	if !q.enabled {
+		return
+	}
+
+	event := events.CreateQueueEventRecord(q.queue.QueuePath, common.Empty, common.Empty, si.EventRecord_SET,
+		si.EventRecord_QUEUE_GUARANTEED, q.queue.guaranteedResource)
+	q.eventSystem.AddEvent(event)
+}
+
+func (q *queueEvents) sendTypeChangedEvent() {
+	if !q.enabled {
+		return
+	}
+
+	var message string
+	if q.queue.isLeaf {
+		message = "leaf queue: true"
+	} else {
+		message = "leaf queue: false"
+	}

Review Comment:
   same like before



##########
pkg/scheduler/objects/queue.go:
##########
@@ -296,12 +302,18 @@ func (sq *Queue) applyConf(conf configs.QueueConfig) error {
 		sq.isManaged = true
 	}
 
+	prevLeaf := sq.isLeaf
 	sq.isLeaf = !conf.Parent
 	// Make sure the parent flag is set correctly: config might expect auto parent type creation
 	if len(conf.Queues) > 0 {
 		sq.isLeaf = false
 	}
 
+	if prevLeaf != sq.isLeaf && sq.initialized {

Review Comment:
   Do we really need `sq.initialized`? Instead, Can we use `sq.queueEvents != nil` or something like that? 



-- 
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


[GitHub] [yunikorn-core] pbacsko commented on a diff in pull request #574: [YUNIKORN-1803] Add queue events

Posted by "pbacsko (via GitHub)" <gi...@apache.org>.
pbacsko commented on code in PR #574:
URL: https://github.com/apache/yunikorn-core/pull/574#discussion_r1259413626


##########
pkg/scheduler/partition.go:
##########
@@ -196,6 +196,7 @@ func (pc *PartitionContext) addQueue(conf []configs.QueueConfig, parent *objects
 		if err != nil {
 			return err
 		}
+		thisQueue.SendNewQueueEvent()

Review Comment:
   I was thinking about this before, it seemed to be a better idea because it makes testing slightly easier (no extra event when creating queues). I can check it and see how much changes is needed.



-- 
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


[GitHub] [yunikorn-core] manirajv06 commented on a diff in pull request #574: [YUNIKORN-1803] Add queue events

Posted by "manirajv06 (via GitHub)" <gi...@apache.org>.
manirajv06 commented on code in PR #574:
URL: https://github.com/apache/yunikorn-core/pull/574#discussion_r1259603248


##########
pkg/scheduler/objects/queue_events.go:
##########
@@ -0,0 +1,127 @@
+/*
+ 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 objects
+
+import (
+	"github.com/apache/yunikorn-core/pkg/common"
+	"github.com/apache/yunikorn-core/pkg/events"
+	"github.com/apache/yunikorn-scheduler-interface/lib/go/si"
+)
+
+type queueEvents struct {
+	enabled     bool
+	eventSystem events.EventSystem
+	queue       *Queue
+}
+
+func (q *queueEvents) sendNewQueueEvent() {
+	if !q.enabled {
+		return
+	}
+
+	var detail si.EventRecord_ChangeDetail
+	if q.queue.IsManaged() {
+		detail = si.EventRecord_DETAILS_NONE
+	} else {
+		detail = si.EventRecord_QUEUE_DYNAMIC
+	}

Review Comment:
   I think you missed this place.



-- 
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


[GitHub] [yunikorn-core] pbacsko commented on a diff in pull request #574: [YUNIKORN-1803] Add queue events

Posted by "pbacsko (via GitHub)" <gi...@apache.org>.
pbacsko commented on code in PR #574:
URL: https://github.com/apache/yunikorn-core/pull/574#discussion_r1259412359


##########
pkg/scheduler/objects/queue_events.go:
##########
@@ -0,0 +1,127 @@
+/*
+ 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 objects
+
+import (
+	"github.com/apache/yunikorn-core/pkg/common"
+	"github.com/apache/yunikorn-core/pkg/events"
+	"github.com/apache/yunikorn-scheduler-interface/lib/go/si"
+)
+
+type queueEvents struct {
+	enabled     bool
+	eventSystem events.EventSystem
+	queue       *Queue
+}
+
+func (q *queueEvents) sendNewQueueEvent() {
+	if !q.enabled {
+		return
+	}

Review Comment:
   This will change due to #579. Not sure if this or that PR goes in first, but let's keep this as it is for now.



-- 
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


[GitHub] [yunikorn-core] manirajv06 commented on a diff in pull request #574: [YUNIKORN-1803] Add queue events

Posted by "manirajv06 (via GitHub)" <gi...@apache.org>.
manirajv06 commented on code in PR #574:
URL: https://github.com/apache/yunikorn-core/pull/574#discussion_r1259601331


##########
pkg/scheduler/partition.go:
##########
@@ -196,6 +196,7 @@ func (pc *PartitionContext) addQueue(conf []configs.QueueConfig, parent *objects
 		if err != nil {
 			return err
 		}
+		thisQueue.SendNewQueueEvent()

Review Comment:
   Ok, see if you can address now or file a new jira



-- 
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


[GitHub] [yunikorn-core] codecov[bot] commented on pull request #574: [YUNIKORN-1803] Add wrapper for queue events

Posted by "codecov[bot] (via GitHub)" <gi...@apache.org>.
codecov[bot] commented on PR #574:
URL: https://github.com/apache/yunikorn-core/pull/574#issuecomment-1598507106

   ## [Codecov](https://app.codecov.io/gh/apache/yunikorn-core/pull/574?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#574](https://app.codecov.io/gh/apache/yunikorn-core/pull/574?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (aea6407) into [master](https://app.codecov.io/gh/apache/yunikorn-core/commit/a6a90bb0408a3004592f06d8a79335ac326dcfe4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (a6a90bb) will **increase** coverage by `0.14%`.
   > The diff coverage is `93.33%`.
   
   > :exclamation: Current head aea6407 differs from pull request most recent head b5cb786. Consider uploading reports for the commit b5cb786 to get more accurate results
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #574      +/-   ##
   ==========================================
   + Coverage   75.45%   75.60%   +0.14%     
   ==========================================
     Files          73       74       +1     
     Lines       12057    12158     +101     
   ==========================================
   + Hits         9098     9192      +94     
   - Misses       2638     2643       +5     
   - Partials      321      323       +2     
   ```
   
   
   | [Impacted Files](https://app.codecov.io/gh/apache/yunikorn-core/pull/574?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [pkg/scheduler/objects/queue.go](https://app.codecov.io/gh/apache/yunikorn-core/pull/574?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL3F1ZXVlLmdv) | `77.09% <87.09%> (+0.19%)` | :arrow_up: |
   | [pkg/scheduler/objects/queue\_events.go](https://app.codecov.io/gh/apache/yunikorn-core/pull/574?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL3F1ZXVlX2V2ZW50cy5nbw==) | `95.89% <95.89%> (ø)` | |
   | [pkg/events/events.go](https://app.codecov.io/gh/apache/yunikorn-core/pull/574?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGtnL2V2ZW50cy9ldmVudHMuZ28=) | `100.00% <100.00%> (ø)` | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   


-- 
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


[GitHub] [yunikorn-core] pbacsko commented on a diff in pull request #574: [YUNIKORN-1803] Add queue events

Posted by "pbacsko (via GitHub)" <gi...@apache.org>.
pbacsko commented on code in PR #574:
URL: https://github.com/apache/yunikorn-core/pull/574#discussion_r1259416256


##########
pkg/scheduler/objects/queue.go:
##########
@@ -296,12 +302,18 @@ func (sq *Queue) applyConf(conf configs.QueueConfig) error {
 		sq.isManaged = true
 	}
 
+	prevLeaf := sq.isLeaf
 	sq.isLeaf = !conf.Parent
 	// Make sure the parent flag is set correctly: config might expect auto parent type creation
 	if len(conf.Queues) > 0 {
 		sq.isLeaf = false
 	}
 
+	if prevLeaf != sq.isLeaf && sq.initialized {

Review Comment:
   Let me check, it looks reasonable... 



##########
pkg/scheduler/objects/queue.go:
##########
@@ -296,12 +302,18 @@ func (sq *Queue) applyConf(conf configs.QueueConfig) error {
 		sq.isManaged = true
 	}
 
+	prevLeaf := sq.isLeaf
 	sq.isLeaf = !conf.Parent
 	// Make sure the parent flag is set correctly: config might expect auto parent type creation
 	if len(conf.Queues) > 0 {
 		sq.isLeaf = false
 	}
 
+	if prevLeaf != sq.isLeaf && sq.initialized {

Review Comment:
   Let me check, it looks reasonable to change this. 



-- 
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


[GitHub] [yunikorn-core] pbacsko closed pull request #574: [YUNIKORN-1803] Add queue events

Posted by "pbacsko (via GitHub)" <gi...@apache.org>.
pbacsko closed pull request #574: [YUNIKORN-1803] Add queue events
URL: https://github.com/apache/yunikorn-core/pull/574


-- 
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