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/07/22 17:56:44 UTC

[GitHub] [yunikorn-core] craigcondit opened a new pull request, #424: [YUNIKORN-1264] Refactor asks and allocations to use consistent locking and member access

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

   ### What is this PR for?
   Refactoring of Allocation and AllocationAsk.
   
   ### What type of PR is it?
   * [ ] - Bug Fix
   * [ ] - Improvement
   * [ ] - Feature
   * [ ] - Documentation
   * [ ] - Hot Fix
   * [x] - Refactoring
   
   ### Todos
   * [ ] - Task
   
   ### What is the Jira issue?
   https://issues.apache.org/jira/browse/YUNIKORN-1264
   
   ### How should this be tested?
   No functional changes, so existing unit tests will work.
   
   ### 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] craigcondit commented on a diff in pull request #424: [YUNIKORN-1264] Refactor asks and allocations to use consistent locking and member access

Posted by GitBox <gi...@apache.org>.
craigcondit commented on code in PR #424:
URL: https://github.com/apache/yunikorn-core/pull/424#discussion_r929053145


##########
pkg/scheduler/objects/node.go:
##########
@@ -100,7 +100,7 @@ func (sn *Node) String() string {
 	if sn == nil {
 		return "node is nil"
 	}
-	return fmt.Sprintf("NodeID %s, Partition %s, Schedulable %t, Total %s, Allocated %s, #allocations %d",
+	return fmt.Sprintf("nodeID %s, Partition %s, Schedulable %t, Total %s, Allocated %s, #allocations %d",

Review Comment:
   Will revert this part during commit.



##########
pkg/scheduler/objects/node_collection_test.go:
##########
@@ -197,7 +197,7 @@ func TestSetNodeSortingPolicy(t *testing.T) {
 			iter := nc.GetNodeIterator()
 			for id := 0; id < len(tt.nodesOrder); id++ {
 				if n := iter.Next(); n.NodeID != tt.nodesOrder[id] {
-					t.Errorf("%s: NodeID wanted %s, but it got %s.", nc.GetNodeSortingPolicy().PolicyType().String(), tt.nodesOrder[id], n.NodeID)
+					t.Errorf("%s: nodeID wanted %s, but it got %s.", nc.GetNodeSortingPolicy().PolicyType().String(), tt.nodesOrder[id], n.NodeID)

Review Comment:
   Will revert this part during commit.



-- 
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] craigcondit closed pull request #424: [YUNIKORN-1264] Refactor asks and allocations to use consistent locking and member access

Posted by GitBox <gi...@apache.org>.
craigcondit closed pull request #424: [YUNIKORN-1264] Refactor asks and allocations to use consistent locking and member access
URL: https://github.com/apache/yunikorn-core/pull/424


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@yunikorn.apache.org

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


[GitHub] [yunikorn-core] manirajv06 commented on a diff in pull request #424: [YUNIKORN-1264] Refactor asks and allocations to use consistent locking and member access

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on code in PR #424:
URL: https://github.com/apache/yunikorn-core/pull/424#discussion_r928987211


##########
pkg/scheduler/objects/node.go:
##########
@@ -100,7 +100,7 @@ func (sn *Node) String() string {
 	if sn == nil {
 		return "node is nil"
 	}
-	return fmt.Sprintf("NodeID %s, Partition %s, Schedulable %t, Total %s, Allocated %s, #allocations %d",
+	return fmt.Sprintf("nodeID %s, Partition %s, Schedulable %t, Total %s, Allocated %s, #allocations %d",

Review Comment:
   Required?



##########
pkg/scheduler/objects/allocation_ask.go:
##########
@@ -62,15 +62,24 @@ type AllocationLogEntry struct {
 	Count          int32
 }
 
-func NewAllocationAsk(ask *si.AllocationAsk) *AllocationAsk {
+func NewAllocationAsk(allocationKey string, applicationID string, allocatedResource *resources.Resource) *AllocationAsk {
+	return &AllocationAsk{
+		allocationKey:     allocationKey,
+		applicationID:     applicationID,
+		allocatedResource: allocatedResource,
+		allocLog:          make(map[string]*AllocationLogEntry),
+	}
+}
+
+func NewAllocationAskFromSI(ask *si.AllocationAsk) *AllocationAsk {
 	saa := &AllocationAsk{
-		AllocationKey:     ask.AllocationKey,
-		AllocatedResource: resources.NewResourceFromProto(ask.ResourceAsk),
-		pendingRepeatAsk:  ask.MaxAllocations,
+		allocationKey:     ask.AllocationKey,
+		allocatedResource: resources.NewResourceFromProto(ask.ResourceAsk),
+		pendingAskRepeat:  ask.MaxAllocations,
 		maxAllocations:    ask.MaxAllocations,
-		ApplicationID:     ask.ApplicationID,
-		PartitionName:     ask.PartitionName,
-		Tags:              ask.Tags,
+		applicationID:     ask.ApplicationID,
+		partitionName:     ask.PartitionName,
+		tags:              CloneAllocationTags(ask.Tags),

Review Comment:
   Do we really need to call CloneAllocationTags? Can we tags directly like other variables?



##########
pkg/scheduler/objects/node_collection_test.go:
##########
@@ -197,7 +197,7 @@ func TestSetNodeSortingPolicy(t *testing.T) {
 			iter := nc.GetNodeIterator()
 			for id := 0; id < len(tt.nodesOrder); id++ {
 				if n := iter.Next(); n.NodeID != tt.nodesOrder[id] {
-					t.Errorf("%s: NodeID wanted %s, but it got %s.", nc.GetNodeSortingPolicy().PolicyType().String(), tt.nodesOrder[id], n.NodeID)
+					t.Errorf("%s: nodeID wanted %s, but it got %s.", nc.GetNodeSortingPolicy().PolicyType().String(), tt.nodesOrder[id], n.NodeID)

Review Comment:
   Required?



##########
pkg/scheduler/objects/allocation.go:
##########
@@ -153,66 +154,220 @@ func (a *Allocation) NewSIFromAllocation() *si.Allocation {
 		return nil
 	}
 	return &si.Allocation{
-		NodeID:           a.NodeID,
-		ApplicationID:    a.ApplicationID,
-		AllocationKey:    a.AllocationKey,
-		UUID:             a.UUID,
-		ResourcePerAlloc: a.AllocatedResource.ToProto(), // needed in tests for restore
-		TaskGroupName:    a.taskGroupName,
-		Placeholder:      a.placeholder,
+		NodeID:           a.GetNodeID(),
+		ApplicationID:    a.GetApplicationID(),
+		AllocationKey:    a.GetAllocationKey(),
+		UUID:             a.GetUUID(),
+		ResourcePerAlloc: a.GetAllocatedResource().ToProto(), // needed in tests for restore
+		TaskGroupName:    a.GetTaskGroup(),
+		Placeholder:      a.IsPlaceholder(),
 	}
 }
 
 func (a *Allocation) String() string {
 	if a == nil {
 		return "nil allocation"
 	}
-	uuid := a.UUID
-	if a.Result == Reserved || a.Result == Unreserved {
+	a.RLock()
+	defer a.RUnlock()
+	uuid := a.uuid
+	if a.result == Reserved || a.result == Unreserved {

Review Comment:
   Can we getter itself to fetch result and uuid similar to other places?



-- 
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 #424: [YUNIKORN-1264] Refactor asks and allocations to use consistent locking and member access

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

   # [Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/424?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 [#424](https://codecov.io/gh/apache/yunikorn-core/pull/424?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (bd87847) into [master](https://codecov.io/gh/apache/yunikorn-core/commit/6d1ccc6957c4578aa60385337960a1d11b2155b1?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6d1ccc6) will **decrease** coverage by `0.00%`.
   > The diff coverage is `68.39%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #424      +/-   ##
   ==========================================
   - Coverage   69.52%   69.52%   -0.01%     
   ==========================================
     Files          68       68              
     Lines        9931    10014      +83     
   ==========================================
   + Hits         6905     6962      +57     
   - Misses       2783     2809      +26     
     Partials      243      243              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/yunikorn-core/pull/424?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/context.go](https://codecov.io/gh/apache/yunikorn-core/pull/424/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) | `31.01% <0.00%> (ø)` | |
   | [pkg/scheduler/drf\_preemption\_policy.go](https://codecov.io/gh/apache/yunikorn-core/pull/424/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/scheduler.go](https://codecov.io/gh/apache/yunikorn-core/pull/424/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/scheduler/objects/application.go](https://codecov.io/gh/apache/yunikorn-core/pull/424/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-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uLmdv) | `56.58% <51.30%> (-0.05%)` | :arrow_down: |
   | [pkg/webservice/handlers.go](https://codecov.io/gh/apache/yunikorn-core/pull/424/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-cGtnL3dlYnNlcnZpY2UvaGFuZGxlcnMuZ28=) | `69.38% <56.66%> (ø)` | |
   | [pkg/scheduler/partition.go](https://codecov.io/gh/apache/yunikorn-core/pull/424/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-cGtnL3NjaGVkdWxlci9wYXJ0aXRpb24uZ28=) | `74.78% <77.77%> (ø)` | |
   | [pkg/scheduler/objects/allocation.go](https://codecov.io/gh/apache/yunikorn-core/pull/424/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=) | `88.23% <82.25%> (-9.81%)` | :arrow_down: |
   | [pkg/scheduler/objects/allocation\_ask.go](https://codecov.io/gh/apache/yunikorn-core/pull/424/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-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FsbG9jYXRpb25fYXNrLmdv) | `92.85% <84.44%> (-4.05%)` | :arrow_down: |
   | [pkg/scheduler/objects/node.go](https://codecov.io/gh/apache/yunikorn-core/pull/424/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-cGtnL3NjaGVkdWxlci9vYmplY3RzL25vZGUuZ28=) | `82.68% <87.50%> (ø)` | |
   | [pkg/scheduler/health\_checker.go](https://codecov.io/gh/apache/yunikorn-core/pull/424/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-cGtnL3NjaGVkdWxlci9oZWFsdGhfY2hlY2tlci5nbw==) | `85.20% <100.00%> (ø)` | |
   | ... and [4 more](https://codecov.io/gh/apache/yunikorn-core/pull/424/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) | |
   
   Help us with your feedback. Take ten seconds to tell us [how you rate us](https://about.codecov.io/nps?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] craigcondit commented on a diff in pull request #424: [YUNIKORN-1264] Refactor asks and allocations to use consistent locking and member access

Posted by GitBox <gi...@apache.org>.
craigcondit commented on code in PR #424:
URL: https://github.com/apache/yunikorn-core/pull/424#discussion_r929051534


##########
pkg/scheduler/objects/allocation.go:
##########
@@ -153,66 +154,220 @@ func (a *Allocation) NewSIFromAllocation() *si.Allocation {
 		return nil
 	}
 	return &si.Allocation{
-		NodeID:           a.NodeID,
-		ApplicationID:    a.ApplicationID,
-		AllocationKey:    a.AllocationKey,
-		UUID:             a.UUID,
-		ResourcePerAlloc: a.AllocatedResource.ToProto(), // needed in tests for restore
-		TaskGroupName:    a.taskGroupName,
-		Placeholder:      a.placeholder,
+		NodeID:           a.GetNodeID(),
+		ApplicationID:    a.GetApplicationID(),
+		AllocationKey:    a.GetAllocationKey(),
+		UUID:             a.GetUUID(),
+		ResourcePerAlloc: a.GetAllocatedResource().ToProto(), // needed in tests for restore
+		TaskGroupName:    a.GetTaskGroup(),
+		Placeholder:      a.IsPlaceholder(),
 	}
 }
 
 func (a *Allocation) String() string {
 	if a == nil {
 		return "nil allocation"
 	}
-	uuid := a.UUID
-	if a.Result == Reserved || a.Result == Unreserved {
+	a.RLock()
+	defer a.RUnlock()
+	uuid := a.uuid
+	if a.result == Reserved || a.result == Unreserved {

Review Comment:
   This is referring to a member object within itself, therefore no getter.



-- 
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] craigcondit commented on a diff in pull request #424: [YUNIKORN-1264] Refactor asks and allocations to use consistent locking and member access

Posted by GitBox <gi...@apache.org>.
craigcondit commented on code in PR #424:
URL: https://github.com/apache/yunikorn-core/pull/424#discussion_r929051834


##########
pkg/scheduler/objects/allocation_ask.go:
##########
@@ -62,15 +62,24 @@ type AllocationLogEntry struct {
 	Count          int32
 }
 
-func NewAllocationAsk(ask *si.AllocationAsk) *AllocationAsk {
+func NewAllocationAsk(allocationKey string, applicationID string, allocatedResource *resources.Resource) *AllocationAsk {
+	return &AllocationAsk{
+		allocationKey:     allocationKey,
+		applicationID:     applicationID,
+		allocatedResource: allocatedResource,
+		allocLog:          make(map[string]*AllocationLogEntry),
+	}
+}
+
+func NewAllocationAskFromSI(ask *si.AllocationAsk) *AllocationAsk {
 	saa := &AllocationAsk{
-		AllocationKey:     ask.AllocationKey,
-		AllocatedResource: resources.NewResourceFromProto(ask.ResourceAsk),
-		pendingRepeatAsk:  ask.MaxAllocations,
+		allocationKey:     ask.AllocationKey,
+		allocatedResource: resources.NewResourceFromProto(ask.ResourceAsk),
+		pendingAskRepeat:  ask.MaxAllocations,
 		maxAllocations:    ask.MaxAllocations,
-		ApplicationID:     ask.ApplicationID,
-		PartitionName:     ask.PartitionName,
-		Tags:              ask.Tags,
+		applicationID:     ask.ApplicationID,
+		partitionName:     ask.PartitionName,
+		tags:              CloneAllocationTags(ask.Tags),

Review Comment:
   Yes, this is because a caller can mutate these, and that should not be allowed.



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