You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@yunikorn.apache.org by GitBox <gi...@apache.org> on 2022/07/15 21:21:32 UTC

[GitHub] [yunikorn-core] craigcondit opened a new pull request, #423: [YUNIKORN-1263] Add pending requests to applciation object in REST API

craigcondit opened a new pull request, #423:
URL: https://github.com/apache/yunikorn-core/pull/423

   ### What is this PR for?
   The REST API responses for application objects include allocations, but not requests. Pending requests (along with diagnostic information related to predicate failure) would be very useful.
   
   ### What type of PR is it?
   * [ ] - Bug Fix
   * [x] - Improvement
   * [ ] - Feature
   * [ ] - Documentation
   * [ ] - Hot Fix
   * [ ] - Refactoring
   
   ### Todos
   * [ ] - Task
   
   ### What is the Jira issue?
   https://issues.apache.org/jira/browse/YUNIKORN-1263
   
   ### How should this be tested?
   Added unit tests for log functionality, verified that new response objects appear as expected.
   
   ### Screenshots (if appropriate)
   
   Example addition to the application object:
   
   ```
   "requests": [
         {
           "allocationKey": "f137fab6-3cfa-4536-93f7-bfff92689382",
           "allocationTags": {
             "kubernetes.io/label/app": "pause",
             "kubernetes.io/label/applicationId": "pod-with-node-affinity-01",
             "kubernetes.io/label/queue": "root.sandbox",
             "kubernetes.io/meta/namespace": "default",
             "kubernetes.io/meta/podName": "with-node-affinity"
           },
           "requestTime": 1657917851845743911,
           "resource": {
             "memory": 500000000,
             "vcore": 100
           },
           "pendingCount": 1,
           "priority": "0",
           "queueName": "root.default",
           "requiredNodeId": null,
           "applicationId": "pod-with-node-affinity-01",
           "partition": "default",
           "placeholder": false,
           "placeholderTimeout": 0,
           "taskGroupName": "",
           "allocationLog": [
             {
               "message": "node(s) didn't match Pod's node affinity, node(s) had taint {node-role.kubernetes.io/master: }, that the pod didn't tolerate",
               "timestamp": 1657917910919138494,
               "count": 81
             },
             {
               "message": "node(s) had taint {node-role.kubernetes.io/master: }, that the pod didn't tolerate, node(s) didn't match Pod's node affinity",
               "timestamp": 1657917911531769848,
               "count": 504
             },
             {
               "message": "node(s) didn't match Pod's node affinity",
               "timestamp": 1657917911532076451,
               "count": 1170
             }
           ]
         }
       ],
   ```
   ### Questions:
   * [ ] - The licenses files need update.
   * [ ] - There is breaking changes for older versions.
   * [ ] - It needs documentation.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@yunikorn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [yunikorn-core] craigcondit commented on a diff in pull request #423: [YUNIKORN-1263] Add pending requests to applciation object in REST API

Posted by GitBox <gi...@apache.org>.
craigcondit commented on code in PR #423:
URL: https://github.com/apache/yunikorn-core/pull/423#discussion_r923937989


##########
pkg/scheduler/objects/allocation_ask.go:
##########
@@ -51,10 +52,17 @@ type AllocationAsk struct {
 	requiredNode     string
 	allowPreemption  bool
 	originator       bool
+	allocLog         map[string]*AllocationLogEntry
 
 	sync.RWMutex
 }
 
+type AllocationLogEntry struct {
+	Message   string
+	Timestamp time.Time

Review Comment:
   Renamed to `LastOccurrence`.



##########
pkg/scheduler/objects/allocation_ask.go:
##########
@@ -188,6 +197,48 @@ func (aa *AllocationAsk) GetAllocatedResource() *resources.Resource {
 	return aa.AllocatedResource
 }
 
+// LogSchedulingFailure keeps track of preconditions not being met for an allocation
+func (aa *AllocationAsk) LogAllocationFailure(message string, allocate bool) {

Review Comment:
   Fixed.



##########
pkg/scheduler/objects/allocation_ask.go:
##########
@@ -188,6 +197,48 @@ func (aa *AllocationAsk) GetAllocatedResource() *resources.Resource {
 	return aa.AllocatedResource
 }
 
+// LogSchedulingFailure keeps track of preconditions not being met for an allocation
+func (aa *AllocationAsk) LogAllocationFailure(message string, allocate bool) {
+	// for now, don't log reservations
+	if !allocate {
+		return
+	}
+
+	aa.Lock()
+	defer aa.Unlock()
+
+	entry, ok := aa.allocLog[message]
+	if !ok {
+		entry = &AllocationLogEntry{
+			Message: message,
+		}
+		aa.allocLog[message] = entry
+	}
+	entry.Timestamp = time.Now()
+	entry.Count++
+}
+
+// GetSchedulingLog returns a list of log entries corresponding to allocation preconditions not being met
+func (aa *AllocationAsk) GetAllocationLog() []*AllocationLogEntry {

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@yunikorn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [yunikorn-core] craigcondit commented on pull request #423: [YUNIKORN-1263] Add pending requests to applciation object in REST API

Posted by GitBox <gi...@apache.org>.
craigcondit commented on PR #423:
URL: https://github.com/apache/yunikorn-core/pull/423#issuecomment-1188429630

   > As a generic comment: we need to assess direct access to fields inside the allocationAsk. The Ask fields are almost all write only on create and read only but we have introduced a mutex for access. The Tags for instance are directly referenced as `allocation.Tags`. The `ask.AllocatedResource` is also directly referenced throughout the scheduling cycle.
   
   Agreed. I'll open a follow-up JIRA for the ask / allocation refactoring.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@yunikorn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [yunikorn-core] craigcondit commented on a diff in pull request #423: [YUNIKORN-1263] Add pending requests to applciation object in REST API

Posted by GitBox <gi...@apache.org>.
craigcondit commented on code in PR #423:
URL: https://github.com/apache/yunikorn-core/pull/423#discussion_r923938143


##########
pkg/scheduler/objects/allocation_ask.go:
##########
@@ -188,6 +197,48 @@ func (aa *AllocationAsk) GetAllocatedResource() *resources.Resource {
 	return aa.AllocatedResource
 }
 
+// LogSchedulingFailure keeps track of preconditions not being met for an allocation
+func (aa *AllocationAsk) LogAllocationFailure(message string, allocate bool) {
+	// for now, don't log reservations
+	if !allocate {
+		return
+	}
+
+	aa.Lock()
+	defer aa.Unlock()
+
+	entry, ok := aa.allocLog[message]
+	if !ok {
+		entry = &AllocationLogEntry{
+			Message: message,
+		}
+		aa.allocLog[message] = entry
+	}
+	entry.Timestamp = time.Now()
+	entry.Count++
+}
+
+// GetSchedulingLog returns a list of log entries corresponding to allocation preconditions not being met
+func (aa *AllocationAsk) GetAllocationLog() []*AllocationLogEntry {
+	aa.RLock()
+	defer aa.RUnlock()
+
+	res := make([]*AllocationLogEntry, len(aa.allocLog))
+	i := 0
+	for _, log := range aa.allocLog {
+		res[i] = &AllocationLogEntry{
+			Message:   log.Message,
+			Timestamp: log.Timestamp,
+			Count:     log.Count,
+		}
+		i++
+	}
+	sort.SliceStable(res, func(i, j int) bool {
+		return res[i].Timestamp.Before(res[j].Timestamp)

Review Comment:
   Sort moved to web handler.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@yunikorn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [yunikorn-core] wilfred-s commented on a diff in pull request #423: [YUNIKORN-1263] Add pending requests to applciation object in REST API

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on code in PR #423:
URL: https://github.com/apache/yunikorn-core/pull/423#discussion_r922914933


##########
pkg/scheduler/objects/allocation_ask.go:
##########
@@ -51,10 +52,17 @@ type AllocationAsk struct {
 	requiredNode     string
 	allowPreemption  bool
 	originator       bool
+	allocLog         map[string]*AllocationLogEntry
 
 	sync.RWMutex
 }
 
+type AllocationLogEntry struct {
+	Message   string
+	Timestamp time.Time

Review Comment:
   Timestamp is ambiguous. If count > 1 the timestamp is linked to the latest occurrence. That either needs to show in the name or we need two: first  occurrence and most recent occurrence timestamp.



##########
pkg/webservice/handlers.go:
##########
@@ -224,6 +226,42 @@ func getApplicationJSON(app *objects.Application) *dao.ApplicationDAOInfo {
 		}
 		allocationInfo = append(allocationInfo, allocInfo)
 	}
+	for _, req := range requests {

Review Comment:
   Move all this processing into its own function. Do the sorting in there as per comment earlier around unlocked sorting.



##########
pkg/scheduler/objects/allocation_ask.go:
##########
@@ -188,6 +197,48 @@ func (aa *AllocationAsk) GetAllocatedResource() *resources.Resource {
 	return aa.AllocatedResource
 }
 
+// LogSchedulingFailure keeps track of preconditions not being met for an allocation
+func (aa *AllocationAsk) LogAllocationFailure(message string, allocate bool) {
+	// for now, don't log reservations
+	if !allocate {
+		return
+	}
+
+	aa.Lock()
+	defer aa.Unlock()
+
+	entry, ok := aa.allocLog[message]
+	if !ok {
+		entry = &AllocationLogEntry{
+			Message: message,
+		}
+		aa.allocLog[message] = entry
+	}
+	entry.Timestamp = time.Now()
+	entry.Count++
+}
+
+// GetSchedulingLog returns a list of log entries corresponding to allocation preconditions not being met
+func (aa *AllocationAsk) GetAllocationLog() []*AllocationLogEntry {

Review Comment:
   Comment differs from real function name



##########
pkg/webservice/dao/allocation_ask_info.go:
##########
@@ -0,0 +1,42 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package dao
+
+type AllocationAskLogDAOInfo struct {
+	Message   string `json:"message"`
+	Timestamp int64  `json:"timestamp"`
+	Count     int32  `json:"count"`
+}
+
+type AllocationAskDAOInfo struct {
+	AllocationKey      string                    `json:"allocationKey"`
+	AllocationTags     map[string]string         `json:"allocationTags"`
+	RequestTime        int64                     `json:"requestTime"`
+	ResourcePerAlloc   map[string]int64          `json:"resource"`
+	PendingCount       int32                     `json:"pendingCount"`
+	Priority           string                    `json:"priority"`
+	QueueName          string                    `json:"queueName"`
+	RequiredNodeID     *string                   `json:"requiredNodeId"`

Review Comment:
   Why handle this one different than other optional values in objects?
   A simple empty string would suffice, unless we go back and do this for all optional values.



##########
pkg/scheduler/objects/allocation_ask.go:
##########
@@ -188,6 +197,48 @@ func (aa *AllocationAsk) GetAllocatedResource() *resources.Resource {
 	return aa.AllocatedResource
 }
 
+// LogSchedulingFailure keeps track of preconditions not being met for an allocation
+func (aa *AllocationAsk) LogAllocationFailure(message string, allocate bool) {
+	// for now, don't log reservations
+	if !allocate {
+		return
+	}
+
+	aa.Lock()
+	defer aa.Unlock()
+
+	entry, ok := aa.allocLog[message]
+	if !ok {
+		entry = &AllocationLogEntry{
+			Message: message,
+		}
+		aa.allocLog[message] = entry
+	}
+	entry.Timestamp = time.Now()
+	entry.Count++
+}
+
+// GetSchedulingLog returns a list of log entries corresponding to allocation preconditions not being met
+func (aa *AllocationAsk) GetAllocationLog() []*AllocationLogEntry {
+	aa.RLock()
+	defer aa.RUnlock()
+
+	res := make([]*AllocationLogEntry, len(aa.allocLog))
+	i := 0
+	for _, log := range aa.allocLog {
+		res[i] = &AllocationLogEntry{
+			Message:   log.Message,
+			Timestamp: log.Timestamp,
+			Count:     log.Count,
+		}
+		i++
+	}
+	sort.SliceStable(res, func(i, j int) bool {
+		return res[i].Timestamp.Before(res[j].Timestamp)

Review Comment:
   Can we sort outside of the lock? This could even be postponed to the web handling when it is all done out of band.



##########
pkg/scheduler/objects/allocation_ask.go:
##########
@@ -188,6 +197,48 @@ func (aa *AllocationAsk) GetAllocatedResource() *resources.Resource {
 	return aa.AllocatedResource
 }
 
+// LogSchedulingFailure keeps track of preconditions not being met for an allocation
+func (aa *AllocationAsk) LogAllocationFailure(message string, allocate bool) {

Review Comment:
   Comment differs from real function name



##########
pkg/webservice/handlers.go:
##########
@@ -224,6 +226,42 @@ func getApplicationJSON(app *objects.Application) *dao.ApplicationDAOInfo {
 		}
 		allocationInfo = append(allocationInfo, allocInfo)
 	}
+	for _, req := range requests {
+		count := req.GetPendingAskRepeat()
+		if count > 0 {
+			nodeID := req.GetRequiredNode()
+			nodePtr := &nodeID
+			if nodeID == "" {
+				nodePtr = nil
+			}

Review Comment:
   See comment above on other optional values.



##########
pkg/scheduler/objects/application.go:
##########
@@ -1387,6 +1386,17 @@ func (sa *Application) getPlaceholderAllocations() []*Allocation {
 	return allocations
 }
 
+// get a copy of all requests of the application
+func (sa *Application) GetAllRequests() []*AllocationAsk {
+	sa.RLock()
+	defer sa.RUnlock()
+	var requests []*AllocationAsk
+	for _, req := range sa.requests {
+		requests = append(requests, req)
+	}
+	return requests
+}

Review Comment:
   Code duplication we already have an unlocked `getAllReqeusts()` This locked version should be a wrapper around the unlocked version. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@yunikorn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [yunikorn-core] codecov[bot] commented on pull request #423: [YUNIKORN-1263] Add pending requests to applciation object in REST API

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

   # [Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/423?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#423](https://codecov.io/gh/apache/yunikorn-core/pull/423?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8ea9c59) into [master](https://codecov.io/gh/apache/yunikorn-core/commit/85c2fce19b0971c1513218d78b51008f41353bda?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (85c2fce) will **increase** coverage by `0.10%`.
   > The diff coverage is `76.84%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #423      +/-   ##
   ==========================================
   + Coverage   69.43%   69.54%   +0.10%     
   ==========================================
     Files          68       68              
     Lines        9856     9938      +82     
   ==========================================
   + Hits         6844     6911      +67     
   - Misses       2771     2785      +14     
   - Partials      241      242       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/yunikorn-core/pull/423?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/scheduler/objects/application.go](https://codecov.io/gh/apache/yunikorn-core/pull/423/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FwcGxpY2F0aW9uLmdv) | `56.45% <0.00%> (-0.33%)` | :arrow_down: |
   | [pkg/scheduler/objects/node.go](https://codecov.io/gh/apache/yunikorn-core/pull/423/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL25vZGUuZ28=) | `82.68% <57.14%> (-0.17%)` | :arrow_down: |
   | [pkg/webservice/handlers.go](https://codecov.io/gh/apache/yunikorn-core/pull/423/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3dlYnNlcnZpY2UvaGFuZGxlcnMuZ28=) | `69.81% <81.08%> (+0.62%)` | :arrow_up: |
   | [pkg/scheduler/objects/allocation\_ask.go](https://codecov.io/gh/apache/yunikorn-core/pull/423/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3NjaGVkdWxlci9vYmplY3RzL2FsbG9jYXRpb25fYXNrLmdv) | `96.96% <100.00%> (+1.13%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/423?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/423?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [85c2fce...8ea9c59](https://codecov.io/gh/apache/yunikorn-core/pull/423?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@yunikorn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [yunikorn-core] wilfred-s closed pull request #423: [YUNIKORN-1263] Add pending requests to applciation object in REST API

Posted by GitBox <gi...@apache.org>.
wilfred-s closed pull request #423: [YUNIKORN-1263] Add pending requests to applciation object in REST API
URL: https://github.com/apache/yunikorn-core/pull/423


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@yunikorn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [yunikorn-core] craigcondit commented on a diff in pull request #423: [YUNIKORN-1263] Add pending requests to applciation object in REST API

Posted by GitBox <gi...@apache.org>.
craigcondit commented on code in PR #423:
URL: https://github.com/apache/yunikorn-core/pull/423#discussion_r923938340


##########
pkg/scheduler/objects/application.go:
##########
@@ -1387,6 +1386,17 @@ func (sa *Application) getPlaceholderAllocations() []*Allocation {
 	return allocations
 }
 
+// get a copy of all requests of the application
+func (sa *Application) GetAllRequests() []*AllocationAsk {
+	sa.RLock()
+	defer sa.RUnlock()
+	var requests []*AllocationAsk
+	for _, req := range sa.requests {
+		requests = append(requests, req)
+	}
+	return requests
+}

Review Comment:
   Fixed, also renamed to `getAllRequestsInternal()` to match `addAllocationInternal()`.



##########
pkg/webservice/dao/allocation_ask_info.go:
##########
@@ -0,0 +1,42 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package dao
+
+type AllocationAskLogDAOInfo struct {
+	Message   string `json:"message"`
+	Timestamp int64  `json:"timestamp"`
+	Count     int32  `json:"count"`
+}
+
+type AllocationAskDAOInfo struct {
+	AllocationKey      string                    `json:"allocationKey"`
+	AllocationTags     map[string]string         `json:"allocationTags"`
+	RequestTime        int64                     `json:"requestTime"`
+	ResourcePerAlloc   map[string]int64          `json:"resource"`
+	PendingCount       int32                     `json:"pendingCount"`
+	Priority           string                    `json:"priority"`
+	QueueName          string                    `json:"queueName"`
+	RequiredNodeID     *string                   `json:"requiredNodeId"`

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@yunikorn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [yunikorn-core] craigcondit commented on a diff in pull request #423: [YUNIKORN-1263] Add pending requests to applciation object in REST API

Posted by GitBox <gi...@apache.org>.
craigcondit commented on code in PR #423:
URL: https://github.com/apache/yunikorn-core/pull/423#discussion_r923938440


##########
pkg/webservice/handlers.go:
##########
@@ -224,6 +226,42 @@ func getApplicationJSON(app *objects.Application) *dao.ApplicationDAOInfo {
 		}
 		allocationInfo = append(allocationInfo, allocInfo)
 	}
+	for _, req := range requests {

Review Comment:
   Done.



##########
pkg/webservice/handlers.go:
##########
@@ -224,6 +226,42 @@ func getApplicationJSON(app *objects.Application) *dao.ApplicationDAOInfo {
 		}
 		allocationInfo = append(allocationInfo, allocInfo)
 	}
+	for _, req := range requests {
+		count := req.GetPendingAskRepeat()
+		if count > 0 {
+			nodeID := req.GetRequiredNode()
+			nodePtr := &nodeID
+			if nodeID == "" {
+				nodePtr = nil
+			}

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@yunikorn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org