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/05/03 12:55:44 UTC

[GitHub] [yunikorn-k8shim] pbacsko opened a new pull request, #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()

pbacsko opened a new pull request, #415:
URL: https://github.com/apache/yunikorn-k8shim/pull/415

   ### What is this PR for?
   The code inside `AppManagementService.recoverApps()` does not call `Context.AddTask()`. Which means that the generated `Application` object will be invalid since it won't include any `Task` object in the `taskMap`.
   
   ### What type of PR is it?
   * [x] - Bug Fix
   * [ ] - Improvement
   * [ ] - Feature
   * [ ] - Documentation
   * [ ] - Hot Fix
   * [ ] - Refactoring
   
   ### Todos
   * [ ] - Task
   
   ### What is the Jira issue?
   https://issues.apache.org/jira/browse/YUNIKORN-1194
   
   ### How should this be tested?
   
   ### Screenshots (if appropriate)
   
   ### 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-k8shim] pbacsko closed pull request #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()

Posted by GitBox <gi...@apache.org>.
pbacsko closed pull request #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()
URL: https://github.com/apache/yunikorn-k8shim/pull/415


-- 
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-k8shim] pbacsko commented on a diff in pull request #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()

Posted by GitBox <gi...@apache.org>.
pbacsko commented on code in PR #415:
URL: https://github.com/apache/yunikorn-k8shim/pull/415#discussion_r863763987


##########
pkg/cache/context.go:
##########
@@ -595,7 +595,9 @@ func (ctx *Context) getNamespaceObject(namespace string) *v1.Namespace {
 
 func (ctx *Context) AddApplication(request *interfaces.AddApplicationRequest) interfaces.ManagedApp {
 	log.Logger().Debug("AddApplication", zap.Any("Request", request))
-	if app := ctx.GetApplication(request.Metadata.ApplicationID); app != nil {
+	appID := request.Metadata.ApplicationID
+	if app := ctx.GetApplication(request.Metadata.ApplicationID); app != nil && !request.ForceUpdate {
+		log.Logger().Debug("Application already in the context", zap.String("appID", appID))

Review Comment:
   This part will likely change after YUNIKORN-1186. With proper coordination, the replacement won't be needed. But in this case, the patch stands on its own.
   
   Alternative approach: solve YUNIKORN-1186 first (which probably isn't good because the recovery path will fail), or solve both JIRAs in a single commit (too much change?).



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

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

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


[GitHub] [yunikorn-k8shim] pbacsko commented on a diff in pull request #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()

Posted by GitBox <gi...@apache.org>.
pbacsko commented on code in PR #415:
URL: https://github.com/apache/yunikorn-k8shim/pull/415#discussion_r863762575


##########
pkg/appmgmt/general/general.go:
##########
@@ -113,7 +113,7 @@ func (os *Manager) getTaskMetadata(pod *v1.Pod) (interfaces.TaskMetadata, bool)
 	}, true
 }
 
-func (os *Manager) getAppMetadata(pod *v1.Pod, recovery bool) (interfaces.ApplicationMetadata, bool) {
+func (os *Manager) GetAppMetadata(pod *v1.Pod, recovery bool) (interfaces.ApplicationMetadata, bool) {

Review Comment:
   Changed the visibility to be consistent with `GetTaskMetaData()`, but not strictly necessary.



-- 
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-k8shim] pbacsko commented on a diff in pull request #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()

Posted by GitBox <gi...@apache.org>.
pbacsko commented on code in PR #415:
URL: https://github.com/apache/yunikorn-k8shim/pull/415#discussion_r863763987


##########
pkg/cache/context.go:
##########
@@ -595,7 +595,9 @@ func (ctx *Context) getNamespaceObject(namespace string) *v1.Namespace {
 
 func (ctx *Context) AddApplication(request *interfaces.AddApplicationRequest) interfaces.ManagedApp {
 	log.Logger().Debug("AddApplication", zap.Any("Request", request))
-	if app := ctx.GetApplication(request.Metadata.ApplicationID); app != nil {
+	appID := request.Metadata.ApplicationID
+	if app := ctx.GetApplication(request.Metadata.ApplicationID); app != nil && !request.ForceUpdate {
+		log.Logger().Debug("Application already in the context", zap.String("appID", appID))

Review Comment:
   This part will likely change after YUNIKORN-1186. With proper coordination, the replacement won't be needed. But in this case, the patch stands on its own.
   
   Alternative approach: solve YUNIKORN-1186 first (which probably isn't good because the recovery path fail), or solve both JIRAs in a single commit (too much change?).



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

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

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


[GitHub] [yunikorn-k8shim] pbacsko commented on a diff in pull request #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()

Posted by GitBox <gi...@apache.org>.
pbacsko commented on code in PR #415:
URL: https://github.com/apache/yunikorn-k8shim/pull/415#discussion_r863761838


##########
pkg/appmgmt/appmgmt.go:
##########
@@ -37,24 +37,28 @@ type AppManagementService struct {
 	apiProvider client.APIProvider
 	amProtocol  interfaces.ApplicationManagementProtocol
 	managers    []interfaces.AppManager
+	generalMgr  *general.Manager
 }
 
 func NewAMService(amProtocol interfaces.ApplicationManagementProtocol,
 	apiProvider client.APIProvider) *AppManagementService {
+	generalMgr := general.NewManager(amProtocol, apiProvider)
 	appManager := &AppManagementService{
 		amProtocol:  amProtocol,
 		apiProvider: apiProvider,
 		managers:    make([]interfaces.AppManager, 0),
+		generalMgr:  generalMgr,

Review Comment:
   Needed for `GetTaskMetaData()`.



-- 
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-k8shim] codecov[bot] commented on pull request #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()

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

   # [Codecov](https://codecov.io/gh/apache/yunikorn-k8shim/pull/415?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 [#415](https://codecov.io/gh/apache/yunikorn-k8shim/pull/415?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1a6d376) into [master](https://codecov.io/gh/apache/yunikorn-k8shim/commit/32dc52653c03880e860d85a0389c582a67e9f833?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (32dc526) will **decrease** coverage by `0.04%`.
   > The diff coverage is `60.78%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #415      +/-   ##
   ==========================================
   - Coverage   65.43%   65.38%   -0.05%     
   ==========================================
     Files          40       40              
     Lines        6379     6408      +29     
   ==========================================
   + Hits         4174     4190      +16     
   - Misses       2043     2054      +11     
   - Partials      162      164       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/yunikorn-k8shim/pull/415?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/appmgmt/appmgmt\_recovery.go](https://codecov.io/gh/apache/yunikorn-k8shim/pull/415/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-cGtnL2FwcG1nbXQvYXBwbWdtdF9yZWNvdmVyeS5nbw==) | `63.88% <46.15%> (-4.04%)` | :arrow_down: |
   | [pkg/appmgmt/appmgmt.go](https://codecov.io/gh/apache/yunikorn-k8shim/pull/415/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-cGtnL2FwcG1nbXQvYXBwbWdtdC5nbw==) | `30.37% <50.00%> (+1.43%)` | :arrow_up: |
   | [pkg/cache/context.go](https://codecov.io/gh/apache/yunikorn-k8shim/pull/415/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-cGtnL2NhY2hlL2NvbnRleHQuZ28=) | `45.81% <62.50%> (-0.05%)` | :arrow_down: |
   | [pkg/appmgmt/general/general.go](https://codecov.io/gh/apache/yunikorn-k8shim/pull/415/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-cGtnL2FwcG1nbXQvZ2VuZXJhbC9nZW5lcmFsLmdv) | `72.96% <92.30%> (+0.20%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/yunikorn-k8shim/pull/415?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-k8shim/pull/415?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 [32dc526...1a6d376](https://codecov.io/gh/apache/yunikorn-k8shim/pull/415?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-k8shim] pbacsko commented on a diff in pull request #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()

Posted by GitBox <gi...@apache.org>.
pbacsko commented on code in PR #415:
URL: https://github.com/apache/yunikorn-k8shim/pull/415#discussion_r863763987


##########
pkg/cache/context.go:
##########
@@ -595,7 +595,9 @@ func (ctx *Context) getNamespaceObject(namespace string) *v1.Namespace {
 
 func (ctx *Context) AddApplication(request *interfaces.AddApplicationRequest) interfaces.ManagedApp {
 	log.Logger().Debug("AddApplication", zap.Any("Request", request))
-	if app := ctx.GetApplication(request.Metadata.ApplicationID); app != nil {
+	appID := request.Metadata.ApplicationID
+	if app := ctx.GetApplication(request.Metadata.ApplicationID); app != nil && !request.ForceUpdate {
+		log.Logger().Debug("Application already in the context", zap.String("appID", appID))

Review Comment:
   This part will likely change after YUNIKORN-1186. With proper coordination, the replacement won't be needed. But in this case, the patch stands on its own.
   
   Alternative approach: solve YUNIKORN-1186 first.



-- 
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-k8shim] pbacsko commented on a diff in pull request #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()

Posted by GitBox <gi...@apache.org>.
pbacsko commented on code in PR #415:
URL: https://github.com/apache/yunikorn-k8shim/pull/415#discussion_r863765079


##########
pkg/appmgmt/general/general.go:
##########
@@ -90,7 +90,7 @@ func (os *Manager) Stop() {
 	// noop
 }
 
-func (os *Manager) getTaskMetadata(pod *v1.Pod) (interfaces.TaskMetadata, bool) {
+func (os *Manager) GetTaskMetadata(pod *v1.Pod) (interfaces.TaskMetadata, bool) {

Review Comment:
   I tried moving both methods to `utils`, but ended up with circular deps. So leaving them here.



-- 
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-k8shim] pbacsko commented on a diff in pull request #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()

Posted by GitBox <gi...@apache.org>.
pbacsko commented on code in PR #415:
URL: https://github.com/apache/yunikorn-k8shim/pull/415#discussion_r863763987


##########
pkg/cache/context.go:
##########
@@ -595,7 +595,9 @@ func (ctx *Context) getNamespaceObject(namespace string) *v1.Namespace {
 
 func (ctx *Context) AddApplication(request *interfaces.AddApplicationRequest) interfaces.ManagedApp {
 	log.Logger().Debug("AddApplication", zap.Any("Request", request))
-	if app := ctx.GetApplication(request.Metadata.ApplicationID); app != nil {
+	appID := request.Metadata.ApplicationID
+	if app := ctx.GetApplication(request.Metadata.ApplicationID); app != nil && !request.ForceUpdate {
+		log.Logger().Debug("Application already in the context", zap.String("appID", appID))

Review Comment:
   This part will likely change after YUNIKORN-1186. With proper coordination, the replacement won't be needed. But in this case, the patch stands on its own.



-- 
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-k8shim] pbacsko commented on pull request #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()

Posted by GitBox <gi...@apache.org>.
pbacsko commented on PR #415:
URL: https://github.com/apache/yunikorn-k8shim/pull/415#issuecomment-1117236329

   Update: I'm working on YUNIKORN-1186 and as far as I can see, fixing that also fixes this. So no need to review it right now, I'll post further updates whether it's necessary or not.


-- 
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-k8shim] wilfred-s commented on a diff in pull request #415: [POC][YUNIKORN-1194] Task objects are not created when applications are restored in recoverApps()

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


##########
pkg/appmgmt/appmgmt_recovery.go:
##########
@@ -47,27 +47,54 @@ func (svc *AppManagementService) recoverApps() (map[string]interfaces.ManagedApp
 	recoveringApps := make(map[string]interfaces.ManagedApp)
 	for _, mgr := range svc.managers {
 		if m, ok := mgr.(interfaces.Recoverable); ok {
-			appMetas, err := m.ListApplications()
+			appMetas, pods, err := m.ListApplications()
 			if err != nil {
 				log.Logger().Error("failed to list apps", zap.Error(err))
 				return recoveringApps, err
 			}
 
-			// trigger recovery of the apps
-			// this is simply submit the app again
+			// Collect the ManagedApp objects and also call AddApplication()
+			// to register the app
+			log.Logger().Debug("Adding retrieved applications")
+			apps := make([]interfaces.ManagedApp, 0)
 			for _, appMeta := range appMetas {
+				log.Logger().Debug("Adding application", zap.String("appId", appMeta.ApplicationID))
 				if app := svc.amProtocol.AddApplication(
 					&interfaces.AddApplicationRequest{
-						Metadata: appMeta,
+						Metadata:    appMeta,
+						ForceUpdate: true,
 					}); app != nil {
+
 					recoveringApps[app.GetApplicationID()] = app
-					if err := app.TriggerAppRecovery(); err != nil {
-						log.Logger().Error("failed to recover app", zap.Error(err))
-						return recoveringApps, fmt.Errorf("failed to recover app %s, reason: %v",
-							app.GetApplicationID(), err)
+					apps = append(apps, app)
+				}
+			}
+
+			// Collect the tasks and call AddTask()
+			log.Logger().Debug("Adding tasks from retrieved pods")
+			for _, pod := range pods {
+				if taskMeta, ok := svc.generalMgr.GetTaskMetadata(pod); ok {
+					if app := svc.amProtocol.GetApplication(taskMeta.ApplicationID); app != nil {
+						if _, taskErr := app.GetTask(string(pod.UID)); taskErr != nil {
+							log.Logger().Debug("Adding task", zap.Any("taskMeta", taskMeta.TaskID))
+							svc.amProtocol.AddTask(&interfaces.AddTaskRequest{
+								Metadata: taskMeta,
+							})
+						}
 					}
 				}
 			}
+
+			// trigger recovery of the apps
+			// this is simply submit the app again
+			for _, app := range apps {
+				log.Logger().Info("Triggering recovery for app", zap.String("appId", app.GetApplicationID()))
+				if err := app.TriggerAppRecovery(); err != nil {
+					log.Logger().Error("failed to recover app", zap.Error(err))
+					return recoveringApps, fmt.Errorf("failed to recover app %s, reason: %v",

Review Comment:
   This will break out if we fail. Do we need to continue recovery and collect all errors or do we fail completely when this happens.
   Does this leave YuniKorn in a broken state?



##########
pkg/appmgmt/general/general.go:
##########
@@ -90,7 +90,7 @@ func (os *Manager) Stop() {
 	// noop
 }
 
-func (os *Manager) getTaskMetadata(pod *v1.Pod) (interfaces.TaskMetadata, bool) {
+func (os *Manager) GetTaskMetadata(pod *v1.Pod) (interfaces.TaskMetadata, bool) {

Review Comment:
   That is caused by the `interfaces.TaskMetadata`.
   We need to leave it here and look at this whole area from YUNIKORN-1195



##########
pkg/cache/context.go:
##########
@@ -595,7 +595,9 @@ func (ctx *Context) getNamespaceObject(namespace string) *v1.Namespace {
 
 func (ctx *Context) AddApplication(request *interfaces.AddApplicationRequest) interfaces.ManagedApp {
 	log.Logger().Debug("AddApplication", zap.Any("Request", request))
-	if app := ctx.GetApplication(request.Metadata.ApplicationID); app != nil {
+	appID := request.Metadata.ApplicationID
+	if app := ctx.GetApplication(request.Metadata.ApplicationID); app != nil && !request.ForceUpdate {
+		log.Logger().Debug("Application already in the context", zap.String("appID", appID))

Review Comment:
   keep separate, solve this first then modify in follow up for YUNIKORN-1186



##########
pkg/appmgmt/general/general.go:
##########
@@ -324,32 +324,34 @@ func (os *Manager) deletePod(obj interface{}) {
 		zap.String("podName", pod.Name),
 		zap.String("podUID", string(pod.UID)))
 
-	if taskMeta, ok := os.getTaskMetadata(pod); ok {
+	if taskMeta, ok := os.GetTaskMetadata(pod); ok {
 		if app := os.amProtocol.GetApplication(taskMeta.ApplicationID); app != nil {
 			os.amProtocol.NotifyTaskComplete(taskMeta.ApplicationID, taskMeta.TaskID)
 		}
 	}
 }
 
-func (os *Manager) ListApplications() (map[string]interfaces.ApplicationMetadata, error) {
+func (os *Manager) ListApplications() (map[string]interfaces.ApplicationMetadata, []*v1.Pod, error) {
 	log.Logger().Info("Retrieving pod list")
 	// list all pods on this cluster
 	appPods, err := os.apiProvider.GetAPIs().PodInformer.Lister().List(labels.NewSelector())
 	if err != nil {
-		return nil, err
+		return nil, nil, err
 	}
 	log.Logger().Info("Pod list retrieved from api server", zap.Int("nr of pods", len(appPods)))
 	// get existing apps
 	existingApps := make(map[string]interfaces.ApplicationMetadata)
 	podsRecovered := 0
 	podsWithoutMetaData := 0
+	pods := make([]*v1.Pod, 0)
 	for _, pod := range appPods {
 		log.Logger().Debug("Looking at pod for recovery candidates", zap.String("podNamespace", pod.Namespace), zap.String("podName", pod.Name))
 		// general filter passes, and pod is assigned
 		// this means the pod is already scheduled by scheduler for an existing app
 		if utils.GeneralPodFilter(pod) && utils.IsAssignedPod(pod) {
-			if meta, ok := os.getAppMetadata(pod, true); ok {
+			if meta, ok := os.GetAppMetadata(pod, true); ok {
 				podsRecovered++
+				pods = append(pods, pod)

Review Comment:
   Should we track this as part of the app meta data instead of returning a separate list of pods?



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