Merge pull request #124557 from danwinship/metrics-and-stuff

kube-proxy metrics cleanup (and stuff)
This commit is contained in:
Kubernetes Prow Robot 2024-04-26 18:31:57 -07:00 committed by GitHub
commit ae8474adcd
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
13 changed files with 114 additions and 89 deletions

View File

@ -80,6 +80,7 @@ import (
"k8s.io/kubernetes/pkg/proxy/apis/config/validation"
"k8s.io/kubernetes/pkg/proxy/config"
"k8s.io/kubernetes/pkg/proxy/healthcheck"
proxymetrics "k8s.io/kubernetes/pkg/proxy/metrics"
proxyutil "k8s.io/kubernetes/pkg/proxy/util"
"k8s.io/kubernetes/pkg/util/filesystem"
utilflag "k8s.io/kubernetes/pkg/util/flag"
@ -919,6 +920,8 @@ func (s *ProxyServer) Run(ctx context.Context) error {
logger.Info("Golang settings", "GOGC", os.Getenv("GOGC"), "GOMAXPROCS", os.Getenv("GOMAXPROCS"), "GOTRACEBACK", os.Getenv("GOTRACEBACK"))
proxymetrics.RegisterMetrics(s.Config.Mode)
// TODO(vmarmol): Use container config for this.
var oomAdjuster *oom.OOMAdjuster
if s.Config.OOMScoreAdj != nil {

View File

@ -50,7 +50,6 @@ import (
"k8s.io/kubernetes/pkg/proxy/ipvs"
utilipset "k8s.io/kubernetes/pkg/proxy/ipvs/ipset"
utilipvs "k8s.io/kubernetes/pkg/proxy/ipvs/util"
proxymetrics "k8s.io/kubernetes/pkg/proxy/metrics"
"k8s.io/kubernetes/pkg/proxy/nftables"
proxyutil "k8s.io/kubernetes/pkg/proxy/util"
utiliptables "k8s.io/kubernetes/pkg/util/iptables"
@ -100,7 +99,6 @@ func (s *ProxyServer) platformSetup(ctx context.Context) error {
return err
}
proxymetrics.RegisterMetrics()
return nil
}

View File

@ -51,7 +51,6 @@ func (o *Options) platformApplyDefaults(config *proxyconfigapi.KubeProxyConfigur
// Proxier. It should fill in any platform-specific fields and perform other
// platform-specific setup.
func (s *ProxyServer) platformSetup(ctx context.Context) error {
winkernel.RegisterMetrics()
// Preserve backward-compatibility with the old secondary IP behavior
if s.PrimaryIPFamily == v1.IPv4Protocol {
s.NodeIPs[v1.IPv6Protocol] = net.IPv6zero

View File

@ -465,7 +465,7 @@ type serverTest struct {
}
func TestHealthzServer(t *testing.T) {
metrics.RegisterMetrics()
metrics.RegisterMetrics("")
listener := newFakeListener()
httpFactory := newFakeHTTPServerFactory()
fakeClock := testingclock.NewFakeClock(time.Now())
@ -500,7 +500,7 @@ func TestHealthzServer(t *testing.T) {
}
func TestLivezServer(t *testing.T) {
metrics.RegisterMetrics()
metrics.RegisterMetrics("")
listener := newFakeListener()
httpFactory := newFakeHTTPServerFactory()
fakeClock := testingclock.NewFakeClock(time.Now())

View File

@ -450,7 +450,7 @@ func CleanupLeftovers(ctx context.Context, ipt utiliptables.Interface) (encounte
err = ipt.Restore(utiliptables.TableNAT, natLines, utiliptables.NoFlushTables, utiliptables.RestoreCounters)
if err != nil {
logger.Error(err, "Failed to execute iptables-restore", "table", utiliptables.TableNAT)
metrics.IptablesRestoreFailuresTotal.Inc()
metrics.IPTablesRestoreFailuresTotal.Inc()
encounteredError = true
}
}
@ -477,7 +477,7 @@ func CleanupLeftovers(ctx context.Context, ipt utiliptables.Interface) (encounte
// Write it.
if err := ipt.Restore(utiliptables.TableFilter, filterLines, utiliptables.NoFlushTables, utiliptables.RestoreCounters); err != nil {
logger.Error(err, "Failed to execute iptables-restore", "table", utiliptables.TableFilter)
metrics.IptablesRestoreFailuresTotal.Inc()
metrics.IPTablesRestoreFailuresTotal.Inc()
encounteredError = true
}
}
@ -817,7 +817,7 @@ func (proxier *Proxier) syncProxyRules() {
proxier.logger.Info("Sync failed", "retryingTime", proxier.syncPeriod)
proxier.syncRunner.RetryAfter(proxier.syncPeriod)
if tryPartialSync {
metrics.IptablesPartialRestoreFailuresTotal.Inc()
metrics.IPTablesPartialRestoreFailuresTotal.Inc()
}
// proxier.serviceChanges and proxier.endpointChanges have already
// been flushed, so we've lost the state needed to be able to do
@ -1482,10 +1482,10 @@ func (proxier *Proxier) syncProxyRules() {
"-j", "ACCEPT",
)
metrics.IptablesRulesTotal.WithLabelValues(string(utiliptables.TableFilter)).Set(float64(proxier.filterRules.Lines()))
metrics.IptablesRulesLastSync.WithLabelValues(string(utiliptables.TableFilter)).Set(float64(proxier.filterRules.Lines()))
metrics.IptablesRulesTotal.WithLabelValues(string(utiliptables.TableNAT)).Set(float64(proxier.natRules.Lines() + skippedNatRules.Lines() - deletedChains))
metrics.IptablesRulesLastSync.WithLabelValues(string(utiliptables.TableNAT)).Set(float64(proxier.natRules.Lines() - deletedChains))
metrics.IPTablesRulesTotal.WithLabelValues(string(utiliptables.TableFilter)).Set(float64(proxier.filterRules.Lines()))
metrics.IPTablesRulesLastSync.WithLabelValues(string(utiliptables.TableFilter)).Set(float64(proxier.filterRules.Lines()))
metrics.IPTablesRulesTotal.WithLabelValues(string(utiliptables.TableNAT)).Set(float64(proxier.natRules.Lines() + skippedNatRules.Lines() - deletedChains))
metrics.IPTablesRulesLastSync.WithLabelValues(string(utiliptables.TableNAT)).Set(float64(proxier.natRules.Lines() - deletedChains))
// Sync rules.
proxier.iptablesData.Reset()
@ -1517,7 +1517,7 @@ func (proxier *Proxier) syncProxyRules() {
} else {
proxier.logger.Error(err, "Failed to execute iptables-restore")
}
metrics.IptablesRestoreFailuresTotal.Inc()
metrics.IPTablesRestoreFailuresTotal.Inc()
return
}
success = true

View File

@ -49,10 +49,10 @@ import (
klogtesting "k8s.io/klog/v2/ktesting"
"k8s.io/kubernetes/pkg/features"
"k8s.io/kubernetes/pkg/proxy"
kubeproxyconfig "k8s.io/kubernetes/pkg/proxy/apis/config"
"k8s.io/kubernetes/pkg/proxy/conntrack"
"k8s.io/kubernetes/pkg/proxy/metrics"
"k8s.io/kubernetes/pkg/proxy/healthcheck"
"k8s.io/kubernetes/pkg/proxy/metrics"
proxyutil "k8s.io/kubernetes/pkg/proxy/util"
proxyutiltest "k8s.io/kubernetes/pkg/proxy/util/testing"
"k8s.io/kubernetes/pkg/util/async"
@ -408,7 +408,7 @@ func countRules(logger klog.Logger, tableName utiliptables.Table, ruleData strin
}
func countRulesFromMetric(logger klog.Logger, tableName utiliptables.Table) int {
numRulesFloat, err := testutil.GetGaugeMetricValue(metrics.IptablesRulesTotal.WithLabelValues(string(tableName)))
numRulesFloat, err := testutil.GetGaugeMetricValue(metrics.IPTablesRulesTotal.WithLabelValues(string(tableName)))
if err != nil {
logger.Error(err, "metrics are not registered?")
return -1
@ -417,7 +417,7 @@ func countRulesFromMetric(logger klog.Logger, tableName utiliptables.Table) int
}
func countRulesFromLastSyncMetric(logger klog.Logger, tableName utiliptables.Table) int {
numRulesFloat, err := testutil.GetGaugeMetricValue(metrics.IptablesRulesLastSync.WithLabelValues(string(tableName)))
numRulesFloat, err := testutil.GetGaugeMetricValue(metrics.IPTablesRulesLastSync.WithLabelValues(string(tableName)))
if err != nil {
logger.Error(err, "metrics are not registered?")
return -1
@ -1543,7 +1543,7 @@ func TestOverallIPTablesRules(t *testing.T) {
logger, _ := klogtesting.NewTestContext(t)
ipt := iptablestest.NewFake()
fp := NewFakeProxier(ipt)
metrics.RegisterMetrics()
metrics.RegisterMetrics(kubeproxyconfig.ProxyModeIPTables)
makeServiceMap(fp,
// create ClusterIP service
@ -4142,12 +4142,12 @@ func TestHealthCheckNodePortWhenTerminating(t *testing.T) {
}
}
func TestProxierMetricsIptablesTotalRules(t *testing.T) {
func TestProxierMetricsIPTablesTotalRules(t *testing.T) {
logger, _ := klogtesting.NewTestContext(t)
ipt := iptablestest.NewFake()
fp := NewFakeProxier(ipt)
metrics.RegisterMetrics()
metrics.RegisterMetrics(kubeproxyconfig.ProxyModeIPTables)
svcIP := "172.30.0.41"
svcPort := 80
@ -5827,7 +5827,7 @@ func TestSyncProxyRulesRepeated(t *testing.T) {
logger, _ := klogtesting.NewTestContext(t)
ipt := iptablestest.NewFake()
fp := NewFakeProxier(ipt)
metrics.RegisterMetrics()
metrics.RegisterMetrics(kubeproxyconfig.ProxyModeIPTables)
defer legacyregistry.Reset()
// Create initial state
@ -6356,7 +6356,7 @@ func TestSyncProxyRulesRepeated(t *testing.T) {
if fp.needFullSync {
t.Fatalf("Proxier unexpectedly already needs a full sync?")
}
partialRestoreFailures, err := testutil.GetCounterMetricValue(metrics.IptablesPartialRestoreFailuresTotal)
partialRestoreFailures, err := testutil.GetCounterMetricValue(metrics.IPTablesPartialRestoreFailuresTotal)
if err != nil {
t.Fatalf("Could not get partial restore failures metric: %v", err)
}
@ -6390,7 +6390,7 @@ func TestSyncProxyRulesRepeated(t *testing.T) {
if !fp.needFullSync {
t.Errorf("Proxier did not fail on previous partial resync?")
}
updatedPartialRestoreFailures, err := testutil.GetCounterMetricValue(metrics.IptablesPartialRestoreFailuresTotal)
updatedPartialRestoreFailures, err := testutil.GetCounterMetricValue(metrics.IPTablesPartialRestoreFailuresTotal)
if err != nil {
t.Errorf("Could not get partial restore failures metric: %v", err)
}
@ -6471,7 +6471,7 @@ func TestNoEndpointsMetric(t *testing.T) {
hostname string
}
metrics.RegisterMetrics()
metrics.RegisterMetrics(kubeproxyconfig.ProxyModeIPTables)
testCases := []struct {
name string
internalTrafficPolicy *v1.ServiceInternalTrafficPolicy

View File

@ -1444,7 +1444,7 @@ func (proxier *Proxier) syncProxyRules() {
} else {
proxier.logger.Error(err, "Failed to execute iptables-restore", "rules", proxier.iptablesData.Bytes())
}
metrics.IptablesRestoreFailuresTotal.Inc()
metrics.IPTablesRestoreFailuresTotal.Inc()
return
}
for name, lastChangeTriggerTimes := range endpointUpdateResult.LastChangeTriggerTimes {

View File

@ -42,6 +42,7 @@ import (
"k8s.io/component-base/metrics/testutil"
"k8s.io/kubernetes/pkg/features"
"k8s.io/kubernetes/pkg/proxy"
kubeproxyconfig "k8s.io/kubernetes/pkg/proxy/apis/config"
"k8s.io/kubernetes/pkg/proxy/conntrack"
"k8s.io/kubernetes/pkg/proxy/healthcheck"
utilipset "k8s.io/kubernetes/pkg/proxy/ipvs/ipset"
@ -5615,7 +5616,7 @@ func TestNoEndpointsMetric(t *testing.T) {
hostname string
}
metrics.RegisterMetrics()
metrics.RegisterMetrics(kubeproxyconfig.ProxyModeIPVS)
testCases := []struct {
name string

View File

@ -22,6 +22,7 @@ import (
"k8s.io/component-base/metrics"
"k8s.io/component-base/metrics/legacyregistry"
kubeproxyconfig "k8s.io/kubernetes/pkg/proxy/apis/config"
)
const kubeProxySubsystem = "kubeproxy"
@ -138,9 +139,9 @@ var (
},
)
// IptablesRestoreFailuresTotal is the number of iptables restore failures that the proxy has
// IPTablesRestoreFailuresTotal is the number of iptables restore failures that the proxy has
// seen.
IptablesRestoreFailuresTotal = metrics.NewCounter(
IPTablesRestoreFailuresTotal = metrics.NewCounter(
&metrics.CounterOpts{
Subsystem: kubeProxySubsystem,
Name: "sync_proxy_rules_iptables_restore_failures_total",
@ -149,9 +150,9 @@ var (
},
)
// IptablesPartialRestoreFailuresTotal is the number of iptables *partial* restore
// IPTablesPartialRestoreFailuresTotal is the number of iptables *partial* restore
// failures (resulting in a fall back to a full restore) that the proxy has seen.
IptablesPartialRestoreFailuresTotal = metrics.NewCounter(
IPTablesPartialRestoreFailuresTotal = metrics.NewCounter(
&metrics.CounterOpts{
Subsystem: kubeProxySubsystem,
Name: "sync_proxy_rules_iptables_partial_restore_failures_total",
@ -160,9 +161,9 @@ var (
},
)
// IptablesRulesTotal is the total number of iptables rules that the iptables
// IPTablesRulesTotal is the total number of iptables rules that the iptables
// proxy has installed.
IptablesRulesTotal = metrics.NewGaugeVec(
IPTablesRulesTotal = metrics.NewGaugeVec(
&metrics.GaugeOpts{
Subsystem: kubeProxySubsystem,
Name: "sync_proxy_rules_iptables_total",
@ -172,9 +173,9 @@ var (
[]string{"table"},
)
// IptablesRulesLastSync is the number of iptables rules that the iptables proxy
// IPTablesRulesLastSync is the number of iptables rules that the iptables proxy
// updated in the last sync.
IptablesRulesLastSync = metrics.NewGaugeVec(
IPTablesRulesLastSync = metrics.NewGaugeVec(
&metrics.GaugeOpts{
Subsystem: kubeProxySubsystem,
Name: "sync_proxy_rules_iptables_last",
@ -184,6 +185,28 @@ var (
[]string{"table"},
)
// NFTablesSyncFailuresTotal is the number of nftables sync failures that the
// proxy has seen.
NFTablesSyncFailuresTotal = metrics.NewCounter(
&metrics.CounterOpts{
Subsystem: kubeProxySubsystem,
Name: "sync_proxy_rules_nftables_sync_failures_total",
Help: "Cumulative proxy nftables sync failures",
StabilityLevel: metrics.ALPHA,
},
)
// NFTablesCleanupFailuresTotal is the number of nftables stale chain cleanup
// failures that the proxy has seen.
NFTablesCleanupFailuresTotal = metrics.NewCounter(
&metrics.CounterOpts{
Subsystem: kubeProxySubsystem,
Name: "sync_proxy_rules_nftables_cleanup_failures_total",
Help: "Cumulative proxy nftables cleanup failures",
StabilityLevel: metrics.ALPHA,
},
)
// ProxyHealthzTotal is the number of returned HTTP Status for each
// healthz probe.
ProxyHealthzTotal = metrics.NewCounterVec(
@ -237,26 +260,42 @@ var (
var registerMetricsOnce sync.Once
// RegisterMetrics registers kube-proxy metrics.
func RegisterMetrics() {
func RegisterMetrics(mode kubeproxyconfig.ProxyMode) {
registerMetricsOnce.Do(func() {
// Core kube-proxy metrics for all backends
legacyregistry.MustRegister(SyncProxyRulesLatency)
legacyregistry.MustRegister(SyncFullProxyRulesLatency)
legacyregistry.MustRegister(SyncPartialProxyRulesLatency)
legacyregistry.MustRegister(SyncProxyRulesLastQueuedTimestamp)
legacyregistry.MustRegister(SyncProxyRulesLastTimestamp)
legacyregistry.MustRegister(NetworkProgrammingLatency)
legacyregistry.MustRegister(EndpointChangesPending)
legacyregistry.MustRegister(EndpointChangesTotal)
legacyregistry.MustRegister(ServiceChangesPending)
legacyregistry.MustRegister(ServiceChangesTotal)
legacyregistry.MustRegister(IptablesRulesTotal)
legacyregistry.MustRegister(IptablesRulesLastSync)
legacyregistry.MustRegister(IptablesRestoreFailuresTotal)
legacyregistry.MustRegister(IptablesPartialRestoreFailuresTotal)
legacyregistry.MustRegister(SyncProxyRulesLastQueuedTimestamp)
legacyregistry.MustRegister(SyncProxyRulesNoLocalEndpointsTotal)
legacyregistry.MustRegister(ProxyHealthzTotal)
legacyregistry.MustRegister(ProxyLivezTotal)
// FIXME: winkernel does not implement these
legacyregistry.MustRegister(NetworkProgrammingLatency)
legacyregistry.MustRegister(SyncProxyRulesNoLocalEndpointsTotal)
switch mode {
case kubeproxyconfig.ProxyModeIPTables:
legacyregistry.MustRegister(SyncFullProxyRulesLatency)
legacyregistry.MustRegister(SyncPartialProxyRulesLatency)
legacyregistry.MustRegister(IPTablesRestoreFailuresTotal)
legacyregistry.MustRegister(IPTablesPartialRestoreFailuresTotal)
legacyregistry.MustRegister(IPTablesRulesTotal)
legacyregistry.MustRegister(IPTablesRulesLastSync)
case kubeproxyconfig.ProxyModeIPVS:
legacyregistry.MustRegister(IPTablesRestoreFailuresTotal)
case kubeproxyconfig.ProxyModeNFTables:
legacyregistry.MustRegister(NFTablesSyncFailuresTotal)
legacyregistry.MustRegister(NFTablesCleanupFailuresTotal)
case kubeproxyconfig.ProxyModeKernelspace:
// currently no winkernel-specific metrics
}
})
}

View File

@ -1032,7 +1032,7 @@ func (proxier *Proxier) syncProxyRules() {
// the chains still exist, they'll just get added back
// (with a later timestamp) at the end of the sync.
proxier.logger.Error(err, "Unable to delete stale chains; will retry later")
// FIXME: metric
metrics.NFTablesCleanupFailuresTotal.Inc()
}
}
}
@ -1613,13 +1613,18 @@ func (proxier *Proxier) syncProxyRules() {
"numEndpoints", totalEndpoints,
)
// FIXME
// klog.V(9).InfoS("Running nftables transaction", "transaction", tx.Bytes())
if klogV9 := klog.V(9); klogV9.Enabled() {
klogV9.InfoS("Running nftables transaction", "transaction", tx.String())
}
err = proxier.nftables.Run(context.TODO(), tx)
if err != nil {
proxier.logger.Error(err, "nftables sync failed")
metrics.IptablesRestoreFailuresTotal.Inc()
metrics.NFTablesSyncFailuresTotal.Inc()
// staleChains is now incorrect since we didn't actually flush the
// chains in it. We can recompute it next time.
clear(proxier.staleChains)
return
}
success = true

View File

@ -40,6 +40,7 @@ import (
"k8s.io/component-base/metrics/testutil"
"k8s.io/kubernetes/pkg/features"
"k8s.io/kubernetes/pkg/proxy"
kubeproxyconfig "k8s.io/kubernetes/pkg/proxy/apis/config"
"k8s.io/kubernetes/pkg/proxy/conntrack"
"k8s.io/kubernetes/pkg/proxy/healthcheck"
"k8s.io/kubernetes/pkg/proxy/metrics"
@ -140,7 +141,7 @@ func NewFakeProxier(ipFamily v1.IPFamily) (*knftables.Fake, *Proxier) {
// rules are exactly as expected.
func TestOverallNFTablesRules(t *testing.T) {
nft, fp := NewFakeProxier(v1.IPv4Protocol)
metrics.RegisterMetrics()
metrics.RegisterMetrics(kubeproxyconfig.ProxyModeNFTables)
makeServiceMap(fp,
// create ClusterIP service
@ -4454,7 +4455,7 @@ func TestNoEndpointsMetric(t *testing.T) {
hostname string
}
metrics.RegisterMetrics()
metrics.RegisterMetrics(kubeproxyconfig.ProxyModeNFTables)
testCases := []struct {
name string
internalTrafficPolicy *v1.ServiceInternalTrafficPolicy

View File

@ -0,0 +1,18 @@
/*
Copyright 2024 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 winkernel implements the Windows-kernel-based proxy
package winkernel // import "k8s.io/kubernetes/pkg/proxy/winkernel"

View File

@ -1,39 +0,0 @@
/*
Copyright 2017 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 winkernel
import (
"sync"
"k8s.io/component-base/metrics/legacyregistry"
"k8s.io/kubernetes/pkg/proxy/metrics"
)
var registerMetricsOnce sync.Once
// RegisterMetrics registers kube-proxy metrics for Windows modes.
func RegisterMetrics() {
registerMetricsOnce.Do(func() {
legacyregistry.MustRegister(metrics.SyncProxyRulesLatency)
legacyregistry.MustRegister(metrics.SyncProxyRulesLastTimestamp)
legacyregistry.MustRegister(metrics.EndpointChangesPending)
legacyregistry.MustRegister(metrics.EndpointChangesTotal)
legacyregistry.MustRegister(metrics.ServiceChangesPending)
legacyregistry.MustRegister(metrics.ServiceChangesTotal)
legacyregistry.MustRegister(metrics.SyncProxyRulesLastQueuedTimestamp)
})
}