You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@yunikorn.apache.org by GitBox <gi...@apache.org> on 2020/03/24 20:41:43 UTC

[GitHub] [incubator-yunikorn-core] yangwwei opened a new pull request #108: [YUNIKORN-47] Support node capacity update

yangwwei opened a new pull request #108: [YUNIKORN-47] Support node capacity update
URL: https://github.com/apache/incubator-yunikorn-core/pull/108
 
 
   We need to support the live updates of node capacity.
   There are multiple use cases for this:
   1) users could reconfigure kubelet with new capacity (e.g reserve more resources for system usage). See more in https://kubernetes.io/docs/tasks/administer-cluster/reconfigure-kubelet/#reconfiguring-the-kubelet-on-a-running-node-in-your-cluster, and https://kubernetes.io/docs/tasks/administer-cluster/reserve-compute-resources/. We need to update our node capacity accordingly.
   2) use this API to dynamically change node capacity to exclude resources allocated by other schedulers (see more in https://issues.apache.org/jira/browse/YUNIKORN-16)

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s closed pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s closed pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108
 
 
   

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


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on issue #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on issue #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#issuecomment-604736924
 
 
   changes committed closing PR

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


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398176943
 
 

 ##########
 File path: pkg/scheduler/scheduler.go
 ##########
 @@ -199,10 +199,12 @@ func (s *Scheduler) processAllocationReleaseByAllocationKey(allocationAsksToRele
 				log.Logger().Info("release allocation",
 					zap.String("allocation", toRelease.Allocationkey),
 					zap.String("appID", toRelease.ApplicationID),
-					zap.String("message", toRelease.Message))
+					zap.String("message", toRelease.Message),
+					zap.Int("ReservedToRelease", reservedAsks))
 				// update the partition if the asks were reserved (clean up)
 				if reservedAsks != 0 {
-					s.clusterSchedulingContext.getPartition(toRelease.PartitionName).unReserveUpdate(toRelease.ApplicationID, reservedAsks)
+					s.clusterSchedulingContext.getPartition(toRelease.PartitionName).
+						unReserveUpdate(toRelease.ApplicationID, reservedAsks)
 
 Review comment:
   reverted the change. 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397823212
 
 

 ##########
 File path: pkg/scheduler/tests/scheduler_operation_test.go
 ##########
 @@ -0,0 +1,514 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package tests
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/common/resources"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/si"
+	"gotest.tools/assert"
+)
+
+// this test simulates the scenario the cluster starts up with 0 nodes
+// then we submit an app, the app tasks will be pending; then we add a
+// node to the cluster, then we see the app gets the allocation it needed.
+func TestSchedulerWithoutNodes(t *testing.T) {
+	// Register RM
+	configData := `
+partitions:
+  -
+    name: default
+    queues:
+      - name: root
+        submitacl: "*"
+        queues:
+          - name: a
+            resources:
+              guaranteed:
+                memory: 100
+                vcore: 10
+              max:
+                memory: 150
+                vcore: 20
+`
+	// Start all tests
+	ms := &mockScheduler{}
+	defer ms.Stop()
+
+	err := ms.Init(configData, false)
+	if err != nil {
+		t.Fatalf("RegisterResourceManager failed: %v", err)
+	}
 
 Review comment:
   We should use `assert.NilError(t, err, "RegisterResourceManager failed")`
   I started to move all smoke tests to use this assert

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398171122
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -109,6 +112,40 @@ func (ni *NodeInfo) GetCapacity() *resources.Resource {
 	return ni.totalResource.Clone()
 }
 
+func (ni *NodeInfo) SetCapacity(newCapacity *resources.Resource) {
+	ni.lock.Lock()
+	defer ni.lock.Unlock()
+	if resources.Equals(ni.totalResource, newCapacity) {
+		log.Logger().Info("skip updating capacity, not changed")
+		return
+	}
+
+	ni.totalResource = newCapacity
+	ni.availableResource = ni.totalResource.Clone()
+	ni.availableResource.SubFrom(ni.allocatedResource)
+	ni.availableResource.SubFrom(ni.occupiedResource)
+}
+
+func (ni *NodeInfo) GetOccupiedResource() *resources.Resource {
+	ni.lock.RLock()
+	defer ni.lock.RUnlock()
+	return ni.occupiedResource.Clone()
+}
+
+func (ni *NodeInfo) SetOccupiedResource(occupiedResource *resources.Resource) {
 
 Review comment:
   Done

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397808114
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -109,6 +112,40 @@ func (ni *NodeInfo) GetCapacity() *resources.Resource {
 	return ni.totalResource.Clone()
 }
 
+func (ni *NodeInfo) SetCapacity(newCapacity *resources.Resource) {
+	ni.lock.Lock()
+	defer ni.lock.Unlock()
+	if resources.Equals(ni.totalResource, newCapacity) {
+		log.Logger().Info("skip updating capacity, not changed")
+		return
+	}
+
+	ni.totalResource = newCapacity
+	ni.availableResource = ni.totalResource.Clone()
+	ni.availableResource.SubFrom(ni.allocatedResource)
+	ni.availableResource.SubFrom(ni.occupiedResource)
+}
+
+func (ni *NodeInfo) GetOccupiedResource() *resources.Resource {
+	ni.lock.RLock()
+	defer ni.lock.RUnlock()
+	return ni.occupiedResource.Clone()
+}
+
+func (ni *NodeInfo) SetOccupiedResource(occupiedResource *resources.Resource) {
+	ni.lock.Lock()
+	defer ni.lock.Unlock()
+	if resources.Equals(ni.occupiedResource, occupiedResource) {
+		log.Logger().Info("skip updating occupiedResource, not changed")
 
 Review comment:
   Info seems to high to log this, Debug is better

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398171547
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -109,6 +112,40 @@ func (ni *NodeInfo) GetCapacity() *resources.Resource {
 	return ni.totalResource.Clone()
 }
 
+func (ni *NodeInfo) SetCapacity(newCapacity *resources.Resource) {
+	ni.lock.Lock()
+	defer ni.lock.Unlock()
+	if resources.Equals(ni.totalResource, newCapacity) {
+		log.Logger().Info("skip updating capacity, not changed")
+		return
+	}
+
+	ni.totalResource = newCapacity
+	ni.availableResource = ni.totalResource.Clone()
+	ni.availableResource.SubFrom(ni.allocatedResource)
+	ni.availableResource.SubFrom(ni.occupiedResource)
+}
+
+func (ni *NodeInfo) GetOccupiedResource() *resources.Resource {
+	ni.lock.RLock()
+	defer ni.lock.RUnlock()
+	return ni.occupiedResource.Clone()
+}
+
+func (ni *NodeInfo) SetOccupiedResource(occupiedResource *resources.Resource) {
+	ni.lock.Lock()
+	defer ni.lock.Unlock()
+	if resources.Equals(ni.occupiedResource, occupiedResource) {
+		log.Logger().Info("skip updating occupiedResource, not changed")
 
 Review comment:
   done

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398317556
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -56,6 +58,7 @@ func NewNodeInfo(proto *si.NewNodeInfo) *NodeInfo {
 		NodeID:            proto.NodeID,
 		totalResource:     resources.NewResourceFromProto(proto.SchedulableResource),
 		allocatedResource: resources.NewResource(),
+		occupiedResource:  resources.NewResource(),
 
 Review comment:
   perfect that is what I wanted to know

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397808027
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -109,6 +112,40 @@ func (ni *NodeInfo) GetCapacity() *resources.Resource {
 	return ni.totalResource.Clone()
 }
 
+func (ni *NodeInfo) SetCapacity(newCapacity *resources.Resource) {
+	ni.lock.Lock()
+	defer ni.lock.Unlock()
+	if resources.Equals(ni.totalResource, newCapacity) {
+		log.Logger().Info("skip updating capacity, not changed")
 
 Review comment:
   Info seems to high to log this, Debug is better

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on issue #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on issue #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#issuecomment-604238650
 
 
   Unit tests passed on second run. Not sure what happened but even a simple partition create failed in one test causing one of the failures. This is not the test code but an environment issue.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398170589
 
 

 ##########
 File path: pkg/scheduler/scheduling_node.go
 ##########
 @@ -74,13 +74,28 @@ func (sn *SchedulingNode) GetReservations() []string {
 	return keys
 }
 
+func (sn *SchedulingNode) updateNodeInfo(newNodeInfo *cache.NodeInfo) {
+	sn.Lock()
+	defer sn.Unlock()
+
+	sn.nodeInfo = newNodeInfo
+	sn.cachedAvailableUpdateNeeded = true
+}
+
 // Get the allocated resource on this node.
 // These resources are just the confirmed allocations (tracked in the cache node).
 // This does not lock the cache node as it will take its own lock.
 func (sn *SchedulingNode) GetAllocatedResource() *resources.Resource {
+	sn.RLock()
+	defer sn.RUnlock()
 	return sn.nodeInfo.GetAllocatedResource()
 }
 
+// expose this for tests
+func (sn *SchedulingNode) GetAvailableResourceForTest() *resources.Resource {
+	return sn.getAvailableResource()
+}
+
 
 Review comment:
   Yes, we can do that. This simplifies the contract.
   I have renamed `getAvailableResource()` to `GetAvailableResource ()`

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397807292
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -109,6 +112,40 @@ func (ni *NodeInfo) GetCapacity() *resources.Resource {
 	return ni.totalResource.Clone()
 }
 
+func (ni *NodeInfo) SetCapacity(newCapacity *resources.Resource) {
+	ni.lock.Lock()
+	defer ni.lock.Unlock()
+	if resources.Equals(ni.totalResource, newCapacity) {
+		log.Logger().Info("skip updating capacity, not changed")
+		return
+	}
+
+	ni.totalResource = newCapacity
+	ni.availableResource = ni.totalResource.Clone()
+	ni.availableResource.SubFrom(ni.allocatedResource)
+	ni.availableResource.SubFrom(ni.occupiedResource)
+}
+
+func (ni *NodeInfo) GetOccupiedResource() *resources.Resource {
+	ni.lock.RLock()
+	defer ni.lock.RUnlock()
+	return ni.occupiedResource.Clone()
+}
+
+func (ni *NodeInfo) SetOccupiedResource(occupiedResource *resources.Resource) {
 
 Review comment:
   Should not be exported

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398325946
 
 

 ##########
 File path: pkg/scheduler/tests/scheduler_utils.go
 ##########
 @@ -103,6 +103,20 @@ func waitForNodesAllocatedResource(t *testing.T, cache *cache.ClusterInfo, parti
 	}
 }
 
+func waitForNodesAvailableResource(t *testing.T, cache *cache.ClusterInfo, partitionName string, nodeIDs []string, availableMemory resources.Quantity, timeoutMs int) {
+	var totalNodeResource resources.Quantity
+	err := common.WaitFor(10*time.Millisecond, time.Duration(timeoutMs)*time.Millisecond, func() bool {
+		totalNodeResource = 0
+		for _, nodeID := range nodeIDs {
+			totalNodeResource += cache.GetPartition(partitionName).GetNode(nodeID).GetAvailableResource().Resources[resources.MEMORY]
+		}
+		return totalNodeResource == availableMemory
+	})
+	if err != nil {
+		t.Fatalf("Failed to wait for available resource %v and node %v, called from: %s", availableMemory, nodeIDs, caller())
 
 Review comment:
   filed follow up jira: YUNIKORN-53

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397784697
 
 

 ##########
 File path: pkg/scheduler/scheduling_partition.go
 ##########
 @@ -308,6 +308,19 @@ func (psc *partitionSchedulingContext) addSchedulingNode(info *cache.NodeInfo) {
 	psc.nodes[info.NodeID] = newSchedulingNode(info)
 }
 
+func (psc *partitionSchedulingContext) updateSchedulingNode(info *cache.NodeInfo) {
+	if info == nil {
+		return
+	}
+
+	psc.Lock()
+	defer psc.Unlock()
+	// check consistency and reset to make sure it is consistent again
+	if schedulingNode, ok := psc.nodes[info.NodeID]; ok {
+		schedulingNode.updateNodeInfo(info)
 
 Review comment:
   The comment is not correct.
   The behaviour is also not trackable: if we have no node we ignore the update. We really need to at least log this.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398176308
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -56,6 +58,7 @@ func NewNodeInfo(proto *si.NewNodeInfo) *NodeInfo {
 		NodeID:            proto.NodeID,
 		totalResource:     resources.NewResourceFromProto(proto.SchedulableResource),
 		allocatedResource: resources.NewResource(),
+		occupiedResource:  resources.NewResource(),
 
 Review comment:
   The existing allocations for non-yunikorn-scheduled pods will be reported through UPDATE call. Not part of existing allocations.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397811926
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -56,6 +58,7 @@ func NewNodeInfo(proto *si.NewNodeInfo) *NodeInfo {
 		NodeID:            proto.NodeID,
 		totalResource:     resources.NewResourceFromProto(proto.SchedulableResource),
 		allocatedResource: resources.NewResource(),
+		occupiedResource:  resources.NewResource(),
 
 Review comment:
   Are you sure that this is always zero when we register a new node or could we have an allocation already?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398318824
 
 

 ##########
 File path: pkg/cache/partition_info.go
 ##########
 @@ -477,12 +477,17 @@ func (pi *PartitionInfo) releaseAllocationsForApplication(toRelease *commonevent
 	for _, alloc := range allocationsToRelease {
 		// remove allocation from node
 		node := pi.nodes[alloc.AllocationProto.NodeID]
+		// node could be nil if when release happens, the node is already
+		// removed from the cluster. in this case, we need to make sure
+		// resources are released correctly. otherwise these resources will
+		// be leaked.
 		if node == nil || node.GetAllocation(alloc.AllocationProto.UUID) == nil {
-			log.Logger().Info("node is not found for allocation",
-				zap.Any("allocation", alloc))
-			continue
+			log.Logger().Info("node is not found or the allocation is not found",
+				zap.String("node", alloc.AllocationProto.NodeID),
+				zap.String("allocationID", alloc.AllocationProto.UUID))
+		} else {
+			node.RemoveAllocation(alloc.AllocationProto.UUID)
 		}
-		node.RemoveAllocation(alloc.AllocationProto.UUID)
 		totalReleasedResource.AddTo(alloc.AllocatedResource)
 
 Review comment:
   This needs to be reverted too. If we have a nil node we need to continue. It is almost an error case but we can only update if we have a node and an alloc.
   The queue will get out of sync if we remove the alloc while the node is nil. The nil node is a result of the queue already being updated.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on issue #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on issue #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#issuecomment-604136728
 
 
   Hi @wilfred-s  I have made the changes, please check.
   The UT failure seems not to relate to my change, as it was failing at https://github.com/apache/incubator-yunikorn-core/pull/110 too, that was just some images update.
   We might need to look at this separately.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398176746
 
 

 ##########
 File path: pkg/scheduler/scheduler.go
 ##########
 @@ -199,10 +199,12 @@ func (s *Scheduler) processAllocationReleaseByAllocationKey(allocationAsksToRele
 				log.Logger().Info("release allocation",
 					zap.String("allocation", toRelease.Allocationkey),
 					zap.String("appID", toRelease.ApplicationID),
-					zap.String("message", toRelease.Message))
+					zap.String("message", toRelease.Message),
+					zap.Int("ReservedToRelease", reservedAsks))
 
 Review comment:
   Done

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397816756
 
 

 ##########
 File path: pkg/scheduler/scheduler.go
 ##########
 @@ -199,10 +199,12 @@ func (s *Scheduler) processAllocationReleaseByAllocationKey(allocationAsksToRele
 				log.Logger().Info("release allocation",
 					zap.String("allocation", toRelease.Allocationkey),
 					zap.String("appID", toRelease.ApplicationID),
-					zap.String("message", toRelease.Message))
+					zap.String("message", toRelease.Message),
+					zap.Int("ReservedToRelease", reservedAsks))
 
 Review comment:
   `ReservedToRelease` should be `reservedAskReleased`

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398177345
 
 

 ##########
 File path: pkg/scheduler/tests/scheduler_operation_test.go
 ##########
 @@ -0,0 +1,514 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package tests
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/common/resources"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/si"
+	"gotest.tools/assert"
+)
+
+// this test simulates the scenario the cluster starts up with 0 nodes
+// then we submit an app, the app tasks will be pending; then we add a
+// node to the cluster, then we see the app gets the allocation it needed.
+func TestSchedulerWithoutNodes(t *testing.T) {
+	// Register RM
+	configData := `
+partitions:
+  -
+    name: default
+    queues:
+      - name: root
+        submitacl: "*"
+        queues:
+          - name: a
+            resources:
+              guaranteed:
+                memory: 100
+                vcore: 10
+              max:
+                memory: 150
+                vcore: 20
+`
+	// Start all tests
+	ms := &mockScheduler{}
+	defer ms.Stop()
+
+	err := ms.Init(configData, false)
+	if err != nil {
+		t.Fatalf("RegisterResourceManager failed: %v", err)
+	}
 
 Review comment:
   Ah, I did not know this short cut. Let me use this for the tests added

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398318824
 
 

 ##########
 File path: pkg/cache/partition_info.go
 ##########
 @@ -477,12 +477,17 @@ func (pi *PartitionInfo) releaseAllocationsForApplication(toRelease *commonevent
 	for _, alloc := range allocationsToRelease {
 		// remove allocation from node
 		node := pi.nodes[alloc.AllocationProto.NodeID]
+		// node could be nil if when release happens, the node is already
+		// removed from the cluster. in this case, we need to make sure
+		// resources are released correctly. otherwise these resources will
+		// be leaked.
 		if node == nil || node.GetAllocation(alloc.AllocationProto.UUID) == nil {
-			log.Logger().Info("node is not found for allocation",
-				zap.Any("allocation", alloc))
-			continue
+			log.Logger().Info("node is not found or the allocation is not found",
+				zap.String("node", alloc.AllocationProto.NodeID),
+				zap.String("allocationID", alloc.AllocationProto.UUID))
+		} else {
+			node.RemoveAllocation(alloc.AllocationProto.UUID)
 		}
-		node.RemoveAllocation(alloc.AllocationProto.UUID)
 		totalReleasedResource.AddTo(alloc.AllocatedResource)
 
 Review comment:
   This needs to be reverted too. If we have a nil node we need to continue. It is almost an error case but we can only update if we have a node and an alloc.
   The queue will get out of sync if we remove the alloc while the node is nil. The nil node is a result of the queue already being updated.
   
   After that I am OK with a merge

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397786337
 
 

 ##########
 File path: pkg/scheduler/scheduling_node.go
 ##########
 @@ -74,13 +74,28 @@ func (sn *SchedulingNode) GetReservations() []string {
 	return keys
 }
 
+func (sn *SchedulingNode) updateNodeInfo(newNodeInfo *cache.NodeInfo) {
+	sn.Lock()
+	defer sn.Unlock()
+
+	sn.nodeInfo = newNodeInfo
+	sn.cachedAvailableUpdateNeeded = true
+}
+
 // Get the allocated resource on this node.
 // These resources are just the confirmed allocations (tracked in the cache node).
 // This does not lock the cache node as it will take its own lock.
 func (sn *SchedulingNode) GetAllocatedResource() *resources.Resource {
+	sn.RLock()
+	defer sn.RUnlock()
 	return sn.nodeInfo.GetAllocatedResource()
 }
 
+// expose this for tests
+func (sn *SchedulingNode) GetAvailableResourceForTest() *resources.Resource {
+	return sn.getAvailableResource()
+}
+
 
 Review comment:
   I would prefer we export `GetAvailableResource()`
   The major use is in the unit tests already so it is not really clear that this is purely for the smoke tests.
   We can do this in a follow up jira but we should try and get this in for 0.8

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398331257
 
 

 ##########
 File path: pkg/cache/partition_info.go
 ##########
 @@ -477,12 +477,17 @@ func (pi *PartitionInfo) releaseAllocationsForApplication(toRelease *commonevent
 	for _, alloc := range allocationsToRelease {
 		// remove allocation from node
 		node := pi.nodes[alloc.AllocationProto.NodeID]
+		// node could be nil if when release happens, the node is already
+		// removed from the cluster. in this case, we need to make sure
+		// resources are released correctly. otherwise these resources will
+		// be leaked.
 		if node == nil || node.GetAllocation(alloc.AllocationProto.UUID) == nil {
-			log.Logger().Info("node is not found for allocation",
-				zap.Any("allocation", alloc))
-			continue
+			log.Logger().Info("node is not found or the allocation is not found",
+				zap.String("node", alloc.AllocationProto.NodeID),
+				zap.String("allocationID", alloc.AllocationProto.UUID))
+		} else {
+			node.RemoveAllocation(alloc.AllocationProto.UUID)
 		}
-		node.RemoveAllocation(alloc.AllocationProto.UUID)
 		totalReleasedResource.AddTo(alloc.AllocatedResource)
 
 Review comment:
   Sorry, I missed this one earlier. Just done in the latest 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398180367
 
 

 ##########
 File path: pkg/scheduler/tests/scheduler_utils.go
 ##########
 @@ -103,6 +103,20 @@ func waitForNodesAllocatedResource(t *testing.T, cache *cache.ClusterInfo, parti
 	}
 }
 
+func waitForNodesAvailableResource(t *testing.T, cache *cache.ClusterInfo, partitionName string, nodeIDs []string, availableMemory resources.Quantity, timeoutMs int) {
+	var totalNodeResource resources.Quantity
+	err := common.WaitFor(10*time.Millisecond, time.Duration(timeoutMs)*time.Millisecond, func() bool {
+		totalNodeResource = 0
+		for _, nodeID := range nodeIDs {
+			totalNodeResource += cache.GetPartition(partitionName).GetNode(nodeID).GetAvailableResource().Resources[resources.MEMORY]
+		}
+		return totalNodeResource == availableMemory
+	})
+	if err != nil {
+		t.Fatalf("Failed to wait for available resource %v and node %v, called from: %s", availableMemory, nodeIDs, caller())
 
 Review comment:
   yeah, but we might need another cleanup JIRA for this. Given all checks in this file are using a similar way.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397815327
 
 

 ##########
 File path: pkg/cache/partition_info.go
 ##########
 @@ -477,12 +477,17 @@ func (pi *PartitionInfo) releaseAllocationsForApplication(toRelease *commonevent
 	for _, alloc := range allocationsToRelease {
 		// remove allocation from node
 		node := pi.nodes[alloc.AllocationProto.NodeID]
+		// node could be nil if when release happens, the node is already
+		// removed from the cluster. in this case, we need to make sure
+		// resources are released correctly. otherwise these resources will
+		// be leaked.
 
 Review comment:
   This cannot happen, and is impossible.
   When a node is removed the allocations are released by calling `removeNodeAllocations()` at that point they are passed back and should have been released. The queue will get out of sync if we do this twice.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398179935
 
 

 ##########
 File path: pkg/scheduler/tests/scheduler_operation_test.go
 ##########
 @@ -0,0 +1,514 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package tests
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/common/resources"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/si"
+	"gotest.tools/assert"
+)
+
+// this test simulates the scenario the cluster starts up with 0 nodes
+// then we submit an app, the app tasks will be pending; then we add a
+// node to the cluster, then we see the app gets the allocation it needed.
+func TestSchedulerWithoutNodes(t *testing.T) {
+	// Register RM
+	configData := `
+partitions:
+  -
+    name: default
+    queues:
+      - name: root
+        submitacl: "*"
+        queues:
+          - name: a
+            resources:
+              guaranteed:
+                memory: 100
+                vcore: 10
+              max:
+                memory: 150
+                vcore: 20
+`
+	// Start all tests
+	ms := &mockScheduler{}
+	defer ms.Stop()
+
+	err := ms.Init(configData, false)
+	if err != nil {
+		t.Fatalf("RegisterResourceManager failed: %v", err)
+	}
+
+	// Check queues of cache and scheduler.
+	partitionInfo := ms.clusterInfo.GetPartition("[rm:123]default")
+	assert.Assert(t, nil == partitionInfo.Root.GetMaxResource(), "partition info max resource nil")
+
+	// Check scheduling queue root
+	schedulerQueueRoot := ms.getSchedulingQueue("root")
+	assert.Assert(t, nil == schedulerQueueRoot.QueueInfo.GetMaxResource())
+
+	// Check scheduling queue a
+	schedulerQueueA := ms.getSchedulingQueue("root.a")
+	assert.Assert(t, 150 == schedulerQueueA.QueueInfo.GetMaxResource().Resources[resources.MEMORY])
+
+	// Add one application
+	err = ms.proxy.Update(&si.UpdateRequest{
+		NewApplications: newAddAppRequest(map[string]string{"app-1": "root.a"}),
+		RmID:            "rm:123",
+	})
+
+	// Application should be accepted
+	ms.mockRM.waitForAcceptedApplication(t, "app-1", 1000)
+
+	// Check scheduling app
+	schedulingApp := ms.getSchedulingApplication("app-1")
+	assert.Equal(t, schedulingApp.ApplicationInfo.ApplicationID, "app-1")
+	assert.Equal(t, len(schedulingApp.ApplicationInfo.GetAllAllocations()), 0)
+
+	// App asks for 2 allocations
+	err = ms.proxy.Update(&si.UpdateRequest{
+		Asks: []*si.AllocationAsk{
+			{
+				AllocationKey: "alloc-1",
+				ResourceAsk: &si.Resource{
+					Resources: map[string]*si.Quantity{
+						"memory": {Value: 10},
+						"vcore":  {Value: 1},
+					},
+				},
+				MaxAllocations: 2,
+				ApplicationID:  "app-1",
+			},
+		},
+		RmID: "rm:123",
+	})
+
+	if err != nil {
+		t.Fatalf("UpdateRequest 2 failed: %v", err)
+	}
+
+	waitForPendingQueueResource(t, schedulerQueueA, 20, 1000)
+	waitForPendingQueueResource(t, schedulerQueueRoot, 20, 1000)
+	waitForPendingAppResource(t, schedulingApp, 20, 1000)
+
+	// no nodes available, no allocation can be made
+	ms.scheduler.MultiStepSchedule(16)
+
+	// pending resources should not change
+	waitForPendingQueueResource(t, schedulerQueueA, 20, 1000)
+	waitForPendingQueueResource(t, schedulerQueueRoot, 20, 1000)
+	waitForPendingAppResource(t, schedulingApp, 20, 1000)
+
+	// Register a node
+	err = ms.proxy.Update(&si.UpdateRequest{
+		NewSchedulableNodes: []*si.NewNodeInfo{
+			{
+				NodeID: "node-1:1234",
+				Attributes: map[string]string{
+					"si.io/hostname": "node-1",
+					"si.io/rackname": "rack-1",
 
 Review comment:
   done

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397807543
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -109,6 +112,40 @@ func (ni *NodeInfo) GetCapacity() *resources.Resource {
 	return ni.totalResource.Clone()
 }
 
+func (ni *NodeInfo) SetCapacity(newCapacity *resources.Resource) {
 
 Review comment:
   Should not be exported

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398171387
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -109,6 +112,40 @@ func (ni *NodeInfo) GetCapacity() *resources.Resource {
 	return ni.totalResource.Clone()
 }
 
+func (ni *NodeInfo) SetCapacity(newCapacity *resources.Resource) {
+	ni.lock.Lock()
+	defer ni.lock.Unlock()
+	if resources.Equals(ni.totalResource, newCapacity) {
+		log.Logger().Info("skip updating capacity, not changed")
 
 Review comment:
   Done

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397817048
 
 

 ##########
 File path: pkg/scheduler/scheduler.go
 ##########
 @@ -199,10 +199,12 @@ func (s *Scheduler) processAllocationReleaseByAllocationKey(allocationAsksToRele
 				log.Logger().Info("release allocation",
 					zap.String("allocation", toRelease.Allocationkey),
 					zap.String("appID", toRelease.ApplicationID),
-					zap.String("message", toRelease.Message))
+					zap.String("message", toRelease.Message),
+					zap.Int("ReservedToRelease", reservedAsks))
 				// update the partition if the asks were reserved (clean up)
 				if reservedAsks != 0 {
-					s.clusterSchedulingContext.getPartition(toRelease.PartitionName).unReserveUpdate(toRelease.ApplicationID, reservedAsks)
+					s.clusterSchedulingContext.getPartition(toRelease.PartitionName).
+						unReserveUpdate(toRelease.ApplicationID, reservedAsks)
 
 Review comment:
   we do not limit line length I don't think we should split this

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397825029
 
 

 ##########
 File path: pkg/scheduler/tests/scheduler_operation_test.go
 ##########
 @@ -0,0 +1,514 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package tests
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/common/resources"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/si"
+	"gotest.tools/assert"
+)
+
+// this test simulates the scenario the cluster starts up with 0 nodes
+// then we submit an app, the app tasks will be pending; then we add a
+// node to the cluster, then we see the app gets the allocation it needed.
+func TestSchedulerWithoutNodes(t *testing.T) {
+	// Register RM
+	configData := `
+partitions:
+  -
+    name: default
+    queues:
+      - name: root
+        submitacl: "*"
+        queues:
+          - name: a
+            resources:
+              guaranteed:
+                memory: 100
+                vcore: 10
+              max:
+                memory: 150
+                vcore: 20
+`
+	// Start all tests
+	ms := &mockScheduler{}
+	defer ms.Stop()
+
+	err := ms.Init(configData, false)
+	if err != nil {
+		t.Fatalf("RegisterResourceManager failed: %v", err)
+	}
+
+	// Check queues of cache and scheduler.
+	partitionInfo := ms.clusterInfo.GetPartition("[rm:123]default")
+	assert.Assert(t, nil == partitionInfo.Root.GetMaxResource(), "partition info max resource nil")
+
+	// Check scheduling queue root
+	schedulerQueueRoot := ms.getSchedulingQueue("root")
+	assert.Assert(t, nil == schedulerQueueRoot.QueueInfo.GetMaxResource())
+
+	// Check scheduling queue a
+	schedulerQueueA := ms.getSchedulingQueue("root.a")
+	assert.Assert(t, 150 == schedulerQueueA.QueueInfo.GetMaxResource().Resources[resources.MEMORY])
+
+	// Add one application
+	err = ms.proxy.Update(&si.UpdateRequest{
+		NewApplications: newAddAppRequest(map[string]string{"app-1": "root.a"}),
+		RmID:            "rm:123",
+	})
+
+	// Application should be accepted
+	ms.mockRM.waitForAcceptedApplication(t, "app-1", 1000)
+
+	// Check scheduling app
+	schedulingApp := ms.getSchedulingApplication("app-1")
+	assert.Equal(t, schedulingApp.ApplicationInfo.ApplicationID, "app-1")
+	assert.Equal(t, len(schedulingApp.ApplicationInfo.GetAllAllocations()), 0)
+
+	// App asks for 2 allocations
+	err = ms.proxy.Update(&si.UpdateRequest{
+		Asks: []*si.AllocationAsk{
+			{
+				AllocationKey: "alloc-1",
+				ResourceAsk: &si.Resource{
+					Resources: map[string]*si.Quantity{
+						"memory": {Value: 10},
+						"vcore":  {Value: 1},
+					},
+				},
+				MaxAllocations: 2,
+				ApplicationID:  "app-1",
+			},
+		},
+		RmID: "rm:123",
+	})
+
+	if err != nil {
+		t.Fatalf("UpdateRequest 2 failed: %v", err)
+	}
+
+	waitForPendingQueueResource(t, schedulerQueueA, 20, 1000)
+	waitForPendingQueueResource(t, schedulerQueueRoot, 20, 1000)
+	waitForPendingAppResource(t, schedulingApp, 20, 1000)
+
+	// no nodes available, no allocation can be made
+	ms.scheduler.MultiStepSchedule(16)
+
+	// pending resources should not change
+	waitForPendingQueueResource(t, schedulerQueueA, 20, 1000)
+	waitForPendingQueueResource(t, schedulerQueueRoot, 20, 1000)
+	waitForPendingAppResource(t, schedulingApp, 20, 1000)
+
+	// Register a node
+	err = ms.proxy.Update(&si.UpdateRequest{
+		NewSchedulableNodes: []*si.NewNodeInfo{
+			{
+				NodeID: "node-1:1234",
+				Attributes: map[string]string{
+					"si.io/hostname": "node-1",
+					"si.io/rackname": "rack-1",
 
 Review comment:
   unused in tests should be removed and made an empty map

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398183686
 
 

 ##########
 File path: pkg/scheduler/tests/scheduler_operation_test.go
 ##########
 @@ -0,0 +1,514 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package tests
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/common/resources"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/si"
+	"gotest.tools/assert"
+)
+
+// this test simulates the scenario the cluster starts up with 0 nodes
+// then we submit an app, the app tasks will be pending; then we add a
+// node to the cluster, then we see the app gets the allocation it needed.
+func TestSchedulerWithoutNodes(t *testing.T) {
+	// Register RM
+	configData := `
+partitions:
+  -
+    name: default
+    queues:
+      - name: root
+        submitacl: "*"
+        queues:
+          - name: a
+            resources:
+              guaranteed:
+                memory: 100
+                vcore: 10
+              max:
+                memory: 150
+                vcore: 20
+`
+	// Start all tests
+	ms := &mockScheduler{}
+	defer ms.Stop()
+
+	err := ms.Init(configData, false)
+	if err != nil {
+		t.Fatalf("RegisterResourceManager failed: %v", err)
+	}
+
+	// Check queues of cache and scheduler.
+	partitionInfo := ms.clusterInfo.GetPartition("[rm:123]default")
+	assert.Assert(t, nil == partitionInfo.Root.GetMaxResource(), "partition info max resource nil")
 
 Review comment:
   fixed

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398171155
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -109,6 +112,40 @@ func (ni *NodeInfo) GetCapacity() *resources.Resource {
 	return ni.totalResource.Clone()
 }
 
+func (ni *NodeInfo) SetCapacity(newCapacity *resources.Resource) {
 
 Review comment:
   Done

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398139862
 
 

 ##########
 File path: pkg/scheduler/scheduling_partition.go
 ##########
 @@ -308,6 +308,19 @@ func (psc *partitionSchedulingContext) addSchedulingNode(info *cache.NodeInfo) {
 	psc.nodes[info.NodeID] = newSchedulingNode(info)
 }
 
+func (psc *partitionSchedulingContext) updateSchedulingNode(info *cache.NodeInfo) {
+	if info == nil {
+		return
+	}
+
+	psc.Lock()
+	defer psc.Unlock()
+	// check consistency and reset to make sure it is consistent again
+	if schedulingNode, ok := psc.nodes[info.NodeID]; ok {
+		schedulingNode.updateNodeInfo(info)
 
 Review comment:
   The comment was misleading. I have removed it.
   I added a WARN log when the node is not there but we attempt to do an update.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397826429
 
 

 ##########
 File path: pkg/scheduler/tests/scheduler_utils.go
 ##########
 @@ -103,6 +103,20 @@ func waitForNodesAllocatedResource(t *testing.T, cache *cache.ClusterInfo, parti
 	}
 }
 
+func waitForNodesAvailableResource(t *testing.T, cache *cache.ClusterInfo, partitionName string, nodeIDs []string, availableMemory resources.Quantity, timeoutMs int) {
+	var totalNodeResource resources.Quantity
+	err := common.WaitFor(10*time.Millisecond, time.Duration(timeoutMs)*time.Millisecond, func() bool {
+		totalNodeResource = 0
+		for _, nodeID := range nodeIDs {
+			totalNodeResource += cache.GetPartition(partitionName).GetNode(nodeID).GetAvailableResource().Resources[resources.MEMORY]
+		}
+		return totalNodeResource == availableMemory
+	})
+	if err != nil {
+		t.Fatalf("Failed to wait for available resource %v and node %v, called from: %s", availableMemory, nodeIDs, caller())
 
 Review comment:
   use `assert.NilError(t, err, "...")
   

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r398175840
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -109,6 +112,40 @@ func (ni *NodeInfo) GetCapacity() *resources.Resource {
 	return ni.totalResource.Clone()
 }
 
+func (ni *NodeInfo) SetCapacity(newCapacity *resources.Resource) {
+	ni.lock.Lock()
+	defer ni.lock.Unlock()
+	if resources.Equals(ni.totalResource, newCapacity) {
+		log.Logger().Info("skip updating capacity, not changed")
+		return
+	}
+
+	ni.totalResource = newCapacity
+	ni.availableResource = ni.totalResource.Clone()
+	ni.availableResource.SubFrom(ni.allocatedResource)
+	ni.availableResource.SubFrom(ni.occupiedResource)
 
 Review comment:
   Make sense to me.
   I moved to code to:
   
   ```
   // refresh node available resource based on the latest allocated and occupied resources.
   // this call assumes the caller already acquires the lock.
   func (ni *NodeInfo) refreshAvailableResource() {
   	ni.availableResource = ni.totalResource.Clone()
   	ni.availableResource.SubFrom(ni.allocatedResource)
   	ni.availableResource.SubFrom(ni.occupiedResource)
   }
   ```

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397811487
 
 

 ##########
 File path: pkg/cache/node_info.go
 ##########
 @@ -109,6 +112,40 @@ func (ni *NodeInfo) GetCapacity() *resources.Resource {
 	return ni.totalResource.Clone()
 }
 
+func (ni *NodeInfo) SetCapacity(newCapacity *resources.Resource) {
+	ni.lock.Lock()
+	defer ni.lock.Unlock()
+	if resources.Equals(ni.totalResource, newCapacity) {
+		log.Logger().Info("skip updating capacity, not changed")
+		return
+	}
+
+	ni.totalResource = newCapacity
+	ni.availableResource = ni.totalResource.Clone()
+	ni.availableResource.SubFrom(ni.allocatedResource)
+	ni.availableResource.SubFrom(ni.occupiedResource)
 
 Review comment:
   Move to a separate function `setAvailabaleResource()` and call from:
   * SetCapacity()
   * SetOccupiedResource()
   * NewNodeInfo()
   
   Keeps the logic the same everywhere.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #108: [YUNIKORN-47] Support node update action
URL: https://github.com/apache/incubator-yunikorn-core/pull/108#discussion_r397824102
 
 

 ##########
 File path: pkg/scheduler/tests/scheduler_operation_test.go
 ##########
 @@ -0,0 +1,514 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package tests
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/common/resources"
+	"github.com/apache/incubator-yunikorn-scheduler-interface/lib/go/si"
+	"gotest.tools/assert"
+)
+
+// this test simulates the scenario the cluster starts up with 0 nodes
+// then we submit an app, the app tasks will be pending; then we add a
+// node to the cluster, then we see the app gets the allocation it needed.
+func TestSchedulerWithoutNodes(t *testing.T) {
+	// Register RM
+	configData := `
+partitions:
+  -
+    name: default
+    queues:
+      - name: root
+        submitacl: "*"
+        queues:
+          - name: a
+            resources:
+              guaranteed:
+                memory: 100
+                vcore: 10
+              max:
+                memory: 150
+                vcore: 20
+`
+	// Start all tests
+	ms := &mockScheduler{}
+	defer ms.Stop()
+
+	err := ms.Init(configData, false)
+	if err != nil {
+		t.Fatalf("RegisterResourceManager failed: %v", err)
+	}
+
+	// Check queues of cache and scheduler.
+	partitionInfo := ms.clusterInfo.GetPartition("[rm:123]default")
+	assert.Assert(t, nil == partitionInfo.Root.GetMaxResource(), "partition info max resource nil")
 
 Review comment:
   no Yoda expressions: partiotion.. == nil

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org