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/11/08 15:53:22 UTC

[GitHub] [yunikorn-core] craigcondit opened a new pull request, #446: [YUNIKORN-1366] Read core configuration from shim

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

   ### What is this PR for?
   Reads the core scheduler configuration from the shim instead of the filesystem. Removes config watching and update logic as this is now the responsibility of the shim. Allows log configuration to be updated dynamically.
   
   ### 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-1366
   
   ### How should this be tested?
   Updated unit tests.
   
   ### 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] craigcondit commented on pull request #446: [YUNIKORN-1366] Read core configuration from shim

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

   > One last change to make sure the default config works
   
   @wilfred-s I don't see a specific code comment here. What do we need to 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-core] pbacsko commented on a diff in pull request #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/common/configs/configs.go:
##########
@@ -53,3 +52,51 @@ func (ctx *SchedulerConfigContext) Get(policyGroup string) *SchedulerConfig {
 	defer ctx.lock.RUnlock()
 	return ctx.configs[policyGroup]
 }
+
+func AddConfigMapCallback(id string, callback func()) {
+	configMapLock.Lock()
+	defer configMapLock.Unlock()
+	configMapCallbacks[id] = callback
+}
+
+func RemoveConfigMapCallback(id string) {
+	configMapLock.Lock()
+	defer configMapLock.Unlock()
+	delete(configMapCallbacks, id)
+}

Review Comment:
   2 things:
   * Comments are missing
   * When are these functions called? I can see only one reference from the unit test.



-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/log/logger_test.go:
##########
@@ -62,8 +63,9 @@ func TestIsDebugEnabled(t *testing.T) {
 		Encoding: "console",
 	}
 	var err error
-	logger, err = zapConfig.Build()
+	logger, err := zapConfig.Build()
 	assert.NilError(t, err, "debug level logger create failed")
+	InitializeLogger(logger, &zapConfig)

Review Comment:
   Per offline discussions, reverting all logging changes.



##########
pkg/log/logger.go:
##########
@@ -23,50 +23,63 @@ import (
 	"fmt"
 	"reflect"
 	"sync"
+	"sync/atomic"
 
 	"go.uber.org/zap"
 	"go.uber.org/zap/zapcore"
 )
 
 var once sync.Once
-var logger *zap.Logger
-var config *zap.Config
+var holderValue atomic.Value
+
+type loggerHolder struct {
+	logger     *zap.Logger
+	zapConfigs *zap.Config
+}
 
 func Logger() *zap.Logger {
-	once.Do(func() {
-		if logger = zap.L(); isNopLogger(logger) {
-			// If a global logger is not found, this could be either scheduler-core
-			// is running as a deployment mode, or running with another non-go code
-			// shim. In this case, we need to create our own logger.
-			// TODO support log options when a global logger is not there
-			config = createConfig()
-			var err error
-			logger, err = config.Build()
-			// this should really not happen so just write to stdout and set a Nop logger
-			if err != nil {
-				fmt.Printf("Logging disabled, logger init failed with error: %v\n", err)
-				logger = zap.NewNop()
-			}
+	once.Do(initLogger)
+	return holderValue.Load().(loggerHolder).logger
+}
+
+func GetConfig() *zap.Config {
+	once.Do(initLogger)
+	return holderValue.Load().(loggerHolder).zapConfigs
+}
+
+func initLogger() {
+	var logger *zap.Logger
+	var config *zap.Config
+
+	if logger = zap.L(); isNopLogger(logger) {
+		// If a global logger is not found, this could be either scheduler-core
+		// is running as a deployment mode, or running with another non-go code
+		// shim. In this case, we need to create our own logger.
+		config = createConfig()
+		var err error
+		logger, err = config.Build()
+		// this should really not happen so just write to stdout and set a Nop logger
+		if err != nil {
+			fmt.Printf("Logging disabled, logger init failed with error: %v\n", err)
+			logger = zap.NewNop()
 		}
+	}
+	holderValue.Store(loggerHolder{
+		logger:     logger,
+		zapConfigs: config,
 	})
-
-	return logger
 }
 
 func InitializeLogger(log *zap.Logger, zapConfig *zap.Config) {
-	once.Do(func() {
-		logger = log
-		config = zapConfig
-		logger.Info("Using an already initialized logger")
+	once.Do(initLogger)

Review Comment:
   Per offline discussions, reverting all logging changes.



-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/common/configs/configs.go:
##########
@@ -53,3 +52,51 @@ func (ctx *SchedulerConfigContext) Get(policyGroup string) *SchedulerConfig {
 	defer ctx.lock.RUnlock()
 	return ctx.configs[policyGroup]
 }
+
+func AddConfigMapCallback(id string, callback func()) {
+	configMapLock.Lock()
+	defer configMapLock.Unlock()
+	configMapCallbacks[id] = callback
+}
+
+func RemoveConfigMapCallback(id string) {
+	configMapLock.Lock()
+	defer configMapLock.Unlock()
+	delete(configMapCallbacks, id)
+}

Review Comment:
   I'll add the comments.
   
   Currently, nothing calls these, as we don't have any config vars specific to the core. See comments on https://issues.apache.org/jira/browse/YUNIKORN-1213 for the first likely candidate (controlling health check interval).
   
   The idea is that code which can respond to config changes will get a callback when new values are loaded, and can update internal state.



-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

Posted by GitBox <gi...@apache.org>.
wilfred-s closed pull request #446: [YUNIKORN-1366] Read core configuration from shim
URL: https://github.com/apache/yunikorn-core/pull/446


-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/entrypoint/entrypoint.go:
##########
@@ -55,6 +55,10 @@ func StartAllServicesWithLogger(logger *zap.Logger, zapConfigs *zap.Config) *Ser
 	return StartAllServices()
 }
 
+func InitializeLogger(logger *zap.Logger, zapConfigs *zap.Config) {
+	log.InitializeLogger(logger, zapConfigs)
+}
+

Review Comment:
   It gets called in the updated shim code when the configuration changes, so it's needed 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-core] craigcondit commented on a diff in pull request #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/scheduler/context.go:
##########
@@ -313,14 +333,14 @@ func (cc *ClusterContext) UpdateSchedulerConfig(conf *configs.SchedulerConfig) e
 // Locked version of the configuration update called outside of event system.
 // Updates the current config via the config loader.
 // Used in test only, normal updates use the internal call and the webservice must use the UpdateSchedulerConfig

Review Comment:
   Will do.



-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/log/logger_test.go:
##########
@@ -119,9 +121,9 @@ func TestInitializeLogger(t *testing.T) {
 
 	InitializeLogger(localLogger, &zapConfig)
 	assert.Equal(t, Logger(), localLogger)
-	// second initialization should not do anything
+	// second initialization should update
 	InitializeLogger(localLogger2, &zapConfig)
-	assert.Equal(t, Logger(), localLogger)
+	assert.Equal(t, Logger(), localLogger2)

Review Comment:
   Removed logging changes.



-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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

   > Have you looked at codecov complaints? Anything to do to make it green?
   
   That's due to all the code which is being deleted. The number of lines is actually increasing, but the overall delta is slightly negative. All the new code is being tested.


-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/common/configs/config.go:
##########
@@ -227,5 +193,14 @@ func GetConfigurationString(requestBytes []byte) string {
 	return strings.ReplaceAll(conf, checksum, "")
 }
 
-// Default loader, can be updated by tests
-var SchedulerConfigLoader LoadSchedulerConfigFunc = loadSchedulerConfigFromFile
+// DefaultSchedulerConfig contains the default scheduler configuration; used if no other is provided
+var DefaultSchedulerConfig = `
+partitions:
+  - name: default
+    placementrules:
+      - name: tag
+        value: namespace
+        create: true
+    queues:
+      - name: root

Review Comment:
   We are missing the following line from the code:
   ```
             submitacl: '*'
   ```



-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/log/logger.go:
##########
@@ -23,50 +23,63 @@ import (
 	"fmt"
 	"reflect"
 	"sync"
+	"sync/atomic"
 
 	"go.uber.org/zap"
 	"go.uber.org/zap/zapcore"
 )
 
 var once sync.Once
-var logger *zap.Logger
-var config *zap.Config
+var holderValue atomic.Value

Review Comment:
   This is here to allow dynamic reconfiguration of the logger. The overhead is minimal (probably a few machine ops per logged statement).



-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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

   # [Codecov](https://codecov.io/gh/apache/yunikorn-core/pull/446?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 [#446](https://codecov.io/gh/apache/yunikorn-core/pull/446?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (effbf9c) into [master](https://codecov.io/gh/apache/yunikorn-core/commit/decbabd8f9b8bf9150aee2f4cda2a47f3f42383a?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (decbabd) will **decrease** coverage by `0.06%`.
   > The diff coverage is `70.34%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #446      +/-   ##
   ==========================================
   - Coverage   72.63%   72.56%   -0.07%     
   ==========================================
     Files          65       67       +2     
     Lines        9569     9778     +209     
   ==========================================
   + Hits         6950     7095     +145     
   - Misses       2381     2445      +64     
     Partials      238      238              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/yunikorn-core/pull/446?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/446/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==) | `100.00% <ø> (+23.33%)` | :arrow_up: |
   | [pkg/scheduler/partition.go](https://codecov.io/gh/apache/yunikorn-core/pull/446/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-cGtnL3NjaGVkdWxlci9wYXJ0aXRpb24uZ28=) | `75.21% <ø> (+0.42%)` | :arrow_up: |
   | [pkg/scheduler/tests/mock\_rm\_callback.go](https://codecov.io/gh/apache/yunikorn-core/pull/446/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-cGtnL3NjaGVkdWxlci90ZXN0cy9tb2NrX3JtX2NhbGxiYWNrLmdv) | `33.33% <ø> (+6.66%)` | :arrow_up: |
   | [pkg/webservice/handlers.go](https://codecov.io/gh/apache/yunikorn-core/pull/446/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=) | `76.88% <ø> (+0.33%)` | :arrow_up: |
   | [pkg/scheduler/context.go](https://codecov.io/gh/apache/yunikorn-core/pull/446/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-cGtnL3NjaGVkdWxlci9jb250ZXh0Lmdv) | `30.19% <8.33%> (-0.83%)` | :arrow_down: |
   | [pkg/scheduler/ugm/manager.go](https://codecov.io/gh/apache/yunikorn-core/pull/446/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-cGtnL3NjaGVkdWxlci91Z20vbWFuYWdlci5nbw==) | `55.05% <55.05%> (ø)` | |
   | [pkg/scheduler/ugm/queue\_tracker.go](https://codecov.io/gh/apache/yunikorn-core/pull/446/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-cGtnL3NjaGVkdWxlci91Z20vcXVldWVfdHJhY2tlci5nbw==) | `84.84% <84.84%> (ø)` | |
   | [pkg/log/logger.go](https://codecov.io/gh/apache/yunikorn-core/pull/446/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-cGtnL2xvZy9sb2dnZXIuZ28=) | `91.46% <90.62%> (+11.97%)` | :arrow_up: |
   | [pkg/scheduler/objects/node\_collection.go](https://codecov.io/gh/apache/yunikorn-core/pull/446/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-cGtnL3NjaGVkdWxlci9vYmplY3RzL25vZGVfY29sbGVjdGlvbi5nbw==) | `93.23% <100.00%> (-2.26%)` | :arrow_down: |
   | [pkg/scheduler/ugm/group\_tracker.go](https://codecov.io/gh/apache/yunikorn-core/pull/446/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-cGtnL3NjaGVkdWxlci91Z20vZ3JvdXBfdHJhY2tlci5nbw==) | `100.00% <100.00%> (ø)` | |
   | ... and [6 more](https://codecov.io/gh/apache/yunikorn-core/pull/446/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) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?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] craigcondit commented on a diff in pull request #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/common/configs/config.go:
##########
@@ -227,5 +193,14 @@ func GetConfigurationString(requestBytes []byte) string {
 	return strings.ReplaceAll(conf, checksum, "")
 }
 
-// Default loader, can be updated by tests
-var SchedulerConfigLoader LoadSchedulerConfigFunc = loadSchedulerConfigFromFile
+// DefaultSchedulerConfig contains the default scheduler configuration; used if no other is provided
+var DefaultSchedulerConfig = `
+partitions:
+  - name: default
+    placementrules:
+      - name: tag
+        value: namespace
+        create: true
+    queues:
+      - name: root

Review Comment:
   Updated.



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

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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/log/logger_test.go:
##########
@@ -62,8 +63,9 @@ func TestIsDebugEnabled(t *testing.T) {
 		Encoding: "console",
 	}
 	var err error
-	logger, err = zapConfig.Build()
+	logger, err := zapConfig.Build()
 	assert.NilError(t, err, "debug level logger create failed")
+	InitializeLogger(logger, &zapConfig)

Review Comment:
   These tests explicitly did not trigger the `once.Do()` to make sure we re-try things.



##########
pkg/log/logger.go:
##########
@@ -23,50 +23,63 @@ import (
 	"fmt"
 	"reflect"
 	"sync"
+	"sync/atomic"
 
 	"go.uber.org/zap"
 	"go.uber.org/zap/zapcore"
 )
 
 var once sync.Once
-var logger *zap.Logger
-var config *zap.Config
+var holderValue atomic.Value

Review Comment:
   We do not need an atomic here. The logger variable is protected by the `sync.Once`. The logger internally protects itself. We cannot build a logger without a config. Neither must change after creation.
   On read we just need to return the global. There is nothing that warrants the overhead of the load and the cast of the atomic value.



##########
pkg/entrypoint/entrypoint.go:
##########
@@ -55,6 +55,10 @@ func StartAllServicesWithLogger(logger *zap.Logger, zapConfigs *zap.Config) *Ser
 	return StartAllServices()
 }
 
+func InitializeLogger(logger *zap.Logger, zapConfigs *zap.Config) {
+	log.InitializeLogger(logger, zapConfigs)
+}
+

Review Comment:
   This is never called, the `StartAllServicesWithLogger()` is called from the k8shim entrypoint we never just want to call this as part of the entry 



##########
pkg/log/logger_test.go:
##########
@@ -119,9 +121,9 @@ func TestInitializeLogger(t *testing.T) {
 
 	InitializeLogger(localLogger, &zapConfig)
 	assert.Equal(t, Logger(), localLogger)
-	// second initialization should not do anything
+	// second initialization should update
 	InitializeLogger(localLogger2, &zapConfig)
-	assert.Equal(t, Logger(), localLogger)
+	assert.Equal(t, Logger(), localLogger2)

Review Comment:
   See comment earlier: this should not change anything as it can break the link between the k8shim and the core logger which breaks the ability to update the k8shim log level.



##########
pkg/log/logger.go:
##########
@@ -116,14 +126,17 @@ func createConfig() *zap.Config {
 }
 
 func SetLogLevel(newLevel string) error {
+	logger := Logger()
+	config := GetConfig()
+
 	oldLevel := config.Level.String()
 
 	// noop if the input is the same as what is set
 	if newLevel == oldLevel {
 		return nil
 	}
 
-	logger.Info("Updating log level",
+	Logger().Info("Updating log level",

Review Comment:
   consistency: either use  `Logger()` or use the local var logger throughout the function:  line 129, 139 & 147



##########
pkg/log/logger.go:
##########
@@ -23,50 +23,63 @@ import (
 	"fmt"
 	"reflect"
 	"sync"
+	"sync/atomic"
 
 	"go.uber.org/zap"
 	"go.uber.org/zap/zapcore"
 )
 
 var once sync.Once
-var logger *zap.Logger
-var config *zap.Config
+var holderValue atomic.Value
+
+type loggerHolder struct {
+	logger     *zap.Logger
+	zapConfigs *zap.Config
+}
 
 func Logger() *zap.Logger {
-	once.Do(func() {
-		if logger = zap.L(); isNopLogger(logger) {
-			// If a global logger is not found, this could be either scheduler-core
-			// is running as a deployment mode, or running with another non-go code
-			// shim. In this case, we need to create our own logger.
-			// TODO support log options when a global logger is not there
-			config = createConfig()
-			var err error
-			logger, err = config.Build()
-			// this should really not happen so just write to stdout and set a Nop logger
-			if err != nil {
-				fmt.Printf("Logging disabled, logger init failed with error: %v\n", err)
-				logger = zap.NewNop()
-			}
+	once.Do(initLogger)
+	return holderValue.Load().(loggerHolder).logger
+}
+
+func GetConfig() *zap.Config {
+	once.Do(initLogger)
+	return holderValue.Load().(loggerHolder).zapConfigs
+}
+
+func initLogger() {
+	var logger *zap.Logger
+	var config *zap.Config
+
+	if logger = zap.L(); isNopLogger(logger) {
+		// If a global logger is not found, this could be either scheduler-core
+		// is running as a deployment mode, or running with another non-go code
+		// shim. In this case, we need to create our own logger.
+		config = createConfig()
+		var err error
+		logger, err = config.Build()
+		// this should really not happen so just write to stdout and set a Nop logger
+		if err != nil {
+			fmt.Printf("Logging disabled, logger init failed with error: %v\n", err)
+			logger = zap.NewNop()
 		}
+	}
+	holderValue.Store(loggerHolder{
+		logger:     logger,
+		zapConfigs: config,
 	})
-
-	return logger
 }
 
 func InitializeLogger(log *zap.Logger, zapConfig *zap.Config) {
-	once.Do(func() {
-		logger = log
-		config = zapConfig
-		logger.Info("Using an already initialized logger")
+	once.Do(initLogger)

Review Comment:
   This changes the init for the logger from the shim into the core. The `InitializeLogger()` passes in a pre-build config from the k8shim and makes sure that the shim and the core run with *the exact same* config object. Without that changing the log level via the rest call is only going to change the core log level.
   
   It also does the work twice in this case: we have a logger and config we just want to set it. We can change that in a later jira to decouple things but not as part of this change.



##########
pkg/scheduler/context.go:
##########
@@ -313,14 +333,14 @@ func (cc *ClusterContext) UpdateSchedulerConfig(conf *configs.SchedulerConfig) e
 // Locked version of the configuration update called outside of event system.
 // Updates the current config via the config loader.
 // Used in test only, normal updates use the internal call and the webservice must use the UpdateSchedulerConfig

Review Comment:
   NIT: Can we update this comment top remove the web service reference?



-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/log/logger_test.go:
##########
@@ -119,9 +121,9 @@ func TestInitializeLogger(t *testing.T) {
 
 	InitializeLogger(localLogger, &zapConfig)
 	assert.Equal(t, Logger(), localLogger)
-	// second initialization should not do anything
+	// second initialization should update
 	InitializeLogger(localLogger2, &zapConfig)
-	assert.Equal(t, Logger(), localLogger)
+	assert.Equal(t, Logger(), localLogger2)

Review Comment:
   As above, InitializeLogger() is how the shim will reconfigure the logger when the configmap changes. I've tested this locally in my shim, and it works as expected.



-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/log/logger.go:
##########
@@ -116,14 +126,17 @@ func createConfig() *zap.Config {
 }
 
 func SetLogLevel(newLevel string) error {
+	logger := Logger()
+	config := GetConfig()
+
 	oldLevel := config.Level.String()
 
 	// noop if the input is the same as what is set
 	if newLevel == oldLevel {
 		return nil
 	}
 
-	logger.Info("Updating log level",
+	Logger().Info("Updating log level",

Review Comment:
   good catch, this should be logger from Line 129.



-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/log/logger_test.go:
##########
@@ -62,8 +63,9 @@ func TestIsDebugEnabled(t *testing.T) {
 		Encoding: "console",
 	}
 	var err error
-	logger, err = zapConfig.Build()
+	logger, err := zapConfig.Build()
 	assert.NilError(t, err, "debug level logger create failed")
+	InitializeLogger(logger, &zapConfig)

Review Comment:
   I will remove the (now incorrect) comment at the top of the function in the next commit.



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

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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/log/logger_test.go:
##########
@@ -62,8 +63,9 @@ func TestIsDebugEnabled(t *testing.T) {
 		Encoding: "console",
 	}
 	var err error
-	logger, err = zapConfig.Build()
+	logger, err := zapConfig.Build()
 	assert.NilError(t, err, "debug level logger create failed")
+	InitializeLogger(logger, &zapConfig)

Review Comment:
   re-init is no longer a problem as we handle this case cleanly.



-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/entrypoint/entrypoint.go:
##########
@@ -55,6 +55,10 @@ func StartAllServicesWithLogger(logger *zap.Logger, zapConfigs *zap.Config) *Ser
 	return StartAllServices()
 }
 
+func InitializeLogger(logger *zap.Logger, zapConfigs *zap.Config) {
+	log.InitializeLogger(logger, zapConfigs)
+}
+

Review Comment:
   Per offline discussions, reverting all logging changes.



##########
pkg/log/logger.go:
##########
@@ -23,50 +23,63 @@ import (
 	"fmt"
 	"reflect"
 	"sync"
+	"sync/atomic"
 
 	"go.uber.org/zap"
 	"go.uber.org/zap/zapcore"
 )
 
 var once sync.Once
-var logger *zap.Logger
-var config *zap.Config
+var holderValue atomic.Value

Review Comment:
   Per offline discussions, reverting all logging changes.



-- 
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 #446: [YUNIKORN-1366] Read core configuration from shim

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


##########
pkg/log/logger.go:
##########
@@ -23,50 +23,63 @@ import (
 	"fmt"
 	"reflect"
 	"sync"
+	"sync/atomic"
 
 	"go.uber.org/zap"
 	"go.uber.org/zap/zapcore"
 )
 
 var once sync.Once
-var logger *zap.Logger
-var config *zap.Config
+var holderValue atomic.Value
+
+type loggerHolder struct {
+	logger     *zap.Logger
+	zapConfigs *zap.Config
+}
 
 func Logger() *zap.Logger {
-	once.Do(func() {
-		if logger = zap.L(); isNopLogger(logger) {
-			// If a global logger is not found, this could be either scheduler-core
-			// is running as a deployment mode, or running with another non-go code
-			// shim. In this case, we need to create our own logger.
-			// TODO support log options when a global logger is not there
-			config = createConfig()
-			var err error
-			logger, err = config.Build()
-			// this should really not happen so just write to stdout and set a Nop logger
-			if err != nil {
-				fmt.Printf("Logging disabled, logger init failed with error: %v\n", err)
-				logger = zap.NewNop()
-			}
+	once.Do(initLogger)
+	return holderValue.Load().(loggerHolder).logger
+}
+
+func GetConfig() *zap.Config {
+	once.Do(initLogger)
+	return holderValue.Load().(loggerHolder).zapConfigs
+}
+
+func initLogger() {
+	var logger *zap.Logger
+	var config *zap.Config
+
+	if logger = zap.L(); isNopLogger(logger) {
+		// If a global logger is not found, this could be either scheduler-core
+		// is running as a deployment mode, or running with another non-go code
+		// shim. In this case, we need to create our own logger.
+		config = createConfig()
+		var err error
+		logger, err = config.Build()
+		// this should really not happen so just write to stdout and set a Nop logger
+		if err != nil {
+			fmt.Printf("Logging disabled, logger init failed with error: %v\n", err)
+			logger = zap.NewNop()
 		}
+	}
+	holderValue.Store(loggerHolder{
+		logger:     logger,
+		zapConfigs: config,
 	})
-
-	return logger
 }
 
 func InitializeLogger(log *zap.Logger, zapConfig *zap.Config) {
-	once.Do(func() {
-		logger = log
-		config = zapConfig
-		logger.Info("Using an already initialized logger")
+	once.Do(initLogger)

Review Comment:
   initLogger is used now only to provide a "fallback" logger in case something is logged early. Once the scheduler is initialized propery, the dynamic configuration will replace the logger with the real, configured one. This is all part of allowing dynamic reconfiguration via the configmap.



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