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

[GitHub] [incubator-yunikorn-core] wilfred-s opened a new pull request #356: [YUNIKORN-982] placeholder size checks

wilfred-s opened a new pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356


   ### What is this PR for?
   Check placeholder during allocation cycle if placeholder is smaller than the real allocation the placeholder(s) get removed.
   Check placeholder during replacement if placeholder is larger than the real allocation the node and queue usage is adjusted.
   Additional unit test:
   partition: rewrite of placeholder and real alloc tests
   node: new test for delta processing on replace
   application: test fixes only
   
   ### What type of PR is it?
   * [X] - Bug Fix
   
   ### What is the Jira issue?
   https://issues.apache.org/jira/browse/YUNIKORN-982
   
   ### How should this be tested?
   New unit tests added for all use cases
   


-- 
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] [incubator-yunikorn-core] pbacsko commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
pbacsko commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781482493



##########
File path: pkg/scheduler/objects/node_test.go
##########
@@ -488,6 +488,40 @@ func TestRemoveAllocation(t *testing.T) {
 	}
 }
 
+func TestNodeReplaceAllocation(t *testing.T) {
+	node := newNode("node-123", map[string]resources.Quantity{"first": 100, "second": 200})
+	assert.Assert(t, resources.IsZero(node.GetAllocatedResource()), "failed to initialize node")
+
+	// allocate half of the resources available and check the calculation
+	phID := "ph-1"
+	half := resources.NewResourceFromMap(map[string]resources.Quantity{"first": 50, "second": 100})
+	ph := newPlaceholderAlloc(appID1, phID, nodeID1, "queue-1", half)
+	node.AddAllocation(ph)
+	if node.GetAllocation(phID) == nil {
+		t.Fatal("failed to add placeholder allocation")
+	}
+	assert.Assert(t, resources.Equals(node.GetAllocatedResource(), half), "allocated resource not set correctly %v got %v", half, node.GetAllocatedResource())
+
+	allocID := "real-1"
+	piece := resources.NewResourceFromMap(map[string]resources.Quantity{"first": 25, "second": 50})
+	alloc := newAllocation(appID1, allocID, nodeID1, "queue-1", piece)
+	// calculate the delta: new allocation resource - placeholder (should be negative!)
+	delta := resources.Sub(piece, half)
+	assert.Assert(t, delta.HasNegativeValue(), "expected negative values in delta")
+	// swap and check the calculation
+	node.ReplaceAllocation(phID, alloc, delta)
+	if node.GetAllocation(allocID) == nil {
+		t.Fatal("failed to replace allocation: allocation not returned")

Review comment:
       Nit: can't this be an assert as well? 

##########
File path: pkg/scheduler/objects/node_test.go
##########
@@ -488,6 +488,40 @@ func TestRemoveAllocation(t *testing.T) {
 	}
 }
 
+func TestNodeReplaceAllocation(t *testing.T) {
+	node := newNode("node-123", map[string]resources.Quantity{"first": 100, "second": 200})
+	assert.Assert(t, resources.IsZero(node.GetAllocatedResource()), "failed to initialize node")
+
+	// allocate half of the resources available and check the calculation
+	phID := "ph-1"
+	half := resources.NewResourceFromMap(map[string]resources.Quantity{"first": 50, "second": 100})
+	ph := newPlaceholderAlloc(appID1, phID, nodeID1, "queue-1", half)
+	node.AddAllocation(ph)
+	if node.GetAllocation(phID) == nil {
+		t.Fatal("failed to add placeholder allocation")
+	}
+	assert.Assert(t, resources.Equals(node.GetAllocatedResource(), half), "allocated resource not set correctly %v got %v", half, node.GetAllocatedResource())
+
+	allocID := "real-1"
+	piece := resources.NewResourceFromMap(map[string]resources.Quantity{"first": 25, "second": 50})
+	alloc := newAllocation(appID1, allocID, nodeID1, "queue-1", piece)
+	// calculate the delta: new allocation resource - placeholder (should be negative!)
+	delta := resources.Sub(piece, half)
+	assert.Assert(t, delta.HasNegativeValue(), "expected negative values in delta")
+	// swap and check the calculation
+	node.ReplaceAllocation(phID, alloc, delta)
+	if node.GetAllocation(allocID) == nil {
+		t.Fatal("failed to replace allocation: allocation not returned")
+	}
+	assert.Assert(t, resources.Equals(node.GetAllocatedResource(), piece), "allocated resource not set correctly %v got %v", piece, node.GetAllocatedResource())
+
+	// clean up all should be zero
+	if node.RemoveAllocation(allocID) == nil {
+		t.Error("allocation should have been removed but was not")

Review comment:
       Same 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] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781751662



##########
File path: pkg/scheduler/objects/application.go
##########
@@ -827,13 +827,41 @@ func (sa *Application) tryPlaceholderAllocate(nodeIterator func() NodeIterator,
 			if ph.released || request.taskGroupName != ph.taskGroupName {
 				continue
 			}
+			// before we check anything we need to check the resources equality
+			delta := resources.Sub(ph.AllocatedResource, request.AllocatedResource)

Review comment:
       too big a change




-- 
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] [incubator-yunikorn-core] pbacsko commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
pbacsko commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781477109



##########
File path: pkg/scheduler/objects/application.go
##########
@@ -827,13 +827,41 @@ func (sa *Application) tryPlaceholderAllocate(nodeIterator func() NodeIterator,
 			if ph.released || request.taskGroupName != ph.taskGroupName {
 				continue
 			}
+			// before we check anything we need to check the resources equality
+			delta := resources.Sub(ph.AllocatedResource, request.AllocatedResource)
+			// Any negative value in the delta means that at least one of the requested resource in the real
+			// allocation is larger than the placeholder. We need to cancel the placeholder and check the next
+			// placeholder. This should trigger a cleanup of all the placeholder as a task group is always one size.

Review comment:
       I would rephrase it as "task group has a size of one", sounds more correct.




-- 
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] [incubator-yunikorn-core] wilfred-s closed pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
wilfred-s closed pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356


   


-- 
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] [incubator-yunikorn-core] pbacsko commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
pbacsko commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781487258



##########
File path: pkg/scheduler/objects/node.go
##########
@@ -316,15 +316,17 @@ func (sn *Node) AddAllocation(alloc *Allocation) bool {
 	return false
 }
 
-// Replace the paceholder allocation on the node. No usage changes as the placeholder must
-// be the same size as the real allocation.
-func (sn *Node) ReplaceAllocation(uuid string, replace *Allocation) {
+// ReplaceAllocation replaces the placeholder with the real allocation on the node.
+// The delta passed in is the difference in resource usage between placeholder and real allocation.
+// It should always be a negative value (really a decreased usage).

Review comment:
       > // It should always be a negative value (really a decreased usage).
   
   If this is a strong requirement, then it might make sense to at least log a warning if this expectation (`delta.HasNegativeValue() == true`) is not met. This can lead to bugs that are hard to figure out.




-- 
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] [incubator-yunikorn-core] pbacsko commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
pbacsko commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781487258



##########
File path: pkg/scheduler/objects/node.go
##########
@@ -316,15 +316,17 @@ func (sn *Node) AddAllocation(alloc *Allocation) bool {
 	return false
 }
 
-// Replace the paceholder allocation on the node. No usage changes as the placeholder must
-// be the same size as the real allocation.
-func (sn *Node) ReplaceAllocation(uuid string, replace *Allocation) {
+// ReplaceAllocation replaces the placeholder with the real allocation on the node.
+// The delta passed in is the difference in resource usage between placeholder and real allocation.
+// It should always be a negative value (really a decreased usage).

Review comment:
       If this is a strong requirement, then it might make sense to at least log a warning if this expectation (`delta.HasNegativeValue() == true`) is not met. This can lead to bugs that are hard to figure out.




-- 
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] [incubator-yunikorn-core] chenya-zhang commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
chenya-zhang commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781384680



##########
File path: pkg/scheduler/objects/application.go
##########
@@ -827,13 +827,41 @@ func (sa *Application) tryPlaceholderAllocate(nodeIterator func() NodeIterator,
 			if ph.released || request.taskGroupName != ph.taskGroupName {
 				continue
 			}
+			// before we check anything we need to check the resources equality
+			delta := resources.Sub(ph.AllocatedResource, request.AllocatedResource)
+			// Any negative value in the delta means that at least one of the requested resource in the real
+			// allocation is larger than the placeholder. We need to cancel the placeholder and check the next
+			// placeholder. This should trigger a cleanup of all the placeholder as a task group is always one size.
+			if delta.HasNegativeValue() {
+				log.Logger().Warn("releasing placeholder: real allocation is larger than placeholder",
+					zap.String("requested resource", request.AllocatedResource.String()),
+					zap.String("placeholderID", ph.UUID),
+					zap.String("placeholder resource", ph.AllocatedResource.String()))
+				// release the placeholder and tell the RM
+				ph.released = true
+				sa.notifyRMAllocationReleased(sa.rmID, []*Allocation{ph}, si.TerminationType_TIMEOUT, "cancel placeholder: resource incompatible")

Review comment:
       > cancel placeholder: resource incompatible
   
   Will it be easier to understand if we say "requested resource larger than placeholder" directly? "resource incompatible" may also refer to resource type.

##########
File path: pkg/scheduler/objects/application.go
##########
@@ -827,13 +827,41 @@ func (sa *Application) tryPlaceholderAllocate(nodeIterator func() NodeIterator,
 			if ph.released || request.taskGroupName != ph.taskGroupName {
 				continue
 			}
+			// before we check anything we need to check the resources equality
+			delta := resources.Sub(ph.AllocatedResource, request.AllocatedResource)
+			// Any negative value in the delta means that at least one of the requested resource in the real
+			// allocation is larger than the placeholder. We need to cancel the placeholder and check the next
+			// placeholder. This should trigger a cleanup of all the placeholder as a task group is always one size.

Review comment:
       > This should trigger a cleanup of all the placeholder as a task group is always one size.
   
   It sounds to me that all other placeholders will be removed (= cleaned up)?




-- 
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] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781727509



##########
File path: pkg/scheduler/objects/application.go
##########
@@ -827,13 +827,41 @@ func (sa *Application) tryPlaceholderAllocate(nodeIterator func() NodeIterator,
 			if ph.released || request.taskGroupName != ph.taskGroupName {
 				continue
 			}
+			// before we check anything we need to check the resources equality
+			delta := resources.Sub(ph.AllocatedResource, request.AllocatedResource)
+			// Any negative value in the delta means that at least one of the requested resource in the real
+			// allocation is larger than the placeholder. We need to cancel the placeholder and check the next
+			// placeholder. This should trigger a cleanup of all the placeholder as a task group is always one size.
+			if delta.HasNegativeValue() {
+				log.Logger().Warn("releasing placeholder: real allocation is larger than placeholder",
+					zap.String("requested resource", request.AllocatedResource.String()),
+					zap.String("placeholderID", ph.UUID),
+					zap.String("placeholder resource", ph.AllocatedResource.String()))
+				// release the placeholder and tell the RM
+				ph.released = true
+				sa.notifyRMAllocationReleased(sa.rmID, []*Allocation{ph}, si.TerminationType_TIMEOUT, "cancel placeholder: resource incompatible")

Review comment:
       Correct because the resource could be incompatible. The real allocation has a reference to resource type GPU with a value of 1, but the placeholder does not which results in GPU becoming negative in the `delta`. The same real allocation could also ask for memory 2GB while the placeholder specifies 4GB. The "incompatible resource" covers these mixed cases better than just the larger than message.
   The event generated,  which is what the end user sees, has all the detail 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] [incubator-yunikorn-core] codecov[bot] edited a comment on pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#issuecomment-1008565409


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?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 [#356](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1a62cf4) into [master](https://codecov.io/gh/apache/incubator-yunikorn-core/commit/5a1c19e280f66d6cb4fc206a675116260d6f2ff1?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5a1c19e) will **increase** coverage by `4.10%`.
   > The diff coverage is `68.06%`.
   
   > :exclamation: Current head 1a62cf4 differs from pull request most recent head 6ad1a7d. Consider uploading reports for the commit 6ad1a7d to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/graphs/tree.svg?width=650&height=150&src=pr&token=SB9NrIi3Hy&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #356      +/-   ##
   ==========================================
   + Coverage   63.46%   67.57%   +4.10%     
   ==========================================
     Files          60       64       +4     
     Lines        5220     8987    +3767     
   ==========================================
   + Hits         3313     6073    +2760     
   - Misses       1747     2679     +932     
   - Partials      160      235      +75     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?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/metrics/init.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL21ldHJpY3MvaW5pdC5nbw==) | `63.15% <0.00%> (+0.19%)` | :arrow_up: |
   | [pkg/metrics/queue.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL21ldHJpY3MvcXVldWUuZ28=) | `0.00% <0.00%> (ø)` | |
   | [pkg/scheduler/drf\_preemption\_policy.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9kcmZfcHJlZW1wdGlvbl9wb2xpY3kuZ28=) | `0.00% <0.00%> (ø)` | |
   | [pkg/scheduler/nodes\_usage\_monitor.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9ub2Rlc191c2FnZV9tb25pdG9yLmdv) | `0.00% <0.00%> (ø)` | |
   | [pkg/scheduler/objects/sorters.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL3NvcnRlcnMuZ28=) | `100.00% <ø> (ø)` | |
   | [pkg/scheduler/scheduler.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9zY2hlZHVsZXIuZ28=) | `0.00% <0.00%> (ø)` | |
   | [pkg/webservice/webservice.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3dlYnNlcnZpY2Uvd2Vic2VydmljZS5nbw==) | `28.57% <ø> (+15.66%)` | :arrow_up: |
   | [pkg/scheduler/context.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9jb250ZXh0Lmdv) | `17.52% <30.05%> (+11.25%)` | :arrow_up: |
   | [pkg/scheduler/objects/allocation.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FsbG9jYXRpb24uZ28=) | `97.22% <33.33%> (-2.78%)` | :arrow_down: |
   | [pkg/metrics/metrics\_collector.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL21ldHJpY3MvbWV0cmljc19jb2xsZWN0b3IuZ28=) | `65.90% <46.42%> (-8.01%)` | :arrow_down: |
   | ... and [88 more](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [c8ac87f...6ad1a7d](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-core] codecov[bot] commented on pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
codecov[bot] commented on pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#issuecomment-1008565409


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?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 [#356](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1a62cf4) into [master](https://codecov.io/gh/apache/incubator-yunikorn-core/commit/5a1c19e280f66d6cb4fc206a675116260d6f2ff1?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5a1c19e) will **increase** coverage by `4.10%`.
   > The diff coverage is `68.06%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/graphs/tree.svg?width=650&height=150&src=pr&token=SB9NrIi3Hy&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #356      +/-   ##
   ==========================================
   + Coverage   63.46%   67.57%   +4.10%     
   ==========================================
     Files          60       64       +4     
     Lines        5220     8987    +3767     
   ==========================================
   + Hits         3313     6073    +2760     
   - Misses       1747     2679     +932     
   - Partials      160      235      +75     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?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/metrics/init.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL21ldHJpY3MvaW5pdC5nbw==) | `63.15% <0.00%> (+0.19%)` | :arrow_up: |
   | [pkg/metrics/queue.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL21ldHJpY3MvcXVldWUuZ28=) | `0.00% <0.00%> (ø)` | |
   | [pkg/scheduler/drf\_preemption\_policy.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9kcmZfcHJlZW1wdGlvbl9wb2xpY3kuZ28=) | `0.00% <0.00%> (ø)` | |
   | [pkg/scheduler/nodes\_usage\_monitor.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9ub2Rlc191c2FnZV9tb25pdG9yLmdv) | `0.00% <0.00%> (ø)` | |
   | [pkg/scheduler/objects/sorters.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL3NvcnRlcnMuZ28=) | `100.00% <ø> (ø)` | |
   | [pkg/scheduler/scheduler.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9zY2hlZHVsZXIuZ28=) | `0.00% <0.00%> (ø)` | |
   | [pkg/webservice/webservice.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3dlYnNlcnZpY2Uvd2Vic2VydmljZS5nbw==) | `28.57% <ø> (+15.66%)` | :arrow_up: |
   | [pkg/scheduler/context.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9jb250ZXh0Lmdv) | `17.52% <30.05%> (+11.25%)` | :arrow_up: |
   | [pkg/scheduler/objects/allocation.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FsbG9jYXRpb24uZ28=) | `97.22% <33.33%> (-2.78%)` | :arrow_down: |
   | [pkg/metrics/metrics\_collector.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL21ldHJpY3MvbWV0cmljc19jb2xsZWN0b3IuZ28=) | `65.90% <46.42%> (-8.01%)` | :arrow_down: |
   | ... and [88 more](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [c8ac87f...1a62cf4](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-core] chenya-zhang commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
chenya-zhang commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781384680



##########
File path: pkg/scheduler/objects/application.go
##########
@@ -827,13 +827,41 @@ func (sa *Application) tryPlaceholderAllocate(nodeIterator func() NodeIterator,
 			if ph.released || request.taskGroupName != ph.taskGroupName {
 				continue
 			}
+			// before we check anything we need to check the resources equality
+			delta := resources.Sub(ph.AllocatedResource, request.AllocatedResource)
+			// Any negative value in the delta means that at least one of the requested resource in the real
+			// allocation is larger than the placeholder. We need to cancel the placeholder and check the next
+			// placeholder. This should trigger a cleanup of all the placeholder as a task group is always one size.
+			if delta.HasNegativeValue() {
+				log.Logger().Warn("releasing placeholder: real allocation is larger than placeholder",
+					zap.String("requested resource", request.AllocatedResource.String()),
+					zap.String("placeholderID", ph.UUID),
+					zap.String("placeholder resource", ph.AllocatedResource.String()))
+				// release the placeholder and tell the RM
+				ph.released = true
+				sa.notifyRMAllocationReleased(sa.rmID, []*Allocation{ph}, si.TerminationType_TIMEOUT, "cancel placeholder: resource incompatible")

Review comment:
       > cancel placeholder: resource incompatible
   
   Will it be easier to understand if we say "requested resource larger than placeholder" directly? "resource incompatible" may also refer to resource type, unit, etc.




-- 
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] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781725480



##########
File path: pkg/scheduler/objects/application.go
##########
@@ -827,13 +827,41 @@ func (sa *Application) tryPlaceholderAllocate(nodeIterator func() NodeIterator,
 			if ph.released || request.taskGroupName != ph.taskGroupName {
 				continue
 			}
+			// before we check anything we need to check the resources equality
+			delta := resources.Sub(ph.AllocatedResource, request.AllocatedResource)
+			// Any negative value in the delta means that at least one of the requested resource in the real
+			// allocation is larger than the placeholder. We need to cancel the placeholder and check the next
+			// placeholder. This should trigger a cleanup of all the placeholder as a task group is always one size.
+			if delta.HasNegativeValue() {
+				log.Logger().Warn("releasing placeholder: real allocation is larger than placeholder",
+					zap.String("requested resource", request.AllocatedResource.String()),
+					zap.String("placeholderID", ph.UUID),
+					zap.String("placeholder resource", ph.AllocatedResource.String()))
+				// release the placeholder and tell the RM
+				ph.released = true
+				sa.notifyRMAllocationReleased(sa.rmID, []*Allocation{ph}, si.TerminationType_TIMEOUT, "cancel placeholder: resource incompatible")

Review comment:
       I did not introduce a new state as the processing for both would be exactly the same in both the shim and the core. It would also require a new SI and a new shim to make the same changes work.




-- 
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] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781737383



##########
File path: pkg/scheduler/objects/node_test.go
##########
@@ -488,6 +488,40 @@ func TestRemoveAllocation(t *testing.T) {
 	}
 }
 
+func TestNodeReplaceAllocation(t *testing.T) {
+	node := newNode("node-123", map[string]resources.Quantity{"first": 100, "second": 200})
+	assert.Assert(t, resources.IsZero(node.GetAllocatedResource()), "failed to initialize node")
+
+	// allocate half of the resources available and check the calculation
+	phID := "ph-1"
+	half := resources.NewResourceFromMap(map[string]resources.Quantity{"first": 50, "second": 100})
+	ph := newPlaceholderAlloc(appID1, phID, nodeID1, "queue-1", half)
+	node.AddAllocation(ph)
+	if node.GetAllocation(phID) == nil {
+		t.Fatal("failed to add placeholder allocation")
+	}
+	assert.Assert(t, resources.Equals(node.GetAllocatedResource(), half), "allocated resource not set correctly %v got %v", half, node.GetAllocatedResource())
+
+	allocID := "real-1"
+	piece := resources.NewResourceFromMap(map[string]resources.Quantity{"first": 25, "second": 50})
+	alloc := newAllocation(appID1, allocID, nodeID1, "queue-1", piece)
+	// calculate the delta: new allocation resource - placeholder (should be negative!)
+	delta := resources.Sub(piece, half)
+	assert.Assert(t, delta.HasNegativeValue(), "expected negative values in delta")
+	// swap and check the calculation
+	node.ReplaceAllocation(phID, alloc, delta)
+	if node.GetAllocation(allocID) == nil {
+		t.Fatal("failed to replace allocation: allocation not returned")

Review comment:
       We should file a follow up for that, the node testing code has a large number of places where it should  be using asserts.
   
   Nil object checks do look a bit weird in the assert, might not improve readability:
   ```
   assert.Assert(t, node.GetAllocation(allocID) != nil, "failed to replace allocation: allocation not returned")
   ```




-- 
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] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781738962



##########
File path: pkg/scheduler/partition.go
##########
@@ -704,6 +704,28 @@ func (pc *PartitionContext) removeNodeAllocations(node *objects.Node) ([]*object
 				if alloc.NodeID != release.NodeID {
 					// ignore the return as that is the same as alloc, the alloc is gone after this call
 					_ = app.ReplaceAllocation(allocID)
+					// we need to check the resources equality
+					delta := resources.Sub(release.AllocatedResource, alloc.AllocatedResource)
+					// Any negative value in the delta means that at least one of the requested resource in the
+					// placeholder is larger than the real allocation. The nodes are correct the queue needs adjusting.
+					// The reverse case is handled during allocation.
+					if delta.HasNegativeValue() {
+						// this looks incorrect but the delta is negative and the result will be a real decrease
+						err := app.GetQueue().IncAllocatedResource(delta, false)

Review comment:
       The update of the queue is enclosed in a check for nodeID difference. The last line of that block is a continue call. when we get into the block we can never reach line 770 which does the queue update for the other cases.




-- 
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] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781247167



##########
File path: pkg/scheduler/partition.go
##########
@@ -704,6 +704,28 @@ func (pc *PartitionContext) removeNodeAllocations(node *objects.Node) ([]*object
 				if alloc.NodeID != release.NodeID {
 					// ignore the return as that is the same as alloc, the alloc is gone after this call
 					_ = app.ReplaceAllocation(allocID)
+					// we need to check the resources equality
+					delta := resources.Sub(release.AllocatedResource, alloc.AllocatedResource)
+					// Any negative value in the delta means that at least one of the requested resource in the
+					// placeholder is larger than the real allocation. The nodes are correct the queue needs adjusting.
+					// The reverse case is handled during allocation.
+					if delta.HasNegativeValue() {
+						// this looks incorrect but the delta is negative and the result will be a real decrease
+						err := app.GetQueue().IncAllocatedResource(delta, false)

Review comment:
       Would this new queue adjustment cause any issues in conjunction with below adjustment in place? app.GetQueue().DecAllocatedResource(alloc.AllocatedResource)
   
   Can we double check? May be unit test to removeNode with replacement on different node ensures this?
   
   Other than this, overall changes looks good.




-- 
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] [incubator-yunikorn-core] chenya-zhang commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
chenya-zhang commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781379822



##########
File path: pkg/scheduler/objects/application.go
##########
@@ -827,13 +827,41 @@ func (sa *Application) tryPlaceholderAllocate(nodeIterator func() NodeIterator,
 			if ph.released || request.taskGroupName != ph.taskGroupName {
 				continue
 			}
+			// before we check anything we need to check the resources equality
+			delta := resources.Sub(ph.AllocatedResource, request.AllocatedResource)

Review comment:
       Maybe we could replace "ph" (placeholder) with "hold" (v.s. "request") to make our codebase easier to understand. :) But if it appears in too many places, it's okay.




-- 
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] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781734588



##########
File path: pkg/scheduler/objects/node.go
##########
@@ -316,15 +316,17 @@ func (sn *Node) AddAllocation(alloc *Allocation) bool {
 	return false
 }
 
-// Replace the paceholder allocation on the node. No usage changes as the placeholder must
-// be the same size as the real allocation.
-func (sn *Node) ReplaceAllocation(uuid string, replace *Allocation) {
+// ReplaceAllocation replaces the placeholder with the real allocation on the node.
+// The delta passed in is the difference in resource usage between placeholder and real allocation.
+// It should always be a negative value (really a decreased usage).

Review comment:
       added log entry




-- 
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] [incubator-yunikorn-core] codecov[bot] edited a comment on pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#issuecomment-1008565409


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?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 [#356](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6ad1a7d) into [master](https://codecov.io/gh/apache/incubator-yunikorn-core/commit/5a1c19e280f66d6cb4fc206a675116260d6f2ff1?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5a1c19e) will **increase** coverage by `4.05%`.
   > The diff coverage is `67.87%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/graphs/tree.svg?width=650&height=150&src=pr&token=SB9NrIi3Hy&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #356      +/-   ##
   ==========================================
   + Coverage   63.46%   67.52%   +4.05%     
   ==========================================
     Files          60       64       +4     
     Lines        5220     8995    +3775     
   ==========================================
   + Hits         3313     6074    +2761     
   - Misses       1747     2685     +938     
   - Partials      160      236      +76     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?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/metrics/init.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL21ldHJpY3MvaW5pdC5nbw==) | `63.15% <0.00%> (+0.19%)` | :arrow_up: |
   | [pkg/metrics/queue.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL21ldHJpY3MvcXVldWUuZ28=) | `0.00% <0.00%> (ø)` | |
   | [pkg/scheduler/drf\_preemption\_policy.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9kcmZfcHJlZW1wdGlvbl9wb2xpY3kuZ28=) | `0.00% <0.00%> (ø)` | |
   | [pkg/scheduler/nodes\_usage\_monitor.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9ub2Rlc191c2FnZV9tb25pdG9yLmdv) | `0.00% <0.00%> (ø)` | |
   | [pkg/scheduler/objects/sorters.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL3NvcnRlcnMuZ28=) | `100.00% <ø> (ø)` | |
   | [pkg/scheduler/scheduler.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9zY2hlZHVsZXIuZ28=) | `0.00% <0.00%> (ø)` | |
   | [pkg/webservice/webservice.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3dlYnNlcnZpY2Uvd2Vic2VydmljZS5nbw==) | `28.57% <ø> (+15.66%)` | :arrow_up: |
   | [pkg/scheduler/context.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9jb250ZXh0Lmdv) | `17.52% <30.05%> (+11.25%)` | :arrow_up: |
   | [pkg/scheduler/objects/allocation.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FsbG9jYXRpb24uZ28=) | `97.22% <33.33%> (-2.78%)` | :arrow_down: |
   | [pkg/metrics/metrics\_collector.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL21ldHJpY3MvbWV0cmljc19jb2xsZWN0b3IuZ28=) | `65.90% <46.42%> (-8.01%)` | :arrow_down: |
   | ... and [88 more](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [c8ac87f...6ad1a7d](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/356?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] [incubator-yunikorn-core] pbacsko commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
pbacsko commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781479241



##########
File path: pkg/scheduler/objects/application.go
##########
@@ -827,13 +827,41 @@ func (sa *Application) tryPlaceholderAllocate(nodeIterator func() NodeIterator,
 			if ph.released || request.taskGroupName != ph.taskGroupName {
 				continue
 			}
+			// before we check anything we need to check the resources equality
+			delta := resources.Sub(ph.AllocatedResource, request.AllocatedResource)
+			// Any negative value in the delta means that at least one of the requested resource in the real
+			// allocation is larger than the placeholder. We need to cancel the placeholder and check the next
+			// placeholder. This should trigger a cleanup of all the placeholder as a task group is always one size.
+			if delta.HasNegativeValue() {
+				log.Logger().Warn("releasing placeholder: real allocation is larger than placeholder",
+					zap.String("requested resource", request.AllocatedResource.String()),
+					zap.String("placeholderID", ph.UUID),
+					zap.String("placeholder resource", ph.AllocatedResource.String()))
+				// release the placeholder and tell the RM
+				ph.released = true
+				sa.notifyRMAllocationReleased(sa.rmID, []*Allocation{ph}, si.TerminationType_TIMEOUT, "cancel placeholder: resource incompatible")

Review comment:
       `si.TerminationType_TIMEOUT` - can this be changed to something else or would we break API with that? Something like `si.TerminationType_RESOURCE_MISMATCH`.




-- 
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] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781738962



##########
File path: pkg/scheduler/partition.go
##########
@@ -704,6 +704,28 @@ func (pc *PartitionContext) removeNodeAllocations(node *objects.Node) ([]*object
 				if alloc.NodeID != release.NodeID {
 					// ignore the return as that is the same as alloc, the alloc is gone after this call
 					_ = app.ReplaceAllocation(allocID)
+					// we need to check the resources equality
+					delta := resources.Sub(release.AllocatedResource, alloc.AllocatedResource)
+					// Any negative value in the delta means that at least one of the requested resource in the
+					// placeholder is larger than the real allocation. The nodes are correct the queue needs adjusting.
+					// The reverse case is handled during allocation.
+					if delta.HasNegativeValue() {
+						// this looks incorrect but the delta is negative and the result will be a real decrease
+						err := app.GetQueue().IncAllocatedResource(delta, false)

Review comment:
       The update of the queue is enclosed in a check for nodeID difference (line 704 - 740). The last line of that block is a continue call (line 739) which starts processing the next allocation (for loop). If and when we get into the block we can never reach line 770 which does the queue update for the other cases.




-- 
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] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #356: [YUNIKORN-982] placeholder size checks

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #356:
URL: https://github.com/apache/incubator-yunikorn-core/pull/356#discussion_r781720362



##########
File path: pkg/scheduler/objects/application.go
##########
@@ -827,13 +827,41 @@ func (sa *Application) tryPlaceholderAllocate(nodeIterator func() NodeIterator,
 			if ph.released || request.taskGroupName != ph.taskGroupName {
 				continue
 			}
+			// before we check anything we need to check the resources equality
+			delta := resources.Sub(ph.AllocatedResource, request.AllocatedResource)
+			// Any negative value in the delta means that at least one of the requested resource in the real
+			// allocation is larger than the placeholder. We need to cancel the placeholder and check the next
+			// placeholder. This should trigger a cleanup of all the placeholder as a task group is always one size.

Review comment:
       Updated comment to be clearer




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