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/03/25 11:52:52 UTC

[GitHub] [yunikorn-core] lowc1012 opened a new pull request #395: [YUNIKORN-1107] Make health check occur in the background

lowc1012 opened a new pull request #395:
URL: https://github.com/apache/yunikorn-core/pull/395


   ### What is this PR for?
   Change "ws/v1/scheduler/healthcheck" API to fetch health check cache in scheduler context
   
   
   ### What type of PR is it?
   * [ ] - Bug Fix
   * [x] - Improvement
   * [ ] - Feature
   * [ ] - Documentation
   * [ ] - Hot Fix
   * [ ] - Refactoring
   
   ### Todos
   * [ ] - Task
   
   ### What is the Jira issue?
   [YUNIKORN-1107](https://issues.apache.org/jira/projects/YUNIKORN/issues/YUNIKORN-1107)
   
   ### 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-core] HuangTing-Yao commented on a change in pull request #395: [YUNIKORN-1107] Make health check occur in the background

Posted by GitBox <gi...@apache.org>.
HuangTing-Yao commented on a change in pull request #395:
URL: https://github.com/apache/yunikorn-core/pull/395#discussion_r835743979



##########
File path: pkg/webservice/handlers.go
##########
@@ -516,14 +516,23 @@ func isChecksumEqual(checksum string) bool {
 
 func checkHealthStatus(w http.ResponseWriter, r *http.Request) {
 	writeHeaders(w)
-	metrics := metrics2.GetSchedulerMetrics()
-	result := scheduler.GetSchedulerHealthStatus(metrics, schedulerContext)
-	if !result.Healthy {
-		log.Logger().Error("Scheduler is not healthy", zap.Any("health check values", result.HealthChecks))
-		buildJSONErrorResponse(w, "Scheduler is not healthy", http.StatusServiceUnavailable)
-	}
-	if err := json.NewEncoder(w).Encode(result); err != nil {
-		buildJSONErrorResponse(w, err.Error(), http.StatusInternalServerError)
+
+	// Fetch last healthCheck result
+	result := schedulerContext.GetLastHealthCheckResult()
+	if result != nil {
+		if !result.Healthy {
+			log.Logger().Error("Scheduler is not healthy", zap.Any("health check info", *result))
+			buildJSONErrorResponse(w, "Scheduler is not healthy", http.StatusServiceUnavailable)
+		} else {
+			log.Logger().Info("Scheduler is healthy", zap.Any("health check info", *result))
+			buildJSONErrorResponse(w, "Scheduler is healthy", http.StatusOK)

Review comment:
       When scheduler is healthy, we shouldn't log it with `ERROR` level.




-- 
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] pbacsko commented on a change in pull request #395: [YUNIKORN-1107] Make health check occur in the background

Posted by GitBox <gi...@apache.org>.
pbacsko commented on a change in pull request #395:
URL: https://github.com/apache/yunikorn-core/pull/395#discussion_r835663967



##########
File path: pkg/scheduler/health_checker.go
##########
@@ -20,14 +20,73 @@ package scheduler
 
 import (
 	"fmt"
+	"time"
 
+	"go.uber.org/zap"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/log"
 	"github.com/apache/incubator-yunikorn-core/pkg/scheduler/objects"
 
 	"github.com/apache/incubator-yunikorn-core/pkg/common/resources"
 	"github.com/apache/incubator-yunikorn-core/pkg/metrics"
 	"github.com/apache/incubator-yunikorn-core/pkg/webservice/dao"
 )
 
+const defaultPeriod = 30 * time.Second
+
+type HealthChecker struct {
+	period   time.Duration
+	stopChan chan struct{}
+}
+
+func NewHealthChecker() *HealthChecker {
+	return &HealthChecker{
+		period:   defaultPeriod,
+		stopChan: make(chan struct{}),
+	}
+}
+
+// start execute healthCheck service in the background,
+func (c *HealthChecker) start(schedulerContext *ClusterContext) {
+	go func() {
+		ticker := time.NewTicker(c.period)
+		for {
+			select {
+			case <-c.stopChan:
+				ticker.Stop()
+				return
+			case <-ticker.C:
+				schedulerMetrics := metrics.GetSchedulerMetrics()
+				result := GetSchedulerHealthStatus(schedulerMetrics, schedulerContext)
+				UpdateSchedulerHealthStatusCache(result, schedulerContext)
+				if !result.Healthy {
+					log.Logger().Error("Scheduler is not healthy",
+						zap.Any("health check values", result.HealthChecks))
+				} else {
+					log.Logger().Info("Scheduler is healthy",
+						zap.Any("health check values", result.HealthChecks))
+				}
+			}
+		}
+	}()
+}
+
+//nolint:unused
+func (c *HealthChecker) stop() {
+	c.stopChan <- struct{}{}
+	close(c.stopChan)
+}
+
+func UpdateSchedulerHealthStatusCache(latest dao.SchedulerHealthDAOInfo, schedulerContext *ClusterContext) {

Review comment:
       Is this method need to be exported? (capital "u")




-- 
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] HuangTing-Yao commented on pull request #395: [YUNIKORN-1107] Make health check occur in the background

Posted by GitBox <gi...@apache.org>.
HuangTing-Yao commented on pull request #395:
URL: https://github.com/apache/yunikorn-core/pull/395#issuecomment-1079674450


   The codecov check doesn't triggered, also we can add `TestHealthCheckerStartStop()` for `start()` and `stop()` to make sure the panic won't happened.


-- 
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] pbacsko commented on a change in pull request #395: [YUNIKORN-1107] Make health check occur in the background

Posted by GitBox <gi...@apache.org>.
pbacsko commented on a change in pull request #395:
URL: https://github.com/apache/yunikorn-core/pull/395#discussion_r835664168



##########
File path: pkg/scheduler/context.go
##########
@@ -918,3 +921,11 @@ func (cc *ClusterContext) SetRMInfo(rmID string, rmBuildInformation map[string]s
 		RMBuildInformation: buildInfo,
 	}
 }
+
+func (cc *ClusterContext) GetHealthCheckCache() *dao.SchedulerHealthDAOInfo {
+	return cc.healthCheckCache
+}
+
+func (cc *ClusterContext) SetHealthCheckCache(c *dao.SchedulerHealthDAOInfo) {
+	cc.healthCheckCache = c
+}

Review comment:
       These methods are invoked from different goroutines. Add the necessary `RLock()` and `RUnlock()` calls.




-- 
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] lowc1012 commented on a change in pull request #395: [YUNIKORN-1107] Make health check occur in the background

Posted by GitBox <gi...@apache.org>.
lowc1012 commented on a change in pull request #395:
URL: https://github.com/apache/yunikorn-core/pull/395#discussion_r835758297



##########
File path: pkg/webservice/handlers.go
##########
@@ -516,14 +516,23 @@ func isChecksumEqual(checksum string) bool {
 
 func checkHealthStatus(w http.ResponseWriter, r *http.Request) {
 	writeHeaders(w)
-	metrics := metrics2.GetSchedulerMetrics()
-	result := scheduler.GetSchedulerHealthStatus(metrics, schedulerContext)
-	if !result.Healthy {
-		log.Logger().Error("Scheduler is not healthy", zap.Any("health check values", result.HealthChecks))
-		buildJSONErrorResponse(w, "Scheduler is not healthy", http.StatusServiceUnavailable)
-	}
-	if err := json.NewEncoder(w).Encode(result); err != nil {
-		buildJSONErrorResponse(w, err.Error(), http.StatusInternalServerError)
+
+	// Fetch last healthCheck result
+	result := schedulerContext.GetLastHealthCheckResult()
+	if result != nil {
+		if !result.Healthy {
+			log.Logger().Error("Scheduler is not healthy", zap.Any("health check info", *result))
+			buildJSONErrorResponse(w, "Scheduler is not healthy", http.StatusServiceUnavailable)
+		} else {
+			log.Logger().Info("Scheduler is healthy", zap.Any("health check info", *result))
+			buildJSONErrorResponse(w, "Scheduler is healthy", http.StatusOK)

Review comment:
       done

##########
File path: pkg/webservice/handlers.go
##########
@@ -516,14 +516,23 @@ func isChecksumEqual(checksum string) bool {
 
 func checkHealthStatus(w http.ResponseWriter, r *http.Request) {
 	writeHeaders(w)
-	metrics := metrics2.GetSchedulerMetrics()
-	result := scheduler.GetSchedulerHealthStatus(metrics, schedulerContext)
-	if !result.Healthy {
-		log.Logger().Error("Scheduler is not healthy", zap.Any("health check values", result.HealthChecks))
-		buildJSONErrorResponse(w, "Scheduler is not healthy", http.StatusServiceUnavailable)
-	}
-	if err := json.NewEncoder(w).Encode(result); err != nil {
-		buildJSONErrorResponse(w, err.Error(), http.StatusInternalServerError)
+
+	// Fetch last healthCheck result
+	result := schedulerContext.GetLastHealthCheckResult()
+	if result != nil {
+		if !result.Healthy {
+			log.Logger().Error("Scheduler is not healthy", zap.Any("health check info", *result))
+			buildJSONErrorResponse(w, "Scheduler is not healthy", http.StatusServiceUnavailable)
+		} else {
+			log.Logger().Info("Scheduler is healthy", zap.Any("health check info", *result))
+			buildJSONErrorResponse(w, "Scheduler is healthy", http.StatusOK)

Review comment:
       Done. 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.

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 #395: [YUNIKORN-1107] Make health check occur in the background

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


   # [Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/395?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 [#395](https://codecov.io/gh/apache/yunikorn-core/pull/395?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5442622) into [master](https://codecov.io/gh/apache/yunikorn-core/commit/affcb27657a9794cf5ef3cacb9a38a59ba25a711?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (affcb27) will **increase** coverage by `0.01%`.
   > The diff coverage is `62.68%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #395      +/-   ##
   ==========================================
   + Coverage   69.54%   69.55%   +0.01%     
   ==========================================
     Files          67       67              
     Lines        9528     9587      +59     
   ==========================================
   + Hits         6626     6668      +42     
   - Misses       2657     2672      +15     
   - Partials      245      247       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/yunikorn-core/pull/395?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/common/configs/config.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZy5nbw==) | `76.66% <ø> (ø)` | |
   | [pkg/common/configs/configvalidator.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZ3ZhbGlkYXRvci5nbw==) | `86.97% <ø> (ø)` | |
   | [pkg/common/configs/configwatcher.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZ3dhdGNoZXIuZ28=) | `85.71% <ø> (ø)` | |
   | [pkg/common/resources/resources.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9yZXNvdXJjZXMvcmVzb3VyY2VzLmdv) | `97.68% <ø> (ø)` | |
   | [pkg/common/security/acl.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9zZWN1cml0eS9hY2wuZ28=) | `84.14% <ø> (ø)` | |
   | [pkg/common/security/usergroup.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9zZWN1cml0eS91c2VyZ3JvdXAuZ28=) | `85.71% <ø> (ø)` | |
   | [pkg/common/server.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9zZXJ2ZXIuZ28=) | `0.00% <ø> (ø)` | |
   | [pkg/common/utils.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi91dGlscy5nbw==) | `71.01% <ø> (ø)` | |
   | [pkg/events/event\_cache.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2V2ZW50cy9ldmVudF9jYWNoZS5nbw==) | `93.18% <ø> (ø)` | |
   | [pkg/events/event\_publisher.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2V2ZW50cy9ldmVudF9wdWJsaXNoZXIuZ28=) | `100.00% <ø> (ø)` | |
   | ... and [43 more](https://codecov.io/gh/apache/yunikorn-core/pull/395/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/395?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/395?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 [73d5528...5442622](https://codecov.io/gh/apache/yunikorn-core/pull/395?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] codecov[bot] edited a comment on pull request #395: [YUNIKORN-1107] Make health check occur in the background

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


   # [Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/395?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 [#395](https://codecov.io/gh/apache/yunikorn-core/pull/395?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5442622) into [master](https://codecov.io/gh/apache/yunikorn-core/commit/affcb27657a9794cf5ef3cacb9a38a59ba25a711?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (affcb27) will **increase** coverage by `0.01%`.
   > The diff coverage is `62.68%`.
   
   > :exclamation: Current head 5442622 differs from pull request most recent head 4119702. Consider uploading reports for the commit 4119702 to get more accurate results
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #395      +/-   ##
   ==========================================
   + Coverage   69.54%   69.55%   +0.01%     
   ==========================================
     Files          67       67              
     Lines        9528     9587      +59     
   ==========================================
   + Hits         6626     6668      +42     
   - Misses       2657     2672      +15     
   - Partials      245      247       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/yunikorn-core/pull/395?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/common/configs/config.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZy5nbw==) | `76.66% <ø> (ø)` | |
   | [pkg/common/configs/configvalidator.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZ3ZhbGlkYXRvci5nbw==) | `86.97% <ø> (ø)` | |
   | [pkg/common/configs/configwatcher.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZ3dhdGNoZXIuZ28=) | `85.71% <ø> (ø)` | |
   | [pkg/common/resources/resources.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9yZXNvdXJjZXMvcmVzb3VyY2VzLmdv) | `97.68% <ø> (ø)` | |
   | [pkg/common/security/acl.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9zZWN1cml0eS9hY2wuZ28=) | `84.14% <ø> (ø)` | |
   | [pkg/common/security/usergroup.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9zZWN1cml0eS91c2VyZ3JvdXAuZ28=) | `85.71% <ø> (ø)` | |
   | [pkg/common/server.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9zZXJ2ZXIuZ28=) | `0.00% <ø> (ø)` | |
   | [pkg/common/utils.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi91dGlscy5nbw==) | `71.01% <ø> (ø)` | |
   | [pkg/events/event\_cache.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2V2ZW50cy9ldmVudF9jYWNoZS5nbw==) | `93.18% <ø> (ø)` | |
   | [pkg/events/event\_publisher.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2V2ZW50cy9ldmVudF9wdWJsaXNoZXIuZ28=) | `100.00% <ø> (ø)` | |
   | ... and [43 more](https://codecov.io/gh/apache/yunikorn-core/pull/395/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/395?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/395?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 [73d5528...4119702](https://codecov.io/gh/apache/yunikorn-core/pull/395?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] pbacsko commented on a change in pull request #395: [YUNIKORN-1107] Make health check occur in the background

Posted by GitBox <gi...@apache.org>.
pbacsko commented on a change in pull request #395:
URL: https://github.com/apache/yunikorn-core/pull/395#discussion_r835663967



##########
File path: pkg/scheduler/health_checker.go
##########
@@ -20,14 +20,73 @@ package scheduler
 
 import (
 	"fmt"
+	"time"
 
+	"go.uber.org/zap"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/log"
 	"github.com/apache/incubator-yunikorn-core/pkg/scheduler/objects"
 
 	"github.com/apache/incubator-yunikorn-core/pkg/common/resources"
 	"github.com/apache/incubator-yunikorn-core/pkg/metrics"
 	"github.com/apache/incubator-yunikorn-core/pkg/webservice/dao"
 )
 
+const defaultPeriod = 30 * time.Second
+
+type HealthChecker struct {
+	period   time.Duration
+	stopChan chan struct{}
+}
+
+func NewHealthChecker() *HealthChecker {
+	return &HealthChecker{
+		period:   defaultPeriod,
+		stopChan: make(chan struct{}),
+	}
+}
+
+// start execute healthCheck service in the background,
+func (c *HealthChecker) start(schedulerContext *ClusterContext) {
+	go func() {
+		ticker := time.NewTicker(c.period)
+		for {
+			select {
+			case <-c.stopChan:
+				ticker.Stop()
+				return
+			case <-ticker.C:
+				schedulerMetrics := metrics.GetSchedulerMetrics()
+				result := GetSchedulerHealthStatus(schedulerMetrics, schedulerContext)
+				UpdateSchedulerHealthStatusCache(result, schedulerContext)
+				if !result.Healthy {
+					log.Logger().Error("Scheduler is not healthy",
+						zap.Any("health check values", result.HealthChecks))
+				} else {
+					log.Logger().Info("Scheduler is healthy",
+						zap.Any("health check values", result.HealthChecks))
+				}
+			}
+		}
+	}()
+}
+
+//nolint:unused
+func (c *HealthChecker) stop() {
+	c.stopChan <- struct{}{}
+	close(c.stopChan)
+}
+
+func UpdateSchedulerHealthStatusCache(latest dao.SchedulerHealthDAOInfo, schedulerContext *ClusterContext) {

Review comment:
       Is this method neet to be exported? (capital "u")




-- 
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] pbacsko commented on a change in pull request #395: [YUNIKORN-1107] Make health check occur in the background

Posted by GitBox <gi...@apache.org>.
pbacsko commented on a change in pull request #395:
URL: https://github.com/apache/yunikorn-core/pull/395#discussion_r835664659



##########
File path: pkg/scheduler/context.go
##########
@@ -54,6 +55,8 @@ type ClusterContext struct {
 	startTime time.Time
 
 	sync.RWMutex
+
+	healthCheckCache *dao.SchedulerHealthDAOInfo

Review comment:
       The name "lastHealthCheckResult" sounds more appropriate to me. "cache" makes me think that it's something more complicated, like a data structure or something.




-- 
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] lowc1012 commented on pull request #395: [YUNIKORN-1107] Make health check occur in the background

Posted by GitBox <gi...@apache.org>.
lowc1012 commented on pull request #395:
URL: https://github.com/apache/yunikorn-core/pull/395#issuecomment-1079620765


   @pbacsko Thanks for your review!
   The above problems have been solved. 


-- 
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] HuangTing-Yao merged pull request #395: [YUNIKORN-1107] Make health check occur in the background

Posted by GitBox <gi...@apache.org>.
HuangTing-Yao merged pull request #395:
URL: https://github.com/apache/yunikorn-core/pull/395


   


-- 
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] edited a comment on pull request #395: [YUNIKORN-1107] Make health check occur in the background

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


   # [Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/395?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 [#395](https://codecov.io/gh/apache/yunikorn-core/pull/395?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5442622) into [master](https://codecov.io/gh/apache/yunikorn-core/commit/affcb27657a9794cf5ef3cacb9a38a59ba25a711?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (affcb27) will **increase** coverage by `0.01%`.
   > The diff coverage is `62.68%`.
   
   > :exclamation: Current head 5442622 differs from pull request most recent head acfdd01. Consider uploading reports for the commit acfdd01 to get more accurate results
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #395      +/-   ##
   ==========================================
   + Coverage   69.54%   69.55%   +0.01%     
   ==========================================
     Files          67       67              
     Lines        9528     9587      +59     
   ==========================================
   + Hits         6626     6668      +42     
   - Misses       2657     2672      +15     
   - Partials      245      247       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/yunikorn-core/pull/395?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/common/configs/config.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZy5nbw==) | `76.66% <ø> (ø)` | |
   | [pkg/common/configs/configvalidator.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZ3ZhbGlkYXRvci5nbw==) | `86.97% <ø> (ø)` | |
   | [pkg/common/configs/configwatcher.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZ3dhdGNoZXIuZ28=) | `85.71% <ø> (ø)` | |
   | [pkg/common/resources/resources.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9yZXNvdXJjZXMvcmVzb3VyY2VzLmdv) | `97.68% <ø> (ø)` | |
   | [pkg/common/security/acl.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9zZWN1cml0eS9hY2wuZ28=) | `84.14% <ø> (ø)` | |
   | [pkg/common/security/usergroup.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9zZWN1cml0eS91c2VyZ3JvdXAuZ28=) | `85.71% <ø> (ø)` | |
   | [pkg/common/server.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9zZXJ2ZXIuZ28=) | `0.00% <ø> (ø)` | |
   | [pkg/common/utils.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi91dGlscy5nbw==) | `71.01% <ø> (ø)` | |
   | [pkg/events/event\_cache.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2V2ZW50cy9ldmVudF9jYWNoZS5nbw==) | `93.18% <ø> (ø)` | |
   | [pkg/events/event\_publisher.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2V2ZW50cy9ldmVudF9wdWJsaXNoZXIuZ28=) | `100.00% <ø> (ø)` | |
   | ... and [43 more](https://codecov.io/gh/apache/yunikorn-core/pull/395/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/395?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/395?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 [73d5528...acfdd01](https://codecov.io/gh/apache/yunikorn-core/pull/395?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] codecov[bot] edited a comment on pull request #395: [YUNIKORN-1107] Make health check occur in the background

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


   # [Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/395?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 [#395](https://codecov.io/gh/apache/yunikorn-core/pull/395?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (acfdd01) into [master](https://codecov.io/gh/apache/yunikorn-core/commit/affcb27657a9794cf5ef3cacb9a38a59ba25a711?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (affcb27) will **increase** coverage by `0.06%`.
   > The diff coverage is `70.83%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #395      +/-   ##
   ==========================================
   + Coverage   69.54%   69.61%   +0.06%     
   ==========================================
     Files          67       67              
     Lines        9528     9592      +64     
   ==========================================
   + Hits         6626     6677      +51     
   - Misses       2657     2670      +13     
     Partials      245      245              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/yunikorn-core/pull/395?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/common/configs/config.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZy5nbw==) | `76.66% <ø> (ø)` | |
   | [pkg/common/configs/configvalidator.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZ3ZhbGlkYXRvci5nbw==) | `86.97% <ø> (ø)` | |
   | [pkg/common/configs/configwatcher.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9jb25maWdzL2NvbmZpZ3dhdGNoZXIuZ28=) | `85.71% <ø> (ø)` | |
   | [pkg/common/resources/resources.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9yZXNvdXJjZXMvcmVzb3VyY2VzLmdv) | `97.68% <ø> (ø)` | |
   | [pkg/common/security/acl.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9zZWN1cml0eS9hY2wuZ28=) | `84.14% <ø> (ø)` | |
   | [pkg/common/security/usergroup.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9zZWN1cml0eS91c2VyZ3JvdXAuZ28=) | `85.71% <ø> (ø)` | |
   | [pkg/common/server.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi9zZXJ2ZXIuZ28=) | `0.00% <ø> (ø)` | |
   | [pkg/common/utils.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2NvbW1vbi91dGlscy5nbw==) | `71.01% <ø> (ø)` | |
   | [pkg/events/event\_cache.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2V2ZW50cy9ldmVudF9jYWNoZS5nbw==) | `93.18% <ø> (ø)` | |
   | [pkg/events/event\_publisher.go](https://codecov.io/gh/apache/yunikorn-core/pull/395/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-cGtnL2V2ZW50cy9ldmVudF9wdWJsaXNoZXIuZ28=) | `100.00% <ø> (ø)` | |
   | ... and [43 more](https://codecov.io/gh/apache/yunikorn-core/pull/395/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/395?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/395?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 [73d5528...acfdd01](https://codecov.io/gh/apache/yunikorn-core/pull/395?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