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 2020/10/14 23:38:09 UTC

[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #215: [YUNIKORN-352] Validate queue capacity setup

yangwwei commented on a change in pull request #215:
URL: https://github.com/apache/incubator-yunikorn-core/pull/215#discussion_r504993591



##########
File path: pkg/common/configs/configvalidator.go
##########
@@ -72,37 +72,68 @@ func checkACL(acl string) error {
 	return nil
 }
 
-// Temporary convenience method: should use resource package to do this
-// currently no check for the type of resource as long as the value is OK all is OK
-func checkResource(res map[string]string) (int64, error) {
-	var totalres int64
-	for _, val := range res {
-		rescount, err := strconv.ParseInt(val, 10, 64)
-		if err != nil {
-			return 0, fmt.Errorf("resource parsing failed: %v", err)
+// Check the queue resource configuration settings.
+// - child or children cannot have higher maximum or guaranteed limits than parents
+// - children (added together) cannot have a higher guaranteed setting than a parent
+func checkResourceConfigurationsForQueue(cur *QueueConfig, parent *QueueConfig) error {
+	// If cur has children, make sure sum of children's guaranteed <= cur.guaranteed
+	curGuaranteedRes, err := resources.NewResourceFromConf(cur.Resources.Guaranteed)
+	if err != nil {
+		return err
+	}
+	if curGuaranteedRes.HasNegativeValue() {
+		return fmt.Errorf("invalid guaranteed resource for queue %s, cannot be negative", cur.Name)

Review comment:
       it makes sense to print the resource as well

##########
File path: pkg/common/configs/configvalidator.go
##########
@@ -72,37 +72,68 @@ func checkACL(acl string) error {
 	return nil
 }
 
-// Temporary convenience method: should use resource package to do this
-// currently no check for the type of resource as long as the value is OK all is OK
-func checkResource(res map[string]string) (int64, error) {
-	var totalres int64
-	for _, val := range res {
-		rescount, err := strconv.ParseInt(val, 10, 64)
-		if err != nil {
-			return 0, fmt.Errorf("resource parsing failed: %v", err)
+// Check the queue resource configuration settings.
+// - child or children cannot have higher maximum or guaranteed limits than parents
+// - children (added together) cannot have a higher guaranteed setting than a parent
+func checkResourceConfigurationsForQueue(cur *QueueConfig, parent *QueueConfig) error {
+	// If cur has children, make sure sum of children's guaranteed <= cur.guaranteed
+	curGuaranteedRes, err := resources.NewResourceFromConf(cur.Resources.Guaranteed)
+	if err != nil {
+		return err
+	}
+	if curGuaranteedRes.HasNegativeValue() {
+		return fmt.Errorf("invalid guaranteed resource for queue %s, cannot be negative", cur.Name)
+	}
+	curMaxRes, err := resources.NewResourceFromConf(cur.Resources.Max)
+	if err != nil {
+		return err
+	}
+	if curMaxRes.HasNegativeValue() {
+		return fmt.Errorf("invalid max resource for queue %s, cannot be negative", cur.Name)

Review comment:
       same as above

##########
File path: pkg/common/configs/configvalidator.go
##########
@@ -72,37 +72,68 @@ func checkACL(acl string) error {
 	return nil
 }
 
-// Temporary convenience method: should use resource package to do this
-// currently no check for the type of resource as long as the value is OK all is OK
-func checkResource(res map[string]string) (int64, error) {
-	var totalres int64
-	for _, val := range res {
-		rescount, err := strconv.ParseInt(val, 10, 64)
-		if err != nil {
-			return 0, fmt.Errorf("resource parsing failed: %v", err)
+// Check the queue resource configuration settings.
+// - child or children cannot have higher maximum or guaranteed limits than parents
+// - children (added together) cannot have a higher guaranteed setting than a parent
+func checkResourceConfigurationsForQueue(cur *QueueConfig, parent *QueueConfig) error {
+	// If cur has children, make sure sum of children's guaranteed <= cur.guaranteed
+	curGuaranteedRes, err := resources.NewResourceFromConf(cur.Resources.Guaranteed)
+	if err != nil {
+		return err
+	}
+	if curGuaranteedRes.HasNegativeValue() {
+		return fmt.Errorf("invalid guaranteed resource for queue %s, cannot be negative", cur.Name)
+	}
+	curMaxRes, err := resources.NewResourceFromConf(cur.Resources.Max)
+	if err != nil {
+		return err
+	}
+	if curMaxRes.HasNegativeValue() {
+		return fmt.Errorf("invalid max resource for queue %s, cannot be negative", cur.Name)
+	}
+
+	if len(cur.Queues) > 0 {
+		// Check children
+		for i := range cur.Queues {
+			if err := checkResourceConfigurationsForQueue(&cur.Queues[i], cur); err != nil {
+				return err
+			}
 		}
-		if rescount < 0 {
-			return 0, fmt.Errorf("resource value invalid, cannot be negative: %d", rescount)
+		sum := resources.NewResource()
+		for _, child := range cur.Queues {
+			childGuaranteed, err := resources.NewResourceFromConf(child.Resources.Guaranteed)
+			if err != nil {
+				return err
+			}
+			sum.AddTo(childGuaranteed)
 		}
-		totalres += rescount
-	}
-	return totalres, nil
-}
 
-// Check the resource configuration
-func checkResources(resource Resources) error {
-	// check guaranteed resources
-	if resource.Guaranteed != nil && len(resource.Guaranteed) != 0 {
-		_, err := checkResource(resource.Guaranteed)
-		if err != nil {
-			return err
+		if cur.Resources.Guaranteed != nil {
+			if !resources.FitIn(curGuaranteedRes, sum) {
+				return fmt.Errorf("queue %s has guaranteed-resources (%v) smaller than sum of children guaranteed resources (%v)", cur.Name, curGuaranteedRes, sum)
+			}
+		} else {
+			cur.Resources.Guaranteed = sum.ToConf()

Review comment:
       this seems to be modifying the value: `cur *QueueConfig`.
   Could you pls check if this is the case? I don't think we should modify the configs.




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

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