You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@yunikorn.apache.org by "craigcondit (via GitHub)" <gi...@apache.org> on 2023/06/08 21:17:24 UTC

[GitHub] [yunikorn-k8shim] craigcondit opened a new pull request, #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

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

   Add API support for scoped logging to the Kubernetes shim and admission controller. This will allow for fine-grained logging control per-subsystem.
   
   ### What type of PR is it?
   * [ ] - Bug Fix
   * [ ] - Improvement
   * [x] - Feature
   * [ ] - Documentation
   * [ ] - Hot Fix
   * [ ] - Refactoring
   
   ### Todos
   * [ ] - Task
   
   ### What is the Jira issue?
   https://issues.apache.org/jira/browse/YUNIKORN-1797
   
   ### How should this be tested?
   All shim log messages should now go to the "k8shim" logger, and admission logs to the "admission" logger. Future PRs will move individual subsystems to even more specific loggers.
   
   ### 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 commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "pbacsko (via GitHub)" <gi...@apache.org>.
pbacsko commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1230874642


##########
pkg/log/logger_test.go:
##########
@@ -0,0 +1,244 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package log
+
+import (
+	"fmt"
+	"os"
+	"testing"
+	"time"
+
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+	"gotest.tools/v3/assert"
+)
+
+var logDir string
+var logFile string
+
+var iterations = 100000
+
+func TestLoggerIds(t *testing.T) {
+	_ = Logger()
+
+	// validate logger count
+	assert.Equal(t, 4, len(loggers), "wrong logger count")
+
+	// validate that all loggers are populated and have sequential ids
+	for i := 0; i < len(loggers); i++ {
+		handle := loggers[i]
+		assert.Assert(t, handle != nil, "nil handle for index", i)
+		assert.Equal(t, handle.id, i+1, "wrong id", handle.name)
+	}
+}
+
+func BenchmarkLegacyLoggerDebug(b *testing.B) {
+	benchmarkLegacyLoggerDebug(b.N)
+}
+
+func TestLegacyLoggerDebug(t *testing.T) {
+	nsOp := benchmarkLegacyLoggerDebug(iterations)
+	RootLogger().Info("log.Logger() performance", zap.Int64("debug (ns/op)", nsOp))
+}
+
+func benchmarkLegacyLoggerDebug(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		RootLogger().Debug("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkLegacyLoggerInfo(b *testing.B) {
+	benchmarkLegacyLoggerInfo(b.N)
+}
+
+func TestLegacyLoggerInfo(t *testing.T) {
+	nsOp := benchmarkLegacyLoggerInfo(iterations)
+	RootLogger().Info("log.Logger() performance", zap.Int64("info (ns/op)", nsOp))
+}
+
+func benchmarkLegacyLoggerInfo(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		RootLogger().Info("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerDebug(b *testing.B) {
+	benchmarkScopedLoggerDebug(b.N)
+}
+
+func TestScopedLoggerDebug(t *testing.T) {
+	nsOp := benchmarkScopedLoggerDebug(iterations)
+	Log(Test).Info("log.Log(...) performance (root=INFO)", zap.Int64("debug (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerDebug(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.level": "INFO",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(K8Shim).Debug("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerInfo(b *testing.B) {
+	benchmarkScopedLoggerInfo(b.N)
+}
+
+func TestScopedLoggerInfo(t *testing.T) {
+	nsOp := benchmarkScopedLoggerInfo(iterations)
+	Log(Test).Info("log.Log(...) performance (root=INFO)", zap.Int64("info (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerInfo(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.level": "INFO",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(K8Shim).Info("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerDebugEnabled(b *testing.B) {
+	benchmarkScopedLoggerDebugEnabled(b.N)
+}
+
+func TestScopedLoggerDebugEnabled(t *testing.T) {
+	nsOp := benchmarkScopedLoggerDebugEnabled(iterations)
+	Log(Test).Info("log.Log(...) performance (root=DEBUG)", zap.Int64("debug (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerDebugEnabled(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.test.level": "DEBUG",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(Test).Debug("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerInfoFiltered(b *testing.B) {
+	benchmarkScopedLoggerInfoFiltered(b.N)
+}
+
+func TestScopedLoggerInfoFiltered(t *testing.T) {
+	nsOp := benchmarkScopedLoggerInfoFiltered(iterations)
+	Log(Test).Info("log.Log(...) performance (root=DEBUG)", zap.Int64("info (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerInfoFiltered(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.test.level": "DEBUG",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(K8Shim).Info("test", zap.String("foo", "bar"))

Review Comment:
   I suppose the message "test" is unrelated to logger named "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-k8shim] craigcondit commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1228517267


##########
pkg/log/logger_test.go:
##########
@@ -0,0 +1,202 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package log
+
+import (
+	"fmt"
+	"os"
+	"testing"
+	"time"
+
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+	"gotest.tools/v3/assert"
+)
+
+var logDir string
+var logFile string
+
+var iterations = 100000
+
+func TestLegacyLoggerPerformance(t *testing.T) {

Review Comment:
   Refactored tests to be able to be run in either benchmark (slower but more reliable) or unit test mode. Also added "make bench" target.



-- 
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 #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "pbacsko (via GitHub)" <gi...@apache.org>.
pbacsko commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1230859687


##########
pkg/log/logger.go:
##########
@@ -30,12 +33,95 @@ var once sync.Once
 var logger *zap.Logger
 var zapConfigs *zap.Config
 
+// LoggerHandle is used to efficiently look up logger references
+type LoggerHandle struct {
+	id   int
+	name string
+}
+
+func (h LoggerHandle) String() string {
+	return h.name
+}
+
+// Logger constants for configuration
+const (
+	defaultLog  = "log.level"
+	logPrefix   = "log."
+	levelSuffix = ".level"
+)
+
+// Predefined loggers: when adding new loggers, ids must be sequential, and all must be added to the loggers slice in the same order
+
+var K8Shim = &LoggerHandle{id: 1, name: "k8shim"}
+var Kubernetes = &LoggerHandle{id: 2, name: "kubernetes"}
+var Admission = &LoggerHandle{id: 3, name: "admission"}
+var Test = &LoggerHandle{id: 4, name: "test"}
+
+var loggers = []*LoggerHandle{
+	K8Shim,
+	Kubernetes,
+	Admission,
+	Test,
+}
+
+type loggerConfig struct {
+	loggers []*zap.Logger
+}
+
+var currentLoggerConfig = atomic.Pointer[loggerConfig]{}
+var defaultLogger = atomic.Pointer[LoggerHandle]{}
+
+// Logger retrieves the global logger
 func Logger() *zap.Logger {
+	once.Do(initLogger)
+	return Log(defaultLogger.Load())
+}
+
+// RootLogger retrieves the root logger
+func RootLogger() *zap.Logger {
 	once.Do(initLogger)
 	return logger
 }
 
+// Log retrieves a standard logger
+func Log(handle *LoggerHandle) *zap.Logger {
+	once.Do(initLogger)
+	if handle == nil || handle.id == 0 {
+		handle = defaultLogger.Load()
+	}
+	conf := currentLoggerConfig.Load()
+	return conf.loggers[handle.id-1]
+}
+
+func createLogger(levelMap map[string]zapcore.Level, name string) *zap.Logger {
+	level := loggerLevel(levelMap, name)
+	return logger.Named(name).WithOptions(zap.WrapCore(func(inner zapcore.Core) zapcore.Core {
+		return filteredCore{inner: inner, level: level}
+	}))
+}
+
+func loggerLevel(levelMap map[string]zapcore.Level, name string) zapcore.Level {

Review Comment:
   Can we have some explanation or comments about what happens here? Just by looking at this, I have to spend some time to understand how we retrieve the 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-k8shim] craigcondit commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1231330511


##########
pkg/log/logger_test.go:
##########
@@ -0,0 +1,244 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package log
+
+import (
+	"fmt"
+	"os"
+	"testing"
+	"time"
+
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+	"gotest.tools/v3/assert"
+)
+
+var logDir string
+var logFile string
+
+var iterations = 100000
+
+func TestLoggerIds(t *testing.T) {
+	_ = Logger()
+
+	// validate logger count
+	assert.Equal(t, 4, len(loggers), "wrong logger count")
+
+	// validate that all loggers are populated and have sequential ids
+	for i := 0; i < len(loggers); i++ {
+		handle := loggers[i]
+		assert.Assert(t, handle != nil, "nil handle for index", i)
+		assert.Equal(t, handle.id, i+1, "wrong id", handle.name)
+	}
+}
+
+func BenchmarkLegacyLoggerDebug(b *testing.B) {
+	benchmarkLegacyLoggerDebug(b.N)
+}
+
+func TestLegacyLoggerDebug(t *testing.T) {
+	nsOp := benchmarkLegacyLoggerDebug(iterations)
+	RootLogger().Info("log.Logger() performance", zap.Int64("debug (ns/op)", nsOp))
+}
+
+func benchmarkLegacyLoggerDebug(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		RootLogger().Debug("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkLegacyLoggerInfo(b *testing.B) {
+	benchmarkLegacyLoggerInfo(b.N)
+}
+
+func TestLegacyLoggerInfo(t *testing.T) {
+	nsOp := benchmarkLegacyLoggerInfo(iterations)
+	RootLogger().Info("log.Logger() performance", zap.Int64("info (ns/op)", nsOp))
+}
+
+func benchmarkLegacyLoggerInfo(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		RootLogger().Info("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerDebug(b *testing.B) {
+	benchmarkScopedLoggerDebug(b.N)
+}
+
+func TestScopedLoggerDebug(t *testing.T) {
+	nsOp := benchmarkScopedLoggerDebug(iterations)
+	Log(Test).Info("log.Log(...) performance (root=INFO)", zap.Int64("debug (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerDebug(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.level": "INFO",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(K8Shim).Debug("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerInfo(b *testing.B) {
+	benchmarkScopedLoggerInfo(b.N)
+}
+
+func TestScopedLoggerInfo(t *testing.T) {
+	nsOp := benchmarkScopedLoggerInfo(iterations)
+	Log(Test).Info("log.Log(...) performance (root=INFO)", zap.Int64("info (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerInfo(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.level": "INFO",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(K8Shim).Info("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerDebugEnabled(b *testing.B) {
+	benchmarkScopedLoggerDebugEnabled(b.N)
+}
+
+func TestScopedLoggerDebugEnabled(t *testing.T) {
+	nsOp := benchmarkScopedLoggerDebugEnabled(iterations)
+	Log(Test).Info("log.Log(...) performance (root=DEBUG)", zap.Int64("debug (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerDebugEnabled(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.test.level": "DEBUG",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(Test).Debug("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerInfoFiltered(b *testing.B) {
+	benchmarkScopedLoggerInfoFiltered(b.N)
+}
+
+func TestScopedLoggerInfoFiltered(t *testing.T) {
+	nsOp := benchmarkScopedLoggerInfoFiltered(iterations)
+	Log(Test).Info("log.Log(...) performance (root=DEBUG)", zap.Int64("info (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerInfoFiltered(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.test.level": "DEBUG",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(K8Shim).Info("test", zap.String("foo", "bar"))

Review Comment:
   The message is purposely identical so that there's no variation in potential processing time (i.e. make sure it's not what we log but how we log it). The logger is chosen to determine which logging path we use relative to the config given.



-- 
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] craigcondit commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1231371631


##########
pkg/log/logger.go:
##########
@@ -82,3 +171,87 @@ func GetZapConfigs() *zap.Config {
 	_ = Logger()
 	return zapConfigs
 }
+
+// SetDefaultLogger allows customization of the default logger
+func SetDefaultLogger(handle *LoggerHandle) {
+	once.Do(initLogger)
+	defaultLogger.Store(handle)
+}
+
+// UpdateLoggingConfig is used to reconfigure logging.
+// This uses config keys of the form log.{logger}.level={level}.
+// The default level is set by log.level={level}
+func UpdateLoggingConfig(config map[string]string) {
+	once.Do(initLogger)
+	initLoggingConfig(config)
+}
+
+func initLoggingConfig(config map[string]string) {
+	levelMap := make(map[string]zapcore.Level)
+	levelMap[""] = zapcore.InfoLevel

Review Comment:
   Will update to `nullLogger` in next PR.



-- 
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] craigcondit commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1231371631


##########
pkg/log/logger.go:
##########
@@ -82,3 +171,87 @@ func GetZapConfigs() *zap.Config {
 	_ = Logger()
 	return zapConfigs
 }
+
+// SetDefaultLogger allows customization of the default logger
+func SetDefaultLogger(handle *LoggerHandle) {
+	once.Do(initLogger)
+	defaultLogger.Store(handle)
+}
+
+// UpdateLoggingConfig is used to reconfigure logging.
+// This uses config keys of the form log.{logger}.level={level}.
+// The default level is set by log.level={level}
+func UpdateLoggingConfig(config map[string]string) {
+	once.Do(initLogger)
+	initLoggingConfig(config)
+}
+
+func initLoggingConfig(config map[string]string) {
+	levelMap := make(map[string]zapcore.Level)
+	levelMap[""] = zapcore.InfoLevel

Review Comment:
   Updated to use `nullLogger` in latest PR.



-- 
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] craigcondit commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1228356529


##########
pkg/log/logger_test.go:
##########
@@ -0,0 +1,202 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package log
+
+import (
+	"fmt"
+	"os"
+	"testing"
+	"time"
+
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+	"gotest.tools/v3/assert"
+)
+
+var logDir string
+var logFile string
+
+var iterations = 100000
+
+func TestLegacyLoggerPerformance(t *testing.T) {

Review Comment:
   The code also functions as a way to exercise these code paths for coverage. I could probably do some benchmarks and factor out common code, but I'd like to keep some variant of this on the default test runs. We also don't (currently) have a benchmark makefile target, so that would need to be added as well.



-- 
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 #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "pbacsko (via GitHub)" <gi...@apache.org>.
pbacsko commented on PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#issuecomment-1589589190

   Looks good, I'll examine this in my IDE in more detail tomorrow just to have a slightly better understanding. 


-- 
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 #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "codecov[bot] (via GitHub)" <gi...@apache.org>.
codecov[bot] commented on PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#issuecomment-1583387822

   ## [Codecov](https://app.codecov.io/gh/apache/yunikorn-k8shim/pull/611?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#611](https://app.codecov.io/gh/apache/yunikorn-k8shim/pull/611?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (b906e5a) into [master](https://app.codecov.io/gh/apache/yunikorn-k8shim/commit/c90673fbe5e82103e511cde9923fb09fb6988942?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (c90673f) will **decrease** coverage by `0.17%`.
   > The diff coverage is `63.11%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #611      +/-   ##
   ==========================================
   - Coverage   70.87%   70.71%   -0.17%     
   ==========================================
     Files          47       49       +2     
     Lines        7972     8123     +151     
   ==========================================
   + Hits         5650     5744      +94     
   - Misses       2117     2166      +49     
   - Partials      205      213       +8     
   ```
   
   
   | [Impacted Files](https://app.codecov.io/gh/apache/yunikorn-k8shim/pull/611?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [pkg/log/filtered\_core.go](https://app.codecov.io/gh/apache/yunikorn-k8shim/pull/611?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGtnL2xvZy9maWx0ZXJlZF9jb3JlLmdv) | `43.75% <43.75%> (ø)` | |
   | [pkg/log/logger.go](https://app.codecov.io/gh/apache/yunikorn-k8shim/pull/611?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGtnL2xvZy9sb2dnZXIuZ28=) | `72.34% <64.28%> (ø)` | |
   | [pkg/conf/schedulerconf.go](https://app.codecov.io/gh/apache/yunikorn-k8shim/pull/611?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGtnL2NvbmYvc2NoZWR1bGVyY29uZi5nbw==) | `80.89% <80.00%> (-0.36%)` | :arrow_down: |
   | [pkg/admission/conf/am\_conf.go](https://app.codecov.io/gh/apache/yunikorn-k8shim/pull/611?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGtnL2FkbWlzc2lvbi9jb25mL2FtX2NvbmYuZ28=) | `69.03% <100.00%> (-3.88%)` | :arrow_down: |
   
   :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=apache)
   


-- 
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] craigcondit commented on pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#issuecomment-1589803439

   @pbacsko made some updates per your input, please re-review.


-- 
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] craigcondit commented on pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#issuecomment-1593536910

   Updated PR (rebased on master) and addressed review comments. Added significant comments throughout, and additional test cases. 


-- 
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 #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "pbacsko (via GitHub)" <gi...@apache.org>.
pbacsko commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1230873818


##########
pkg/log/logger_test.go:
##########
@@ -0,0 +1,244 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package log
+
+import (
+	"fmt"
+	"os"
+	"testing"
+	"time"
+
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+	"gotest.tools/v3/assert"
+)
+
+var logDir string
+var logFile string
+
+var iterations = 100000
+
+func TestLoggerIds(t *testing.T) {
+	_ = Logger()
+
+	// validate logger count
+	assert.Equal(t, 4, len(loggers), "wrong logger count")
+
+	// validate that all loggers are populated and have sequential ids
+	for i := 0; i < len(loggers); i++ {
+		handle := loggers[i]
+		assert.Assert(t, handle != nil, "nil handle for index", i)
+		assert.Equal(t, handle.id, i+1, "wrong id", handle.name)
+	}
+}
+
+func BenchmarkLegacyLoggerDebug(b *testing.B) {
+	benchmarkLegacyLoggerDebug(b.N)
+}
+
+func TestLegacyLoggerDebug(t *testing.T) {
+	nsOp := benchmarkLegacyLoggerDebug(iterations)
+	RootLogger().Info("log.Logger() performance", zap.Int64("debug (ns/op)", nsOp))
+}
+
+func benchmarkLegacyLoggerDebug(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		RootLogger().Debug("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkLegacyLoggerInfo(b *testing.B) {
+	benchmarkLegacyLoggerInfo(b.N)
+}
+
+func TestLegacyLoggerInfo(t *testing.T) {
+	nsOp := benchmarkLegacyLoggerInfo(iterations)
+	RootLogger().Info("log.Logger() performance", zap.Int64("info (ns/op)", nsOp))
+}
+
+func benchmarkLegacyLoggerInfo(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		RootLogger().Info("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerDebug(b *testing.B) {
+	benchmarkScopedLoggerDebug(b.N)
+}
+
+func TestScopedLoggerDebug(t *testing.T) {
+	nsOp := benchmarkScopedLoggerDebug(iterations)
+	Log(Test).Info("log.Log(...) performance (root=INFO)", zap.Int64("debug (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerDebug(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.level": "INFO",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(K8Shim).Debug("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerInfo(b *testing.B) {
+	benchmarkScopedLoggerInfo(b.N)
+}
+
+func TestScopedLoggerInfo(t *testing.T) {
+	nsOp := benchmarkScopedLoggerInfo(iterations)
+	Log(Test).Info("log.Log(...) performance (root=INFO)", zap.Int64("info (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerInfo(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.level": "INFO",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(K8Shim).Info("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerDebugEnabled(b *testing.B) {
+	benchmarkScopedLoggerDebugEnabled(b.N)
+}
+
+func TestScopedLoggerDebugEnabled(t *testing.T) {
+	nsOp := benchmarkScopedLoggerDebugEnabled(iterations)
+	Log(Test).Info("log.Log(...) performance (root=DEBUG)", zap.Int64("debug (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerDebugEnabled(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.test.level": "DEBUG",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(Test).Debug("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerInfoFiltered(b *testing.B) {
+	benchmarkScopedLoggerInfoFiltered(b.N)
+}
+
+func TestScopedLoggerInfoFiltered(t *testing.T) {
+	nsOp := benchmarkScopedLoggerInfoFiltered(iterations)
+	Log(Test).Info("log.Log(...) performance (root=DEBUG)", zap.Int64("info (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerInfoFiltered(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.test.level": "DEBUG",

Review Comment:
   What's exactly the difference between this and the non-filtered test? What are we filtering 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 #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "pbacsko (via GitHub)" <gi...@apache.org>.
pbacsko commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1228349786


##########
pkg/log/logger_test.go:
##########
@@ -0,0 +1,202 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package log
+
+import (
+	"fmt"
+	"os"
+	"testing"
+	"time"
+
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+	"gotest.tools/v3/assert"
+)
+
+var logDir string
+var logFile string
+
+var iterations = 100000
+
+func TestLegacyLoggerPerformance(t *testing.T) {

Review Comment:
   Have you considered benchmarking testcases for this (the ones that start with the word "benchmark", like `func BenchmarkLegacyLoggerPerformance(b *testing.B)`? Seems like this is exactly what you need. Especially because benchmarks are skipped during normal unit test run.



-- 
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] craigcondit commented on pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#issuecomment-1593459843

   > Ok, I did a re-review. Some parts are not entirely clear, it's difficult to understand what happens without comments. You can also consider using `Example*` functions which documents how the new logger approach should be used (filtering, parent loggers, etc).
   
   I'll try to add some more comments but I don't see the need for example functions. The original JIRA demonstrates this, and eventually all log statements will use the new form. Essentially:
   
   `log.Logger().Info(...)` becomes `log.Log(log.LogName).Info(...)`
   
   Everything else remains the same. The specific loggers will be subsystem-specific, and probably added to the codebase as we segregate things out. I don't want to do a high-level "dump everything by package" translation as I'd like there to be some thought given to organizing our loggers.
   
   The existing `log.Logger()` call remains in place for backwards compatibility. Depending on context (admission controller, shim, or core), this becomes equivalent to either `log.Log(log.Admission)`, `log.Log(log.K8Shim)` or `log.Log(log.Core)` respectively.
   
   Finally, `log.RootLogger()` provides the old behavior, but this is only exposed for initial configuration (i.e. passing the log reference to the core from the shim).


-- 
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 #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "pbacsko (via GitHub)" <gi...@apache.org>.
pbacsko commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1228354935


##########
pkg/log/logger.go:
##########
@@ -30,13 +33,99 @@ var once sync.Once
 var logger *zap.Logger
 var zapConfigs *zap.Config
 
+// LoggerHandle is used to efficiently look up logger references
+type LoggerHandle struct {
+	name string
+}
+
+func (h LoggerHandle) String() string {
+	return h.name
+}
+
+// Logger names
+const (
+	defaultLog  = "log.level"
+	logPrefix   = "log."
+	levelSuffix = ".level"
+)
+
+var K8Shim = LoggerHandle{name: "k8shim"}
+var Kubernetes = LoggerHandle{name: "kubernetes"}
+var Admission = LoggerHandle{name: "admission"}
+var Test = LoggerHandle{name: "test"}

Review Comment:
   If we manage all loggers, we can get faster logger retrieval by defining an extra "int" field in `LoggerHandle` which tells the index of the `*zap.Logger` instance inside a slice. We can still use the map for less commonly used loggers.



-- 
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] craigcondit commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1228355723


##########
pkg/log/logger.go:
##########
@@ -30,13 +33,99 @@ var once sync.Once
 var logger *zap.Logger
 var zapConfigs *zap.Config
 
+// LoggerHandle is used to efficiently look up logger references
+type LoggerHandle struct {
+	name string
+}
+
+func (h LoggerHandle) String() string {
+	return h.name
+}
+
+// Logger names
+const (
+	defaultLog  = "log.level"
+	logPrefix   = "log."
+	levelSuffix = ".level"
+)
+
+var K8Shim = LoggerHandle{name: "k8shim"}
+var Kubernetes = LoggerHandle{name: "kubernetes"}
+var Admission = LoggerHandle{name: "admission"}
+var Test = LoggerHandle{name: "test"}

Review Comment:
   Let me do a little investigating on this. I don't think the performance impact is going to matter much, as the overhead of this is already pretty negligible.



-- 
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 #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "pbacsko (via GitHub)" <gi...@apache.org>.
pbacsko commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1230858544


##########
pkg/log/logger.go:
##########
@@ -30,12 +33,95 @@ var once sync.Once
 var logger *zap.Logger
 var zapConfigs *zap.Config
 
+// LoggerHandle is used to efficiently look up logger references
+type LoggerHandle struct {
+	id   int
+	name string
+}
+
+func (h LoggerHandle) String() string {
+	return h.name
+}
+
+// Logger constants for configuration
+const (
+	defaultLog  = "log.level"
+	logPrefix   = "log."
+	levelSuffix = ".level"
+)
+
+// Predefined loggers: when adding new loggers, ids must be sequential, and all must be added to the loggers slice in the same order
+
+var K8Shim = &LoggerHandle{id: 1, name: "k8shim"}
+var Kubernetes = &LoggerHandle{id: 2, name: "kubernetes"}
+var Admission = &LoggerHandle{id: 3, name: "admission"}
+var Test = &LoggerHandle{id: 4, name: "test"}
+
+var loggers = []*LoggerHandle{
+	K8Shim,
+	Kubernetes,
+	Admission,
+	Test,
+}
+
+type loggerConfig struct {
+	loggers []*zap.Logger
+}
+
+var currentLoggerConfig = atomic.Pointer[loggerConfig]{}
+var defaultLogger = atomic.Pointer[LoggerHandle]{}
+
+// Logger retrieves the global logger
 func Logger() *zap.Logger {
+	once.Do(initLogger)
+	return Log(defaultLogger.Load())
+}
+
+// RootLogger retrieves the root logger
+func RootLogger() *zap.Logger {
 	once.Do(initLogger)
 	return logger
 }
 
+// Log retrieves a standard logger
+func Log(handle *LoggerHandle) *zap.Logger {
+	once.Do(initLogger)
+	if handle == nil || handle.id == 0 {
+		handle = defaultLogger.Load()
+	}
+	conf := currentLoggerConfig.Load()
+	return conf.loggers[handle.id-1]
+}
+
+func createLogger(levelMap map[string]zapcore.Level, name string) *zap.Logger {
+	level := loggerLevel(levelMap, name)
+	return logger.Named(name).WithOptions(zap.WrapCore(func(inner zapcore.Core) zapcore.Core {
+		return filteredCore{inner: inner, level: level}
+	}))
+}
+
+func loggerLevel(levelMap map[string]zapcore.Level, name string) zapcore.Level {
+	for ; name != ""; name = parentLogger(name) {
+		if level, ok := levelMap[name]; ok {
+			return level
+		}
+	}
+	if level, ok := levelMap[""]; ok {
+		return level
+	}
+	return zapcore.InfoLevel
+}
+
+func parentLogger(name string) string {
+	i := strings.LastIndex(name, ".")
+	if i < 0 {
+		return ""
+	}
+	return name[0:i]

Review Comment:
   We need coverage for this, I wanted to see how we use parent loggers & what happens inside, but there's no test for this path.



-- 
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] craigcondit commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1228355723


##########
pkg/log/logger.go:
##########
@@ -30,13 +33,99 @@ var once sync.Once
 var logger *zap.Logger
 var zapConfigs *zap.Config
 
+// LoggerHandle is used to efficiently look up logger references
+type LoggerHandle struct {
+	name string
+}
+
+func (h LoggerHandle) String() string {
+	return h.name
+}
+
+// Logger names
+const (
+	defaultLog  = "log.level"
+	logPrefix   = "log."
+	levelSuffix = ".level"
+)
+
+var K8Shim = LoggerHandle{name: "k8shim"}
+var Kubernetes = LoggerHandle{name: "kubernetes"}
+var Admission = LoggerHandle{name: "admission"}
+var Test = LoggerHandle{name: "test"}

Review Comment:
   This doesn't work because the core and shim loggers are managed separately. The performance impact of these changes is already negligible.



-- 
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] craigcondit commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1228518755


##########
pkg/log/logger.go:
##########
@@ -30,13 +33,99 @@ var once sync.Once
 var logger *zap.Logger
 var zapConfigs *zap.Config
 
+// LoggerHandle is used to efficiently look up logger references
+type LoggerHandle struct {
+	name string
+}
+
+func (h LoggerHandle) String() string {
+	return h.name
+}
+
+// Logger names
+const (
+	defaultLog  = "log.level"
+	logPrefix   = "log."
+	levelSuffix = ".level"
+)
+
+var K8Shim = LoggerHandle{name: "k8shim"}
+var Kubernetes = LoggerHandle{name: "kubernetes"}
+var Admission = LoggerHandle{name: "admission"}
+var Test = LoggerHandle{name: "test"}

Review Comment:
   I tried this locally and it does indeed seem to be faster. Overhead is now in the ballpark of a few ns per log call. The one downside is that there's now some more maintenance required for logging to work properly. I've added a unit test to verify that all the loggers exist and are configured 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-k8shim] pbacsko commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "pbacsko (via GitHub)" <gi...@apache.org>.
pbacsko commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1230857285


##########
pkg/log/logger.go:
##########
@@ -82,3 +171,87 @@ func GetZapConfigs() *zap.Config {
 	_ = Logger()
 	return zapConfigs
 }
+
+// SetDefaultLogger allows customization of the default logger
+func SetDefaultLogger(handle *LoggerHandle) {
+	once.Do(initLogger)
+	defaultLogger.Store(handle)
+}
+
+// UpdateLoggingConfig is used to reconfigure logging.
+// This uses config keys of the form log.{logger}.level={level}.
+// The default level is set by log.level={level}
+func UpdateLoggingConfig(config map[string]string) {
+	once.Do(initLogger)
+	initLoggingConfig(config)
+}
+
+func initLoggingConfig(config map[string]string) {
+	levelMap := make(map[string]zapcore.Level)
+	levelMap[""] = zapcore.InfoLevel

Review Comment:
   Can we have a constant for `""`, it's not entirely clear what it represents. I guess it's a default level, but a naming would help.



##########
pkg/log/logger.go:
##########
@@ -82,3 +171,87 @@ func GetZapConfigs() *zap.Config {
 	_ = Logger()
 	return zapConfigs
 }
+
+// SetDefaultLogger allows customization of the default logger
+func SetDefaultLogger(handle *LoggerHandle) {
+	once.Do(initLogger)
+	defaultLogger.Store(handle)
+}
+
+// UpdateLoggingConfig is used to reconfigure logging.
+// This uses config keys of the form log.{logger}.level={level}.
+// The default level is set by log.level={level}
+func UpdateLoggingConfig(config map[string]string) {
+	once.Do(initLogger)
+	initLoggingConfig(config)
+}
+
+func initLoggingConfig(config map[string]string) {
+	levelMap := make(map[string]zapcore.Level)
+	levelMap[""] = zapcore.InfoLevel

Review Comment:
   Can we have a constant for `""`? It's not entirely clear what it represents. I guess it's a default level, but a naming would help.



-- 
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] craigcondit commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1231328943


##########
pkg/log/logger_test.go:
##########
@@ -0,0 +1,244 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+*/
+
+package log
+
+import (
+	"fmt"
+	"os"
+	"testing"
+	"time"
+
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+	"gotest.tools/v3/assert"
+)
+
+var logDir string
+var logFile string
+
+var iterations = 100000
+
+func TestLoggerIds(t *testing.T) {
+	_ = Logger()
+
+	// validate logger count
+	assert.Equal(t, 4, len(loggers), "wrong logger count")
+
+	// validate that all loggers are populated and have sequential ids
+	for i := 0; i < len(loggers); i++ {
+		handle := loggers[i]
+		assert.Assert(t, handle != nil, "nil handle for index", i)
+		assert.Equal(t, handle.id, i+1, "wrong id", handle.name)
+	}
+}
+
+func BenchmarkLegacyLoggerDebug(b *testing.B) {
+	benchmarkLegacyLoggerDebug(b.N)
+}
+
+func TestLegacyLoggerDebug(t *testing.T) {
+	nsOp := benchmarkLegacyLoggerDebug(iterations)
+	RootLogger().Info("log.Logger() performance", zap.Int64("debug (ns/op)", nsOp))
+}
+
+func benchmarkLegacyLoggerDebug(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		RootLogger().Debug("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkLegacyLoggerInfo(b *testing.B) {
+	benchmarkLegacyLoggerInfo(b.N)
+}
+
+func TestLegacyLoggerInfo(t *testing.T) {
+	nsOp := benchmarkLegacyLoggerInfo(iterations)
+	RootLogger().Info("log.Logger() performance", zap.Int64("info (ns/op)", nsOp))
+}
+
+func benchmarkLegacyLoggerInfo(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		RootLogger().Info("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerDebug(b *testing.B) {
+	benchmarkScopedLoggerDebug(b.N)
+}
+
+func TestScopedLoggerDebug(t *testing.T) {
+	nsOp := benchmarkScopedLoggerDebug(iterations)
+	Log(Test).Info("log.Log(...) performance (root=INFO)", zap.Int64("debug (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerDebug(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.level": "INFO",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(K8Shim).Debug("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerInfo(b *testing.B) {
+	benchmarkScopedLoggerInfo(b.N)
+}
+
+func TestScopedLoggerInfo(t *testing.T) {
+	nsOp := benchmarkScopedLoggerInfo(iterations)
+	Log(Test).Info("log.Log(...) performance (root=INFO)", zap.Int64("info (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerInfo(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.level": "INFO",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(K8Shim).Info("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerDebugEnabled(b *testing.B) {
+	benchmarkScopedLoggerDebugEnabled(b.N)
+}
+
+func TestScopedLoggerDebugEnabled(t *testing.T) {
+	nsOp := benchmarkScopedLoggerDebugEnabled(iterations)
+	Log(Test).Info("log.Log(...) performance (root=DEBUG)", zap.Int64("debug (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerDebugEnabled(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.test.level": "DEBUG",
+	})
+	start := time.Now()
+	for i := 0; i < iterations; i++ {
+		Log(Test).Debug("test", zap.String("foo", "bar"))
+	}
+	return (time.Since(start).Nanoseconds()) / int64(iterations)
+}
+
+func BenchmarkScopedLoggerInfoFiltered(b *testing.B) {
+	benchmarkScopedLoggerInfoFiltered(b.N)
+}
+
+func TestScopedLoggerInfoFiltered(t *testing.T) {
+	nsOp := benchmarkScopedLoggerInfoFiltered(iterations)
+	Log(Test).Info("log.Log(...) performance (root=DEBUG)", zap.Int64("info (ns/op)", nsOp))
+}
+
+func benchmarkScopedLoggerInfoFiltered(iterations int) int64 {
+	_ = Logger()
+	initTestLogger()
+	defer resetTestLogger()
+	UpdateLoggingConfig(map[string]string{
+		"log.test.level": "DEBUG",

Review Comment:
   Because the log level is implemented as a pre-check before we get to the root (real) logger, there's two scenarios:
   
   1) Log level matches at both levels (i.e. root is INFO, specific logger is also INFO)
   2) Log level does not match (i.e. root is DEBUG, specific logger is INFO)
   
   In the second case this is measuring the performance of logging at DEBUG level when the logger is set to INFO, so the filtering has to happen in our code vs. in zap. The overhead should be minimal, but this test just verifies that.



-- 
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] craigcondit commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1231394003


##########
pkg/log/logger.go:
##########
@@ -30,12 +33,95 @@ var once sync.Once
 var logger *zap.Logger
 var zapConfigs *zap.Config
 
+// LoggerHandle is used to efficiently look up logger references
+type LoggerHandle struct {
+	id   int
+	name string
+}
+
+func (h LoggerHandle) String() string {
+	return h.name
+}
+
+// Logger constants for configuration
+const (
+	defaultLog  = "log.level"
+	logPrefix   = "log."
+	levelSuffix = ".level"
+)
+
+// Predefined loggers: when adding new loggers, ids must be sequential, and all must be added to the loggers slice in the same order
+
+var K8Shim = &LoggerHandle{id: 1, name: "k8shim"}
+var Kubernetes = &LoggerHandle{id: 2, name: "kubernetes"}
+var Admission = &LoggerHandle{id: 3, name: "admission"}
+var Test = &LoggerHandle{id: 4, name: "test"}
+
+var loggers = []*LoggerHandle{
+	K8Shim,
+	Kubernetes,
+	Admission,
+	Test,
+}
+
+type loggerConfig struct {
+	loggers []*zap.Logger
+}
+
+var currentLoggerConfig = atomic.Pointer[loggerConfig]{}
+var defaultLogger = atomic.Pointer[LoggerHandle]{}
+
+// Logger retrieves the global logger
 func Logger() *zap.Logger {
+	once.Do(initLogger)
+	return Log(defaultLogger.Load())
+}
+
+// RootLogger retrieves the root logger
+func RootLogger() *zap.Logger {
 	once.Do(initLogger)
 	return logger
 }
 
+// Log retrieves a standard logger
+func Log(handle *LoggerHandle) *zap.Logger {
+	once.Do(initLogger)
+	if handle == nil || handle.id == 0 {
+		handle = defaultLogger.Load()
+	}
+	conf := currentLoggerConfig.Load()
+	return conf.loggers[handle.id-1]
+}
+
+func createLogger(levelMap map[string]zapcore.Level, name string) *zap.Logger {
+	level := loggerLevel(levelMap, name)
+	return logger.Named(name).WithOptions(zap.WrapCore(func(inner zapcore.Core) zapcore.Core {
+		return filteredCore{inner: inner, level: level}
+	}))
+}
+
+func loggerLevel(levelMap map[string]zapcore.Level, name string) zapcore.Level {
+	for ; name != ""; name = parentLogger(name) {
+		if level, ok := levelMap[name]; ok {
+			return level
+		}
+	}
+	if level, ok := levelMap[""]; ok {
+		return level
+	}
+	return zapcore.InfoLevel
+}
+
+func parentLogger(name string) string {
+	i := strings.LastIndex(name, ".")
+	if i < 0 {
+		return ""
+	}
+	return name[0:i]

Review Comment:
   Added test coverage for both `parentLogger()` and `parseLevel()`.



-- 
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] craigcondit commented on a diff in pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit commented on code in PR #611:
URL: https://github.com/apache/yunikorn-k8shim/pull/611#discussion_r1231395464


##########
pkg/log/logger.go:
##########
@@ -30,12 +33,95 @@ var once sync.Once
 var logger *zap.Logger
 var zapConfigs *zap.Config
 
+// LoggerHandle is used to efficiently look up logger references
+type LoggerHandle struct {
+	id   int
+	name string
+}
+
+func (h LoggerHandle) String() string {
+	return h.name
+}
+
+// Logger constants for configuration
+const (
+	defaultLog  = "log.level"
+	logPrefix   = "log."
+	levelSuffix = ".level"
+)
+
+// Predefined loggers: when adding new loggers, ids must be sequential, and all must be added to the loggers slice in the same order
+
+var K8Shim = &LoggerHandle{id: 1, name: "k8shim"}
+var Kubernetes = &LoggerHandle{id: 2, name: "kubernetes"}
+var Admission = &LoggerHandle{id: 3, name: "admission"}
+var Test = &LoggerHandle{id: 4, name: "test"}
+
+var loggers = []*LoggerHandle{
+	K8Shim,
+	Kubernetes,
+	Admission,
+	Test,
+}
+
+type loggerConfig struct {
+	loggers []*zap.Logger
+}
+
+var currentLoggerConfig = atomic.Pointer[loggerConfig]{}
+var defaultLogger = atomic.Pointer[LoggerHandle]{}
+
+// Logger retrieves the global logger
 func Logger() *zap.Logger {
+	once.Do(initLogger)
+	return Log(defaultLogger.Load())
+}
+
+// RootLogger retrieves the root logger
+func RootLogger() *zap.Logger {
 	once.Do(initLogger)
 	return logger
 }
 
+// Log retrieves a standard logger
+func Log(handle *LoggerHandle) *zap.Logger {
+	once.Do(initLogger)
+	if handle == nil || handle.id == 0 {
+		handle = defaultLogger.Load()
+	}
+	conf := currentLoggerConfig.Load()
+	return conf.loggers[handle.id-1]
+}
+
+func createLogger(levelMap map[string]zapcore.Level, name string) *zap.Logger {
+	level := loggerLevel(levelMap, name)
+	return logger.Named(name).WithOptions(zap.WrapCore(func(inner zapcore.Core) zapcore.Core {
+		return filteredCore{inner: inner, level: level}
+	}))
+}
+
+func loggerLevel(levelMap map[string]zapcore.Level, name string) zapcore.Level {

Review Comment:
   Added extensive comments throughout.



-- 
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] craigcondit closed pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging

Posted by "craigcondit (via GitHub)" <gi...@apache.org>.
craigcondit closed pull request #611: [YUNIKORN-1797] K8Shim: Implement scoped logging
URL: https://github.com/apache/yunikorn-k8shim/pull/611


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