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/10/22 09:02:42 UTC

[GitHub] [incubator-yunikorn-core] kingamarton commented on a change in pull request #330: [YUNIKORN-879] Extend health check with allocation check

kingamarton commented on a change in pull request #330:
URL: https://github.com/apache/incubator-yunikorn-core/pull/330#discussion_r734364198



##########
File path: pkg/scheduler/health_checker.go
##########
@@ -151,5 +161,40 @@ func checkSchedulingContext(schedulerContext *ClusterContext) []dao.HealthCheckI
 	infos[6] = CreateCheckInfo(true, "Reservation check",
 		"Check the reservation nr compared to the number of nodes",
 		fmt.Sprintf("Reservation/node nr ratio: %f", partitionReservationRatio))
+	infos[7] = CreateCheckInfo(len(orphanAllocationsOnNode) == 0, "Orphan allocation on node check",
+		"Check if there are orphan allocations on the nodes",
+		fmt.Sprintf("Orphan allocations: %v", orphanAllocationsOnNode))
+	infos[8] = CreateCheckInfo(len(orphanAllocationsOnApp) == 0, "Orphan allocation on app check",
+		"Check if there are orphan allocations on the applications",
+		fmt.Sprintf("OrphanAllocations: %v", orphanAllocationsOnApp))
 	return infos
 }
+
+func checkAppAllocations(app *objects.Application, nodes objects.NodeCollection) []*objects.Allocation {
+	orphanAllocationsOnApp := make([]*objects.Allocation, 0)
+	for _, alloc := range app.GetAllAllocations() {
+		if node := nodes.GetNode(alloc.NodeID); node != nil {
+			node.GetAllocation(alloc.UUID)

Review comment:
       Good catch. Is not neeeded.

##########
File path: pkg/scheduler/health_checker_test.go
##########
@@ -84,6 +84,28 @@ func TestGetSchedulerHealthStatusContext(t *testing.T) {
 	assert.NilError(t, err, "Unexpected error while adding a new node")
 	healthInfo = GetSchedulerHealthStatus(schedulerMetrics, schedulerContext)
 	assert.Assert(t, !healthInfo.Healthy, "Scheduler should not be healthy")
+
+	// add orphan allocation to a node
+	node := schedulerContext.partitions[partName].nodes.GetNode("node")
+	alloc := objects.NewAllocation(allocID, "node", newAllocationAsk("key", "appID", resources.NewResource()))
+	node.AddAllocation(alloc)
+	healthInfo = GetSchedulerHealthStatus(schedulerMetrics, schedulerContext)
+	assert.Assert(t, !healthInfo.Healthy, "Scheduler should not be healthy")
+	assert.Assert(t, !healthInfo.HealthChecks[9].Succeeded, "The orphan allocation check on the node should not be successful")
+
+	// add the allocation to the app as well
+	part := schedulerContext.partitions[partName]
+	app := newApplication("appID", partName, "queue")
+	app.AddAllocation(alloc)
+	part.applications["appID"] = app
+	healthInfo = GetSchedulerHealthStatus(schedulerMetrics, schedulerContext)
+	assert.Assert(t, healthInfo.HealthChecks[9].Succeeded, "The orphan allocation check on the node should be successful")
+
+	// remove the allocation from the node, so we will have an orphan allocation assigned to the app
+	node.RemoveAllocation(allocID)
+	healthInfo = GetSchedulerHealthStatus(schedulerMetrics, schedulerContext)
+	assert.Assert(t, healthInfo.HealthChecks[9].Succeeded, "The orphan allocation check on the node should be successful")
+	assert.Assert(t, !healthInfo.HealthChecks[10].Succeeded, "The orphan allocation check on the app should not be successful")

Review comment:
       In the normal cases no, but we have seen this kind of behaviour in some escalations, when a node was unexpectedly removed and we had some placeholders processing. 




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