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 2021/01/04 03:13:25 UTC

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #229: YUNIKORN-466: Node capacity update does not update root queue or partition

wilfred-s commented on a change in pull request #229:
URL: https://github.com/apache/incubator-yunikorn-core/pull/229#discussion_r551097097



##########
File path: pkg/scheduler/partition_test.go
##########
@@ -1079,3 +1079,55 @@ func TestUpdateRootQueue(t *testing.T) {
 	assert.Equal(t, partition.GetQueue("root.leaf").CurrentState(), objects.Draining.String(), "leaf queue should have been marked for removal")
 	assert.Equal(t, partition.GetQueue("root.parent").CurrentState(), objects.Draining.String(), "parent queue should have been marked for removal")
 }
+
+func TestUpdateNode(t *testing.T) {
+	partition, err := newBasePartition()
+	assert.NilError(t, err, "test partition create failed with error")
+
+	newRes, err := resources.NewResourceFromConf(map[string]string{"memory": "400", "vcore": "30"})
+	assert.NilError(t, err, "failed to create resource")
+
+	err = partition.AddNode(newNodeMaxResource("test", newRes), nil)
+	assert.NilError(t, err, "test node add failed unexpected")
+	assert.Equal(t, 1, len(partition.nodes), "node list not correct")
+
+	if !resources.Equals(newRes, partition.GetTotalPartitionResource()) {
+		t.Errorf("Expected partition resource %s, doesn't match with actual partition resource %s", newRes, partition.GetTotalPartitionResource())
+	}
+
+	//delta resource for a node with mem as 450 and vcores as 40 (both mem and vcores has increeased)

Review comment:
       This causes the lint check to fail: comments must have a space between the `//` and the text.

##########
File path: pkg/scheduler/tests/operation_test.go
##########
@@ -371,13 +371,162 @@ partitions:
 	})
 
 	assert.NilError(t, err, "UpdateRequest failed")
-
 	waitForAvailableNodeResource(t, ms.scheduler.GetClusterContext(), "[rm:123]default",
 		[]string{"node-1:1234"}, 300, 1000)
 	assert.Equal(t, int64(node1.GetCapacity().Resources[resources.MEMORY]), int64(300))
 	assert.Equal(t, int64(node1.GetCapacity().Resources[resources.VCORE]), int64(10))
 	assert.Equal(t, int64(schedulingNode1.GetAllocatedResource().Resources[resources.MEMORY]), int64(0))
 	assert.Equal(t, int64(schedulingNode1.GetAvailableResource().Resources[resources.MEMORY]), int64(300))
+	newRes, err := resources.NewResourceFromConf(map[string]string{"memory": "300", "vcore": "10"})
+	assert.NilError(t, err, "failed to create resource")
+	if !resources.Equals(newRes, partitionInfo.GetTotalPartitionResource()) {
+		t.Errorf("Expected partition resource %s, doesn't match with actual partition resource %s", newRes, partitionInfo.GetTotalPartitionResource())
+	}
+	if !resources.Equals(newRes, partitionInfo.GetQueue("root").GetMaxResource()) {
+		t.Errorf("Expected partition resource %s, doesn't match with actual partition resource %s", newRes, partitionInfo.GetQueue("root").GetMaxResource())
+	}
+
+	// update node capacity with more mem and same vcores
+	err = ms.proxy.Update(&si.UpdateRequest{
+		UpdatedNodes: []*si.UpdateNodeInfo{
+			{
+				NodeID:     "node-1:1234",
+				Attributes: map[string]string{},
+				SchedulableResource: &si.Resource{
+					Resources: map[string]*si.Quantity{
+						"memory": {Value: 100},
+						"vcore":  {Value: 20},
+					},
+				},
+				Action: si.UpdateNodeInfo_UPDATE,
+			},
+		},
+		RmID: "rm:123",
+	})
+	assert.NilError(t, err, "UpdateRequest failed")
+	waitForAvailableNodeResource(t, ms.scheduler.GetClusterContext(), "[rm:123]default",
+		[]string{"node-1:1234"}, 100, 1000)

Review comment:
       No need to split the line to keep length down, make it one line. (multiple times found)

##########
File path: pkg/scheduler/objects/node.go
##########
@@ -127,15 +127,18 @@ func (sn *Node) GetCapacity() *resources.Resource {
 	return sn.totalResource.Clone()
 }
 
-func (sn *Node) SetCapacity(newCapacity *resources.Resource) {
+func (sn *Node) SetCapacity(newCapacity *resources.Resource) *resources.Resource {
 	sn.Lock()
 	defer sn.Unlock()
+	var delta *resources.Resource = nil
 	if resources.Equals(sn.totalResource, newCapacity) {
 		log.Logger().Debug("skip updating capacity, not changed")
-		return
+		return delta
 	}
+	delta = resources.Sub(newCapacity, sn.totalResource)

Review comment:
       Simplify this to a `return nil` and create a mew var using `:=`, can remove the var delta when you do 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.

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