Skip to content

Commit

Permalink
Introduced viper as the entry for configurations (#328)
Browse files Browse the repository at this point in the history
  • Loading branch information
powerfooI authored Apr 24, 2024
1 parent 0c88386 commit 22c9018
Show file tree
Hide file tree
Showing 7 changed files with 398 additions and 12 deletions.
16 changes: 8 additions & 8 deletions cmd/operator/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,11 @@ package main

import (
"context"
"flag"
"os"

//+kubebuilder:scaffold:imports

"github.com/spf13/pflag"
"go.uber.org/zap/zapcore"
"k8s.io/apimachinery/pkg/runtime"
utilruntime "k8s.io/apimachinery/pkg/util/runtime"
Expand Down Expand Up @@ -66,15 +66,15 @@ func main() {
var enableLeaderElection bool
var probeAddr string
var logVerbosity int
flag.StringVar(&namespace, "namespace", "", "The namespace to run oceanbase, default value is empty means all.")
flag.StringVar(&managerNamespace, "manager-namespace", "oceanbase-system", "The namespace to run manager tools.")
flag.StringVar(&metricsAddr, "metrics-bind-address", ":8080", "The address the metric endpoint binds to.")
flag.StringVar(&probeAddr, "health-probe-bind-address", ":8081", "The address the probe endpoint binds to.")
flag.BoolVar(&enableLeaderElection, "leader-elect", false,
pflag.StringVar(&namespace, "namespace", "", "The namespace to run oceanbase, default value is empty means all.")
pflag.StringVar(&managerNamespace, "manager-namespace", "oceanbase-system", "The namespace to run manager tools.")
pflag.StringVar(&metricsAddr, "metrics-bind-address", ":8080", "The address the metric endpoint binds to.")
pflag.StringVar(&probeAddr, "health-probe-bind-address", ":8081", "The address the probe endpoint binds to.")
pflag.BoolVar(&enableLeaderElection, "leader-elect", false,
"Enable leader election for controller manager. "+
"Enabling this will ensure there is only one active controller manager.")
flag.IntVar(&logVerbosity, "log-verbosity", 0, "Log verbosity level, 0 is info, 1 is debug, 2 is trace")
flag.Parse()
pflag.IntVar(&logVerbosity, "log-verbosity", 0, "Log verbosity level, 0 is info, 1 is debug, 2 is trace")
pflag.Parse()

opts := zap.Options{
Development: logVerbosity > 0,
Expand Down
25 changes: 25 additions & 0 deletions internal/config/operator/config_suite_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
Copyright (c) 2023 OceanBase
ob-operator is licensed under Mulan PSL v2.
You can use this software according to the terms and conditions of the Mulan PSL v2.
You may obtain a copy of Mulan PSL v2 at:
http://license.coscl.org.cn/MulanPSL2
THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
See the Mulan PSL v2 for more details.
*/

package operator

import (
"testing"

. "github.com/onsi/ginkgo/v2"
. "github.com/onsi/gomega"
)

func TestOperator(t *testing.T) {
RegisterFailHandler(Fail)
RunSpecs(t, "Operator Suite")
}
77 changes: 77 additions & 0 deletions internal/config/operator/default.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
Copyright (c) 2023 OceanBase
ob-operator is licensed under Mulan PSL v2.
You can use this software according to the terms and conditions of the Mulan PSL v2.
You may obtain a copy of Mulan PSL v2 at:
http://license.coscl.org.cn/MulanPSL2
THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
See the Mulan PSL v2 for more details.
*/

package operator

import (
"github.com/spf13/viper"

oc "github.com/oceanbase/ob-operator/internal/const/oceanbase"
"github.com/oceanbase/ob-operator/pkg/database"
)

var defaultConfigMap = map[string]any{
"namespace": "",
"manager-namespace": "oceanbase-system",
"metric-addr": ":8080",
"leader-elect": true,
"health-probe-bind-address": ":8081",
"log-verbosity": 0,
"disable-webhooks": false,

"task.debug": false,
"task.poolSize": 10000,

"telemetry.disabled": false,
"telemetry.debug": false,
"telemetry.host": "https://openwebapi.oceanbase.com",

"database.connectionLRUCacheSize": database.DefaultLRUCacheSize,

"resource.defaultDiskExpandPercent": oc.DefaultDiskExpandPercent,
"resource.defaultLogPercent": oc.DefaultLogPercent,
"resource.initialDataDiskUsePercent": oc.InitialDataDiskUsePercent,
"resource.defaultDiskUsePercent": oc.DefaultDiskUsePercent,
"resource.defaultMemoryLimitPercent": oc.DefaultMemoryLimitPercent,
"resource.defaultMemoryLimitSize": oc.DefaultMemoryLimitSize,
"resource.defaultDatafileMaxSize": oc.DefaultDatafileMaxSize,
"resource.defaultDatafileNextSize": oc.DefaultDatafileNextSize,
"resource.minMemorySize": oc.MinMemorySizeS,
"resource.minDataDiskSize": oc.MinDataDiskSizeS,
"resource.minRedoLogDiskSize": oc.MinRedoLogDiskSizeS,
"resource.minLogDiskSize": oc.MinLogDiskSizeS,

"time.tenantOpRetryTimes": oc.TenantOpRetryTimes,
"time.tenantOpRetryGapSeconds": oc.TenantOpRetryGapSeconds,
"time.taskMaxRetryTimes": oc.TaskMaxRetryTimes,
"time.taskRetryBackoffThreshold": oc.TaskRetryBackoffThreshold,
"time.probeCheckPeriodSeconds": oc.ProbeCheckPeriodSeconds,
"time.probeCheckDelaySeconds": oc.ProbeCheckDelaySeconds,
"time.getConnectionMaxRetries": oc.GetConnectionMaxRetries,
"time.checkConnectionInterval": oc.CheckConnectionInterval,
"time.checkJobInterval": oc.CheckJobInterval,
"time.checkJobMaxRetries": oc.CheckJobMaxRetries,
"time.commonCheckInterval": oc.CommonCheckInterval,
"time.bootstrapTimeoutSeconds": oc.BootstrapTimeoutSeconds,
"time.localityChangeTimeoutSeconds": oc.LocalityChangeTimeoutSeconds,
"time.defaultStateWaitTimeout": oc.DefaultStateWaitTimeout,
"time.timeConsumingStateWaitTimeout": oc.TimeConsumingStateWaitTimeout,
"time.waitForJobTimeoutSeconds": oc.WaitForJobTimeoutSeconds,
"time.serverDeleteTimeoutSeconds": oc.ServerDeleteTimeoutSeconds,
"time.tolerateServerPodNotReadyMinutes": oc.TolerateServerPodNotReadyMinutes,
}

func setDefaultConfigs(vp *viper.Viper) {
for k, v := range defaultConfigMap {
vp.SetDefault(k, v)
}
}
70 changes: 70 additions & 0 deletions internal/config/operator/new.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
Copyright (c) 2023 OceanBase
ob-operator is licensed under Mulan PSL v2.
You can use this software according to the terms and conditions of the Mulan PSL v2.
You may obtain a copy of Mulan PSL v2 at:
http://license.coscl.org.cn/MulanPSL2
THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
See the Mulan PSL v2 for more details.
*/

package operator

import (
"flag"
"strings"
"sync"

"github.com/spf13/pflag"
"github.com/spf13/viper"
)

var (
cfgOnce sync.Once
cfg *Config
)

func newConfig() *Config {
v := viper.New()
v.AddConfigPath(".")
v.AddConfigPath("/etc/admin/oceanbase")
v.SetConfigName(".ob-operator")
v.SetConfigType("yaml")

setDefaultConfigs(v)

pflag.CommandLine.AddGoFlagSet(flag.CommandLine)
pflag.Parse()
_ = v.BindPFlags(pflag.CommandLine)

v.AutomaticEnv()
v.SetEnvPrefix("OB_OPERATOR")
v.SetEnvKeyReplacer(strings.NewReplacer(".", "_", "-", ""))

config := &Config{}
if err := v.ReadInConfig(); err != nil {
if _, ok := err.(viper.ConfigFileNotFoundError); !ok {
panic(err)
}
}
if err := v.Unmarshal(config); err != nil {
panic(err)
}
config.v = v
return config
}

func GetConfig() *Config {
if cfg == nil {
cfgOnce.Do(func() {
cfg = newConfig()
})
}
return cfg
}

func (c *Config) Write() error {
return c.v.WriteConfigAs(".ob-operator.yaml")
}
115 changes: 115 additions & 0 deletions internal/config/operator/new_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/*
Copyright (c) 2023 OceanBase
ob-operator is licensed under Mulan PSL v2.
You can use this software according to the terms and conditions of the Mulan PSL v2.
You may obtain a copy of Mulan PSL v2 at:
http://license.coscl.org.cn/MulanPSL2
THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
See the Mulan PSL v2 for more details.
*/

package operator

import (
"flag"
"os"

"github.com/mitchellh/mapstructure"
. "github.com/onsi/ginkgo/v2"
. "github.com/onsi/gomega"
)

var _ = Describe("Config", func() {
GinkgoHelper()

Context("default", func() {
It("should return default config", func() {
output := Config{}
Expect(mapstructure.Decode(defaultConfigMap, &output)).To(Succeed())

got := newConfig()
Expect(got.Database.ConnectionLRUCacheSize).To(BeEquivalentTo(defaultConfigMap["database.connectionLRUCacheSize"]))
Expect(got.Resource.DefaultDiskExpandPercent).To(BeEquivalentTo(defaultConfigMap["resource.defaultDiskExpandPercent"]))
Expect(got.Resource.DefaultLogPercent).To(BeEquivalentTo(defaultConfigMap["resource.defaultLogPercent"]))
Expect(got.Resource.InitialDataDiskUsePercent).To(BeEquivalentTo(defaultConfigMap["resource.initialDataDiskUsePercent"]))
Expect(got.Resource.DefaultDiskUsePercent).To(BeEquivalentTo(defaultConfigMap["resource.defaultDiskUsePercent"]))
Expect(got.Resource.DefaultMemoryLimitPercent).To(BeEquivalentTo(defaultConfigMap["resource.defaultMemoryLimitPercent"]))
Expect(got.Resource.DefaultMemoryLimitSize).To(BeEquivalentTo(defaultConfigMap["resource.defaultMemoryLimitSize"]))
Expect(got.Resource.DefaultDatafileMaxSize).To(BeEquivalentTo(defaultConfigMap["resource.defaultDatafileMaxSize"]))
Expect(got.Resource.DefaultDatafileNextSize).To(BeEquivalentTo(defaultConfigMap["resource.defaultDatafileNextSize"]))
Expect(got.Resource.MinMemorySize).To(BeEquivalentTo(defaultConfigMap["resource.minMemorySize"]))
Expect(got.Resource.MinDataDiskSize).To(BeEquivalentTo(defaultConfigMap["resource.minDataDiskSize"]))
Expect(got.Resource.MinRedoLogDiskSize).To(BeEquivalentTo(defaultConfigMap["resource.minRedoLogDiskSize"]))
Expect(got.Resource.MinLogDiskSize).To(BeEquivalentTo(defaultConfigMap["resource.minLogDiskSize"]))
Expect(got.Time.TenantOpRetryTimes).To(BeEquivalentTo(defaultConfigMap["time.tenantOpRetryTimes"]))
Expect(got.Time.TenantOpRetryGapSeconds).To(BeEquivalentTo(defaultConfigMap["time.tenantOpRetryGapSeconds"]))
Expect(got.Time.TaskMaxRetryTimes).To(BeEquivalentTo(defaultConfigMap["time.taskMaxRetryTimes"]))
Expect(got.Time.TaskRetryBackoffThreshold).To(BeEquivalentTo(defaultConfigMap["time.taskRetryBackoffThreshold"]))
Expect(got.Time.ProbeCheckPeriodSeconds).To(BeEquivalentTo(defaultConfigMap["time.probeCheckPeriodSeconds"]))
Expect(got.Time.ProbeCheckDelaySeconds).To(BeEquivalentTo(defaultConfigMap["time.probeCheckDelaySeconds"]))
Expect(got.Time.GetConnectionMaxRetries).To(BeEquivalentTo(defaultConfigMap["time.getConnectionMaxRetries"]))
Expect(got.Time.CheckConnectionInterval).To(BeEquivalentTo(defaultConfigMap["time.checkConnectionInterval"]))
Expect(got.Time.CheckJobInterval).To(BeEquivalentTo(defaultConfigMap["time.checkJobInterval"]))
Expect(got.Time.CheckJobMaxRetries).To(BeEquivalentTo(defaultConfigMap["time.checkJobMaxRetries"]))
Expect(got.Time.CommonCheckInterval).To(BeEquivalentTo(defaultConfigMap["time.commonCheckInterval"]))
Expect(got.Time.BootstrapTimeoutSeconds).To(BeEquivalentTo(defaultConfigMap["time.bootstrapTimeoutSeconds"]))
Expect(got.Time.LocalityChangeTimeoutSeconds).To(BeEquivalentTo(defaultConfigMap["time.localityChangeTimeoutSeconds"]))
Expect(got.Time.DefaultStateWaitTimeout).To(BeEquivalentTo(defaultConfigMap["time.defaultStateWaitTimeout"]))
Expect(got.Time.TimeConsumingStateWaitTimeout).To(BeEquivalentTo(defaultConfigMap["time.timeConsumingStateWaitTimeout"]))
Expect(got.Time.WaitForJobTimeoutSeconds).To(BeEquivalentTo(defaultConfigMap["time.waitForJobTimeoutSeconds"]))
Expect(got.Time.ServerDeleteTimeoutSeconds).To(BeEquivalentTo(defaultConfigMap["time.serverDeleteTimeoutSeconds"]))
Expect(got.Telemetry.Disabled).To(BeEquivalentTo(defaultConfigMap["telemetry.disabled"]))
Expect(got.Telemetry.Debug).To(BeEquivalentTo(defaultConfigMap["telemetry.debug"]))
Expect(got.Telemetry.Host).To(BeEquivalentTo(defaultConfigMap["telemetry.host"]))
Expect(got.Task.Debug).To(BeEquivalentTo(defaultConfigMap["task.debug"]))
Expect(got.Task.PoolSize).To(BeEquivalentTo(defaultConfigMap["task.poolSize"]))
// Expect(got.Manager.DisableWebhooks).To(BeEquivalentTo(defaultConfigMap["manager.disableWebhooks"]))
// Expect(got.Manager.LogVerbosity).To(BeEquivalentTo(defaultConfigMap["manager.logVerbosity"]))
})
})

Context("envVars", func() {
BeforeEach(func() {
os.Setenv("OB_OPERATOR_TASK_POOLSIZE", "9876")
os.Setenv("OB_OPERATOR_TIME_TASKMAXRETRYTIMES", "1234")
})
AfterEach(func() {
os.Unsetenv("OB_OPERATOR_TASK_POOLSIZE")
os.Unsetenv("OB_OPERATOR_TIME_TASKMAXRETRYTIMES")
})
It("should return config with envVars", func() {
Expect(os.Getenv("OB_OPERATOR_TASK_POOLSIZE")).To(Equal("9876"))
got := newConfig()
Expect(got.Task.PoolSize).To(Equal(9876))
Expect(got.Time.TaskMaxRetryTimes).To(Equal(1234))
})
})

Context("flags", func() {
It("should return config with flags", func() {
var namespace string
var managerNamespace string
var metricsAddr string
var enableLeaderElection bool
var probeAddr string
var logVerbosity int
flag.StringVar(&namespace, "namespace", "", "The namespace to run oceanbase, default value is empty means all.")
flag.StringVar(&managerNamespace, "manager-namespace", "oceanbase-system", "The namespace to run manager tools.")
flag.StringVar(&metricsAddr, "metrics-bind-address", ":8080", "The address the metric endpoint binds to.")
flag.StringVar(&probeAddr, "health-probe-bind-address", ":8081", "The address the probe endpoint binds to.")
flag.BoolVar(&enableLeaderElection, "leader-elect", false,
"Enable leader election for controller manager. "+
"Enabling this will ensure there is only one active controller manager.")
flag.IntVar(&logVerbosity, "log-verbosity", 0, "Log verbosity level, 0 is info, 1 is debug, 2 is trace")
Expect(flag.CommandLine.Parse([]string{
"--log-verbosity", "1",
})).To(Succeed())
GinkgoLogr.Info("logVerbosity", "logVerbosity", logVerbosity)

got := newConfig()
Expect(got.Manager.LogVerbosity).To(Equal(1))
})
})
})
Loading

0 comments on commit 22c9018

Please sign in to comment.