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/03/10 07:53:55 UTC

[GitHub] [incubator-yunikorn-k8shim] wilfred-s commented on a change in pull request #380: [YUNIKORN-1090] remove common.Node from the shim

wilfred-s commented on a change in pull request #380:
URL: https://github.com/apache/incubator-yunikorn-k8shim/pull/380#discussion_r823430214



##########
File path: pkg/cache/nodes.go
##########
@@ -213,21 +212,22 @@ func (nc *schedulerNodes) updateNode(oldNode, newNode *v1.Node) {
 		return
 	}
 
-	if schedulerNode, ok := nc.nodesMap[newNode.Name]; ok {
-		schedulerNode.ready = common.HasReadyCondition(newNode)
+	schedulerNode, ok := nc.nodesMap[newNode.Name]
+	if ok {
+		schedulerNode.ready = hasReadyCondition(newNode)
 		if !schedulerNode.ready {
 			log.Logger().Debug("Node is not ready", zap.String("Node name", newNode.Name))
 		}

Review comment:
       we can log the state of the node always

##########
File path: pkg/cache/nodes.go
##########
@@ -213,21 +212,22 @@ func (nc *schedulerNodes) updateNode(oldNode, newNode *v1.Node) {
 		return

Review comment:
       we must update the resources in the cachedNode, might need locking and a setter method:
   `cachedNode.capacity = common.GetNodeResource(&newNode.Status)`

##########
File path: pkg/common/si_helper.go
##########
@@ -133,18 +134,23 @@ func CreateReleaseAllocationRequestForTask(appID, allocUUID, partition, terminat
 	return result
 }
 
-func CreateUpdateRequestForNewNode(node Node) si.NodeRequest {
+func CreateUpdateRequestForNewNode(nodeID string, capacity *si.Resource, occupied *si.Resource,

Review comment:
       Add comment to the exported function:
   ```
   // CreateUpdateRequestForNewNode builds a NodeRequest ....
   ```
   Should do the same for all si_helper functions changed here.

##########
File path: pkg/common/si_helper.go
##########
@@ -133,18 +134,23 @@ func CreateReleaseAllocationRequestForTask(appID, allocUUID, partition, terminat
 	return result
 }
 
-func CreateUpdateRequestForNewNode(node Node) si.NodeRequest {
+func CreateUpdateRequestForNewNode(nodeID string, capacity *si.Resource, occupied *si.Resource,
+	existingAllocations []*si.Allocation, labels string, ready bool) si.NodeRequest {
 	// Use node's name as the NodeID, this is because when bind pod to node,
 	// name of node is required but uid is optional.
 	nodeInfo := &si.NodeInfo{
-		NodeID:              node.name,
-		SchedulableResource: node.capacity,
+		NodeID:              nodeID,
+		SchedulableResource: capacity,
+		OccupiedResource:    occupied,
 		// TODO is this required?

Review comment:
       Please remove this TODO

##########
File path: pkg/cache/nodes.go
##########
@@ -213,21 +212,22 @@ func (nc *schedulerNodes) updateNode(oldNode, newNode *v1.Node) {
 		return
 	}
 
-	if schedulerNode, ok := nc.nodesMap[newNode.Name]; ok {
-		schedulerNode.ready = common.HasReadyCondition(newNode)
+	schedulerNode, ok := nc.nodesMap[newNode.Name]
+	if ok {
+		schedulerNode.ready = hasReadyCondition(newNode)
 		if !schedulerNode.ready {
 			log.Logger().Debug("Node is not ready", zap.String("Node name", newNode.Name))
 		}
+		request := common.CreateUpdateRequestForUpdatedNode(newNode.Name, common.GetNodeResource(&newNode.Status),
+			schedulerNode.occupied, schedulerNode.ready)
+		log.Logger().Info("report updated nodes to scheduler", zap.Any("request", request))
+		if err := nc.proxy.UpdateNode(&request); err != nil {
+			log.Logger().Info("hitting error while handling UpdateNode", zap.Error(err))
+		}
 	} else {
 		log.Logger().Error("Unable to find scheduler node in nodes map", zap.String("node name",

Review comment:
       can be removed node is always found

##########
File path: pkg/cache/nodes.go
##########
@@ -213,21 +212,22 @@ func (nc *schedulerNodes) updateNode(oldNode, newNode *v1.Node) {
 		return
 	}
 
-	if schedulerNode, ok := nc.nodesMap[newNode.Name]; ok {
-		schedulerNode.ready = common.HasReadyCondition(newNode)
+	schedulerNode, ok := nc.nodesMap[newNode.Name]
+	if ok {
+		schedulerNode.ready = hasReadyCondition(newNode)

Review comment:
       I do think we need to carefully look at what we do here (referencing new line numbers):
   * line 195: we get the node out of the nodesMap store in cachedNode
   * line 215: we retrieve the same node again, we already know it CANNOT be nil and ok is `true, we should re-use cachedNode
   * line 217: should always be executed, even if resources do not change, should this use a locked setter method?

##########
File path: pkg/common/si_helper_test.go
##########
@@ -18,14 +18,21 @@
 package common
 
 import (
+	"strconv"
 	"testing"
 
+	"github.com/apache/incubator-yunikorn-k8shim/pkg/common/constants"
 	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/common"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/si"
+
 	"gotest.tools/assert"
+
 	v1 "k8s.io/api/core/v1"
 	apis "k8s.io/apimachinery/pkg/apis/meta/v1"

Review comment:
       Can we re-order as we are cleaning up, 3 groups separated by an empty line:
   - group system imports
   - group other imports
   - group yunikorn imports

##########
File path: pkg/cache/nodes.go
##########
@@ -213,21 +212,22 @@ func (nc *schedulerNodes) updateNode(oldNode, newNode *v1.Node) {
 		return
 	}
 
-	if schedulerNode, ok := nc.nodesMap[newNode.Name]; ok {
-		schedulerNode.ready = common.HasReadyCondition(newNode)
+	schedulerNode, ok := nc.nodesMap[newNode.Name]
+	if ok {
+		schedulerNode.ready = hasReadyCondition(newNode)
 		if !schedulerNode.ready {
 			log.Logger().Debug("Node is not ready", zap.String("Node name", newNode.Name))
 		}
+		request := common.CreateUpdateRequestForUpdatedNode(newNode.Name, common.GetNodeResource(&newNode.Status),

Review comment:
       should use the cachedNode, does this event need to be generated when ready changes without resource changes?

##########
File path: pkg/common/si_helper.go
##########
@@ -156,15 +162,16 @@ func CreateUpdateRequestForNewNode(node Node) si.NodeRequest {
 	return request
 }
 
-func CreateUpdateRequestForUpdatedNode(node Node) si.NodeRequest {
+func CreateUpdateRequestForUpdatedNode(nodeID string, capacity *si.Resource, occupied *si.Resource,
+	ready bool) si.NodeRequest {
 	// Currently only includes resource in the update request

Review comment:
       Please update this comment as it is incorrect

##########
File path: pkg/cache/nodes.go
##########
@@ -259,3 +258,12 @@ func (nc *schedulerNodes) schedulerNodeEventHandler() func(obj interface{}) {
 		}
 	}
 }
+
+func hasReadyCondition(node *v1.Node) bool {
+	for _, condition := range node.Status.Conditions {

Review comment:
       Can `node` or `Status` be nil? Rather be careful than rely on the externally defined object always being complete.




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