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 2020/10/14 14:28:58 UTC

[GitHub] [incubator-yunikorn-core] manirajv06 opened a new pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

manirajv06 opened a new pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216


   First cut implementation. Need to add few more attributes in JSON response.


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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r509427956



##########
File path: pkg/webservice/routes.go
##########
@@ -126,6 +126,14 @@ var webRoutes = routes{
 		getContainerHistory,
 	},
 
+	//
+	route{
+		"Partitions",
+		"GET",
+		"/ws/v1/partitions",
+		getPartitions,
+	},

Review comment:
       As of now, We don't have such plan.




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



[GitHub] [incubator-yunikorn-core] yangwwei commented on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
yangwwei commented on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-769469434


   Oops, I can't get this merged as there are some conflicts.
   Could you pls fix the conflict? Thanks!


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



[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r568589627



##########
File path: pkg/scheduler/partition.go
##########
@@ -1208,3 +1210,17 @@ func (pc *PartitionContext) removeAllocationAsk(appID string, allocationKey stri
 		}
 	}
 }
+
+func (pc *PartitionContext) GetCurrentState() string {
+	return pc.stateMachine.Current()
+}
+
+func (pc *PartitionContext) GetStateTime() time.Time {
+	pc.RLock()
+	defer pc.RUnlock()
+	return pc.stateTime
+}
+
+func (pc *PartitionContext) GetNodeSortingPolicy() *policies.NodeSortingPolicy {
+	return pc.nodeSortingPolicy

Review comment:
       with the reuse I am ok with 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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r563316841



##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -24,12 +24,29 @@ type PartitionDAOInfo struct {
 	Queues        QueueDAOInfo      `json:"queues"`
 }
 
+type PartitionInfo struct {
+	Name                    string            `json:"name"`
+	Capacity                PartitionCapacity `json:"capacity"`
+	NodeSortingPolicy       string            `json:"nodeSortingPolicy"`
+	Applications            Applications      `json:"applications"`
+	State                   string            `json:"state"`
+	LastStateTransitionTime string            `json:"lastStateTransitionTime"`
+}
+
 type PartitionCapacity struct {
 	Capacity     string `json:"capacity"`
-	UsedCapacity string `json:"usedcapacity"`
+	UsedCapacity string `json:"usedCapacity"`
 }
 
 type NodeInfo struct {
 	NodeID     string `json:"nodeId"`
 	Capability string `json:"capability"`
 }
+
+type Applications struct {
+	Total     int `json:"total"`
+	Running   int `json:"running"`
+	Pending   int `json:"pending"`
+	Completed int `json:"completed"`
+	Failed    int `json:"failed"`

Review comment:
       I was trying to sum up certain states under common state. However, showing up each state count is also detailed enough. 




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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r563316654



##########
File path: pkg/scheduler/partition.go
##########
@@ -54,13 +54,13 @@ type PartitionContext struct {
 	allocations            map[string]*objects.Allocation  // allocations
 	placementManager       *placement.AppPlacementManager  // placement manager for this partition
 	partitionManager       *partitionManager               // manager for this partition
-	stateMachine           *fsm.FSM                        // the state of the partition for scheduling
-	stateTime              time.Time                       // last time the state was updated (needed for cleanup)
+	StateMachine           *fsm.FSM                        // the state of the partition for scheduling
+	StateTime              time.Time                       // last time the state was updated (needed for cleanup)
 	isPreemptable          bool                            // can allocations be preempted
 	rules                  *[]configs.PlacementRule        // placement rules to be loaded by the scheduler
 	userGroupCache         *security.UserGroupCache        // user cache per partition
 	totalPartitionResource *resources.Resource             // Total node resources
-	nodeSortingPolicy      *policies.NodeSortingPolicy     // Global Node Sorting Policies
+	NodeSortingPolicy      *policies.NodeSortingPolicy     // Global Node Sorting Policies

Review comment:
       Taken care.




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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r510261614



##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -24,12 +24,29 @@ type PartitionDAOInfo struct {
 	Queues        QueueDAOInfo      `json:"queues"`
 }
 
+type PartitionInfo struct {
+	Name                    string            `json:"name"`
+	Capacity                PartitionCapacity `json:"capacity"`

Review comment:
       As and when any new node gets added, Node's total resource would get added to this capacity. Similarly, for node removal resource would be subtracted from this capacity.




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



[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r566581972



##########
File path: pkg/webservice/handlers_test.go
##########
@@ -607,3 +627,46 @@ func TestGetNodesUtilJSON(t *testing.T) {
 	assert.Equal(t, subresNon[0].NumOfNodes, int64(-1))
 	assert.Equal(t, subresNon[0].NodeNames[0], "N/A")
 }
+
+func TestPartitions(t *testing.T) {
+	configs.MockSchedulerConfigByData([]byte(configMultiPartitions))
+	var err error
+	schedulerContext, err = scheduler.NewClusterContext(rmID, policyGroup)
+	assert.NilError(t, err, "Error when load clusterInfo from config")
+	assert.Equal(t, 2, len(schedulerContext.GetPartitionMapClone()))
+
+	// Check default partition
+	partitionName := "[" + rmID + "]default"
+	defaultPartition := schedulerContext.GetPartition(partitionName)
+	assert.Equal(t, 0, len(defaultPartition.GetApplications()))
+
+	// add a new app
+	app := newApplication("app-1", partitionName, "root.default", rmID)
+	err = defaultPartition.AddApplication(app)
+	assert.NilError(t, err, "Failed to add Application to Partition.")
+	assert.Equal(t, app.CurrentState(), objects.New.String())
+	assert.Equal(t, 1, len(defaultPartition.GetApplications()))
+
+	NewWebApp(schedulerContext, nil)
+
+	var req *http.Request
+	req, err = http.NewRequest("GET", "/ws/v1/partitions", strings.NewReader(""))
+	assert.NilError(t, err, "App Handler request failed")
+	resp := &MockResponseWriter{}
+	var partitionInfo []dao.PartitionInfo
+	getPartitions(resp, req)
+	err = json.Unmarshal(resp.outputBytes, &partitionInfo)
+	assert.NilError(t, err, "failed to unmarshal applications dao response from response body: %s", string(resp.outputBytes))
+	for _, part := range partitionInfo {
+		if part.Name == partitionName {
+			assert.Equal(t, part.Name, "[rm-123]default")
+			assert.Equal(t, part.NodeSortingPolicy, "fair")
+			assert.Equal(t, part.Applications["total"], 1)

Review comment:
       We should also test the count for the state the application is in.
   ```
   assert.Equal(t, part.Applications["new"], 1)
   ```
   can we also extend this to multiple apps, in different states and make sure they all count correctly and sum up to the total.

##########
File path: pkg/scheduler/partition.go
##########
@@ -264,6 +264,8 @@ func (pc *PartitionContext) isStopped() bool {
 func (pc *PartitionContext) handlePartitionEvent(event objects.ObjectEvent) error {
 	err := pc.stateMachine.Event(event.String(), pc.Name)
 	if err == nil {
+		pc.Lock()
+		defer pc.Unlock()

Review comment:
       Why do we need this locking?
   The event handling makes sure that the events are processed sequentially.
   The only way a partition can change state is via a config update. So there is no way that we can have multiple things happening at the same time here. There is also only one go routine that can make this change.
   
   Unless this has shown as an issue for data race I do not think we need it.

##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -24,12 +24,29 @@ type PartitionDAOInfo struct {
 	Queues        QueueDAOInfo      `json:"queues"`
 }
 
+type PartitionInfo struct {
+	Name                    string            `json:"name"`
+	Capacity                PartitionCapacity `json:"capacity"`
+	NodeSortingPolicy       string            `json:"nodeSortingPolicy"`
+	Applications            Applications      `json:"applications"`
+	State                   string            `json:"state"`
+	LastStateTransitionTime string            `json:"lastStateTransitionTime"`
+}
+
 type PartitionCapacity struct {
 	Capacity     string `json:"capacity"`
-	UsedCapacity string `json:"usedcapacity"`
+	UsedCapacity string `json:"usedCapacity"`
 }
 
 type NodeInfo struct {
 	NodeID     string `json:"nodeId"`
 	Capability string `json:"capability"`
 }
+
+type Applications struct {
+	Total     int `json:"total"`
+	Running   int `json:"running"`
+	Pending   int `json:"pending"`
+	Completed int `json:"completed"`
+	Failed    int `json:"failed"`

Review comment:
       I do not see this structure being used anywhere we should remove it.

##########
File path: pkg/scheduler/partition.go
##########
@@ -1208,3 +1210,17 @@ func (pc *PartitionContext) removeAllocationAsk(appID string, allocationKey stri
 		}
 	}
 }
+
+func (pc *PartitionContext) GetCurrentState() string {
+	return pc.stateMachine.Current()
+}
+
+func (pc *PartitionContext) GetStateTime() time.Time {
+	pc.RLock()
+	defer pc.RUnlock()
+	return pc.stateTime
+}
+
+func (pc *PartitionContext) GetNodeSortingPolicy() *policies.NodeSortingPolicy {
+	return pc.nodeSortingPolicy

Review comment:
       We need locking around this. The policy could be changed via the config update and we could thus be triggering a data race.
   I would also recommend that we return a String instead of the policy itself. We do not need more than the String 




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



[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r563835564



##########
File path: pkg/scheduler/partition.go
##########
@@ -262,6 +262,8 @@ func (pc *PartitionContext) isStopped() bool {
 // Handle the state event for the partition.
 // The state machine handles the locking.
 func (pc *PartitionContext) handlePartitionEvent(event objects.ObjectEvent) error {
+	pc.Lock()
+	defer pc.Unlock()

Review comment:
       Can we move this to line 269?
   I think we do not need to lock the partition when calling `pc.stateMachine.Event(event.String(), pc.Name)`




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



[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r567600463



##########
File path: pkg/scheduler/partition.go
##########
@@ -264,6 +264,8 @@ func (pc *PartitionContext) isStopped() bool {
 func (pc *PartitionContext) handlePartitionEvent(event objects.ObjectEvent) error {
 	err := pc.stateMachine.Event(event.String(), pc.Name)
 	if err == nil {
+		pc.Lock()
+		defer pc.Unlock()

Review comment:
       Good point, agree we can remove this. Thanks!




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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r509427489



##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -33,3 +41,11 @@ type NodeInfo struct {
 	NodeID     string `json:"nodeId"`

Review comment:
       We are using PartitionInfo (not PartitionDAOInfo) struct for this API. May be cleaning up PartitionDAOInfo struct can be taken up later in separate jira?




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



[GitHub] [incubator-yunikorn-core] codecov[bot] edited a comment on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-769434430


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=h1) Report
   > Merging [#216](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=desc) (7c2c271) into [master](https://codecov.io/gh/apache/incubator-yunikorn-core/commit/4e91f3f1e652d4eae1405ad733625c46199bd47c?el=desc) (4e91f3f) will **increase** coverage by `0.06%`.
   > The diff coverage is `72.91%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/graphs/tree.svg?width=650&height=150&src=pr&token=SB9NrIi3Hy)](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #216      +/-   ##
   ==========================================
   + Coverage   63.16%   63.23%   +0.06%     
   ==========================================
     Files          60       60              
     Lines        5242     5271      +29     
   ==========================================
   + Hits         3311     3333      +22     
   - Misses       1768     1774       +6     
   - Partials      163      164       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [pkg/scheduler/objects/application.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uLmdv) | `48.90% <0.00%> (ø)` | |
   | [pkg/scheduler/partition.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9wYXJ0aXRpb24uZ28=) | `62.94% <28.57%> (-0.40%)` | :arrow_down: |
   | [pkg/webservice/handlers.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3dlYnNlcnZpY2UvaGFuZGxlcnMuZ28=) | `56.73% <90.90%> (+2.59%)` | :arrow_up: |
   | [pkg/scheduler/objects/application\_state.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uX3N0YXRlLmdv) | `95.77% <100.00%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=footer). Last update [4e91f3f...7c2c271](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [incubator-yunikorn-core] codecov[bot] edited a comment on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-769434430


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=h1) Report
   > Merging [#216](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=desc) (7c2c271) into [master](https://codecov.io/gh/apache/incubator-yunikorn-core/commit/4e91f3f1e652d4eae1405ad733625c46199bd47c?el=desc) (4e91f3f) will **increase** coverage by `0.06%`.
   > The diff coverage is `72.91%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/graphs/tree.svg?width=650&height=150&src=pr&token=SB9NrIi3Hy)](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #216      +/-   ##
   ==========================================
   + Coverage   63.16%   63.23%   +0.06%     
   ==========================================
     Files          60       60              
     Lines        5242     5271      +29     
   ==========================================
   + Hits         3311     3333      +22     
   - Misses       1768     1774       +6     
   - Partials      163      164       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [pkg/scheduler/objects/application.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uLmdv) | `48.90% <0.00%> (ø)` | |
   | [pkg/scheduler/partition.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9wYXJ0aXRpb24uZ28=) | `62.94% <28.57%> (-0.40%)` | :arrow_down: |
   | [pkg/webservice/handlers.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3dlYnNlcnZpY2UvaGFuZGxlcnMuZ28=) | `56.73% <90.90%> (+2.59%)` | :arrow_up: |
   | [pkg/scheduler/objects/application\_state.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uX3N0YXRlLmdv) | `95.77% <100.00%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=footer). Last update [4e91f3f...7ac9f6f](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [incubator-yunikorn-core] TravisBuddy commented on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
TravisBuddy commented on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-708501255


   ## Travis tests have failed
   
   Hey @manirajv06,
   Please read the following log in order to understand the failure reason.
   It'll be awesome if you fix what's wrong and commit the changes.
   
   
   ### 1st Build
   
   <a href="https:&#x2F;&#x2F;travis-ci.org&#x2F;apache&#x2F;incubator-yunikorn-core&#x2F;jobs&#x2F;735738140">View build log</a>
   
   
   <details>
     <summary>
       <strong>
        curl -sSfL https:&#x2F;&#x2F;raw.githubusercontent.com&#x2F;golangci&#x2F;golangci-lint&#x2F;master&#x2F;install.sh | sh -s -- -b $(go env GOPATH)&#x2F;bin v1.22.2
       </strong>
     </summary>
   
   ```
   golangci/golangci-lint info checking GitHub for tag 'v1.22.2'
   golangci/golangci-lint info found version: 1.22.2 for v1.22.2/linux/amd64
   golangci/golangci-lint info installed /home/travis/gopath/bin/golangci-lint
   ```
   
   </details>
   
   
   <details>
     <summary>
       <strong>
        make lint
       </strong>
     </summary>
   
   ```
   running golangci-lint
   pkg/webservice/handlers.go:557:3: ifElseChain: rewrite if-else to switch statement (gocritic)
   		if app.IsRunning() {
   		^
   Makefile:45: recipe for target 'lint' failed
   make: *** [lint] Error 1
   ```
   
   </details>
   
   
   ###### TravisBuddy Request Identifier: c7643c70-0e36-11eb-964d-5d54b4986782
   


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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r567696290



##########
File path: pkg/scheduler/partition.go
##########
@@ -1208,3 +1210,17 @@ func (pc *PartitionContext) removeAllocationAsk(appID string, allocationKey stri
 		}
 	}
 }
+
+func (pc *PartitionContext) GetCurrentState() string {
+	return pc.stateMachine.Current()
+}
+
+func (pc *PartitionContext) GetStateTime() time.Time {
+	pc.RLock()
+	defer pc.RUnlock()
+	return pc.stateTime
+}
+
+func (pc *PartitionContext) GetNodeSortingPolicy() *policies.NodeSortingPolicy {
+	return pc.nodeSortingPolicy

Review comment:
       Taken care, But returning "SortingPolicy instead of string" approach is re-usable. Hope this is fine.




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



[GitHub] [incubator-yunikorn-core] codecov[bot] edited a comment on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-769434430


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=h1) Report
   > Merging [#216](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=desc) (7ac9f6f) into [master](https://codecov.io/gh/apache/incubator-yunikorn-core/commit/4e91f3f1e652d4eae1405ad733625c46199bd47c?el=desc) (4e91f3f) will **increase** coverage by `0.62%`.
   > The diff coverage is `85.36%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/graphs/tree.svg?width=650&height=150&src=pr&token=SB9NrIi3Hy)](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #216      +/-   ##
   ==========================================
   + Coverage   63.16%   63.78%   +0.62%     
   ==========================================
     Files          60       60              
     Lines        5242     5263      +21     
   ==========================================
   + Hits         3311     3357      +46     
   + Misses       1768     1749      -19     
   + Partials      163      157       -6     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [pkg/scheduler/partition.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9wYXJ0aXRpb24uZ28=) | `65.94% <ø> (+2.60%)` | :arrow_up: |
   | [pkg/scheduler/objects/application.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uLmdv) | `48.73% <33.33%> (-0.18%)` | :arrow_down: |
   | [pkg/webservice/handlers.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3dlYnNlcnZpY2UvaGFuZGxlcnMuZ28=) | `56.73% <90.90%> (+2.59%)` | :arrow_up: |
   | [pkg/scheduler/objects/application\_state.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uX3N0YXRlLmdv) | `95.77% <100.00%> (ø)` | |
   | [pkg/scheduler/context.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9jb250ZXh0Lmdv) | `6.28% <0.00%> (+0.01%)` | :arrow_up: |
   | [pkg/scheduler/objects/node.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL25vZGUuZ28=) | `81.98% <0.00%> (+0.16%)` | :arrow_up: |
   | [pkg/common/resources/resources.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL2NvbW1vbi9yZXNvdXJjZXMvcmVzb3VyY2VzLmdv) | `96.78% <0.00%> (+0.75%)` | :arrow_up: |
   | [pkg/common/configs/configvalidator.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZ3ZhbGlkYXRvci5nbw==) | `90.00% <0.00%> (+2.99%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=footer). Last update [4e91f3f...7ac9f6f](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r563348013



##########
File path: pkg/scheduler/partition.go
##########
@@ -1099,3 +1099,15 @@ func (pc *PartitionContext) removeAllocationAsk(appID string, allocationKey stri
 		}
 	}
 }
+
+func (pc *PartitionContext) GetCurrentState() string {
+	return pc.stateMachine.Current()
+}
+
+func (pc *PartitionContext) GetStateTime() time.Time {
+	return pc.stateTime
+}
+
+func (pc *PartitionContext) GetNodeSortingPolicy() *policies.NodeSortingPolicy {
+	return pc.nodeSortingPolicy
+}

Review comment:
       I think we need to add a read lock for `GetStateTime()`, and the write lock in `handlePartitionEvent()`. because `handlePartitionEvent()` and the REST API could be called at the same time.
   
   ```
   func (pc *PartitionContext) handlePartitionEvent(event objects.ObjectEvent) error {
   	err := pc.stateMachine.Event(event.String(), pc.Name)
   	if err == nil {
                   pc.Lock()
   		pc.stateTime = time.Now()
                   pc.Unlock()
   		return nil
   	}
   	// handle the same state transition not nil error (limit of fsm).
   	if err.Error() == "no transition" {
   		return nil
   	}
   	return err
   }
   ```
   
   `GetCurrentState()` and `GetNodeSortingPolicy()` should be safe because state machine Current() holds the lock already, and the nodeSortingPolicy only gets modified when updating the config, that is a swapping of the old object.




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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r563316625



##########
File path: pkg/webservice/handlers.go
##########
@@ -521,3 +521,39 @@ func updateConfiguration(conf string) (string, error) {
 	}
 	return "", fmt.Errorf("config plugin not found")
 }
+
+func getPartitions(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var partitionsInfo []*dao.PartitionInfo
+	lists := schedulerContext.GetPartitionMapClone()
+	for _, partitionContext := range lists {
+		partitionInfo := &dao.PartitionInfo{}
+		partitionInfo.Name = partitionContext.Name
+		partitionInfo.State = partitionContext.StateMachine.Current()

Review comment:
       Taken care.




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



[GitHub] [incubator-yunikorn-core] codecov[bot] edited a comment on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-769434430


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=h1) Report
   > Merging [#216](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=desc) (a3c65cd) into [master](https://codecov.io/gh/apache/incubator-yunikorn-core/commit/78bf4f68470987c110399fafa1671b3ffc85a358?el=desc) (78bf4f6) will **increase** coverage by `0.10%`.
   > The diff coverage is `78.12%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/graphs/tree.svg?width=650&height=150&src=pr&token=SB9NrIi3Hy)](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #216      +/-   ##
   ==========================================
   + Coverage   62.98%   63.09%   +0.10%     
   ==========================================
     Files          60       60              
     Lines        5203     5232      +29     
   ==========================================
   + Hits         3277     3301      +24     
   - Misses       1764     1769       +5     
     Partials      162      162              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [pkg/scheduler/partition.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9wYXJ0aXRpb24uZ28=) | `62.39% <28.57%> (-0.40%)` | :arrow_down: |
   | [pkg/webservice/handlers.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3dlYnNlcnZpY2UvaGFuZGxlcnMuZ28=) | `56.73% <90.90%> (+2.59%)` | :arrow_up: |
   | [pkg/scheduler/objects/application\_state.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uX3N0YXRlLmdv) | `94.82% <100.00%> (ø)` | |
   | [pkg/events/event\_publisher.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL2V2ZW50cy9ldmVudF9wdWJsaXNoZXIuZ28=) | `100.00% <0.00%> (+11.11%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=footer). Last update [4e91f3f...2a7863d](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r563348013



##########
File path: pkg/scheduler/partition.go
##########
@@ -1099,3 +1099,15 @@ func (pc *PartitionContext) removeAllocationAsk(appID string, allocationKey stri
 		}
 	}
 }
+
+func (pc *PartitionContext) GetCurrentState() string {
+	return pc.stateMachine.Current()
+}
+
+func (pc *PartitionContext) GetStateTime() time.Time {
+	return pc.stateTime
+}
+
+func (pc *PartitionContext) GetNodeSortingPolicy() *policies.NodeSortingPolicy {
+	return pc.nodeSortingPolicy
+}

Review comment:
       I think we need to add a read lock for `GetStateTime()`, and the write lock in `handlePartitionEvent()`. because `handlePartitionEvent()` and the REST API could be called at the same time.
   
   ```
   func (pc *PartitionContext) handlePartitionEvent(event objects.ObjectEvent) error {
   	err := pc.stateMachine.Event(event.String(), pc.Name)
   	if err == nil {
                   pc.Lock()
   		pc.stateTime = time.Now()
                   pc.Unlock()
   		return nil
   	}
   	// handle the same state transition not nil error (limit of fsm).
   	if err.Error() == "no transition" {
   		return nil
   	}
   	return err
   }
   ```
   
   `GetCurrentState()` and `GetNodeSortingPolicy()` should be safe because state machine Current() holds the lock already, and the nodeSortingPolicy only gets modified when updating the config, that is a swapping of the old object.

##########
File path: pkg/scheduler/partition.go
##########
@@ -262,6 +262,8 @@ func (pc *PartitionContext) isStopped() bool {
 // Handle the state event for the partition.
 // The state machine handles the locking.
 func (pc *PartitionContext) handlePartitionEvent(event objects.ObjectEvent) error {
+	pc.Lock()
+	defer pc.Unlock()

Review comment:
       Can we move this to line 269?
   I think we do not need to lock the partition when calling `pc.stateMachine.Event(event.String(), pc.Name)`




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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r566830358



##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -35,18 +35,10 @@ type PartitionInfo struct {
 
 type PartitionCapacity struct {
 	Capacity     string `json:"capacity"`
-	UsedCapacity string `json:"usedCapacity"`
+	UsedCapacity string `json:"usedcapacity"`

Review comment:
       Renaming as before because this change should not break the existing use cases. Once new API has been adopted completely, we can do the naming with similar convention.




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



[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r567535397



##########
File path: pkg/scheduler/partition.go
##########
@@ -1208,3 +1210,17 @@ func (pc *PartitionContext) removeAllocationAsk(appID string, allocationKey stri
 		}
 	}
 }
+
+func (pc *PartitionContext) GetCurrentState() string {
+	return pc.stateMachine.Current()
+}
+
+func (pc *PartitionContext) GetStateTime() time.Time {
+	pc.RLock()
+	defer pc.RUnlock()
+	return pc.stateTime
+}
+
+func (pc *PartitionContext) GetNodeSortingPolicy() *policies.NodeSortingPolicy {
+	return pc.nodeSortingPolicy

Review comment:
       Can we change this to:
   ```
   func (pc *PartitionContext) GetNodeSortingPolicy() String {
   	pc.RLock()
   	defer pc.RUnlock()
   	return pc.nodeSortingPolicy.PolicyType.String()
   }
   ```
   That is inline with the `getNodeIteratorForPolicy()` call.




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



[GitHub] [incubator-yunikorn-core] wilfred-s closed pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
wilfred-s closed pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216


   


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



[GitHub] [incubator-yunikorn-core] codecov[bot] edited a comment on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-769434430


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=h1) Report
   > Merging [#216](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=desc) (2a7863d) into [master](https://codecov.io/gh/apache/incubator-yunikorn-core/commit/4e91f3f1e652d4eae1405ad733625c46199bd47c?el=desc) (4e91f3f) will **increase** coverage by `0.06%`.
   > The diff coverage is `78.12%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/graphs/tree.svg?width=650&height=150&src=pr&token=SB9NrIi3Hy)](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #216      +/-   ##
   ==========================================
   + Coverage   63.16%   63.23%   +0.06%     
   ==========================================
     Files          60       60              
     Lines        5242     5271      +29     
   ==========================================
   + Hits         3311     3333      +22     
   - Misses       1768     1774       +6     
   - Partials      163      164       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [pkg/scheduler/partition.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9wYXJ0aXRpb24uZ28=) | `62.94% <28.57%> (-0.40%)` | :arrow_down: |
   | [pkg/webservice/handlers.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3dlYnNlcnZpY2UvaGFuZGxlcnMuZ28=) | `56.73% <90.90%> (+2.59%)` | :arrow_up: |
   | [pkg/scheduler/objects/application\_state.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uX3N0YXRlLmdv) | `95.77% <100.00%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=footer). Last update [4e91f3f...2a7863d](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [incubator-yunikorn-core] sunilgovind commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
sunilgovind commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r509433129



##########
File path: pkg/webservice/handlers.go
##########
@@ -527,3 +527,45 @@ func updateConfiguration(conf string) (string, error) {
 	}
 	return "", fmt.Errorf("config plugin not found")
 }
+
+func getPartitions(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	lists := gClusterInfo.ListPartitions()
+	for _, k := range lists {
+		partitionInfo := getPartition(k)
+
+		if err := json.NewEncoder(w).Encode(partitionInfo); err != nil {
+			http.Error(w, err.Error(), http.StatusInternalServerError)
+		}
+	}
+}
+
+func getPartition(name string) *dao.PartitionInfo {
+	partitionInfo := &dao.PartitionInfo{}
+	partitionContext := gClusterInfo.GetPartition(name)
+	partitionInfo.Name = partitionContext.Name
+	partitionInfo.State = partitionContext.StateMachine.Current()
+	partitionInfo.LastStateTransitionTime = partitionContext.StateTime.String()
+
+	capacityInfo := dao.PartitionCapacity{}
+	capacityInfo.Capacity = partitionContext.GetTotalPartitionResource().String()
+	capacityInfo.UsedCapacity = partitionContext.Root.GetAllocatedResource().String()
+	partitionInfo.Capacity = capacityInfo
+	partitionInfo.NodeSortingPolicy = partitionContext.GetNodeSortingPolicy().String()
+
+	applicationsInfo := dao.Applications{}
+	appList := partitionContext.GetApplications()
+	applicationsState := make(map[string]int)
+
+	for _, app := range appList {
+		applicationsState[app.GetApplicationState()]++
+	}
+	applicationsInfo.Running = applicationsState["Running"]

Review comment:
       could we optimize this code a bit. too much of impl related to comparison etc.

##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -24,12 +24,29 @@ type PartitionDAOInfo struct {
 	Queues        QueueDAOInfo      `json:"queues"`
 }
 
+type PartitionInfo struct {
+	Name                    string            `json:"name"`
+	Capacity                PartitionCapacity `json:"capacity"`

Review comment:
       This capacity will be the configured capacity one, correct?

##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -33,3 +41,11 @@ type NodeInfo struct {
 	NodeID     string `json:"nodeId"`

Review comment:
       Lets take this up separate, thats more cleaner and 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



[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r510586876



##########
File path: pkg/webservice/handlers.go
##########
@@ -527,3 +527,45 @@ func updateConfiguration(conf string) (string, error) {
 	}
 	return "", fmt.Errorf("config plugin not found")
 }
+
+func getPartitions(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	lists := gClusterInfo.ListPartitions()
+	for _, k := range lists {
+		partitionInfo := getPartition(k)
+
+		if err := json.NewEncoder(w).Encode(partitionInfo); err != nil {
+			http.Error(w, err.Error(), http.StatusInternalServerError)
+		}
+	}
+}
+
+func getPartition(name string) *dao.PartitionInfo {
+	partitionInfo := &dao.PartitionInfo{}
+	partitionContext := gClusterInfo.GetPartition(name)
+	partitionInfo.Name = partitionContext.Name
+	partitionInfo.State = partitionContext.StateMachine.Current()
+	partitionInfo.LastStateTransitionTime = partitionContext.StateTime.String()
+
+	capacityInfo := dao.PartitionCapacity{}
+	capacityInfo.Capacity = partitionContext.GetTotalPartitionResource().String()
+	capacityInfo.UsedCapacity = partitionContext.Root.GetAllocatedResource().String()
+	partitionInfo.Capacity = capacityInfo
+	partitionInfo.NodeSortingPolicy = partitionContext.GetNodeSortingPolicy().String()
+
+	applicationsInfo := dao.Applications{}
+	appList := partitionContext.GetApplications()
+	applicationsState := make(map[string]int)
+
+	for _, app := range appList {
+		applicationsState[app.GetApplicationState()]++
+	}
+	applicationsInfo.Running = applicationsState["Running"]

Review comment:
       hi @manirajv06  for this part. Can we avoid using hardcoded states for Applications struct.
   Maybe we can change it to
   
   ```
   type PartitionInfo struct {
      ...
      appCounts map[string]int
   }
   ```
   then we can simply aggregate the count of apps by their state. What do you think?
   
   
     




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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r510262333



##########
File path: pkg/webservice/handlers.go
##########
@@ -527,3 +527,45 @@ func updateConfiguration(conf string) (string, error) {
 	}
 	return "", fmt.Errorf("config plugin not found")
 }
+
+func getPartitions(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	lists := gClusterInfo.ListPartitions()
+	for _, k := range lists {
+		partitionInfo := getPartition(k)
+
+		if err := json.NewEncoder(w).Encode(partitionInfo); err != nil {
+			http.Error(w, err.Error(), http.StatusInternalServerError)
+		}
+	}
+}
+
+func getPartition(name string) *dao.PartitionInfo {
+	partitionInfo := &dao.PartitionInfo{}
+	partitionContext := gClusterInfo.GetPartition(name)
+	partitionInfo.Name = partitionContext.Name
+	partitionInfo.State = partitionContext.StateMachine.Current()
+	partitionInfo.LastStateTransitionTime = partitionContext.StateTime.String()
+
+	capacityInfo := dao.PartitionCapacity{}
+	capacityInfo.Capacity = partitionContext.GetTotalPartitionResource().String()
+	capacityInfo.UsedCapacity = partitionContext.Root.GetAllocatedResource().String()
+	partitionInfo.Capacity = capacityInfo
+	partitionInfo.NodeSortingPolicy = partitionContext.GetNodeSortingPolicy().String()
+
+	applicationsInfo := dao.Applications{}
+	appList := partitionContext.GetApplications()
+	applicationsState := make(map[string]int)
+
+	for _, app := range appList {
+		applicationsState[app.GetApplicationState()]++
+	}
+	applicationsInfo.Running = applicationsState["Running"]

Review comment:
       Can you please check the recent changes? It has been taken care.




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



[GitHub] [incubator-yunikorn-core] codecov[bot] edited a comment on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-769434430


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=h1) Report
   > Merging [#216](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=desc) (a3c65cd) into [master](https://codecov.io/gh/apache/incubator-yunikorn-core/commit/2bf2767642274b47d94c022d4bc4f36f733da617?el=desc) (2bf2767) will **increase** coverage by `0.02%`.
   > The diff coverage is `78.12%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/graphs/tree.svg?width=650&height=150&src=pr&token=SB9NrIi3Hy)](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #216      +/-   ##
   ==========================================
   + Coverage   63.06%   63.09%   +0.02%     
   ==========================================
     Files          60       60              
     Lines        5199     5232      +33     
   ==========================================
   + Hits         3279     3301      +22     
   - Misses       1759     1769      +10     
   - Partials      161      162       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [pkg/scheduler/partition.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9wYXJ0aXRpb24uZ28=) | `62.39% <28.57%> (-0.40%)` | :arrow_down: |
   | [pkg/webservice/handlers.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3dlYnNlcnZpY2UvaGFuZGxlcnMuZ28=) | `56.73% <90.90%> (+2.59%)` | :arrow_up: |
   | [pkg/scheduler/objects/application\_state.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uX3N0YXRlLmdv) | `94.82% <100.00%> (ø)` | |
   | [pkg/scheduler/objects/application.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uLmdv) | `48.69% <0.00%> (-0.37%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=footer). Last update [2bf2767...a3c65cd](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r566598781



##########
File path: pkg/scheduler/partition.go
##########
@@ -264,6 +264,8 @@ func (pc *PartitionContext) isStopped() bool {
 func (pc *PartitionContext) handlePartitionEvent(event objects.ObjectEvent) error {
 	err := pc.stateMachine.Event(event.String(), pc.Name)
 	if err == nil {
+		pc.Lock()
+		defer pc.Unlock()

Review comment:
       hi @wilfred-s  does this [my earlier comment ](https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r563348013)make sense to you?




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



[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r568589627



##########
File path: pkg/scheduler/partition.go
##########
@@ -1208,3 +1210,17 @@ func (pc *PartitionContext) removeAllocationAsk(appID string, allocationKey stri
 		}
 	}
 }
+
+func (pc *PartitionContext) GetCurrentState() string {
+	return pc.stateMachine.Current()
+}
+
+func (pc *PartitionContext) GetStateTime() time.Time {
+	pc.RLock()
+	defer pc.RUnlock()
+	return pc.stateTime
+}
+
+func (pc *PartitionContext) GetNodeSortingPolicy() *policies.NodeSortingPolicy {
+	return pc.nodeSortingPolicy

Review comment:
       with the reuse I am ok with 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



[GitHub] [incubator-yunikorn-core] codecov[bot] edited a comment on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-769434430


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=h1) Report
   > Merging [#216](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=desc) (a3c65cd) into [master](https://codecov.io/gh/apache/incubator-yunikorn-core/commit/4e91f3f1e652d4eae1405ad733625c46199bd47c?el=desc) (4e91f3f) will **decrease** coverage by `0.07%`.
   > The diff coverage is `78.12%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/graphs/tree.svg?width=650&height=150&src=pr&token=SB9NrIi3Hy)](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #216      +/-   ##
   ==========================================
   - Coverage   63.16%   63.09%   -0.08%     
   ==========================================
     Files          60       60              
     Lines        5242     5232      -10     
   ==========================================
   - Hits         3311     3301      -10     
   - Misses       1768     1769       +1     
   + Partials      163      162       -1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [pkg/scheduler/partition.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9wYXJ0aXRpb24uZ28=) | `62.39% <28.57%> (-0.95%)` | :arrow_down: |
   | [pkg/webservice/handlers.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3dlYnNlcnZpY2UvaGFuZGxlcnMuZ28=) | `56.73% <90.90%> (+2.59%)` | :arrow_up: |
   | [pkg/scheduler/objects/application\_state.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uX3N0YXRlLmdv) | `94.82% <100.00%> (-0.95%)` | :arrow_down: |
   | [pkg/scheduler/partition\_manager.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9wYXJ0aXRpb25fbWFuYWdlci5nbw==) | `16.00% <0.00%> (-4.00%)` | :arrow_down: |
   | [pkg/scheduler/objects/application.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uLmdv) | `48.69% <0.00%> (-0.22%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=footer). Last update [4e91f3f...7c2c271](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r567696397



##########
File path: pkg/scheduler/partition.go
##########
@@ -264,6 +264,8 @@ func (pc *PartitionContext) isStopped() bool {
 func (pc *PartitionContext) handlePartitionEvent(event objects.ObjectEvent) error {
 	err := pc.stateMachine.Event(event.String(), pc.Name)
 	if err == nil {
+		pc.Lock()
+		defer pc.Unlock()

Review comment:
       Taken care.




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



[GitHub] [incubator-yunikorn-core] sunilgovind commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
sunilgovind commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r534127334



##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -24,12 +24,29 @@ type PartitionDAOInfo struct {
 	Queues        QueueDAOInfo      `json:"queues"`
 }
 
+type PartitionInfo struct {
+	Name                    string            `json:"name"`
+	Capacity                PartitionCapacity `json:"capacity"`

Review comment:
       ok. thanks




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



[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r562401523



##########
File path: pkg/scheduler/partition.go
##########
@@ -54,13 +54,13 @@ type PartitionContext struct {
 	allocations            map[string]*objects.Allocation  // allocations
 	placementManager       *placement.AppPlacementManager  // placement manager for this partition
 	partitionManager       *partitionManager               // manager for this partition
-	stateMachine           *fsm.FSM                        // the state of the partition for scheduling
-	stateTime              time.Time                       // last time the state was updated (needed for cleanup)
+	StateMachine           *fsm.FSM                        // the state of the partition for scheduling
+	StateTime              time.Time                       // last time the state was updated (needed for cleanup)
 	isPreemptable          bool                            // can allocations be preempted
 	rules                  *[]configs.PlacementRule        // placement rules to be loaded by the scheduler
 	userGroupCache         *security.UserGroupCache        // user cache per partition
 	totalPartitionResource *resources.Resource             // Total node resources
-	nodeSortingPolicy      *policies.NodeSortingPolicy     // Global Node Sorting Policies
+	NodeSortingPolicy      *policies.NodeSortingPolicy     // Global Node Sorting Policies

Review comment:
       Do not expose this field as well. We can introduce a function to return the sorting policy instead.

##########
File path: pkg/webservice/handlers.go
##########
@@ -521,3 +521,39 @@ func updateConfiguration(conf string) (string, error) {
 	}
 	return "", fmt.Errorf("config plugin not found")
 }
+
+func getPartitions(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var partitionsInfo []*dao.PartitionInfo
+	lists := schedulerContext.GetPartitionMapClone()
+	for _, partitionContext := range lists {
+		partitionInfo := &dao.PartitionInfo{}
+		partitionInfo.Name = partitionContext.Name
+		partitionInfo.State = partitionContext.StateMachine.Current()

Review comment:
       It is not a good idea to expose `StateMachine` and `StateTime`, they are pretty internal objects.
   Instead, we can create a `GetCurrentState()` function that returns a string. Same for the stateTime.

##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -24,12 +24,29 @@ type PartitionDAOInfo struct {
 	Queues        QueueDAOInfo      `json:"queues"`
 }
 
+type PartitionInfo struct {
+	Name                    string            `json:"name"`
+	Capacity                PartitionCapacity `json:"capacity"`
+	NodeSortingPolicy       string            `json:"nodeSortingPolicy"`
+	Applications            Applications      `json:"applications"`
+	State                   string            `json:"state"`
+	LastStateTransitionTime string            `json:"lastStateTransitionTime"`
+}
+
 type PartitionCapacity struct {
 	Capacity     string `json:"capacity"`
-	UsedCapacity string `json:"usedcapacity"`
+	UsedCapacity string `json:"usedCapacity"`
 }
 
 type NodeInfo struct {
 	NodeID     string `json:"nodeId"`
 	Capability string `json:"capability"`
 }
+
+type Applications struct {
+	Total     int `json:"total"`
+	Running   int `json:"running"`
+	Pending   int `json:"pending"`
+	Completed int `json:"completed"`
+	Failed    int `json:"failed"`

Review comment:
       My earlier comment is to remove hardcoded state names.
   Because if we modified app state in the scheduler package, and most likely we are going to do so. It is going to break this. And I see we are doing some formatting like:
   
   ```
   applicationsInfo.Running = applicationsState["Running"]
   applicationsInfo.Pending = applicationsState["Waiting"] + applicationsState["Accepted"] + applicationsState["Starting"] + applicationsState["New"]
   applicationsInfo.Completed = applicationsState["Completed"]
   applicationsInfo.Failed = applicationsState["Killed"] + applicationsState["Rejected"]
   applicationsInfo.Total = applicationsInfo.Running + applicationsInfo.Pending + applicationsInfo.Completed + applicationsInfo.Failed
   ```
   I'd rather stay with the initial values. Because we are documenting all the app states carefully, see http://yunikorn.apache.org/docs/next/design/scheduler_object_states#application-state. We need to keep the REST API response align with the doc. So I think returning its original map plus a total number is enough. Does that make sense to you?

##########
File path: pkg/scheduler/partition.go
##########
@@ -54,13 +54,13 @@ type PartitionContext struct {
 	allocations            map[string]*objects.Allocation  // allocations
 	placementManager       *placement.AppPlacementManager  // placement manager for this partition
 	partitionManager       *partitionManager               // manager for this partition
-	stateMachine           *fsm.FSM                        // the state of the partition for scheduling
-	stateTime              time.Time                       // last time the state was updated (needed for cleanup)

Review comment:
       See my comments later for this, we should not expose the internal objects out of its package.




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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-771658199


   > LGTM, thanks for sticking with me on this long review.
   
   Thanks for detailed review @wilfred-s  @yangwwei 


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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-771658199


   > LGTM, thanks for sticking with me on this long review.
   
   Thanks for detailed review @wilfred-s  @yangwwei 


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



[GitHub] [incubator-yunikorn-core] wilfred-s closed pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
wilfred-s closed pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216


   


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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r563316625



##########
File path: pkg/webservice/handlers.go
##########
@@ -521,3 +521,39 @@ func updateConfiguration(conf string) (string, error) {
 	}
 	return "", fmt.Errorf("config plugin not found")
 }
+
+func getPartitions(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var partitionsInfo []*dao.PartitionInfo
+	lists := schedulerContext.GetPartitionMapClone()
+	for _, partitionContext := range lists {
+		partitionInfo := &dao.PartitionInfo{}
+		partitionInfo.Name = partitionContext.Name
+		partitionInfo.State = partitionContext.StateMachine.Current()

Review comment:
       Taken care.

##########
File path: pkg/scheduler/partition.go
##########
@@ -54,13 +54,13 @@ type PartitionContext struct {
 	allocations            map[string]*objects.Allocation  // allocations
 	placementManager       *placement.AppPlacementManager  // placement manager for this partition
 	partitionManager       *partitionManager               // manager for this partition
-	stateMachine           *fsm.FSM                        // the state of the partition for scheduling
-	stateTime              time.Time                       // last time the state was updated (needed for cleanup)
+	StateMachine           *fsm.FSM                        // the state of the partition for scheduling
+	StateTime              time.Time                       // last time the state was updated (needed for cleanup)
 	isPreemptable          bool                            // can allocations be preempted
 	rules                  *[]configs.PlacementRule        // placement rules to be loaded by the scheduler
 	userGroupCache         *security.UserGroupCache        // user cache per partition
 	totalPartitionResource *resources.Resource             // Total node resources
-	nodeSortingPolicy      *policies.NodeSortingPolicy     // Global Node Sorting Policies
+	NodeSortingPolicy      *policies.NodeSortingPolicy     // Global Node Sorting Policies

Review comment:
       Taken care.

##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -24,12 +24,29 @@ type PartitionDAOInfo struct {
 	Queues        QueueDAOInfo      `json:"queues"`
 }
 
+type PartitionInfo struct {
+	Name                    string            `json:"name"`
+	Capacity                PartitionCapacity `json:"capacity"`
+	NodeSortingPolicy       string            `json:"nodeSortingPolicy"`
+	Applications            Applications      `json:"applications"`
+	State                   string            `json:"state"`
+	LastStateTransitionTime string            `json:"lastStateTransitionTime"`
+}
+
 type PartitionCapacity struct {
 	Capacity     string `json:"capacity"`
-	UsedCapacity string `json:"usedcapacity"`
+	UsedCapacity string `json:"usedCapacity"`
 }
 
 type NodeInfo struct {
 	NodeID     string `json:"nodeId"`
 	Capability string `json:"capability"`
 }
+
+type Applications struct {
+	Total     int `json:"total"`
+	Running   int `json:"running"`
+	Pending   int `json:"pending"`
+	Completed int `json:"completed"`
+	Failed    int `json:"failed"`

Review comment:
       I was trying to sum up certain states under common state. However, showing up each state count is also detailed enough. 

##########
File path: pkg/scheduler/partition.go
##########
@@ -1099,3 +1099,15 @@ func (pc *PartitionContext) removeAllocationAsk(appID string, allocationKey stri
 		}
 	}
 }
+
+func (pc *PartitionContext) GetCurrentState() string {
+	return pc.stateMachine.Current()
+}
+
+func (pc *PartitionContext) GetStateTime() time.Time {
+	return pc.stateTime
+}
+
+func (pc *PartitionContext) GetNodeSortingPolicy() *policies.NodeSortingPolicy {
+	return pc.nodeSortingPolicy
+}

Review comment:
       Taken care.




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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r529201050



##########
File path: pkg/webservice/handlers.go
##########
@@ -527,3 +527,45 @@ func updateConfiguration(conf string) (string, error) {
 	}
 	return "", fmt.Errorf("config plugin not found")
 }
+
+func getPartitions(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	lists := gClusterInfo.ListPartitions()
+	for _, k := range lists {
+		partitionInfo := getPartition(k)
+
+		if err := json.NewEncoder(w).Encode(partitionInfo); err != nil {
+			http.Error(w, err.Error(), http.StatusInternalServerError)
+		}
+	}
+}
+
+func getPartition(name string) *dao.PartitionInfo {
+	partitionInfo := &dao.PartitionInfo{}
+	partitionContext := gClusterInfo.GetPartition(name)
+	partitionInfo.Name = partitionContext.Name
+	partitionInfo.State = partitionContext.StateMachine.Current()
+	partitionInfo.LastStateTransitionTime = partitionContext.StateTime.String()
+
+	capacityInfo := dao.PartitionCapacity{}
+	capacityInfo.Capacity = partitionContext.GetTotalPartitionResource().String()
+	capacityInfo.UsedCapacity = partitionContext.Root.GetAllocatedResource().String()
+	partitionInfo.Capacity = capacityInfo
+	partitionInfo.NodeSortingPolicy = partitionContext.GetNodeSortingPolicy().String()
+
+	applicationsInfo := dao.Applications{}
+	appList := partitionContext.GetApplications()
+	applicationsState := make(map[string]int)
+
+	for _, app := range appList {
+		applicationsState[app.GetApplicationState()]++
+	}
+	applicationsInfo.Running = applicationsState["Running"]

Review comment:
       hi @yangwwei  Sorry for the delay. 
   
   applicationsState in handlers#getPartition() is a map contains the state wise count. 
   
   applicationsInfo ( dao.Applications{} ) in handlers#getPartition() has been used to display the state wise count (after aggregating some app states. For ex, Killed and Rejected as Failed) in JSON output.
   
   I am not sure how defining appCounts map[string]int in PartitionInfo struct helps in this case as we are manipulating some app states. Please correct my understanding if needed.




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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r566828399



##########
File path: pkg/webservice/handlers_test.go
##########
@@ -607,3 +627,46 @@ func TestGetNodesUtilJSON(t *testing.T) {
 	assert.Equal(t, subresNon[0].NumOfNodes, int64(-1))
 	assert.Equal(t, subresNon[0].NodeNames[0], "N/A")
 }
+
+func TestPartitions(t *testing.T) {
+	configs.MockSchedulerConfigByData([]byte(configMultiPartitions))
+	var err error
+	schedulerContext, err = scheduler.NewClusterContext(rmID, policyGroup)
+	assert.NilError(t, err, "Error when load clusterInfo from config")
+	assert.Equal(t, 2, len(schedulerContext.GetPartitionMapClone()))
+
+	// Check default partition
+	partitionName := "[" + rmID + "]default"
+	defaultPartition := schedulerContext.GetPartition(partitionName)
+	assert.Equal(t, 0, len(defaultPartition.GetApplications()))
+
+	// add a new app
+	app := newApplication("app-1", partitionName, "root.default", rmID)
+	err = defaultPartition.AddApplication(app)
+	assert.NilError(t, err, "Failed to add Application to Partition.")
+	assert.Equal(t, app.CurrentState(), objects.New.String())
+	assert.Equal(t, 1, len(defaultPartition.GetApplications()))
+
+	NewWebApp(schedulerContext, nil)
+
+	var req *http.Request
+	req, err = http.NewRequest("GET", "/ws/v1/partitions", strings.NewReader(""))
+	assert.NilError(t, err, "App Handler request failed")
+	resp := &MockResponseWriter{}
+	var partitionInfo []dao.PartitionInfo
+	getPartitions(resp, req)
+	err = json.Unmarshal(resp.outputBytes, &partitionInfo)
+	assert.NilError(t, err, "failed to unmarshal applications dao response from response body: %s", string(resp.outputBytes))
+	for _, part := range partitionInfo {
+		if part.Name == partitionName {
+			assert.Equal(t, part.Name, "[rm-123]default")
+			assert.Equal(t, part.NodeSortingPolicy, "fair")
+			assert.Equal(t, part.Applications["total"], 1)

Review comment:
       Improved UT coverage.




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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-769101989


   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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r563817194



##########
File path: pkg/scheduler/partition.go
##########
@@ -1099,3 +1099,15 @@ func (pc *PartitionContext) removeAllocationAsk(appID string, allocationKey stri
 		}
 	}
 }
+
+func (pc *PartitionContext) GetCurrentState() string {
+	return pc.stateMachine.Current()
+}
+
+func (pc *PartitionContext) GetStateTime() time.Time {
+	return pc.stateTime
+}
+
+func (pc *PartitionContext) GetNodeSortingPolicy() *policies.NodeSortingPolicy {
+	return pc.nodeSortingPolicy
+}

Review comment:
       Taken care.




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



[GitHub] [incubator-yunikorn-core] codecov[bot] commented on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
codecov[bot] commented on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-769434430


   # [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=h1) Report
   > Merging [#216](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=desc) (ea92daf) into [master](https://codecov.io/gh/apache/incubator-yunikorn-core/commit/2bf2767642274b47d94c022d4bc4f36f733da617?el=desc) (2bf2767) will **increase** coverage by `0.02%`.
   > The diff coverage is `80.55%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/graphs/tree.svg?width=650&height=150&src=pr&token=SB9NrIi3Hy)](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #216      +/-   ##
   ==========================================
   + Coverage   63.06%   63.09%   +0.02%     
   ==========================================
     Files          60       60              
     Lines        5199     5232      +33     
   ==========================================
   + Hits         3279     3301      +22     
   - Misses       1759     1769      +10     
   - Partials      161      162       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [pkg/scheduler/partition.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9wYXJ0aXRpb24uZ28=) | `62.39% <28.57%> (-0.40%)` | :arrow_down: |
   | [pkg/webservice/handlers.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3dlYnNlcnZpY2UvaGFuZGxlcnMuZ28=) | `56.73% <90.90%> (+2.59%)` | :arrow_up: |
   | [pkg/scheduler/objects/application.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uLmdv) | `48.69% <100.00%> (-0.37%)` | :arrow_down: |
   | [pkg/scheduler/objects/application\_state.go](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216/diff?src=pr&el=tree#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uX3N0YXRlLmdv) | `94.82% <100.00%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=footer). Last update [2bf2767...a3c65cd](https://codecov.io/gh/apache/incubator-yunikorn-core/pull/216?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [incubator-yunikorn-core] TravisBuddy commented on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
TravisBuddy commented on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-713697083


   ## Travis tests have failed
   
   Hey @manirajv06,
   Please read the following log in order to understand the failure reason.
   It'll be awesome if you fix what's wrong and commit the changes.
   
   
   ### 1st Build
   
   <a href="https:&#x2F;&#x2F;travis-ci.org&#x2F;apache&#x2F;incubator-yunikorn-core&#x2F;jobs&#x2F;737773732">View build log</a>
   
   
   <details>
     <summary>
       <strong>
        curl -sSfL https:&#x2F;&#x2F;raw.githubusercontent.com&#x2F;golangci&#x2F;golangci-lint&#x2F;master&#x2F;install.sh | sh -s -- -b $(go env GOPATH)&#x2F;bin v1.22.2
       </strong>
     </summary>
   
   ```
   golangci/golangci-lint info checking GitHub for tag 'v1.22.2'
   golangci/golangci-lint info found version: 1.22.2 for v1.22.2/linux/amd64
   golangci/golangci-lint info installed /home/travis/gopath/bin/golangci-lint
   ```
   
   </details>
   
   
   <details>
     <summary>
       <strong>
        make lint
       </strong>
     </summary>
   
   ```
   running golangci-lint
   pkg/webservice/handlers.go:557:3: ifElseChain: rewrite if-else to switch statement (gocritic)
   		if app.IsRunning() {
   		^
   Makefile:45: recipe for target 'lint' failed
   make: *** [lint] Error 1
   ```
   
   </details>
   
   
   ###### TravisBuddy Request Identifier: eb7198a0-13b9-11eb-a355-c174999d48b2
   


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



[GitHub] [incubator-yunikorn-core] yangwwei commented on pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
yangwwei commented on pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#issuecomment-768547171


   hi @manirajv06  Looks almost good, but still need to fix the lint issue
   
   ```
   $ make lint
   running golangci-lint
   checking against commit sha 2bf276764227
   pkg/scheduler/objects/application.go:141:1: receiver name ai should be consistent with previous receiver name sa for Application (golint)
   func (ai *Application) IsCompleted() bool {
   ^
   pkg/scheduler/objects/application.go:145:1: receiver name ai should be consistent with previous receiver name sa for Application (golint)
   func (ai *Application) IsKilled() bool {
   ^
   pkg/scheduler/objects/application.go:149:1: receiver name ai should be consistent with previous receiver name sa for Application (golint)
   func (ai *Application) IsRejected() bool {
   ^
   Makefile:58: recipe for target 'lint' failed
   make: *** [lint] Error 1
   ```
   
   you can run `make lint` locally to ensure there is no more lint issues before updating the PR. Thanks


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



[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r567520478



##########
File path: pkg/scheduler/partition.go
##########
@@ -264,6 +264,8 @@ func (pc *PartitionContext) isStopped() bool {
 func (pc *PartitionContext) handlePartitionEvent(event objects.ObjectEvent) error {
 	err := pc.stateMachine.Event(event.String(), pc.Name)
 	if err == nil {
+		pc.Lock()
+		defer pc.Unlock()

Review comment:
       For Time the [documentation](https://golang.org/pkg/time/#Time) shows:
   > A Time value can be used by multiple goroutines simultaneously except that the methods GobDecode, UnmarshalBinary, UnmarshalJSON and UnmarshalText are not concurrency-safe.
   
   Which would mean that we do not need a lock.




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



[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r505079790



##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -24,6 +24,14 @@ type PartitionDAOInfo struct {
 	Queues        QueueDAOInfo      `json:"queues"`
 }
 
+type PartitionInfo struct {
+	Name              string            `json:"name"`
+	Capacity          PartitionCapacity `json:"capacity"`
+	IsPremptable      bool              `json:"isPremptable"`
+	NodeSortingPolicy string            `json:"nodeSortingPolicy"`
+	Applications      Applications      `json:"applications"`
+}
+
 type PartitionCapacity struct {
 	Capacity     string `json:"capacity"`
 	UsedCapacity string `json:"usedcapacity"`

Review comment:
       usedcapacity -> usedCapacity
   and another thing is the usedCapacity seems to be skipped in the API
   I think it can be retrieved via: `partition.Root.GetAllocatedResource()`

##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -33,3 +41,11 @@ type NodeInfo struct {
 	NodeID     string `json:"nodeId"`

Review comment:
       this is not needed
   maybe we can change this to display the number of nodes in this partition?

##########
File path: pkg/webservice/routes.go
##########
@@ -126,6 +126,14 @@ var webRoutes = routes{
 		getContainerHistory,
 	},
 
+	//
+	route{
+		"Partitions",
+		"GET",
+		"/ws/v1/partitions",
+		getPartitions,
+	},

Review comment:
       Are we going to support the sub-path `ws/v1/partitions/{partitionName}`?

##########
File path: pkg/webservice/handlers.go
##########
@@ -527,3 +527,47 @@ func updateConfiguration(conf string) (string, error) {
 	}
 	return "", fmt.Errorf("config plugin not found")
 }
+
+func getPartitions(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	lists := gClusterInfo.ListPartitions()
+	for _, k := range lists {
+		partitionInfo := getPartition(k)
+
+		if err := json.NewEncoder(w).Encode(partitionInfo); err != nil {
+			http.Error(w, err.Error(), http.StatusInternalServerError)
+		}
+	}
+}
+
+func getPartition(name string) *dao.PartitionInfo {
+	partitionInfo := &dao.PartitionInfo{}
+	partitionContext := gClusterInfo.GetPartition(name)
+	partitionInfo.Name = partitionContext.Name
+	capacityInfo := dao.PartitionCapacity{}
+	capacityInfo.Capacity = partitionContext.GetTotalPartitionResource().String()
+	partitionInfo.Capacity = capacityInfo
+	partitionInfo.IsPremptable = partitionContext.NeedPreemption()
+	partitionInfo.NodeSortingPolicy = partitionContext.GetNodeSortingPolicy().String()
+
+	applicationsInfo := dao.Applications{}
+	appList := partitionContext.GetApplications()
+	for _, app := range appList {
+		if app.IsRunning() {
+			applicationsInfo.Running++
+			applicationsInfo.Total++
+		} else if app.IsWaiting() || app.IsNew() || app.IsStarting() || app.IsAccepted() {
+			applicationsInfo.Pending++
+			applicationsInfo.Total++
+		} else if app.IsCompleted() {
+			applicationsInfo.Completed++
+			applicationsInfo.Total++
+		} else if app.IsKilled() || app.IsRejected() {
+			applicationsInfo.Failed++
+			applicationsInfo.Total++
+		}

Review comment:
       Instead of doing these `if-else`. Maybe we can simply do the math based on key-value
   for applications, we just need to have a `map[string]int`, then we can aggregate the app numbers by state

##########
File path: pkg/webservice/handlers.go
##########
@@ -527,3 +527,47 @@ func updateConfiguration(conf string) (string, error) {
 	}
 	return "", fmt.Errorf("config plugin not found")
 }
+
+func getPartitions(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	lists := gClusterInfo.ListPartitions()
+	for _, k := range lists {
+		partitionInfo := getPartition(k)
+
+		if err := json.NewEncoder(w).Encode(partitionInfo); err != nil {
+			http.Error(w, err.Error(), http.StatusInternalServerError)
+		}
+	}
+}
+
+func getPartition(name string) *dao.PartitionInfo {
+	partitionInfo := &dao.PartitionInfo{}
+	partitionContext := gClusterInfo.GetPartition(name)
+	partitionInfo.Name = partitionContext.Name
+	capacityInfo := dao.PartitionCapacity{}
+	capacityInfo.Capacity = partitionContext.GetTotalPartitionResource().String()
+	partitionInfo.Capacity = capacityInfo
+	partitionInfo.IsPremptable = partitionContext.NeedPreemption()

Review comment:
       can we remove this for now? 
   the preemption related code may need some refactoring, this property might be gone in future

##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -24,6 +24,14 @@ type PartitionDAOInfo struct {
 	Queues        QueueDAOInfo      `json:"queues"`
 }
 
+type PartitionInfo struct {
+	Name              string            `json:"name"`
+	Capacity          PartitionCapacity `json:"capacity"`
+	IsPremptable      bool              `json:"isPremptable"`

Review comment:
       remove this

##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -24,6 +24,14 @@ type PartitionDAOInfo struct {
 	Queues        QueueDAOInfo      `json:"queues"`
 }
 
+type PartitionInfo struct {

Review comment:
       Can we add some fields about the partition state?
   - state
   - lastStateTransitionTime




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



[GitHub] [incubator-yunikorn-core] manirajv06 commented on a change in pull request #216: YUNIKORN-414: Add partitions REST API to fetch list of partitions

Posted by GitBox <gi...@apache.org>.
manirajv06 commented on a change in pull request #216:
URL: https://github.com/apache/incubator-yunikorn-core/pull/216#discussion_r566830358



##########
File path: pkg/webservice/dao/partition_info.go
##########
@@ -35,18 +35,10 @@ type PartitionInfo struct {
 
 type PartitionCapacity struct {
 	Capacity     string `json:"capacity"`
-	UsedCapacity string `json:"usedCapacity"`
+	UsedCapacity string `json:"usedcapacity"`

Review comment:
       Renaming as before because 
   1. This change should not break the existing use cases. 
   2. No backward compatibility support
    
   Once new API has been adopted completely, we can do the naming with similar convention.




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