Add initial wrappers for prometheus.Counter and prometheus.CounterVec. Also add wrapper around prometheus.Registry to customize control-flow

This commit is contained in:
Han Kang 2019-04-23 15:59:29 -07:00
parent 91528d60d5
commit cf181cdd9a
8 changed files with 990 additions and 2 deletions

View File

@ -8,20 +8,38 @@ load(
go_library(
name = "go_default_library",
srcs = ["util.go"],
srcs = [
"counter.go",
"metric.go",
"opts.go",
"registry.go",
"util.go",
"wrappers.go",
],
importpath = "k8s.io/kubernetes/pkg/util/metrics",
deps = [
"//staging/src/k8s.io/client-go/util/flowcontrol:go_default_library",
"//vendor/github.com/blang/semver:go_default_library",
"//vendor/github.com/prometheus/client_golang/prometheus:go_default_library",
"//vendor/github.com/prometheus/client_model/go:go_default_library",
"//vendor/k8s.io/klog:go_default_library",
],
)
go_test(
name = "go_default_test",
srcs = ["util_test.go"],
srcs = [
"counter_test.go",
"registry_test.go",
"util_test.go",
],
embed = [":go_default_library"],
deps = [
"//staging/src/k8s.io/client-go/util/flowcontrol:go_default_library",
"//vendor/github.com/blang/semver:go_default_library",
"//vendor/github.com/prometheus/client_golang/prometheus:go_default_library",
"//vendor/github.com/prometheus/common/expfmt:go_default_library",
"//vendor/github.com/stretchr/testify/assert:go_default_library",
],
)

135
pkg/util/metrics/counter.go Normal file
View File

@ -0,0 +1,135 @@
/*
Copyright 2019 The Kubernetes Authors.
Licensed 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 metrics
import (
"github.com/blang/semver"
"github.com/prometheus/client_golang/prometheus"
)
// kubeCounter is our internal representation for our wrapping struct around prometheus
// counters. kubeCounter implements both KubeCollector and KubeCounter.
type kubeCounter struct {
KubeCounter
*CounterOpts
lazyMetric
selfCollector
}
// NewCounter returns an object which satisfies the KubeCollector and KubeCounter interfaces.
// However, the object returned will not measure anything unless the collector is first
// registered, since the metric is lazily instantiated.
func NewCounter(opts CounterOpts) *kubeCounter {
// todo: handle defaulting better
if opts.StabilityLevel == "" {
opts.StabilityLevel = ALPHA
}
kc := &kubeCounter{
CounterOpts: &opts,
lazyMetric: lazyMetric{},
}
kc.setPrometheusCounter(noop)
kc.lazyInit(kc)
return kc
}
// setPrometheusCounter sets the underlying KubeCounter object, i.e. the thing that does the measurement.
func (c *kubeCounter) setPrometheusCounter(counter prometheus.Counter) {
c.KubeCounter = counter
c.initSelfCollection(counter)
}
// GetDeprecatedVersion returns a pointer to the Version or nil
func (c *kubeCounter) GetDeprecatedVersion() *semver.Version {
return c.CounterOpts.DeprecatedVersion
}
// initializeMetric invocation creates the actual underlying Counter. Until this method is called
// our underlying counter is a no-op.
func (c *kubeCounter) initializeMetric() {
c.CounterOpts.annotateStabilityLevel()
// this actually creates the underlying prometheus counter.
c.setPrometheusCounter(prometheus.NewCounter(c.CounterOpts.toPromCounterOpts()))
}
// initializeDeprecatedMetric invocation creates the actual (but deprecated) Counter. Until this method
// is called our underlying counter is a no-op.
func (c *kubeCounter) initializeDeprecatedMetric() {
c.CounterOpts.markDeprecated()
c.initializeMetric()
}
// kubeCounterVec is our internal representation of our wrapping struct around prometheus
// counterVecs. kubeCounterVec implements both KubeCollector and KubeCounterVec.
type kubeCounterVec struct {
*prometheus.CounterVec
*CounterOpts
lazyMetric
originalLabels []string
}
// NewCounterVec returns an object which satisfies the KubeCollector and KubeCounterVec interfaces.
// However, the object returned will not measure anything unless the collector is first
// registered, since the metric is lazily instantiated.
func NewCounterVec(opts CounterOpts, labels []string) *kubeCounterVec {
cv := &kubeCounterVec{
CounterVec: noopCounterVec,
CounterOpts: &opts,
originalLabels: labels,
lazyMetric: lazyMetric{},
}
cv.lazyInit(cv)
return cv
}
// GetDeprecatedVersion returns a pointer to the Version or nil
func (v *kubeCounterVec) GetDeprecatedVersion() *semver.Version {
return v.CounterOpts.DeprecatedVersion
}
// initializeMetric invocation creates the actual underlying CounterVec. Until this method is called
// our underlying counterVec is a no-op.
func (v *kubeCounterVec) initializeMetric() {
v.CounterVec = prometheus.NewCounterVec(v.CounterOpts.toPromCounterOpts(), v.originalLabels)
}
// initializeMetric invocation creates the actual (but deprecated) CounterVec. Until this method is called
// our underlying counterVec is a no-op.
func (v *kubeCounterVec) initializeDeprecatedMetric() {
v.CounterOpts.markDeprecated()
v.initializeMetric()
}
// Default Prometheus behavior actually results in the creation of a new metric
// if a metric with the unique label values is not found in the underlying stored metricMap. This
// is undesirable for us, since we want a way to turn OFF metrics which end up turning into memory
// leaks.
//
// For reference: https://github.com/prometheus/client_golang/blob/master/prometheus/counter.go#L148-L177
func (v *kubeCounterVec) WithLabelValues(lvs ...string) KubeCounter {
if !v.IsCreated() {
return noop // return no-op counter
}
return v.CounterVec.WithLabelValues(lvs...)
}
func (v *kubeCounterVec) With(labels prometheus.Labels) KubeCounter {
if !v.IsCreated() {
return noop // return no-op counter
}
return v.CounterVec.With(labels)
}

View File

@ -0,0 +1,217 @@
/*
Copyright 2019 The Kubernetes Authors.
Licensed 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 metrics
import (
"bytes"
"github.com/blang/semver"
"github.com/prometheus/common/expfmt"
"testing"
)
func TestCounter(t *testing.T) {
v115 := semver.MustParse("1.15.0")
v114 := semver.MustParse("1.14.0")
var tests = []struct {
desc string
CounterOpts
registryVersion *semver.Version
expectedMetricCount int
expectedHelp string
}{
{
desc: "Test non deprecated",
CounterOpts: CounterOpts{
Namespace: "namespace",
Name: "metric_test_name",
Subsystem: "subsystem",
StabilityLevel: ALPHA,
Help: "counter help",
},
registryVersion: &v115,
expectedMetricCount: 1,
expectedHelp: "[ALPHA] counter help",
},
{
desc: "Test deprecated",
CounterOpts: CounterOpts{
Namespace: "namespace",
Name: "metric_test_name",
Subsystem: "subsystem",
Help: "counter help",
StabilityLevel: ALPHA,
DeprecatedVersion: &v115,
},
registryVersion: &v115,
expectedMetricCount: 1,
expectedHelp: "[ALPHA] (Deprecated since 1.15.0) counter help",
},
{
desc: "Test hidden",
CounterOpts: CounterOpts{
Namespace: "namespace",
Name: "metric_test_name",
Subsystem: "subsystem",
Help: "counter help",
StabilityLevel: ALPHA,
DeprecatedVersion: &v114,
},
registryVersion: &v115,
expectedMetricCount: 0,
},
}
for _, test := range tests {
t.Run(test.desc, func(t *testing.T) {
registry := NewKubeRegistry(*test.registryVersion)
c := NewCounter(test.CounterOpts)
registry.MustRegister(c)
ms, err := registry.Gather()
var buf bytes.Buffer
enc := expfmt.NewEncoder(&buf, "text/plain; version=0.0.4; charset=utf-8")
if len(ms) != test.expectedMetricCount {
t.Errorf("Got %v metrics, Want: %v metrics", len(ms), test.expectedMetricCount)
}
if err != nil {
t.Fatalf("Gather failed %v", err)
}
for _, metric := range ms {
err := enc.Encode(metric)
if err != nil {
t.Fatalf("Unexpected err %v in encoding the metric", err)
}
if metric.GetHelp() != test.expectedHelp {
t.Errorf("Got %s as help message, want %s", metric.GetHelp(), test.expectedHelp)
}
}
// let's increment the counter N number of times and verify that the metric retains the count correctly
numberOfTimesToIncrement := 3
for i := 0; i < numberOfTimesToIncrement; i++ {
c.Inc()
}
ms, err = registry.Gather()
if err != nil {
t.Fatalf("Gather failed %v", err)
}
for _, mf := range ms {
for _, m := range mf.GetMetric() {
if int(m.GetCounter().GetValue()) != numberOfTimesToIncrement {
t.Errorf("Got %v, wanted %v as the count", m.GetCounter().GetValue(), numberOfTimesToIncrement)
}
}
}
})
}
}
func TestCounterVec(t *testing.T) {
v115 := semver.MustParse("1.15.0")
v114 := semver.MustParse("1.14.0")
var tests = []struct {
desc string
CounterOpts
labels []string
registryVersion *semver.Version
expectedMetricFamilyCount int
expectedHelp string
}{
{
desc: "Test non deprecated",
CounterOpts: CounterOpts{
Namespace: "namespace",
Name: "metric_test_name",
Subsystem: "subsystem",
Help: "counter help",
},
labels: []string{"label_a", "label_b"},
registryVersion: &v115,
expectedMetricFamilyCount: 1,
expectedHelp: "counter help",
},
{
desc: "Test deprecated",
CounterOpts: CounterOpts{
Namespace: "namespace",
Name: "metric_test_name",
Subsystem: "subsystem",
Help: "counter help",
DeprecatedVersion: &v115,
},
labels: []string{"label_a", "label_b"},
registryVersion: &v115,
expectedMetricFamilyCount: 1,
expectedHelp: "(Deprecated since 1.15.0) counter help",
},
{
desc: "Test hidden",
CounterOpts: CounterOpts{
Namespace: "namespace",
Name: "metric_test_name",
Subsystem: "subsystem",
Help: "counter help",
DeprecatedVersion: &v114,
},
labels: []string{"label_a", "label_b"},
registryVersion: &v115,
expectedMetricFamilyCount: 0,
expectedHelp: "counter help",
},
}
for _, test := range tests {
t.Run(test.desc, func(t *testing.T) {
registry := NewKubeRegistry(*test.registryVersion)
c := NewCounterVec(test.CounterOpts, test.labels)
registry.MustRegister(c)
c.WithLabelValues("1", "2").Inc()
mfs, err := registry.Gather()
if len(mfs) != test.expectedMetricFamilyCount {
t.Errorf("Got %v metric families, Want: %v metric families", len(mfs), test.expectedMetricFamilyCount)
}
if err != nil {
t.Fatalf("Gather failed %v", err)
}
// we no-opt here when we don't have any metric families (i.e. when the metric is hidden)
for _, mf := range mfs {
if len(mf.GetMetric()) != 1 {
t.Errorf("Got %v metrics, wanted 1 as the count", len(mf.GetMetric()))
}
if mf.GetHelp() != test.expectedHelp {
t.Errorf("Got %s as help message, want %s", mf.GetHelp(), test.expectedHelp)
}
}
// let's increment the counter and verify that the metric still works
c.WithLabelValues("1", "3").Inc()
c.WithLabelValues("2", "3").Inc()
mfs, err = registry.Gather()
if err != nil {
t.Fatalf("Gather failed %v", err)
}
// we no-opt here when we don't have any metric families (i.e. when the metric is hidden)
for _, mf := range mfs {
if len(mf.GetMetric()) != 3 {
t.Errorf("Got %v metrics, wanted 3 as the count", len(mf.GetMetric()))
}
}
})
}
}

196
pkg/util/metrics/metric.go Normal file
View File

@ -0,0 +1,196 @@
/*
Copyright 2019 The Kubernetes Authors.
Licensed 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 metrics
import (
"github.com/blang/semver"
"github.com/prometheus/client_golang/prometheus"
dto "github.com/prometheus/client_model/go"
"k8s.io/klog"
"sync"
)
/**
* This extends the prometheus.Collector interface so that we can customize the metric
* registration process. Specifically, we defer metric initialization until ActuallyCreate
* is called, which then delegates to the underlying metric's initializeMetric or
* initializeDeprecatedMetric method call depending on whether the metric is deprecated or not.
*/
type KubeCollector interface {
Collector
LazyMetric
GetDeprecatedVersion() *semver.Version
// Each collector metric should provide an initialization function
// for both deprecated and non-deprecated variants of a metric. This
// is necessary since we are now deferring metric instantiation
// until the metric is actually registered somewhere.
initializeMetric()
initializeDeprecatedMetric()
}
// LazyMetric defines our registration functionality. We expect LazyMetric
// objects to lazily instantiate metrics (i.e defer metric instantiation until when
// ActuallyCreate is explicitly called).
type LazyMetric interface {
ActuallyCreate(*semver.Version) bool
IsCreated() bool
IsHidden() bool
IsDeprecated() bool
}
/*
* lazyMetric implements LazyMetric. A lazy metric is lazy because it waits until metric
* registration time before instantiation. Add it as an anonymous field to a struct that
* implements KubeCollector to get deferred registration behavior. You must call lazyInit
* with the KubeCollector itself as an argument.
*/
type lazyMetric struct {
isDeprecated bool
isHidden bool
isCreated bool
markDeprecationOnce sync.Once
createOnce sync.Once
self KubeCollector
}
func (r *lazyMetric) IsCreated() bool {
return r.isCreated
}
// lazyInit provides the lazyMetric with a reference to the KubeCollector it is supposed
// to allow lazy initialization for. It should be invoked in the factory function which creates new
// KubeCollector type objects.
func (r *lazyMetric) lazyInit(self KubeCollector) {
r.self = self
}
// determineDeprecationStatus figures out whether our lazy metric should be deprecated or not. It takes
// a Version argument which should be the version of the binary in which this code is currently being
// executed.
func (r *lazyMetric) determineDeprecationStatus(version semver.Version) {
selfVersion := r.self.GetDeprecatedVersion()
if selfVersion == nil {
return
}
r.markDeprecationOnce.Do(func() {
if selfVersion.LTE(version) {
r.isDeprecated = true
}
if selfVersion.LT(version) {
klog.Warningf("This metric has been deprecated for more than one release, hiding.")
r.isHidden = true
}
})
}
func (r *lazyMetric) IsHidden() bool {
return r.isHidden
}
func (r *lazyMetric) IsDeprecated() bool {
return r.isDeprecated
}
// Defer initialization of metric until we know if we actually need to
// register the thing. This wrapper just allows us to consolidate the
// syncOnce logic in a single spot and toggle the flag, since this
// behavior will be consistent across metrics.
//
// This no-opts and returns true if metric is already created.
func (r *lazyMetric) ActuallyCreate(version *semver.Version) bool {
if version != nil {
r.determineDeprecationStatus(*version)
}
// let's not create if this metric is slated to be hidden
if r.IsHidden() {
return false
}
r.createOnce.Do(func() {
r.isCreated = true
if r.IsDeprecated() {
r.self.initializeDeprecatedMetric()
} else {
r.self.initializeMetric()
}
})
return r.IsCreated()
}
/**
* This code is directly lifted from the prometheus codebase. It's a convenience struct which
* allows you satisfy the Collector interface automatically if you already satisfy the Metric interface.
*
* For reference: https://github.com/prometheus/client_golang/blob/65d3a96fbaa7c8c9535d7133d6d98cd50eed4db8/prometheus/collector.go#L98-L120
*/
type selfCollector struct {
metric prometheus.Metric
}
func (c *selfCollector) initSelfCollection(m prometheus.Metric) {
c.metric = m
}
func (c *selfCollector) Describe(ch chan<- *prometheus.Desc) {
ch <- c.metric.Desc()
}
func (c *selfCollector) Collect(ch chan<- prometheus.Metric) {
ch <- c.metric
}
// no-op vecs for convenience
var noopCounterVec = &prometheus.CounterVec{}
var noopHistogramVec = &prometheus.HistogramVec{}
var noopSummaryVec = &prometheus.SummaryVec{}
var noopGaugeVec = &prometheus.GaugeVec{}
var noopObserverVec = &noopObserverVector{}
// just use a convenience struct for all the no-ops
var noop = &noopMetric{}
type noopMetric struct{}
func (noopMetric) Inc() {}
func (noopMetric) Add(float64) {}
func (noopMetric) Dec() {}
func (noopMetric) Set(float64) {}
func (noopMetric) Sub(float64) {}
func (noopMetric) Observe(float64) {}
func (noopMetric) SetToCurrentTime() {}
func (noopMetric) Desc() *prometheus.Desc { return nil }
func (noopMetric) Write(*dto.Metric) error { return nil }
func (noopMetric) Describe(chan<- *prometheus.Desc) {}
func (noopMetric) Collect(chan<- prometheus.Metric) {}
type noopObserverVector struct{}
func (noopObserverVector) GetMetricWith(prometheus.Labels) (prometheus.Observer, error) {
return noop, nil
}
func (noopObserverVector) GetMetricWithLabelValues(...string) (prometheus.Observer, error) {
return noop, nil
}
func (noopObserverVector) With(prometheus.Labels) prometheus.Observer { return noop }
func (noopObserverVector) WithLabelValues(...string) prometheus.Observer { return noop }
func (noopObserverVector) CurryWith(prometheus.Labels) (prometheus.ObserverVec, error) {
return noopObserverVec, nil
}
func (noopObserverVector) MustCurryWith(prometheus.Labels) prometheus.ObserverVec {
return noopObserverVec
}
func (noopObserverVector) Describe(chan<- *prometheus.Desc) {}
func (noopObserverVector) Collect(chan<- prometheus.Metric) {}

78
pkg/util/metrics/opts.go Normal file
View File

@ -0,0 +1,78 @@
/*
Copyright 2019 The Kubernetes Authors.
Licensed 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 metrics
import (
"fmt"
"github.com/blang/semver"
"github.com/prometheus/client_golang/prometheus"
"sync"
)
// KubeOpts is superset struct for prometheus.Opts. We choose not to embed
// the prometheus Opts structure here because that would change struct initialization
// in the manner to which people are currently accustomed.
//
// Name must be set to a non-empty string. DeprecatedVersion is defined only
// if the metric for which this options applies is, in fact, deprecated.
type KubeOpts struct {
Namespace string
Subsystem string
Name string
Help string
ConstLabels prometheus.Labels
DeprecatedVersion *semver.Version
deprecateOnce sync.Once
annotateOnce sync.Once
StabilityLevel StabilityLevel
}
type StabilityLevel string
const (
ALPHA StabilityLevel = "ALPHA"
STABLE StabilityLevel = "STABLE"
)
type CounterOpts KubeOpts
// Modify help description on the metric description.
func (o *CounterOpts) markDeprecated() {
o.deprecateOnce.Do(func() {
o.Help = fmt.Sprintf("(Deprecated since %v) %v", o.DeprecatedVersion, o.Help)
})
}
// annotateStabilityLevel annotates help description on the metric description with the stability level
// of the metric
func (o *CounterOpts) annotateStabilityLevel() {
o.annotateOnce.Do(func() {
o.Help = fmt.Sprintf("[%v] %v", o.StabilityLevel, o.Help)
})
}
// convenience function to allow easy transformation to the prometheus
// counterpart. This will do more once we have a proper label abstraction
func (o CounterOpts) toPromCounterOpts() prometheus.CounterOpts {
return prometheus.CounterOpts{
Namespace: o.Namespace,
Subsystem: o.Subsystem,
Name: o.Name,
Help: o.Help,
ConstLabels: o.ConstLabels,
}
}

View File

@ -0,0 +1,79 @@
/*
Copyright 2019 The Kubernetes Authors.
Licensed 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 metrics
import (
"github.com/blang/semver"
"github.com/prometheus/client_golang/prometheus"
dto "github.com/prometheus/client_model/go"
)
var (
// todo: load the version dynamically at application boot.
DefaultGlobalRegistry = NewKubeRegistry(semver.MustParse("1.15.0"))
)
type KubeRegistry struct {
PromRegistry
version semver.Version
}
// Register registers a collectable metric, but it uses a global registry.
func Register(c KubeCollector) error {
return DefaultGlobalRegistry.Register(c)
}
// MustRegister works like Register but registers any number of
// Collectors and panics upon the first registration that causes an
// error.
func MustRegister(cs ...KubeCollector) {
DefaultGlobalRegistry.MustRegister(cs...)
}
func (kr *KubeRegistry) Register(c KubeCollector) error {
if c.ActuallyCreate(&kr.version) {
return kr.PromRegistry.Register(c)
}
return nil
}
func (kr *KubeRegistry) MustRegister(cs ...KubeCollector) {
metrics := make([]prometheus.Collector, 0, len(cs))
for _, c := range cs {
if c.ActuallyCreate(&kr.version) {
metrics = append(metrics, c)
}
}
kr.PromRegistry.MustRegister(metrics...)
}
func (kr *KubeRegistry) Unregister(collector KubeCollector) bool {
return kr.PromRegistry.Unregister(collector)
}
func (kr *KubeRegistry) Gather() ([]*dto.MetricFamily, error) {
return kr.PromRegistry.Gather()
}
// NewRegistry creates a new vanilla Registry without any Collectors
// pre-registered.
func NewKubeRegistry(version semver.Version) *KubeRegistry {
return &KubeRegistry{
PromRegistry: prometheus.NewRegistry(),
version: version,
}
}

View File

@ -0,0 +1,198 @@
/*
Copyright 2019 The Kubernetes Authors.
Licensed 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 metrics
import (
"github.com/blang/semver"
"github.com/prometheus/client_golang/prometheus"
"github.com/stretchr/testify/assert"
"testing"
)
var (
v115 = semver.MustParse("1.15.0")
v114 = semver.MustParse("1.14.0")
alphaCounter = NewCounter(
CounterOpts{
Namespace: "some_namespace",
Name: "test_counter_name",
Subsystem: "subsystem",
StabilityLevel: ALPHA,
Help: "counter help",
},
)
alphaDeprecatedCounter = NewCounter(
CounterOpts{
Namespace: "some_namespace",
Name: "test_alpha_dep_counter",
Subsystem: "subsystem",
StabilityLevel: ALPHA,
Help: "counter help",
DeprecatedVersion: &v115,
},
)
alphaHiddenCounter = NewCounter(
CounterOpts{
Namespace: "some_namespace",
Name: "test_alpha_hidden_counter",
Subsystem: "subsystem",
StabilityLevel: ALPHA,
Help: "counter help",
DeprecatedVersion: &v114,
},
)
stableCounter = NewCounter(
CounterOpts{
Namespace: "some_namespace",
Name: "test_some_other_counter",
Subsystem: "subsystem",
StabilityLevel: STABLE,
Help: "counter help",
},
)
)
func TestRegister(t *testing.T) {
var tests = []struct {
desc string
metrics []*kubeCounter
registryVersion *semver.Version
expectedErrors []error
expectedIsCreatedValues []bool
expectedIsDeprecated []bool
expectedIsHidden []bool
}{
{
desc: "test alpha metric",
metrics: []*kubeCounter{alphaCounter},
registryVersion: &v115,
expectedErrors: []error{nil},
expectedIsCreatedValues: []bool{true},
expectedIsDeprecated: []bool{false},
expectedIsHidden: []bool{false},
},
{
desc: "test registering same metric multiple times",
metrics: []*kubeCounter{alphaCounter, alphaCounter},
registryVersion: &v115,
expectedErrors: []error{nil, prometheus.AlreadyRegisteredError{}},
expectedIsCreatedValues: []bool{true, true},
expectedIsDeprecated: []bool{false, false},
expectedIsHidden: []bool{false, false},
},
{
desc: "test alpha deprecated metric",
metrics: []*kubeCounter{alphaDeprecatedCounter},
registryVersion: &v115,
expectedErrors: []error{nil, prometheus.AlreadyRegisteredError{}},
expectedIsCreatedValues: []bool{true},
expectedIsDeprecated: []bool{true},
expectedIsHidden: []bool{false},
},
{
desc: "test alpha hidden metric",
metrics: []*kubeCounter{alphaHiddenCounter},
registryVersion: &v115,
expectedErrors: []error{nil, prometheus.AlreadyRegisteredError{}},
expectedIsCreatedValues: []bool{false},
expectedIsDeprecated: []bool{true},
expectedIsHidden: []bool{true},
},
}
for _, test := range tests {
t.Run(test.desc, func(t *testing.T) {
registry := NewKubeRegistry(*test.registryVersion)
for i, m := range test.metrics {
err := registry.Register(m)
if err != test.expectedErrors[i] && err.Error() != test.expectedErrors[i].Error() {
t.Errorf("Got unexpected error %v, wanted %v", err, test.expectedErrors[i])
}
if m.IsCreated() != test.expectedIsCreatedValues[i] {
t.Errorf("Got isCreated == %v, wanted isCreated to be %v", m.IsCreated(), test.expectedIsCreatedValues[i])
}
if m.IsDeprecated() != test.expectedIsDeprecated[i] {
t.Errorf("Got IsDeprecated == %v, wanted IsDeprecated to be %v", m.IsDeprecated(), test.expectedIsDeprecated[i])
}
if m.IsHidden() != test.expectedIsHidden[i] {
t.Errorf("Got IsHidden == %v, wanted IsHidden to be %v", m.IsHidden(), test.expectedIsDeprecated[i])
}
}
})
}
}
func TestMustRegister(t *testing.T) {
var tests = []struct {
desc string
metrics []*kubeCounter
registryVersion *semver.Version
expectedPanics []bool
}{
{
desc: "test alpha metric",
metrics: []*kubeCounter{alphaCounter},
registryVersion: &v115,
expectedPanics: []bool{false},
},
{
desc: "test registering same metric multiple times",
metrics: []*kubeCounter{alphaCounter, alphaCounter},
registryVersion: &v115,
expectedPanics: []bool{false, true},
},
{
desc: "test alpha deprecated metric",
metrics: []*kubeCounter{alphaDeprecatedCounter},
registryVersion: &v115,
expectedPanics: []bool{false},
},
{
desc: "test must registering same deprecated metric",
metrics: []*kubeCounter{alphaDeprecatedCounter, alphaDeprecatedCounter},
registryVersion: &v115,
expectedPanics: []bool{false, true},
},
{
desc: "test alpha hidden metric",
metrics: []*kubeCounter{alphaHiddenCounter},
registryVersion: &v115,
expectedPanics: []bool{false},
},
{
desc: "test must registering same hidden metric",
metrics: []*kubeCounter{alphaHiddenCounter, alphaHiddenCounter},
registryVersion: &v115,
expectedPanics: []bool{false, false}, // hidden metrics no-opt
},
}
for _, test := range tests {
t.Run(test.desc, func(t *testing.T) {
registry := NewKubeRegistry(*test.registryVersion)
for i, m := range test.metrics {
if test.expectedPanics[i] {
assert.Panics(t,
func() { registry.MustRegister(m) },
"Did not panic even though we expected it.")
} else {
registry.MustRegister(m)
}
}
})
}
}

View File

@ -0,0 +1,67 @@
/*
Copyright 2019 The Kubernetes Authors.
Licensed 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 metrics
import (
"github.com/prometheus/client_golang/prometheus"
dto "github.com/prometheus/client_model/go"
)
// This file contains a series of interfaces which we explicitly define for
// integrating with prometheus. We redefine the interfaces explicitly here
// so that we can prevent breakage if methods are ever added to prometheus
// variants of them.
/**
* Collector defines a subset of prometheus.Collector interface methods
*/
type Collector interface {
Describe(chan<- *prometheus.Desc)
Collect(chan<- prometheus.Metric)
}
/**
* Metric defines a subset of prometheus.Metric interface methods
*/
type Metric interface {
Desc() *prometheus.Desc
Write(*dto.Metric) error
}
// Counter is a Metric that represents a single numerical value that only ever
// goes up. That implies that it cannot be used to count items whose number can
// also go down, e.g. the number of currently running goroutines. Those
// "counters" are represented by Gauges.
//
// This interface defines a subset of the interface provided by prometheus.Counter
type KubeCounter interface {
Inc()
Add(float64)
}
type KubeCounterVec interface {
WithLabelValues(...string) KubeCounter
With(prometheus.Labels) KubeCounter
}
type PromRegistry interface {
Register(prometheus.Collector) error
MustRegister(...prometheus.Collector)
Unregister(prometheus.Collector) bool
Gather() ([]*dto.MetricFamily, error)
}