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/02/25 12:33:22 UTC

[GitHub] [incubator-yunikorn-core] kingamarton commented on a change in pull request #251: [YUNIKORN-505] placeholder allocate tests

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



##########
File path: pkg/scheduler/partition.go
##########
@@ -1185,18 +1180,29 @@ func (pc *PartitionContext) removeAllocation(release *si.AllocationRelease) ([]*
 // NOTE: this is a lock free call. It must NOT be called holding the PartitionContext lock.
 func (pc *PartitionContext) removeAllocationAsk(appID string, allocationKey string) {
 	app := pc.getApplication(appID)
-	if app != nil {
-		// remove the allocation asks from the app
-		reservedAsks := app.RemoveAllocationAsk(allocationKey)
-		log.Logger().Info("release allocation ask",
+	if app == nil {
+		log.Logger().Info("Invalid ask release requested by shim",
 			zap.String("allocation", allocationKey),
-			zap.String("appID", appID),
-			zap.Int("reservedAskReleased", reservedAsks))
-		// update the partition if the asks were reserved (clean up)
-		if reservedAsks != 0 {
-			pc.unReserveCount(appID, reservedAsks)
-		}
+			zap.String("appID", appID))
+		return
+	}
+	// remove the allocation asks from the app
+	reservedAsks := app.RemoveAllocationAsk(allocationKey)
+	// update the partition if the asks were reserved (clean up)
+	if reservedAsks != 0 {
+		pc.unReserveCount(appID, reservedAsks)
+	}
+}
+
+// Remove the allocation Ask from the specified application

Review comment:
       Please fix the comment, since here we are adding Allocation ask not removing it.

##########
File path: pkg/scheduler/partition.go
##########
@@ -380,9 +379,6 @@ func (pc *PartitionContext) removeApplication(appID string) []*objects.Allocatio
 			}
 		}
 	}
-	log.Logger().Debug("application removed from the scheduler",
-		zap.String("queue", queueName),
-		zap.String("applicationID", appID))

Review comment:
       I think this log message can be useful. Right now we have no logs when we remove an application

##########
File path: pkg/scheduler/context.go
##########
@@ -626,27 +656,18 @@ func (cc *ClusterContext) processAsks(request *si.UpdateRequest) {
 			continue
 		}
 
-		// if app info doesn't exist, reject the request
-		app := partition.getApplication(siAsk.ApplicationID)
-		if app == nil {
-			msg := fmt.Sprintf("Failed to find application %s, for allocation %s", siAsk.ApplicationID, siAsk.AllocationKey)
-			log.Logger().Info(msg)
-			rejectedAsks = append(rejectedAsks,

Review comment:
       after moving this check to the partitionContext, if this check will fail, the rejectedAsks list will not have the actual one.

##########
File path: pkg/scheduler/partition_test.go
##########
@@ -1221,3 +1222,237 @@ func TestUpdateNode(t *testing.T) {
 		t.Errorf("Expected partition resource %s, doesn't match with actual partition resource %s", expectedRes, partition.GetTotalPartitionResource())
 	}
 }
+
+func TestAddTGApplication(t *testing.T) {
+	limit := map[string]string{"first": "1"}
+	partition, err := newLimitedPartition(limit)
+	assert.NilError(t, err, "partition create failed")
+	// add a app with TG that does not fit in the queue
+	var tgRes *resources.Resource
+	tgRes, err = resources.NewResourceFromConf(map[string]string{"first": "10"})
+	assert.NilError(t, err, "failed to create resource")
+	app := newApplicationTG(appID1, "default", "root.limited", tgRes)
+	err = partition.AddApplication(app)
+	if err == nil {
+		t.Error("app-1 should be rejected due to TG request")
+	}
+
+	limit = map[string]string{"first": "100"}
+	partition, err = newLimitedPartition(limit)
+	assert.NilError(t, err, "partition create failed")
+	err = partition.AddApplication(app)
+	assert.NilError(t, err, "app-1 should have been added to the partition")

Review comment:
       Besides checking for error please check if the application was added to the application list as well.

##########
File path: pkg/scheduler/partition.go
##########
@@ -585,10 +580,6 @@ func (pc *PartitionContext) AddNode(node *objects.Node, existingAllocations []*o
 
 	// Node is added update the metrics
 	metrics.GetSchedulerMetrics().IncActiveNodes()
-	log.Logger().Info("added node to partition",
-		zap.String("partitionName", pc.Name),
-		zap.String("nodeID", node.NodeID),
-		zap.String("partitionResource", pc.totalPartitionResource.String()))

Review comment:
       This one can be helpful as well, I think. Maybe at debug level is enough, but I don't think we should remove it.

##########
File path: pkg/scheduler/partition.go
##########
@@ -1185,18 +1180,29 @@ func (pc *PartitionContext) removeAllocation(release *si.AllocationRelease) ([]*
 // NOTE: this is a lock free call. It must NOT be called holding the PartitionContext lock.
 func (pc *PartitionContext) removeAllocationAsk(appID string, allocationKey string) {
 	app := pc.getApplication(appID)
-	if app != nil {
-		// remove the allocation asks from the app
-		reservedAsks := app.RemoveAllocationAsk(allocationKey)
-		log.Logger().Info("release allocation ask",
+	if app == nil {
+		log.Logger().Info("Invalid ask release requested by shim",
 			zap.String("allocation", allocationKey),
-			zap.String("appID", appID),
-			zap.Int("reservedAskReleased", reservedAsks))
-		// update the partition if the asks were reserved (clean up)
-		if reservedAsks != 0 {
-			pc.unReserveCount(appID, reservedAsks)
-		}
+			zap.String("appID", appID))
+		return
+	}
+	// remove the allocation asks from the app
+	reservedAsks := app.RemoveAllocationAsk(allocationKey)
+	// update the partition if the asks were reserved (clean up)
+	if reservedAsks != 0 {
+		pc.unReserveCount(appID, reservedAsks)
+	}
+}
+
+// Remove the allocation Ask from the specified application
+// NOTE: this is a lock free call. It must NOT be called holding the PartitionContext lock.
+func (pc *PartitionContext) addAllocationAsk(siAsk *si.AllocationAsk) error {
+	app := pc.getApplication(siAsk.ApplicationID)
+	if app == nil {
+		return fmt.Errorf("failed to find application %s, for allocation ask %s", siAsk.ApplicationID, siAsk.AllocationKey)
 	}
+	// remove the allocation asks from the app

Review comment:
       please fix the comment




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