You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@yunikorn.apache.org by "manirajv06 (via GitHub)" <gi...@apache.org> on 2023/05/11 16:01:33 UTC

[GitHub] [yunikorn-core] manirajv06 opened a new pull request, #543: [YUNIKORN-1608] Configuration updates and storage for limit

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

   ### What is this PR for?
   Traverse the limits configuration (already passed through validation) for all queues recursively and set the user and groups limits in corresponding QueueTracker object.
   
   ### What type of PR is it?
   * [ ] - Improvement
   
   ### Todos
   * [ ] - Task
   
   ### What is the Jira issue?
   https://issues.apache.org/jira/browse/YUNIKORN-1608
   
   ### 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 closed pull request #543: [YUNIKORN-1608] Configuration updates and storage for limit

Posted by "manirajv06 (via GitHub)" <gi...@apache.org>.
manirajv06 closed pull request #543: [YUNIKORN-1608] Configuration updates and storage for limit
URL: https://github.com/apache/yunikorn-core/pull/543


-- 
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 pull request #543: [YUNIKORN-1608] Configuration updates and storage for limit

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

   Test failure is not related


-- 
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 #543: [YUNIKORN-1608] Configuration updates and storage for limit

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


##########
pkg/scheduler/ugm/manager.go:
##########
@@ -73,11 +80,66 @@ func (m *Manager) IncreaseTrackedResource(queuePath string, applicationID string
 		return fmt.Errorf("mandatory parameters are missing. queuepath: %s, application id: %s, resource usage: %s, user: %s",
 			queuePath, applicationID, usage.String(), user.User)
 	}
-	m.lock.Lock()
-	defer m.lock.Unlock()
+	m.Lock()
+	defer m.Unlock()
 	var userTracker *UserTracker
 	if m.userTrackers[user.User] == nil {
-		userTracker = newUserTracker(user)
+		userTracker = newUserTracker(user.User)
+
+		// Set the limits for all configured queue paths of the user
+		for configQueuePath, config := range m.userLimitsConfig[user.User] {
+			log.Logger().Debug("Setting the limit max applications settings.",
+				zap.String("user", user.User),
+				zap.String("queue path", configQueuePath))
+			maxApps, maxAppsErr := config[maxapplications].(uint64)
+			if !maxAppsErr {
+				log.Logger().Warn("Problem in setting the limit max applications settings. Unable to cast the value from interface to uint64",
+					zap.String("user", user.User),
+					zap.String("queue path", configQueuePath),
+					zap.Uint64("limit max applications", maxApps))
+				return fmt.Errorf("unable to set the max resources. user: %s, queuepath : %s, applicationid: %s",
+					user.User, configQueuePath, applicationID)
+			}
+			err := userTracker.setMaxApplications(maxApps, configQueuePath)
+			if err != nil {
+				log.Logger().Warn("Problem in setting the limit max applications settings.",
+					zap.String("user", user.User),
+					zap.String("queue path", configQueuePath),
+					zap.Uint64("limit max applications", maxApps),
+					zap.Error(err))
+				return fmt.Errorf("unable to set the max applications. user: %s, queuepath : %s, applicationid: %s, usage: %s, reason: %w",
+					user.User, configQueuePath, applicationID, usage.String(), err)
+			}
+			maxResources, maxResourcesErr := config[maxresources].(map[string]string)
+			if !maxResourcesErr {
+				log.Logger().Warn("Problem in setting the limit max resources settings. Unable to cast the value from interface to resource",
+					zap.String("user", user.User),
+					zap.String("queue path", configQueuePath),
+					zap.Any("limit max resources", maxResources))
+				return fmt.Errorf("unable to set the max resources. user: %s, queuepath : %s, applicationid: %s",
+					user.User, configQueuePath, applicationID)
+			}
+			resource, resourceErr := resources.NewResourceFromConf(maxResources)

Review Comment:
   There are some codecov complaints here, please address those if relevant (I'm generally in favor of checking the negative paths).



-- 
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 #543: [YUNIKORN-1608] Configuration updates and storage for limit

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


##########
pkg/scheduler/ugm/manager.go:
##########
@@ -314,15 +432,278 @@ func (m *Manager) isGroupRemovable(gt *GroupTracker) bool {
 	return false
 }
 
+func (m *Manager) UpdateConfig(config configs.QueueConfig, queuePath string) error {
+	m.Lock()
+	defer m.Unlock()
+
+	// clear the local limit config maps before processing the limit config
+	m.userLimitsConfig = make(map[string]map[string]map[string]interface{})
+	m.groupLimitsConfig = make(map[string]map[string]map[string]interface{})
+	return m.internalProcessConfig(config, queuePath)
+}
+
+func (m *Manager) internalProcessConfig(cur configs.QueueConfig, queuePath string) error {

Review Comment:
   This method is excessively long. Can't you split it up to reasonable parts & name them properly?



-- 
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 pull request #543: [YUNIKORN-1608] Configuration updates and storage for limit

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

   Filed https://issues.apache.org/jira/browse/YUNIKORN-1815 based on discussion.


-- 
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] wilfred-s commented on pull request #543: [YUNIKORN-1608] Configuration updates and storage for limit

Posted by "wilfred-s (via GitHub)" <gi...@apache.org>.
wilfred-s commented on PR #543:
URL: https://github.com/apache/yunikorn-core/pull/543#issuecomment-1584085781

   > Test failure is not related
   
   I am also not sure that the test failures are not from this patch.
   We have run other PR pre-commit flows in the last days and all tests pass.
   This one failed on re-run also.


-- 
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] wilfred-s commented on a diff in pull request #543: [YUNIKORN-1608] Configuration updates and storage for limit

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


##########
pkg/scheduler/partition.go:
##########
@@ -178,7 +185,14 @@ func (pc *PartitionContext) updatePartitionDetails(conf configs.PartitionConfig)
 	}
 	root.UpdateQueueProperties()
 	// update the rest of the queues recursively
-	return pc.updateQueues(queueConf.Queues, root)
+	if err := pc.updateQueues(queueConf.Queues, root); err != nil {
+		return err
+	}
+	// update limit settings: start at the root
+	if err := ugm.GetUserManager().UpdateConfig(queueConf, conf.Queues[0].Name); err != nil {
+		return err
+	}
+	return nil

Review Comment:
   make this just a simple `return ugm...`



##########
pkg/scheduler/ugm/manager.go:
##########
@@ -37,6 +38,7 @@ var m *Manager
 type Manager struct {
 	userTrackers  map[string]*UserTracker
 	groupTrackers map[string]*GroupTracker
+	config        configs.QueueConfig

Review Comment:
   Why do we need to store the config?
   We process and then do not need the object any longer, should just live on the stack.



##########
pkg/scheduler/ugm/manager.go:
##########
@@ -32,19 +33,26 @@ import (
 var once sync.Once
 var m *Manager
 
+const maxresources = "maxresources"
+const maxapplications = "maxapplications"
+
 // Manager implements tracker. A User Group Manager to track the usage for both user and groups.
 // Holds object of both user and group trackers
 type Manager struct {
-	userTrackers  map[string]*UserTracker
-	groupTrackers map[string]*GroupTracker
-	lock          sync.RWMutex
+	userTrackers      map[string]*UserTracker
+	groupTrackers     map[string]*GroupTracker
+	userLimitsConfig  map[string]map[string]map[string]interface{} // Hold limits settings of user * queue path
+	groupLimitsConfig map[string]map[string]map[string]interface{} // Hold limits settings of user * queue path

Review Comment:
   holds group limit not user



##########
pkg/scheduler/ugm/manager.go:
##########
@@ -192,7 +255,7 @@ func (m *Manager) DecreaseTrackedResource(queuePath string, applicationID string
 func (m *Manager) GetUserResources(user security.UserGroup) *resources.Resource {
 	m.lock.RLock()
 	defer m.lock.RUnlock()
-	if m.userTrackers[user.User] != nil {
+	if m.userTrackers[user.User] != nil && len(m.userTrackers[user.User].queueTracker.resourceUsage.Resources) > 0 {

Review Comment:
   This uses a direct access without taking a lock on the user object. This can horribly fail with data races. All access to either user and group objects in the manager must go through the locked calls



##########
pkg/scheduler/ugm/manager.go:
##########
@@ -32,19 +33,26 @@ import (
 var once sync.Once
 var m *Manager
 
+const maxresources = "maxresources"
+const maxapplications = "maxapplications"
+
 // Manager implements tracker. A User Group Manager to track the usage for both user and groups.
 // Holds object of both user and group trackers
 type Manager struct {
-	userTrackers  map[string]*UserTracker
-	groupTrackers map[string]*GroupTracker
-	lock          sync.RWMutex
+	userTrackers      map[string]*UserTracker
+	groupTrackers     map[string]*GroupTracker
+	userLimitsConfig  map[string]map[string]map[string]interface{} // Hold limits settings of user * queue path
+	groupLimitsConfig map[string]map[string]map[string]interface{} // Hold limits settings of user * queue path
+	lock              sync.RWMutex

Review Comment:
   can we just implement `sync.RWMutex`
   remove `lock` everywhere in the file and call `m.Unlock()` etc



##########
pkg/scheduler/ugm/manager.go:
##########
@@ -260,6 +323,61 @@ func (m *Manager) ensureGroupTrackerForApp(queuePath string, applicationID strin
 				zap.String("user", user.User),
 				zap.String("group", group))
 			groupTracker = newGroupTracker(group)
+
+			// Set the limits for all configured queue paths of the group
+			for configQueuePath, config := range m.groupLimitsConfig[group] {
+				log.Logger().Debug("Setting the limit max applications settings.",
+					zap.String("group", group),
+					zap.String("queue path", configQueuePath))
+				maxApps, maxAppsErr := config[maxapplications].(uint64)
+				if !maxAppsErr {
+					log.Logger().Warn("Problem in setting the limit max applications settings. Unable to cast the value from interface to uint64",
+						zap.String("group", group),
+						zap.String("queue path", configQueuePath),
+						zap.Any("limit max applications", maxApps))

Review Comment:
   zap.Any only if there is no types logger available (multiple occurrences)



##########
pkg/scheduler/ugm/manager.go:
##########
@@ -77,7 +85,62 @@ func (m *Manager) IncreaseTrackedResource(queuePath string, applicationID string
 	defer m.lock.Unlock()
 	var userTracker *UserTracker
 	if m.userTrackers[user.User] == nil {
-		userTracker = newUserTracker(user)
+		userTracker = newUserTracker(user.User)
+
+		// Set the limits for all configured queue paths of the user
+		for configQueuePath, config := range m.userLimitsConfig[user.User] {
+			log.Logger().Debug("Setting the limit max applications settings.",
+				zap.String("user", user.User),
+				zap.String("queue path", configQueuePath))
+			maxApps, maxAppsErr := config[maxapplications].(uint64)
+			if !maxAppsErr {
+				log.Logger().Warn("Problem in setting the limit max applications settings. Unable to cast the value from interface to uint64",
+					zap.String("user", user.User),
+					zap.String("queue path", configQueuePath),
+					zap.Any("limit max applications", maxApps))
+				return fmt.Errorf("unable to set the max resources. user: %s, queuepath : %s, applicationid: %s",
+					user.User, configQueuePath, applicationID)
+			}
+			err := userTracker.setMaxApplications(maxApps, configQueuePath)
+			if err != nil {
+				log.Logger().Warn("Problem in setting the limit max applications settings.",
+					zap.String("user", user.User),
+					zap.String("queue path", configQueuePath),
+					zap.Any("limit max applications", maxApps),
+					zap.String("error message", err.Error()))
+				return fmt.Errorf("unable to set the max applications. user: %s, queuepath : %s, applicationid: %s, usage: %s, reason: %s",

Review Comment:
   use `%w` for wrapping errors (multiple occurrences)



##########
pkg/scheduler/partition.go:
##########
@@ -128,6 +129,12 @@ func (pc *PartitionContext) initialPartitionFromConfig(conf configs.PartitionCon
 	// TODO get the resolver from the config
 	pc.userGroupCache = security.GetUserGroupCache("")
 	pc.updateNodeSortingPolicy(conf)
+
+	// update limit settings: start at the root
+	if err := ugm.GetUserManager().UpdateConfig(queueConf, conf.Queues[0].Name); err != nil {
+		return err
+	}
+

Review Comment:
   simplify to `return ugm.GetUserManager().UpdateConfig(queueConf, conf.Queues[0].Name)`
   Same construct is used in numerous places



##########
pkg/scheduler/ugm/utilities.go:
##########
@@ -0,0 +1,41 @@
+/*
+ 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 ugm
+
+import (
+	"strings"
+
+	"github.com/apache/yunikorn-core/pkg/common/configs"
+)
+
+func getChildQueuePath(queuePath string) (string, string) {
+	idx := strings.Index(queuePath, configs.DOT)
+	childQueuePath := ""
+	if idx != -1 {
+		childQueuePath = queuePath[idx+1:]
+	}

Review Comment:
   if idx == -1 there is no dot, so the return values are fixed: "" and ""
   less overhead when we do it like this:
   ```
   	idx := strings.Index(queuePath, ".")
   	if idx == -1 {
   		return "", ""
   	}
   
   	childQueuePath := queuePath[idx+1:]
   	idx = strings.Index(childQueuePath, ".")
   	if idx == -1 {
   		return childQueuePath, childQueuePath
   	}
   	return childQueuePath, childQueuePath[:idx]
   ```



##########
pkg/scheduler/ugm/queue_tracker.go:
##########
@@ -48,6 +49,8 @@ func newQueueTracker(queueName string) *QueueTracker {
 		queueName:           queueName,
 		resourceUsage:       resources.NewResource(),
 		runningApplications: make(map[string]bool),
+		maxResourceUsage:    resources.NewResource(),
+		maxRunningApps:      0,

Review Comment:
   unneeded init, we should handle nil for the resource and uint zero value is 0 



-- 
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 pull request #543: [YUNIKORN-1608] Configuration updates and storage for limit

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

   > > Test failure is not related
   > 
   > I am also not sure that the test failures are not from this patch. We have run other PR pre-commit flows in the last days and all tests pass. This one failed on re-run also.
   
   Tests are passing.


-- 
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 #543: [YUNIKORN-1608] Configuration updates and storage for limit

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


##########
pkg/scheduler/utilities_test.go:
##########
@@ -376,9 +587,77 @@ func getTestUserGroup() security.UserGroup {
 }
 
 func assertUserGroupResource(t *testing.T, userGroup security.UserGroup, expected *resources.Resource) {
-	ugm := ugm.GetUserManager()
-	userResource := ugm.GetUserResources(userGroup)
-	groupResource := ugm.GetGroupResources(userGroup.Groups[0])
+	expectedQueuesMaxLimits := make(map[string]map[string]interface{})
+	expectedQueuesMaxLimits["root"] = make(map[string]interface{})
+	expectedQueuesMaxLimits["root.default"] = make(map[string]interface{})
+	expectedQueuesMaxLimits["root"][maxresources] = resources.NewResourceFromMap(map[string]resources.Quantity{"memory": 10, "vcores": 10})
+	expectedQueuesMaxLimits["root.default"][maxresources] = resources.NewResourceFromMap(map[string]resources.Quantity{"memory": 5, "vcores": 5})
+	expectedQueuesMaxLimits["root"][maxapplications] = uint64(2)
+	expectedQueuesMaxLimits["root.default"][maxapplications] = uint64(1)
+	internalAssertUserGroupResource(t, userGroup, expected, expectedQueuesMaxLimits)
+}
+
+func internalAssertUserGroupResource(t *testing.T, userGroup security.UserGroup, expected *resources.Resource, expectedQueuesMaxLimits map[string]map[string]interface{}) {

Review Comment:
   I'd rename this to `assertUserGroupResourceMaxLimits()`, or maybe even just `assertResourcesMaxLimits()` (and the existing can be changed to `assertResources()`). Otherwise I have to scan with my eyes to see why it's called "internal".



-- 
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] wilfred-s commented on a diff in pull request #543: [YUNIKORN-1608] Configuration updates and storage for limit

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


##########
pkg/scheduler/ugm/manager.go:
##########
@@ -286,8 +404,8 @@ func (m *Manager) getGroup(user security.UserGroup) (string, error) {
 // cleaner Auto wakeup go routine to remove the user and group trackers based on applications being tracked upon, its root queueTracker usage etc
 // nolint:unused
 func (m *Manager) cleaner() {

Review Comment:
   follow up jira needed to remove this function



-- 
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 #543: [YUNIKORN-1608] Configuration updates and storage for limit

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

   ## [Codecov](https://app.codecov.io/gh/apache/yunikorn-core/pull/543?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#543](https://app.codecov.io/gh/apache/yunikorn-core/pull/543?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (86ff77c) into [master](https://app.codecov.io/gh/apache/yunikorn-core/commit/8664ce2b584c41fbfe3ea4aa41634d48214295d0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8664ce2) will **decrease** coverage by `1.23%`.
   > The diff coverage is `14.77%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #543      +/-   ##
   ==========================================
   - Coverage   75.06%   73.83%   -1.23%     
   ==========================================
     Files          70       71       +1     
     Lines       11332    11559     +227     
   ==========================================
   + Hits         8506     8535      +29     
   - Misses       2529     2721     +192     
   - Partials      297      303       +6     
   ```
   
   
   | [Impacted Files](https://app.codecov.io/gh/apache/yunikorn-core/pull/543?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/scheduler/ugm/queue\_tracker.go](https://app.codecov.io/gh/apache/yunikorn-core/pull/543?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci91Z20vcXVldWVfdHJhY2tlci5nbw==) | `51.93% <4.76%> (-28.71%)` | :arrow_down: |
   | [pkg/scheduler/ugm/manager.go](https://app.codecov.io/gh/apache/yunikorn-core/pull/543?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci91Z20vbWFuYWdlci5nbw==) | `41.62% <8.26%> (-20.15%)` | :arrow_down: |
   | [pkg/scheduler/partition.go](https://app.codecov.io/gh/apache/yunikorn-core/pull/543?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9wYXJ0aXRpb24uZ28=) | `76.73% <14.28%> (-0.46%)` | :arrow_down: |
   | [pkg/scheduler/ugm/group\_tracker.go](https://app.codecov.io/gh/apache/yunikorn-core/pull/543?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci91Z20vZ3JvdXBfdHJhY2tlci5nbw==) | `82.60% <20.00%> (-17.40%)` | :arrow_down: |
   | [pkg/scheduler/ugm/user\_tracker.go](https://app.codecov.io/gh/apache/yunikorn-core/pull/543?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci91Z20vdXNlcl90cmFja2VyLmdv) | `85.45% <27.27%> (-14.55%)` | :arrow_down: |
   | [pkg/scheduler/ugm/utilities.go](https://app.codecov.io/gh/apache/yunikorn-core/pull/543?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci91Z20vdXRpbGl0aWVzLmdv) | `100.00% <100.00%> (ø)` | |
   
   ... and [5 files with indirect coverage changes](https://app.codecov.io/gh/apache/yunikorn-core/pull/543/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   :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=The+Apache+Software+Foundation)
   


-- 
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 #543: [YUNIKORN-1608] Configuration updates and storage for limit

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


##########
pkg/scheduler/utilities_test.go:
##########
@@ -376,9 +587,77 @@ func getTestUserGroup() security.UserGroup {
 }
 
 func assertUserGroupResource(t *testing.T, userGroup security.UserGroup, expected *resources.Resource) {
-	ugm := ugm.GetUserManager()
-	userResource := ugm.GetUserResources(userGroup)
-	groupResource := ugm.GetGroupResources(userGroup.Groups[0])
+	expectedQueuesMaxLimits := make(map[string]map[string]interface{})
+	expectedQueuesMaxLimits["root"] = make(map[string]interface{})
+	expectedQueuesMaxLimits["root.default"] = make(map[string]interface{})
+	expectedQueuesMaxLimits["root"][maxresources] = resources.NewResourceFromMap(map[string]resources.Quantity{"memory": 10, "vcores": 10})
+	expectedQueuesMaxLimits["root.default"][maxresources] = resources.NewResourceFromMap(map[string]resources.Quantity{"memory": 5, "vcores": 5})
+	expectedQueuesMaxLimits["root"][maxapplications] = uint64(2)
+	expectedQueuesMaxLimits["root.default"][maxapplications] = uint64(1)
+	internalAssertUserGroupResource(t, userGroup, expected, expectedQueuesMaxLimits)
+}
+
+func internalAssertUserGroupResource(t *testing.T, userGroup security.UserGroup, expected *resources.Resource, expectedQueuesMaxLimits map[string]map[string]interface{}) {
+	manager := ugm.GetUserManager()
+	userResource := manager.GetUserResources(userGroup)
+	groupResource := manager.GetGroupResources(userGroup.Groups[0])
+	ut := manager.GetUserTracker(userGroup.User)
+	if ut != nil {
+		maxResources := make(map[string]*resources.Resource)
+		usage := ut.GetUserResourceUsageDAOInfo()
+		internalGetMaxResource(usage.Queues, maxResources)
+		for q, qMaxLimits := range expectedQueuesMaxLimits {
+			if qRes, ok := maxResources[q]; ok {
+				assert.Equal(t, resources.Equals(qRes, qMaxLimits[maxresources].(*resources.Resource)), true)
+			}
+		}
+		maxApplications := make(map[string]uint64)
+		internalGetMaxApplications(usage.Queues, maxApplications)
+		for q, qMaxLimits := range expectedQueuesMaxLimits {
+			if qApps, ok := maxApplications[q]; ok {
+				assert.Equal(t, qApps, qMaxLimits[maxapplications].(uint64), "queue path is "+q+" actual: "+strconv.Itoa(int(qApps))+", expected: "+strconv.Itoa(int(qMaxLimits[maxapplications].(uint64))))
+			}
+		}
+	}
+
+	gt := manager.GetUserTracker(userGroup.User)
+	if gt != nil {
+		gMaxResources := make(map[string]*resources.Resource)
+		gUsage := gt.GetUserResourceUsageDAOInfo()
+		internalGetMaxResource(gUsage.Queues, gMaxResources)
+		for q, qMaxLimits := range expectedQueuesMaxLimits {
+			if qRes, ok := gMaxResources[q]; ok {
+				assert.Equal(t, resources.Equals(qRes, qMaxLimits[maxresources].(*resources.Resource)), true)
+			}
+		}
+		gMaxApps := make(map[string]uint64)
+		internalGetMaxApplications(gUsage.Queues, gMaxApps)
+		for q, qMaxLimits := range expectedQueuesMaxLimits {
+			if qApps, ok := gMaxApps[q]; ok {
+				assert.Equal(t, qApps, qMaxLimits[maxapplications].(uint64))
+			}
+		}
+	}
 	assert.Equal(t, resources.Equals(userResource, expected), true)
 	assert.Equal(t, resources.Equals(groupResource, expected), true)
 }
+
+func internalGetMaxResource(usage *dao.ResourceUsageDAOInfo, maxResources map[string]*resources.Resource) map[string]*resources.Resource {

Review Comment:
   Nit: just call it `getMaxResource()`, it doesn't collide with an existing function. 



##########
pkg/scheduler/utilities_test.go:
##########
@@ -376,9 +587,77 @@ func getTestUserGroup() security.UserGroup {
 }
 
 func assertUserGroupResource(t *testing.T, userGroup security.UserGroup, expected *resources.Resource) {
-	ugm := ugm.GetUserManager()
-	userResource := ugm.GetUserResources(userGroup)
-	groupResource := ugm.GetGroupResources(userGroup.Groups[0])
+	expectedQueuesMaxLimits := make(map[string]map[string]interface{})
+	expectedQueuesMaxLimits["root"] = make(map[string]interface{})
+	expectedQueuesMaxLimits["root.default"] = make(map[string]interface{})
+	expectedQueuesMaxLimits["root"][maxresources] = resources.NewResourceFromMap(map[string]resources.Quantity{"memory": 10, "vcores": 10})
+	expectedQueuesMaxLimits["root.default"][maxresources] = resources.NewResourceFromMap(map[string]resources.Quantity{"memory": 5, "vcores": 5})
+	expectedQueuesMaxLimits["root"][maxapplications] = uint64(2)
+	expectedQueuesMaxLimits["root.default"][maxapplications] = uint64(1)
+	internalAssertUserGroupResource(t, userGroup, expected, expectedQueuesMaxLimits)
+}
+
+func internalAssertUserGroupResource(t *testing.T, userGroup security.UserGroup, expected *resources.Resource, expectedQueuesMaxLimits map[string]map[string]interface{}) {
+	manager := ugm.GetUserManager()
+	userResource := manager.GetUserResources(userGroup)
+	groupResource := manager.GetGroupResources(userGroup.Groups[0])
+	ut := manager.GetUserTracker(userGroup.User)
+	if ut != nil {
+		maxResources := make(map[string]*resources.Resource)
+		usage := ut.GetUserResourceUsageDAOInfo()
+		internalGetMaxResource(usage.Queues, maxResources)
+		for q, qMaxLimits := range expectedQueuesMaxLimits {
+			if qRes, ok := maxResources[q]; ok {
+				assert.Equal(t, resources.Equals(qRes, qMaxLimits[maxresources].(*resources.Resource)), true)
+			}
+		}
+		maxApplications := make(map[string]uint64)
+		internalGetMaxApplications(usage.Queues, maxApplications)
+		for q, qMaxLimits := range expectedQueuesMaxLimits {
+			if qApps, ok := maxApplications[q]; ok {
+				assert.Equal(t, qApps, qMaxLimits[maxapplications].(uint64), "queue path is "+q+" actual: "+strconv.Itoa(int(qApps))+", expected: "+strconv.Itoa(int(qMaxLimits[maxapplications].(uint64))))
+			}
+		}
+	}
+
+	gt := manager.GetUserTracker(userGroup.User)
+	if gt != nil {
+		gMaxResources := make(map[string]*resources.Resource)
+		gUsage := gt.GetUserResourceUsageDAOInfo()
+		internalGetMaxResource(gUsage.Queues, gMaxResources)
+		for q, qMaxLimits := range expectedQueuesMaxLimits {
+			if qRes, ok := gMaxResources[q]; ok {
+				assert.Equal(t, resources.Equals(qRes, qMaxLimits[maxresources].(*resources.Resource)), true)
+			}
+		}
+		gMaxApps := make(map[string]uint64)
+		internalGetMaxApplications(gUsage.Queues, gMaxApps)
+		for q, qMaxLimits := range expectedQueuesMaxLimits {
+			if qApps, ok := gMaxApps[q]; ok {
+				assert.Equal(t, qApps, qMaxLimits[maxapplications].(uint64))
+			}
+		}
+	}
 	assert.Equal(t, resources.Equals(userResource, expected), true)
 	assert.Equal(t, resources.Equals(groupResource, expected), true)
 }
+
+func internalGetMaxResource(usage *dao.ResourceUsageDAOInfo, maxResources map[string]*resources.Resource) map[string]*resources.Resource {
+	maxResources[usage.QueuePath] = usage.MaxResources
+	if len(usage.Children) > 0 {
+		for _, resourceUsage := range usage.Children {
+			internalGetMaxResource(resourceUsage, maxResources)
+		}
+	}
+	return maxResources
+}
+
+func internalGetMaxApplications(usage *dao.ResourceUsageDAOInfo, maxApplications map[string]uint64) map[string]uint64 {

Review Comment:
   Nit: same.



-- 
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 #543: [YUNIKORN-1608] Configuration updates and storage for limit

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


##########
pkg/scheduler/ugm/queue_tracker.go:
##########
@@ -115,6 +129,74 @@ func (qt *QueueTracker) decreaseTrackedResource(queuePath string, applicationID
 	return removeQT, nil
 }
 
+func (qt *QueueTracker) setMaxApplications(count uint64, queuePath string) error {
+	log.Logger().Debug("Setting max applications",
+		zap.String("queue path", queuePath),
+		zap.Uint64("max applications", count))
+	var childQueuePath, immediateChildQueueName string
+	childQueuePath, immediateChildQueueName = getChildQueuePath(queuePath)
+	childQueueTracker := qt
+	if childQueuePath != "" {
+		for childQueuePath != "" {
+			if childQueueTracker != nil {
+				if len(childQueueTracker.childQueueTrackers) == 0 || childQueueTracker.childQueueTrackers[immediateChildQueueName] == nil {
+					newChildQt := newQueueTracker(immediateChildQueueName)
+					childQueueTracker.childQueueTrackers[immediateChildQueueName] = newChildQt
+					childQueueTracker = newChildQt
+				} else {
+					childQueueTracker = childQueueTracker.childQueueTrackers[immediateChildQueueName]
+				}
+			}
+			childQueuePath, immediateChildQueueName = getChildQueuePath(childQueuePath)
+		}
+	}
+	if childQueueTracker.maxRunningApps != 0 && count != 0 && len(childQueueTracker.runningApplications) > int(count) {
+		log.Logger().Warn("Current running applications is greater than config max applications",
+			zap.String("queue path", queuePath),
+			zap.Uint64("current max applications", childQueueTracker.maxRunningApps),
+			zap.Int("total running applications", len(childQueueTracker.runningApplications)),
+			zap.Uint64("config max applications", count))
+		return fmt.Errorf("current running applications is greater than config max applications for %s", queuePath)
+	} else {
+		childQueueTracker.maxRunningApps = count
+	}
+	return nil
+}
+
+func (qt *QueueTracker) setMaxResources(resource *resources.Resource, queuePath string) error {

Review Comment:
   This method looks almost the same as `setMaxApplications()`, there are a lot of common things, could you extract it? Plus, there's missing coverage here, please address 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] manirajv06 commented on pull request #543: [YUNIKORN-1608] Configuration updates and storage for limit

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

   Test failure is not related


-- 
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 #543: [YUNIKORN-1608] Configuration updates and storage for limit

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


##########
pkg/scheduler/objects/queue.go:
##########
@@ -135,7 +135,6 @@ func NewConfiguredQueue(conf configs.QueueConfig, parent *Queue) (*Queue, error)
 		sq.mergeProperties(parent.getProperties(), conf.Properties)
 	}
 	sq.UpdateQueueProperties()
-

Review Comment:
   Nit: unnecessary change here.



-- 
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 #543: [YUNIKORN-1608] Configuration updates and storage for limit

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


##########
pkg/scheduler/ugm/manager.go:
##########
@@ -73,11 +80,66 @@ func (m *Manager) IncreaseTrackedResource(queuePath string, applicationID string
 		return fmt.Errorf("mandatory parameters are missing. queuepath: %s, application id: %s, resource usage: %s, user: %s",
 			queuePath, applicationID, usage.String(), user.User)
 	}
-	m.lock.Lock()
-	defer m.lock.Unlock()
+	m.Lock()
+	defer m.Unlock()
 	var userTracker *UserTracker
 	if m.userTrackers[user.User] == nil {
-		userTracker = newUserTracker(user)
+		userTracker = newUserTracker(user.User)
+
+		// Set the limits for all configured queue paths of the user
+		for configQueuePath, config := range m.userLimitsConfig[user.User] {
+			log.Logger().Debug("Setting the limit max applications settings.",
+				zap.String("user", user.User),
+				zap.String("queue path", configQueuePath))
+			maxApps, maxAppsErr := config[maxapplications].(uint64)
+			if !maxAppsErr {
+				log.Logger().Warn("Problem in setting the limit max applications settings. Unable to cast the value from interface to uint64",
+					zap.String("user", user.User),
+					zap.String("queue path", configQueuePath),
+					zap.Uint64("limit max applications", maxApps))
+				return fmt.Errorf("unable to set the max resources. user: %s, queuepath : %s, applicationid: %s",
+					user.User, configQueuePath, applicationID)
+			}
+			err := userTracker.setMaxApplications(maxApps, configQueuePath)
+			if err != nil {
+				log.Logger().Warn("Problem in setting the limit max applications settings.",
+					zap.String("user", user.User),
+					zap.String("queue path", configQueuePath),
+					zap.Uint64("limit max applications", maxApps),
+					zap.Error(err))
+				return fmt.Errorf("unable to set the max applications. user: %s, queuepath : %s, applicationid: %s, usage: %s, reason: %w",
+					user.User, configQueuePath, applicationID, usage.String(), err)
+			}
+			maxResources, maxResourcesErr := config[maxresources].(map[string]string)

Review Comment:
   Nit: just simply `ok` instead of  `maxResourcesErr`, that's the idiomatic Go approach.



##########
pkg/scheduler/ugm/manager.go:
##########
@@ -73,11 +80,66 @@ func (m *Manager) IncreaseTrackedResource(queuePath string, applicationID string
 		return fmt.Errorf("mandatory parameters are missing. queuepath: %s, application id: %s, resource usage: %s, user: %s",
 			queuePath, applicationID, usage.String(), user.User)
 	}
-	m.lock.Lock()
-	defer m.lock.Unlock()
+	m.Lock()
+	defer m.Unlock()
 	var userTracker *UserTracker
 	if m.userTrackers[user.User] == nil {
-		userTracker = newUserTracker(user)
+		userTracker = newUserTracker(user.User)
+
+		// Set the limits for all configured queue paths of the user
+		for configQueuePath, config := range m.userLimitsConfig[user.User] {
+			log.Logger().Debug("Setting the limit max applications settings.",
+				zap.String("user", user.User),
+				zap.String("queue path", configQueuePath))
+			maxApps, maxAppsErr := config[maxapplications].(uint64)
+			if !maxAppsErr {
+				log.Logger().Warn("Problem in setting the limit max applications settings. Unable to cast the value from interface to uint64",
+					zap.String("user", user.User),
+					zap.String("queue path", configQueuePath),
+					zap.Uint64("limit max applications", maxApps))
+				return fmt.Errorf("unable to set the max resources. user: %s, queuepath : %s, applicationid: %s",
+					user.User, configQueuePath, applicationID)
+			}
+			err := userTracker.setMaxApplications(maxApps, configQueuePath)
+			if err != nil {
+				log.Logger().Warn("Problem in setting the limit max applications settings.",
+					zap.String("user", user.User),
+					zap.String("queue path", configQueuePath),
+					zap.Uint64("limit max applications", maxApps),
+					zap.Error(err))
+				return fmt.Errorf("unable to set the max applications. user: %s, queuepath : %s, applicationid: %s, usage: %s, reason: %w",
+					user.User, configQueuePath, applicationID, usage.String(), err)
+			}
+			maxResources, maxResourcesErr := config[maxresources].(map[string]string)
+			if !maxResourcesErr {
+				log.Logger().Warn("Problem in setting the limit max resources settings. Unable to cast the value from interface to resource",
+					zap.String("user", user.User),
+					zap.String("queue path", configQueuePath),
+					zap.Any("limit max resources", maxResources))
+				return fmt.Errorf("unable to set the max resources. user: %s, queuepath : %s, applicationid: %s",
+					user.User, configQueuePath, applicationID)
+			}
+			resource, resourceErr := resources.NewResourceFromConf(maxResources)

Review Comment:
   Nit: just simply `ok` instead of  `maxResourcesErr`, that's the idiomatic Go approach.



-- 
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 pull request #543: [YUNIKORN-1608] Configuration updates and storage for limit

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

   @wilfred-s Can you also take a final look?


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