Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 8 additions & 0 deletions charts/kminion/values.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -223,6 +223,12 @@ kminion:
# retryInitConnection: false
#
# minion:
# clusterInfo:
# # Enabled specifies whether cluster information shall be scraped and exported or not.
# enabled: true
# brokerInfo:
# # Enabled specifies whether broker information shall be scraped and exported or not.
# enabled: true
# consumerGroups:
# # Enabled specifies whether consumer groups shall be scraped and exported or not.
# enabled: true
Expand All @@ -241,6 +247,8 @@ kminion:
# # take precedence over allowed groups.
# ignoredGroups: [ ]
# topics:
# # Enabled specifies whether topics shall be scraped and exported or not.
# enabled: true
# # Granularity can be per topic or per partition. If you want to reduce the number of exported metric series and
# # you aren't interested in per partition metrics you could choose "topic".
# granularity: partition
Expand Down
8 changes: 8 additions & 0 deletions docs/reference-config.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,12 @@ kafka:
scope: ""

minion:
clusterInfo:
# Enabled specifies whether cluster information shall be scraped and exported or not.
enabled: true
brokerInfo:
# Enabled specifies whether broker information shall be scraped and exported or not.
enabled: true
consumerGroups:
# Enabled specifies whether consumer groups shall be scraped and exported or not.
enabled: true
Expand All @@ -90,6 +96,8 @@ minion:
# take precedence over allowed groups.
ignoredGroups: [ ]
topics:
# Enabled specifies whether topics shall be scraped and exported or not.
enabled: true
# Granularity can be per topic or per partition. If you want to reduce the number of exported metric series and
# you aren't interested in per partition metrics you could choose "topic".
granularity: partition
Expand Down
14 changes: 13 additions & 1 deletion minion/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,8 @@ import (
)

type Config struct {
ClusterInfo ClusterInfoConfig `koanf:"clusterInfo"`
BrokerInfo BrokerInfoConfig `koanf:"brokerInfo"`
ConsumerGroups ConsumerGroupConfig `koanf:"consumerGroups"`
Topics TopicConfig `koanf:"topics"`
LogDirs LogDirsConfig `koanf:"logDirs"`
Expand All @@ -21,7 +23,17 @@ func (c *Config) SetDefaults() {
}

func (c *Config) Validate() error {
err := c.ConsumerGroups.Validate()
err := c.ClusterInfo.Validate()
if err != nil {
return fmt.Errorf("failed to cluster info config: %w", err)
}

err = c.BrokerInfo.Validate()
if err != nil {
return fmt.Errorf("failed to broker info config: %w", err)
}

err = c.ConsumerGroups.Validate()
if err != nil {
return fmt.Errorf("failed to consumer group config: %w", err)
}
Expand Down
14 changes: 14 additions & 0 deletions minion/config_broker_info.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
package minion

type BrokerInfoConfig struct {
// Enabled specifies whether broker information shall be scraped and exported or not.
Enabled bool `koanf:"enabled"`
}

func (c *BrokerInfoConfig) SetDefaults() {
c.Enabled = true
}

func (c *BrokerInfoConfig) Validate() error {
return nil
}
14 changes: 14 additions & 0 deletions minion/config_cluster_info.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
package minion

type ClusterInfoConfig struct {
// Enabled specifies whether cluster information shall be scraped and exported or not.
Enabled bool `koanf:"enabled"`
}

func (c *ClusterInfoConfig) SetDefaults() {
c.Enabled = true
}

func (c *ClusterInfoConfig) Validate() error {
return nil
}
4 changes: 4 additions & 0 deletions minion/config_topic_config.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,9 @@ const (
)

type TopicConfig struct {
// Enabled specifies whether topics shall be scraped and exported or not.
Enabled bool `koanf:"enabled"`

// Granularity can be per topic or per partition. If you want to reduce the number of exported metric series and
// you aren't interested in per partition metrics you could choose "topic".
Granularity string `koanf:"granularity"`
Expand Down Expand Up @@ -60,6 +63,7 @@ func (c *TopicConfig) Validate() error {

// SetDefaults for topic config
func (c *TopicConfig) SetDefaults() {
c.Enabled = true
c.Granularity = TopicGranularityPartition
c.AllowedTopics = []string{"/.*/"}
c.InfoMetric = InfoMetricConfig{ConfigKeys: []string{"cleanup.policy"}}
Expand Down
7 changes: 6 additions & 1 deletion prometheus/collect_broker_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -2,12 +2,17 @@ package prometheus

import (
"context"
"strconv"

"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
"strconv"
)

func (e *Exporter) collectBrokerInfo(ctx context.Context, ch chan<- prometheus.Metric) bool {
if !e.minionSvc.Cfg.BrokerInfo.Enabled {
return true
}

metadata, err := e.minionSvc.GetMetadataCached(ctx)
if err != nil {
e.logger.Error("failed to get kafka metadata", zap.Error(err))
Expand Down
7 changes: 6 additions & 1 deletion prometheus/collect_cluster_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -2,12 +2,17 @@ package prometheus

import (
"context"
"strconv"

"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
"strconv"
)

func (e *Exporter) collectClusterInfo(ctx context.Context, ch chan<- prometheus.Metric) bool {
if !e.minionSvc.Cfg.ClusterInfo.Enabled {
return true
}

version, err := e.minionSvc.GetClusterVersion(ctx)
if err != nil {
e.logger.Error("failed to get kafka cluster version", zap.Error(err))
Expand Down
9 changes: 7 additions & 2 deletions prometheus/collect_consumer_group_lags.go
Original file line number Diff line number Diff line change
Expand Up @@ -2,13 +2,14 @@ package prometheus

import (
"context"
"math"
"strconv"

"github.com/cloudhut/kminion/v2/minion"
"github.com/prometheus/client_golang/prometheus"
"github.com/twmb/franz-go/pkg/kerr"
"github.com/twmb/franz-go/pkg/kmsg"
"go.uber.org/zap"
"math"
"strconv"
)

type waterMark struct {
Expand All @@ -19,6 +20,10 @@ type waterMark struct {
}

func (e *Exporter) collectConsumerGroupLags(ctx context.Context, ch chan<- prometheus.Metric) bool {
if !e.minionSvc.Cfg.ConsumerGroups.Enabled {
return true
}

// Low Watermarks (at the moment they are not needed at all, they could be used to calculate the lag on partitions
// that don't have any active offsets)
lowWaterMarks, err := e.minionSvc.ListOffsetsCached(ctx, -2)
Expand Down
4 changes: 4 additions & 0 deletions prometheus/collect_topic_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,10 @@ import (
)

func (e *Exporter) collectTopicInfo(ctx context.Context, ch chan<- prometheus.Metric) bool {
if !e.minionSvc.Cfg.Topics.Enabled {
return true
}

metadata, err := e.minionSvc.GetMetadataCached(ctx)
if err != nil {
e.logger.Error("failed to get metadata", zap.Error(err))
Expand Down
4 changes: 4 additions & 0 deletions prometheus/collect_topic_partition_offsets.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,10 @@ import (
)

func (e *Exporter) collectTopicPartitionOffsets(ctx context.Context, ch chan<- prometheus.Metric) bool {
if !e.minionSvc.Cfg.Topics.Enabled {
return true
}

isOk := true

// Low Watermarks
Expand Down