diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index 70d111ee73cc..a563a24198ad 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -1031,74 +1031,56 @@ metastore_client: # Configures the gRPC client used to communicate with the metastore. [grpc_client_config: ] -partition_ring: - # The key-value store used to share the hash ring across multiple instances. - # This option needs be set on ingesters, distributors, queriers, and rulers - # when running in microservices mode. - kvstore: - # Backend storage to use for the ring. Supported values are: consul, etcd, - # inmemory, memberlist, multi. - # CLI flag: -ingester.partition-ring.store - [store: | default = "memberlist"] - - # The prefix for the keys in the store. Should end with a /. - # CLI flag: -ingester.partition-ring.prefix - [prefix: | default = "collectors/"] - - # Configuration for a Consul client. Only applies if the selected kvstore is - # consul. - # The CLI flags prefix for this block configuration is: - # ingester.partition-ring.consul - [consul: ] - - # Configuration for an ETCD v3 client. Only applies if the selected kvstore - # is etcd. - # The CLI flags prefix for this block configuration is: - # ingester.partition-ring.etcd - [etcd: ] - - multi: - # Primary backend storage used by multi-client. - # CLI flag: -ingester.partition-ring.multi.primary - [primary: | default = ""] - - # Secondary backend storage used by multi-client. - # CLI flag: -ingester.partition-ring.multi.secondary - [secondary: | default = ""] - - # Mirror writes to secondary store. - # CLI flag: -ingester.partition-ring.multi.mirror-enabled - [mirror_enabled: | default = false] - - # Timeout for storing value to secondary store. - # CLI flag: -ingester.partition-ring.multi.mirror-timeout - [mirror_timeout: | default = 2s] - - # Minimum number of owners to wait before a PENDING partition gets switched to - # ACTIVE. - # CLI flag: -ingester.partition-ring.min-partition-owners-count - [min_partition_owners_count: | default = 1] - - # How long the minimum number of owners are enforced before a PENDING - # partition gets switched to ACTIVE. - # CLI flag: -ingester.partition-ring.min-partition-owners-duration - [min_partition_owners_duration: | default = 10s] - - # How long to wait before an INACTIVE partition is eligible for deletion. The - # partition is deleted only if it has been in INACTIVE state for at least the - # configured duration and it has no owners registered. A value of 0 disables - # partitions deletion. - # CLI flag: -ingester.partition-ring.delete-inactive-partition-after - [delete_inactive_partition_after: | default = 13h] - kafka_config: - # the kafka endpoint to connect to - # CLI flag: -address + # The Kafka backend address. + # CLI flag: -kafka.address [address: | default = "localhost:9092"] # The Kafka topic name. - # CLI flag: -.topic - [topic: | default = "loki.push"] + # CLI flag: -kafka.topic + [topic: | default = ""] + + # The Kafka client ID. + # CLI flag: -kafka.client-id + [client_id: | default = ""] + + # The maximum time allowed to open a connection to a Kafka broker. + # CLI flag: -kafka.dial-timeout + [dial_timeout: | default = 2s] + + # How long to wait for an incoming write request to be successfully committed + # to the Kafka backend. + # CLI flag: -kafka.write-timeout + [write_timeout: | default = 10s] + + # The consumer group used by the consumer to track the last consumed offset. + # The consumer group must be different for each ingester. If the configured + # consumer group contains the '' placeholder, it is replaced with + # the actual partition ID owned by the ingester. When empty (recommended), + # Mimir uses the ingester instance ID to guarantee uniqueness. + # CLI flag: -kafka.consumer-group + [consumer_group: | default = ""] + + # How long to retry a failed request to get the last produced offset. + # CLI flag: -kafka.last-produced-offset-retry-timeout + [last_produced_offset_retry_timeout: | default = 10s] + + # Enable auto-creation of Kafka topic if it doesn't exist. + # CLI flag: -kafka.auto-create-topic-enabled + [auto_create_topic_enabled: | default = true] + + # The maximum size of a Kafka record data that should be generated by the + # producer. An incoming write request larger than this size is split into + # multiple Kafka records. We strongly recommend to not change this setting + # unless for testing purposes. + # CLI flag: -kafka.producer-max-record-size-bytes + [producer_max_record_size_bytes: | default = 15983616] + + # The maximum size of (uncompressed) buffered and unacknowledged produced + # records sent to Kafka. The produce request fails once this limit is reached. + # This limit is per Kafka client. 0 to disable the limit. + # CLI flag: -kafka.producer-max-buffered-bytes + [producer_max_buffered_bytes: | default = 1073741824] kafka_ingester: # Whether the kafka ingester is enabled. @@ -1251,46 +1233,75 @@ kafka_ingester: # CLI flag: -kafka-ingester.shutdown-marker-path [shutdown_marker_path: | default = ""] + # The interval at which the ingester will flush and commit offsets to Kafka. + # If not set, the default flush interval will be used. + # CLI flag: -kafka-ingester.flush-interval + [flush_interval: | default = 15s] + + # The size at which the ingester will flush and commit offsets to Kafka. If + # not set, the default flush size will be used. + # CLI flag: -kafka-ingester.flush-size + [flush_size: | default = 314572800] + partition_ring: # The key-value store used to share the hash ring across multiple instances. # This option needs be set on ingesters, distributors, queriers, and rulers # when running in microservices mode. kvstore: - [store: | default = ""] + # Backend storage to use for the ring. Supported values are: consul, etcd, + # inmemory, memberlist, multi. + # CLI flag: -ingester.partition-ring.store + [store: | default = "memberlist"] - [prefix: | default = ""] + # The prefix for the keys in the store. Should end with a /. + # CLI flag: -ingester.partition-ring.prefix + [prefix: | default = "collectors/"] # Configuration for a Consul client. Only applies if the selected kvstore # is consul. # The CLI flags prefix for this block configuration is: - # common.storage.ring.consul + # ingester.partition-ring.consul [consul: ] # Configuration for an ETCD v3 client. Only applies if the selected # kvstore is etcd. # The CLI flags prefix for this block configuration is: - # common.storage.ring.etcd + # ingester.partition-ring.etcd [etcd: ] multi: + # Primary backend storage used by multi-client. + # CLI flag: -ingester.partition-ring.multi.primary [primary: | default = ""] + # Secondary backend storage used by multi-client. + # CLI flag: -ingester.partition-ring.multi.secondary [secondary: | default = ""] - [mirror_enabled: ] - - [mirror_timeout: ] - - [min_partition_owners_count: ] + # Mirror writes to secondary store. + # CLI flag: -ingester.partition-ring.multi.mirror-enabled + [mirror_enabled: | default = false] - [min_partition_owners_duration: ] + # Timeout for storing value to secondary store. + # CLI flag: -ingester.partition-ring.multi.mirror-timeout + [mirror_timeout: | default = 2s] - [delete_inactive_partition_after: ] + # Minimum number of owners to wait before a PENDING partition gets switched + # to ACTIVE. + # CLI flag: -ingester.partition-ring.min-partition-owners-count + [min_partition_owners_count: | default = 1] - kafkaconfig: - [address: | default = ""] + # How long the minimum number of owners are enforced before a PENDING + # partition gets switched to ACTIVE. + # CLI flag: -ingester.partition-ring.min-partition-owners-duration + [min_partition_owners_duration: | default = 10s] - [topic: | default = ""] + # How long to wait before an INACTIVE partition is eligible for deletion. + # The partition is deleted only if it has been in INACTIVE state for at + # least the configured duration and it has no owners registered. A value of + # 0 disables partitions deletion. + # CLI flag: -ingester.partition-ring.delete-inactive-partition-after + [delete_inactive_partition_after: | default = 13h] # Configuration for 'runtime config' module, responsible for reloading runtime # configuration file. @@ -2244,10 +2255,14 @@ ring: # Configuration for a Consul client. Only applies if the selected kvstore is # consul. + # The CLI flags prefix for this block configuration is: + # common.storage.ring.consul [consul: ] # Configuration for an ETCD v3 client. Only applies if the selected kvstore # is etcd. + # The CLI flags prefix for this block configuration is: + # common.storage.ring.etcd [etcd: ] multi: @@ -3578,26 +3593,16 @@ The `ingester_client` block configures how the distributor will connect to inges ```yaml # Configures how connections are pooled. pool_config: - # How frequently to clean up clients for ingesters that have gone away. - # CLI flag: -distributor.client-cleanup-period - [client_cleanup_period: | default = 15s] + [client_cleanup_period: ] - # Run a health check on each ingester client during periodic cleanup. - # CLI flag: -distributor.health-check-ingesters - [health_check_ingesters: | default = true] + [health_check_ingesters: ] - # How quickly a dead client will be removed after it has been detected to - # disappear. Set this to a value to allow time for a secondary health check to - # recover the missing client. - # CLI flag: -ingester.client.healthcheck-timeout - [remote_timeout: | default = 1s] + [remote_timeout: ] -# The remote request timeout on the client side. -# CLI flag: -ingester.client.timeout -[remote_timeout: | default = 5s] +[remote_timeout: ] # Configures how the gRPC connection to ingesters work as a client. -# The CLI flags prefix for this block configuration is: ingester.client +# The CLI flags prefix for this block configuration is: ingester-rf1.client [grpc_client_config: ] ``` diff --git a/go.mod b/go.mod index 8579f77e0e4b..85d4d76e6e0c 100644 --- a/go.mod +++ b/go.mod @@ -139,6 +139,10 @@ require ( github.com/shirou/gopsutil/v4 v4.24.0-alpha.1 github.com/thanos-io/objstore v0.0.0-20240818203309-0363dadfdfb1 github.com/twmb/franz-go v1.17.1 + github.com/twmb/franz-go/pkg/kadm v1.13.0 + github.com/twmb/franz-go/pkg/kfake v0.0.0-20240821035758-b77dd13e2bfa + github.com/twmb/franz-go/pkg/kmsg v1.8.0 + github.com/twmb/franz-go/plugin/kotel v1.5.0 github.com/twmb/franz-go/plugin/kprom v1.1.0 github.com/willf/bloom v2.0.3+incompatible go.opentelemetry.io/collector/pdata v1.12.0 @@ -178,7 +182,6 @@ require ( github.com/shoenig/go-m1cpu v0.1.6 // indirect github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect - github.com/twmb/franz-go/pkg/kmsg v1.8.0 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect go.opentelemetry.io/otel/sdk v1.28.0 // indirect go.opentelemetry.io/otel/sdk/metric v1.28.0 // indirect @@ -347,9 +350,9 @@ require ( go.opentelemetry.io/collector/semconv v0.105.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.53.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.53.0 // indirect - go.opentelemetry.io/otel v1.28.0 // indirect + go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/metric v1.28.0 // indirect - go.opentelemetry.io/otel/trace v1.28.0 // indirect + go.opentelemetry.io/otel/trace v1.28.0 go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.21.0 // indirect golang.org/x/mod v0.19.0 // indirect diff --git a/go.sum b/go.sum index cc26b57251f0..4cfb97906a97 100644 --- a/go.sum +++ b/go.sum @@ -1816,8 +1816,14 @@ github.com/ttacon/chalk v0.0.0-20160626202418-22c06c80ed31/go.mod h1:onvgF043R+l github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= github.com/twmb/franz-go v1.17.1 h1:0LwPsbbJeJ9R91DPUHSEd4su82WJWcTY1Zzbgbg4CeQ= github.com/twmb/franz-go v1.17.1/go.mod h1:NreRdJ2F7dziDY/m6VyspWd6sNxHKXdMZI42UfQ3GXM= +github.com/twmb/franz-go/pkg/kadm v1.13.0 h1:bJq4C2ZikUE2jh/wl9MtMTQ/kpmnBgVFh8XMQBEC+60= +github.com/twmb/franz-go/pkg/kadm v1.13.0/go.mod h1:VMvpfjz/szpH9WB+vGM+rteTzVv0djyHFimci9qm2C0= +github.com/twmb/franz-go/pkg/kfake v0.0.0-20240821035758-b77dd13e2bfa h1:OmQ4DJhqeOPdIH60Psut1vYU8A6LGyxJbF09w5RAa2w= +github.com/twmb/franz-go/pkg/kfake v0.0.0-20240821035758-b77dd13e2bfa/go.mod h1:nkBI/wGFp7t1NJnnCeJdS4sX5atPAqwCPpDXKuI7SC8= github.com/twmb/franz-go/pkg/kmsg v1.8.0 h1:lAQB9Z3aMrIP9qF9288XcFf/ccaSxEitNA1CDTEIeTA= github.com/twmb/franz-go/pkg/kmsg v1.8.0/go.mod h1:HzYEb8G3uu5XevZbtU0dVbkphaKTHk0X68N5ka4q6mU= +github.com/twmb/franz-go/plugin/kotel v1.5.0 h1:TiPfGUbQK384OO7ZYGdo7JuPCbJn+/8njQ/D9Je9CDE= +github.com/twmb/franz-go/plugin/kotel v1.5.0/go.mod h1:wRXzRo76x1myOUMaVHAyraXoGBdEcvlLChGTVv5+DWU= github.com/twmb/franz-go/plugin/kprom v1.1.0 h1:grGeIJbm4llUBF8jkDjTb/b8rKllWSXjMwIqeCCcNYQ= github.com/twmb/franz-go/plugin/kprom v1.1.0/go.mod h1:cTDrPMSkyrO99LyGx3AtiwF9W6+THHjZrkDE2+TEBIU= github.com/uber-go/atomic v1.3.2/go.mod h1:/Ct5t2lcmbJ4OSe/waGBoaVvVqtO0bmtfVNex1PFV8g= diff --git a/pkg/ingester-kafka/ingester.go b/pkg/ingester-kafka/ingester.go deleted file mode 100644 index 1609eb819780..000000000000 --- a/pkg/ingester-kafka/ingester.go +++ /dev/null @@ -1,499 +0,0 @@ -package ingesterkafka - -import ( - "context" - "errors" - "flag" - "fmt" - "io" - "net/http" - "os" - "path" - "regexp" - "strconv" - "strings" - "sync" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/grafana/dskit/kv" - "github.com/grafana/dskit/modules" - "github.com/grafana/dskit/multierror" - "github.com/grafana/dskit/ring" - "github.com/grafana/dskit/services" - "github.com/prometheus/client_golang/prometheus" - "google.golang.org/grpc/health/grpc_health_v1" - - "github.com/grafana/loki/v3/pkg/ingester-kafka/kafka" - "github.com/grafana/loki/v3/pkg/ingester-kafka/partitionring" - util_log "github.com/grafana/loki/v3/pkg/util/log" - - "github.com/grafana/loki/v3/pkg/analytics" - "github.com/grafana/loki/v3/pkg/logproto" - "github.com/grafana/loki/v3/pkg/util" -) - -const ( - RingName = "kafka-ingester" - PartitionRingName = "kafka-partition" - - shutdownMarkerFilename = "shutdown-requested.txt" -) - -// ErrReadOnly is returned when the ingester is shutting down and a push was -// attempted. -var ( - ErrReadOnly = errors.New("Ingester is shutting down") - - activeTenantsStats = analytics.NewInt("ingester_active_tenants") - ingesterIDRegexp = regexp.MustCompile("ingester(-rf1)-([0-9]+)") -) - -// Config for an ingester. -type Config struct { - Enabled bool `yaml:"enabled" doc:"description=Whether the kafka ingester is enabled."` - - LifecyclerConfig ring.LifecyclerConfig `yaml:"lifecycler,omitempty" doc:"description=Configures how the lifecycle of the ingester will operate and where it will register for discovery."` - ShutdownMarkerPath string `yaml:"shutdown_marker_path"` - - // Used for the kafka ingestion path - PartitionRingConfig partitionring.Config `yaml:"partition_ring" category:"experimental"` - KafkaConfig kafka.Config -} - -// RegisterFlags registers the flags. -func (cfg *Config) RegisterFlags(f *flag.FlagSet) { - cfg.LifecyclerConfig.RegisterFlagsWithPrefix("kafka-ingester", f, util_log.Logger) - - f.BoolVar(&cfg.Enabled, "kafka-ingester.enabled", false, "Whether the Kafka-based ingester path is enabled") - f.StringVar(&cfg.ShutdownMarkerPath, "kafka-ingester.shutdown-marker-path", "", "Path where the shutdown marker file is stored. If not set and common.path_prefix is set then common.path_prefix will be used.") -} - -type Wrapper interface { - Wrap(wrapped Interface) Interface -} - -// Storage is the store interface we need on the ingester. -type Storage interface { - PutObject(ctx context.Context, objectKey string, object io.Reader) error - Stop() -} - -// Interface is an interface for the Ingester -type Interface interface { - services.Service - http.Handler - - logproto.PusherServer - - CheckReady(ctx context.Context) error - FlushHandler(w http.ResponseWriter, _ *http.Request) - ShutdownHandler(w http.ResponseWriter, r *http.Request) - PrepareShutdown(w http.ResponseWriter, r *http.Request) -} - -// Ingester builds chunks for incoming log streams. -type Ingester struct { - services.Service - - cfg Config - logger log.Logger - - metrics *ingesterMetrics - - // Flag for whether stopping the ingester service should also terminate the - // loki process. - // This is set when calling the shutdown handler. - terminateOnShutdown bool - readonly bool - shutdownMtx sync.Mutex // Allows processes to grab a lock and prevent a shutdown - - lifecycler *ring.Lifecycler - lifecyclerWatcher *services.FailureWatcher - ingesterPartitionID int32 - partitionRingLifecycler *ring.PartitionInstanceLifecycler -} - -// New makes a new Ingester. -func New(cfg Config, - registerer prometheus.Registerer, - metricsNamespace string, - logger log.Logger, -) (*Ingester, error) { - metrics := newIngesterMetrics(registerer) - - ingesterPartitionID, err := extractIngesterPartitionID(cfg.LifecyclerConfig.ID) - if err != nil { - return nil, fmt.Errorf("calculating ingester partition ID: %w", err) - } - - partitionRingKV := cfg.LifecyclerConfig.RingConfig.KVStore.Mock - if partitionRingKV == nil { - partitionRingKV, err = kv.NewClient(cfg.LifecyclerConfig.RingConfig.KVStore, ring.GetPartitionRingCodec(), kv.RegistererWithKVName(registerer, PartitionRingName+"-lifecycler"), logger) - if err != nil { - return nil, fmt.Errorf("creating KV store for ingester partition ring: %w", err) - } - } - - partitionRingLifecycler := ring.NewPartitionInstanceLifecycler( - cfg.PartitionRingConfig.ToLifecyclerConfig(ingesterPartitionID, cfg.LifecyclerConfig.ID), - PartitionRingName, - PartitionRingName+"-key", - partitionRingKV, - logger, - prometheus.WrapRegistererWithPrefix("loki_", registerer)) - - i := &Ingester{ - cfg: cfg, - logger: logger, - ingesterPartitionID: ingesterPartitionID, - partitionRingLifecycler: partitionRingLifecycler, - metrics: metrics, - } - - i.lifecycler, err = ring.NewLifecycler(cfg.LifecyclerConfig, i, RingName, RingName+"-ring", true, logger, prometheus.WrapRegistererWithPrefix(metricsNamespace+"_", registerer)) - if err != nil { - return nil, err - } - - i.lifecyclerWatcher = services.NewFailureWatcher() - i.lifecyclerWatcher.WatchService(i.lifecycler) - i.lifecyclerWatcher.WatchService(i.partitionRingLifecycler) - - i.Service = services.NewBasicService(i.starting, i.running, i.stopping) - - return i, nil -} - -// ingesterPartitionID returns the partition ID owner the the given ingester. -func extractIngesterPartitionID(ingesterID string) (int32, error) { - if strings.Contains(ingesterID, "local") { - return 0, nil - } - - match := ingesterIDRegexp.FindStringSubmatch(ingesterID) - if len(match) == 0 { - return 0, fmt.Errorf("ingester ID %s doesn't match regular expression %q", ingesterID, ingesterIDRegexp.String()) - } - // Parse the ingester sequence number. - ingesterSeq, err := strconv.Atoi(match[1]) - if err != nil { - return 0, fmt.Errorf("no ingester sequence number in ingester ID %s", ingesterID) - } - - return int32(ingesterSeq), nil -} - -// ServeHTTP implements the pattern ring status page. -func (i *Ingester) ServeHTTP(w http.ResponseWriter, r *http.Request) { - i.lifecycler.ServeHTTP(w, r) -} - -func (i *Ingester) starting(ctx context.Context) error { - // pass new context to lifecycler, so that it doesn't stop automatically when Ingester's service context is done - err := i.lifecycler.StartAsync(context.Background()) - if err != nil { - return err - } - - err = i.lifecycler.AwaitRunning(ctx) - if err != nil { - return err - } - - err = i.partitionRingLifecycler.StartAsync(context.Background()) - if err != nil { - return err - } - err = i.partitionRingLifecycler.AwaitRunning(ctx) - if err != nil { - return err - } - - shutdownMarkerPath := path.Join(i.cfg.ShutdownMarkerPath, shutdownMarkerFilename) - shutdownMarker, err := shutdownMarkerExists(shutdownMarkerPath) - if err != nil { - return fmt.Errorf("failed to check ingester shutdown marker: %w", err) - } - - if shutdownMarker { - level.Info(i.logger).Log("msg", "detected existing shutdown marker, setting unregister and flush on shutdown", "path", shutdownMarkerPath) - i.setPrepareShutdown() - } - - return nil -} - -func (i *Ingester) running(ctx context.Context) error { - var serviceError error - select { - // wait until service is asked to stop - case <-ctx.Done(): - // stop - case err := <-i.lifecyclerWatcher.Chan(): - serviceError = fmt.Errorf("lifecycler failed: %w", err) - } - - return serviceError -} - -// stopping is called when Ingester transitions to Stopping state. -// -// At this point, loop no longer runs, but flushers are still running. -func (i *Ingester) stopping(_ error) error { - i.stopIncomingRequests() - var errs util.MultiError - - //if i.flushOnShutdownSwitch.Get() { - // i.lifecycler.SetFlushOnShutdown(true) - //} - errs.Add(services.StopAndAwaitTerminated(context.Background(), i.lifecycler)) - - // i.streamRateCalculator.Stop() - - // In case the flag to terminate on shutdown is set or this instance is marked to release its resources, - // we need to mark the ingester service as "failed", so Loki will shut down entirely. - // The module manager logs the failure `modules.ErrStopProcess` in a special way. - if i.terminateOnShutdown && errs.Err() == nil { - i.removeShutdownMarkerFile() - return modules.ErrStopProcess - } - return errs.Err() -} - -// stopIncomingRequests is called when ingester is stopping -func (i *Ingester) stopIncomingRequests() { - i.shutdownMtx.Lock() - defer i.shutdownMtx.Unlock() - - i.readonly = true -} - -// removeShutdownMarkerFile removes the shutdown marker if it exists. Any errors are logged. -func (i *Ingester) removeShutdownMarkerFile() { - shutdownMarkerPath := path.Join(i.cfg.ShutdownMarkerPath, shutdownMarkerFilename) - exists, err := shutdownMarkerExists(shutdownMarkerPath) - if err != nil { - level.Error(i.logger).Log("msg", "error checking shutdown marker file exists", "err", err) - } - if exists { - err = removeShutdownMarker(shutdownMarkerPath) - if err != nil { - level.Error(i.logger).Log("msg", "error removing shutdown marker file", "err", err) - } - } -} - -// PrepareShutdown will handle the /ingester/prepare_shutdown endpoint. -// -// Internally, when triggered, this handler will configure the ingester service to release their resources whenever a SIGTERM is received. -// Releasing resources meaning flushing data, deleting tokens, and removing itself from the ring. -// -// It also creates a file on disk which is used to re-apply the configuration if the -// ingester crashes and restarts before being permanently shutdown. -// -// * `GET` shows the status of this configuration -// * `POST` enables this configuration -// * `DELETE` disables this configuration -func (i *Ingester) PrepareShutdown(w http.ResponseWriter, r *http.Request) { - if i.cfg.ShutdownMarkerPath == "" { - w.WriteHeader(http.StatusInternalServerError) - return - } - shutdownMarkerPath := path.Join(i.cfg.ShutdownMarkerPath, shutdownMarkerFilename) - - switch r.Method { - case http.MethodGet: - exists, err := shutdownMarkerExists(shutdownMarkerPath) - if err != nil { - level.Error(i.logger).Log("msg", "unable to check for prepare-shutdown marker file", "path", shutdownMarkerPath, "err", err) - w.WriteHeader(http.StatusInternalServerError) - return - } - - if exists { - util.WriteTextResponse(w, "set") - } else { - util.WriteTextResponse(w, "unset") - } - case http.MethodPost: - if err := createShutdownMarker(shutdownMarkerPath); err != nil { - level.Error(i.logger).Log("msg", "unable to create prepare-shutdown marker file", "path", shutdownMarkerPath, "err", err) - w.WriteHeader(http.StatusInternalServerError) - return - } - - i.setPrepareShutdown() - level.Info(i.logger).Log("msg", "created prepare-shutdown marker file", "path", shutdownMarkerPath) - - w.WriteHeader(http.StatusNoContent) - case http.MethodDelete: - if err := removeShutdownMarker(shutdownMarkerPath); err != nil { - level.Error(i.logger).Log("msg", "unable to remove prepare-shutdown marker file", "path", shutdownMarkerPath, "err", err) - w.WriteHeader(http.StatusInternalServerError) - return - } - - i.unsetPrepareShutdown() - level.Info(i.logger).Log("msg", "removed prepare-shutdown marker file", "path", shutdownMarkerPath) - - w.WriteHeader(http.StatusNoContent) - default: - w.WriteHeader(http.StatusMethodNotAllowed) - } -} - -// setPrepareShutdown toggles ingester lifecycler config to prepare for shutdown -func (i *Ingester) setPrepareShutdown() { - level.Info(i.logger).Log("msg", "preparing full ingester shutdown, resources will be released on SIGTERM") - i.lifecycler.SetFlushOnShutdown(true) - i.lifecycler.SetUnregisterOnShutdown(true) - i.terminateOnShutdown = true - i.metrics.shutdownMarker.Set(1) -} - -func (i *Ingester) unsetPrepareShutdown() { - level.Info(i.logger).Log("msg", "undoing preparation for full ingester shutdown") - i.lifecycler.SetFlushOnShutdown(true) - i.lifecycler.SetUnregisterOnShutdown(i.cfg.LifecyclerConfig.UnregisterOnShutdown) - i.terminateOnShutdown = false - i.metrics.shutdownMarker.Set(0) -} - -// createShutdownMarker writes a marker file to disk to indicate that an ingester is -// going to be scaled down in the future. The presence of this file means that an ingester -// should flush and upload all data when stopping. -func createShutdownMarker(p string) error { - // Write the file, fsync it, then fsync the containing directory in order to guarantee - // it is persisted to disk. From https://man7.org/linux/man-pages/man2/fsync.2.html - // - // > Calling fsync() does not necessarily ensure that the entry in the - // > directory containing the file has also reached disk. For that an - // > explicit fsync() on a file descriptor for the directory is also - // > needed. - file, err := os.Create(p) - if err != nil { - return err - } - - merr := multierror.New() - _, err = file.WriteString(time.Now().UTC().Format(time.RFC3339)) - merr.Add(err) - merr.Add(file.Sync()) - merr.Add(file.Close()) - - if err := merr.Err(); err != nil { - return err - } - - dir, err := os.OpenFile(path.Dir(p), os.O_RDONLY, 0o777) - if err != nil { - return err - } - - merr.Add(dir.Sync()) - merr.Add(dir.Close()) - return merr.Err() -} - -// removeShutdownMarker removes the shutdown marker file if it exists. -func removeShutdownMarker(p string) error { - err := os.Remove(p) - if err != nil && !os.IsNotExist(err) { - return err - } - - dir, err := os.OpenFile(path.Dir(p), os.O_RDONLY, 0o777) - if err != nil { - return err - } - - merr := multierror.New() - merr.Add(dir.Sync()) - merr.Add(dir.Close()) - return merr.Err() -} - -// shutdownMarkerExists returns true if the shutdown marker file exists, false otherwise -func shutdownMarkerExists(p string) (bool, error) { - s, err := os.Stat(p) - if err != nil && os.IsNotExist(err) { - return false, nil - } - - if err != nil { - return false, err - } - - return s.Mode().IsRegular(), nil -} - -// ShutdownHandler handles a graceful shutdown of the ingester service and -// termination of the Loki process. -func (i *Ingester) ShutdownHandler(w http.ResponseWriter, r *http.Request) { - // Don't allow calling the shutdown handler multiple times - if i.State() != services.Running { - w.WriteHeader(http.StatusServiceUnavailable) - _, _ = w.Write([]byte("Ingester is stopping or already stopped.")) - return - } - params := r.URL.Query() - doFlush := util.FlagFromValues(params, "flush", true) - doDeleteRingTokens := util.FlagFromValues(params, "delete_ring_tokens", false) - doTerminate := util.FlagFromValues(params, "terminate", true) - err := i.handleShutdown(doTerminate, doFlush, doDeleteRingTokens) - - // Stopping the module will return the modules.ErrStopProcess error. This is - // needed so the Loki process is shut down completely. - if err == nil || err == modules.ErrStopProcess { - w.WriteHeader(http.StatusNoContent) - } else { - w.WriteHeader(http.StatusInternalServerError) - _, _ = w.Write([]byte(err.Error())) - } -} - -// handleShutdown triggers the following operations: -// - Change the state of ring to stop accepting writes. -// - optional: Flush all the chunks. -// - optional: Delete ring tokens file -// - Unregister from KV store -// - optional: Terminate process (handled by service manager in loki.go) -func (i *Ingester) handleShutdown(terminate, flush, del bool) error { - i.lifecycler.SetFlushOnShutdown(flush) - i.lifecycler.SetClearTokensOnShutdown(del) - i.lifecycler.SetUnregisterOnShutdown(true) - i.terminateOnShutdown = terminate - return services.StopAndAwaitTerminated(context.Background(), i) -} - -// Push implements logproto.Pusher. -func (i *Ingester) Push(_ context.Context, _ *logproto.PushRequest) (*logproto.PushResponse, error) { - return &logproto.PushResponse{}, nil -} - -// Watch implements grpc_health_v1.HealthCheck. -func (*Ingester) Watch(*grpc_health_v1.HealthCheckRequest, grpc_health_v1.Health_WatchServer) error { - return nil -} - -// ReadinessHandler is used to indicate to k8s when the ingesters are ready for -// the addition removal of another ingester. Returns 204 when the ingester is -// ready, 500 otherwise. -func (i *Ingester) CheckReady(ctx context.Context) error { - if s := i.State(); s != services.Running && s != services.Stopping { - return fmt.Errorf("ingester not ready: %v", s) - } - return i.lifecycler.CheckReady(ctx) -} - -// Flush implements ring.FlushTransferer -// Flush triggers a flush of all the chunks and closes the flush queues. -// Called from the Lifecycler as part of the ingester shutdown. -func (i *Ingester) Flush() { -} - -func (i *Ingester) TransferOut(_ context.Context) error { - return nil -} diff --git a/pkg/ingester-rf1/objstore/test_storage.go b/pkg/ingester-rf1/objstore/test_storage.go new file mode 100644 index 000000000000..db25c8487cca --- /dev/null +++ b/pkg/ingester-rf1/objstore/test_storage.go @@ -0,0 +1,37 @@ +package objstore + +import ( + "os" + "testing" + + "github.com/prometheus/common/model" + + "github.com/grafana/loki/v3/pkg/storage" + "github.com/grafana/loki/v3/pkg/storage/chunk/client/local" + "github.com/grafana/loki/v3/pkg/storage/config" +) + +var metrics *storage.ClientMetrics + +func NewTestStorage(t testing.TB) (*Multi, error) { + if metrics == nil { + m := storage.NewClientMetrics() + metrics = &m + } + dir := t.TempDir() + t.Cleanup(func() { + os.RemoveAll(dir) + metrics.Unregister() + }) + cfg := storage.Config{ + FSConfig: local.FSConfig{ + Directory: dir, + }, + } + return New([]config.PeriodConfig{ + { + From: config.DayTime{Time: model.Now()}, + ObjectType: "filesystem", + }, + }, cfg, *metrics) +} diff --git a/pkg/kafka/config.go b/pkg/kafka/config.go new file mode 100644 index 000000000000..f916b145f008 --- /dev/null +++ b/pkg/kafka/config.go @@ -0,0 +1,109 @@ +package kafka + +import ( + "errors" + "flag" + "fmt" + "strconv" + "strings" + "time" +) + +const ( + consumeFromLastOffset = "last-offset" + consumeFromStart = "start" + consumeFromEnd = "end" + consumeFromTimestamp = "timestamp" + + // writerRequestTimeoutOverhead is the overhead applied by the Writer to every Kafka timeout. + // You can think about this overhead as an extra time for requests sitting in the client's buffer + // before being sent on the wire and the actual time it takes to send it over the network and + // start being processed by Kafka. + writerRequestTimeoutOverhead = 2 * time.Second + + // producerBatchMaxBytes is the max allowed size of a batch of Kafka records. + producerBatchMaxBytes = 16_000_000 + + // maxProducerRecordDataBytesLimit is the max allowed size of a single record data. Given we have a limit + // on the max batch size (producerBatchMaxBytes), a Kafka record data can't be bigger than the batch size + // minus some overhead required to serialise the batch and the record itself. We use 16KB as such overhead + // in the worst case scenario, which is expected to be way above the actual one. + maxProducerRecordDataBytesLimit = producerBatchMaxBytes - 16384 + minProducerRecordDataBytesLimit = 1024 * 1024 + + kafkaConfigFlagPrefix = "ingest-storage.kafka" + targetConsumerLagAtStartupFlag = kafkaConfigFlagPrefix + ".target-consumer-lag-at-startup" + maxConsumerLagAtStartupFlag = kafkaConfigFlagPrefix + ".max-consumer-lag-at-startup" +) + +var ( + ErrMissingKafkaAddress = errors.New("the Kafka address has not been configured") + ErrMissingKafkaTopic = errors.New("the Kafka topic has not been configured") + ErrInvalidProducerMaxRecordSizeBytes = fmt.Errorf("the configured producer max record size bytes must be a value between %d and %d", minProducerRecordDataBytesLimit, maxProducerRecordDataBytesLimit) + + consumeFromPositionOptions = []string{consumeFromLastOffset, consumeFromStart, consumeFromEnd, consumeFromTimestamp} +) + +// Config holds the generic config for the Kafka backend. +type Config struct { + Address string `yaml:"address"` + Topic string `yaml:"topic"` + ClientID string `yaml:"client_id"` + DialTimeout time.Duration `yaml:"dial_timeout"` + WriteTimeout time.Duration `yaml:"write_timeout"` + + ConsumerGroup string `yaml:"consumer_group"` + + LastProducedOffsetRetryTimeout time.Duration `yaml:"last_produced_offset_retry_timeout"` + + AutoCreateTopicEnabled bool `yaml:"auto_create_topic_enabled"` + // AutoCreateTopicDefaultPartitions int `yaml:"auto_create_topic_default_partitions"` + + ProducerMaxRecordSizeBytes int `yaml:"producer_max_record_size_bytes"` + ProducerMaxBufferedBytes int64 `yaml:"producer_max_buffered_bytes"` +} + +func (cfg *Config) RegisterFlags(f *flag.FlagSet) { + cfg.RegisterFlagsWithPrefix("kafka", f) +} + +func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { + f.StringVar(&cfg.Address, prefix+".address", "localhost:9092", "The Kafka backend address.") + f.StringVar(&cfg.Topic, prefix+".topic", "", "The Kafka topic name.") + f.StringVar(&cfg.ClientID, prefix+".client-id", "", "The Kafka client ID.") + f.DurationVar(&cfg.DialTimeout, prefix+".dial-timeout", 2*time.Second, "The maximum time allowed to open a connection to a Kafka broker.") + f.DurationVar(&cfg.WriteTimeout, prefix+".write-timeout", 10*time.Second, "How long to wait for an incoming write request to be successfully committed to the Kafka backend.") + + f.StringVar(&cfg.ConsumerGroup, prefix+".consumer-group", "", "The consumer group used by the consumer to track the last consumed offset. The consumer group must be different for each ingester. If the configured consumer group contains the '' placeholder, it is replaced with the actual partition ID owned by the ingester. When empty (recommended), Mimir uses the ingester instance ID to guarantee uniqueness.") + + f.DurationVar(&cfg.LastProducedOffsetRetryTimeout, prefix+".last-produced-offset-retry-timeout", 10*time.Second, "How long to retry a failed request to get the last produced offset.") + + f.BoolVar(&cfg.AutoCreateTopicEnabled, prefix+".auto-create-topic-enabled", true, "Enable auto-creation of Kafka topic if it doesn't exist.") + // f.IntVar(&cfg.AutoCreateTopicDefaultPartitions, prefix+".auto-create-topic-default-partitions", 0, "When auto-creation of Kafka topic is enabled and this value is positive, Kafka's num.partitions configuration option is set on Kafka brokers with this value when Mimir component that uses Kafka starts. This configuration option specifies the default number of partitions that the Kafka broker uses for auto-created topics. Note that this is a Kafka-cluster wide setting, and applies to any auto-created topic. If the setting of num.partitions fails, Mimir proceeds anyways, but auto-created topics could have an incorrect number of partitions.") + + f.IntVar(&cfg.ProducerMaxRecordSizeBytes, prefix+".producer-max-record-size-bytes", maxProducerRecordDataBytesLimit, "The maximum size of a Kafka record data that should be generated by the producer. An incoming write request larger than this size is split into multiple Kafka records. We strongly recommend to not change this setting unless for testing purposes.") + f.Int64Var(&cfg.ProducerMaxBufferedBytes, prefix+".producer-max-buffered-bytes", 1024*1024*1024, "The maximum size of (uncompressed) buffered and unacknowledged produced records sent to Kafka. The produce request fails once this limit is reached. This limit is per Kafka client. 0 to disable the limit.") +} + +func (cfg *Config) Validate() error { + if cfg.Address == "" { + return ErrMissingKafkaAddress + } + if cfg.Topic == "" { + return ErrMissingKafkaTopic + } + if cfg.ProducerMaxRecordSizeBytes < minProducerRecordDataBytesLimit || cfg.ProducerMaxRecordSizeBytes > maxProducerRecordDataBytesLimit { + return ErrInvalidProducerMaxRecordSizeBytes + } + + return nil +} + +// GetConsumerGroup returns the consumer group to use for the given instanceID and partitionID. +func (cfg *Config) GetConsumerGroup(instanceID string, partitionID int32) string { + if cfg.ConsumerGroup == "" { + return instanceID + } + + return strings.ReplaceAll(cfg.ConsumerGroup, "", strconv.Itoa(int(partitionID))) +} diff --git a/pkg/kafka/encoding.go b/pkg/kafka/encoding.go new file mode 100644 index 000000000000..c4977054f32f --- /dev/null +++ b/pkg/kafka/encoding.go @@ -0,0 +1,175 @@ +// Package kafka provides encoding and decoding functionality for Loki's Kafka integration. +package kafka + +import ( + "errors" + "fmt" + math_bits "math/bits" + "sync" + + "github.com/twmb/franz-go/pkg/kgo" + + lru "github.com/hashicorp/golang-lru" + "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" +) + +var encoderPool = sync.Pool{ + New: func() any { + return &logproto.Stream{} + }, +} + +// Encode converts a logproto.Stream into one or more Kafka records. +// It handles splitting large streams into multiple records if necessary. +// +// The encoding process works as follows: +// 1. If the stream size is smaller than maxSize, it's encoded into a single record. +// 2. For larger streams, it splits the entries into multiple batches, each under maxSize. +// 3. The data is wrapped in a Kafka record with the tenant ID as the key. +// +// The format of each record is: +// - Key: Tenant ID (used for routing, not for partitioning) +// - Value: Protobuf serialized logproto.Stream +// - Partition: As specified in the partitionID parameter +// +// Parameters: +// - partitionID: The Kafka partition ID for the record +// - tenantID: The tenant ID for the stream +// - stream: The logproto.Stream to be encoded +// - maxSize: The maximum size of each Kafka record +func Encode(partitionID int32, tenantID string, stream logproto.Stream, maxSize int) ([]*kgo.Record, error) { + reqSize := stream.Size() + + // Fast path for small requests + if reqSize <= maxSize { + rec, err := marshalWriteRequestToRecord(partitionID, tenantID, stream) + if err != nil { + return nil, err + } + return []*kgo.Record{rec}, nil + } + + var records []*kgo.Record + batch := encoderPool.Get().(*logproto.Stream) + defer encoderPool.Put(batch) + + batch.Labels = stream.Labels + batch.Hash = stream.Hash + + if batch.Entries == nil { + batch.Entries = make([]logproto.Entry, 0, 1024) + } + batch.Entries = batch.Entries[:0] + labelsSize := batch.Size() + currentSize := labelsSize + + for i, entry := range stream.Entries { + l := entry.Size() + // Size of the entry in the stream + entrySize := 1 + l + sovPush(uint64(l)) + + // Check if a single entry is too big + if entrySize > maxSize || (i == 0 && currentSize+entrySize > maxSize) { + return nil, fmt.Errorf("single entry size (%d) exceeds maximum allowed size (%d)", entrySize, maxSize) + } + + if currentSize+entrySize > maxSize { + // Current stream is full, create a record and start a new stream + if len(batch.Entries) > 0 { + rec, err := marshalWriteRequestToRecord(partitionID, tenantID, *batch) + if err != nil { + return nil, err + } + records = append(records, rec) + } + // Reset currentStream + batch.Entries = batch.Entries[:0] + currentSize = labelsSize + } + batch.Entries = append(batch.Entries, entry) + currentSize += entrySize + } + + // Handle any remaining entries + if len(batch.Entries) > 0 { + rec, err := marshalWriteRequestToRecord(partitionID, tenantID, *batch) + if err != nil { + return nil, err + } + records = append(records, rec) + } + + if len(records) == 0 { + return nil, errors.New("no valid records created") + } + + return records, nil +} + +func marshalWriteRequestToRecord(partitionID int32, tenantID string, stream logproto.Stream) (*kgo.Record, error) { + data, err := stream.Marshal() + if err != nil { + return nil, fmt.Errorf("failed to marshal stream: %w", err) + } + + return &kgo.Record{ + Key: []byte(tenantID), + Value: data, + Partition: partitionID, + }, nil +} + +// Decoder is responsible for decoding Kafka record data back into logproto.Stream format. +// It caches parsed labels for efficiency. +type Decoder struct { + stream *logproto.Stream + cache *lru.Cache +} + +func NewDecoder() (*Decoder, error) { + cache, err := lru.New(5000) // Set LRU size to 5000, adjust as needed + if err != nil { + return nil, fmt.Errorf("failed to create LRU cache: %w", err) + } + return &Decoder{ + stream: &logproto.Stream{}, + cache: cache, + }, nil +} + +// Decode converts a Kafka record's byte data back into a logproto.Stream and labels.Labels. +// The decoding process works as follows: +// 1. Unmarshal the data into a logproto.Stream. +// 2. Parse and cache the labels for efficiency in future decodes. +// +// Returns the decoded logproto.Stream, parsed labels, and any error encountered. +func (d *Decoder) Decode(data []byte) (logproto.Stream, labels.Labels, error) { + d.stream.Entries = d.stream.Entries[:0] + if err := d.stream.Unmarshal(data); err != nil { + return logproto.Stream{}, nil, fmt.Errorf("failed to unmarshal stream: %w", err) + } + + var ls labels.Labels + if cachedLabels, ok := d.cache.Get(d.stream.Labels); ok { + ls = cachedLabels.(labels.Labels) + } else { + var err error + ls, err = syntax.ParseLabels(d.stream.Labels) + if err != nil { + return logproto.Stream{}, nil, fmt.Errorf("failed to parse labels: %w", err) + } + d.cache.Add(d.stream.Labels, ls) + } + + return *d.stream, ls, nil +} + +// sovPush calculates the size of varint-encoded uint64. +// It is used to determine the number of bytes needed to encode a uint64 value +// in Protocol Buffers' variable-length integer format. +func sovPush(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} diff --git a/pkg/kafka/encoding_test.go b/pkg/kafka/encoding_test.go new file mode 100644 index 000000000000..3b058b782fda --- /dev/null +++ b/pkg/kafka/encoding_test.go @@ -0,0 +1,151 @@ +package kafka + +import ( + "math/rand" + "testing" + "time" + + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/v3/pkg/logproto" +) + +func TestEncoderDecoder(t *testing.T) { + tests := []struct { + name string + stream logproto.Stream + maxSize int + expectSplit bool + }{ + { + name: "Small stream, no split", + stream: generateStream(10, 100), + maxSize: 1024 * 1024, + expectSplit: false, + }, + { + name: "Large stream, expect split", + stream: generateStream(1000, 1000), + maxSize: 1024 * 10, + expectSplit: true, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + decoder, err := NewDecoder() + require.NoError(t, err) + + records, err := Encode(0, "test-tenant", tt.stream, tt.maxSize) + require.NoError(t, err) + + if tt.expectSplit { + require.Greater(t, len(records), 1) + } else { + require.Equal(t, 1, len(records)) + } + + var decodedEntries []logproto.Entry + var decodedLabels labels.Labels + + for _, record := range records { + stream, ls, err := decoder.Decode(record.Value) + require.NoError(t, err) + decodedEntries = append(decodedEntries, stream.Entries...) + if decodedLabels == nil { + decodedLabels = ls + } else { + require.Equal(t, decodedLabels, ls) + } + } + + require.Equal(t, tt.stream.Labels, decodedLabels.String()) + require.Equal(t, len(tt.stream.Entries), len(decodedEntries)) + for i, entry := range tt.stream.Entries { + require.Equal(t, entry.Timestamp.UTC(), decodedEntries[i].Timestamp.UTC()) + require.Equal(t, entry.Line, decodedEntries[i].Line) + } + }) + } +} + +func TestEncoderSingleEntryTooLarge(t *testing.T) { + stream := generateStream(1, 1000) + + _, err := Encode(0, "test-tenant", stream, 100) + require.Error(t, err) + require.Contains(t, err.Error(), "single entry size") +} + +func TestDecoderInvalidData(t *testing.T) { + decoder, err := NewDecoder() + require.NoError(t, err) + + _, _, err = decoder.Decode([]byte("invalid data")) + require.Error(t, err) +} + +func TestEncoderDecoderEmptyStream(t *testing.T) { + decoder, err := NewDecoder() + require.NoError(t, err) + + stream := logproto.Stream{ + Labels: `{app="test"}`, + } + + records, err := Encode(0, "test-tenant", stream, 10<<20) + require.NoError(t, err) + require.Len(t, records, 1) + + decodedStream, decodedLabels, err := decoder.Decode(records[0].Value) + require.NoError(t, err) + require.Equal(t, stream.Labels, decodedLabels.String()) + require.Empty(t, decodedStream.Entries) +} + +func BenchmarkEncodeDecode(b *testing.B) { + decoder, _ := NewDecoder() + stream := generateStream(1000, 200) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + records, err := Encode(0, "test-tenant", stream, 10<<20) + if err != nil { + b.Fatal(err) + } + for _, record := range records { + _, _, err := decoder.Decode(record.Value) + if err != nil { + b.Fatal(err) + } + } + } +} + +// Helper function to generate a test stream +func generateStream(entries, lineLength int) logproto.Stream { + stream := logproto.Stream{ + Labels: `{app="test", env="prod"}`, + Entries: make([]logproto.Entry, entries), + } + + for i := 0; i < entries; i++ { + stream.Entries[i] = logproto.Entry{ + Timestamp: time.Now(), + Line: generateRandomString(lineLength), + } + } + + return stream +} + +// Helper function to generate a random string +func generateRandomString(length int) string { + const charset = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789" + b := make([]byte, length) + for i := range b { + b[i] = charset[rand.Intn(len(charset))] + } + return string(b) +} diff --git a/pkg/kafka/ingester/consumer.go b/pkg/kafka/ingester/consumer.go new file mode 100644 index 000000000000..352916838c40 --- /dev/null +++ b/pkg/kafka/ingester/consumer.go @@ -0,0 +1,307 @@ +package ingester + +import ( + "bytes" + "context" + "crypto/rand" + "fmt" + "io" + "math" + "sync" + "time" + + "github.com/dustin/go-humanize" + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "google.golang.org/grpc" + + "github.com/grafana/dskit/backoff" + + "github.com/grafana/loki/v3/pkg/ingester-rf1/metastore/metastorepb" + "github.com/grafana/loki/v3/pkg/kafka" + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/storage/wal" +) + +// ObjectStorage defines an interface for object storage operations +type ObjectStorage interface { + PutObject(ctx context.Context, objectKey string, object io.Reader) error +} + +// MetadataStore defines an interface for metadata storage operations +type MetadataStore interface { + AddBlock(ctx context.Context, in *metastorepb.AddBlockRequest, opts ...grpc.CallOption) (*metastorepb.AddBlockResponse, error) +} + +// Committer defines an interface for committing offsets +type Committer interface { + Commit(ctx context.Context, offset int64) error +} + +// consumer represents a Kafka consumer that processes and stores log entries +type consumer struct { + metastoreClient MetadataStore + storage ObjectStorage + writer *wal.SegmentWriter + committer Committer + flushInterval time.Duration + maxFlushSize int64 + lastOffset int64 + + flushBuf *bytes.Buffer + decoder *kafka.Decoder + toStore []*logproto.Entry + + metrics *consumerMetrics + logger log.Logger +} + +// NewConsumerFactory creates and initializes a new consumer instance +func NewConsumerFactory( + metastoreClient MetadataStore, + storage ObjectStorage, + flushInterval time.Duration, + maxFlushSize int64, + logger log.Logger, + reg prometheus.Registerer, +) ConsumerFactory { + return func(committer Committer) (Consumer, error) { + writer, err := wal.NewWalSegmentWriter() + if err != nil { + return nil, err + } + decoder, err := kafka.NewDecoder() + if err != nil { + return nil, err + } + return &consumer{ + logger: logger, + metastoreClient: metastoreClient, + storage: storage, + writer: writer, + metrics: newConsumerMetrics(reg), + flushBuf: bytes.NewBuffer(make([]byte, 0, 10<<20)), // 10 MB + decoder: decoder, + committer: committer, + flushInterval: flushInterval, + maxFlushSize: maxFlushSize, + lastOffset: -1, + }, nil + } +} + +// Start starts the consumer and returns a function to wait for it to finish +// It consumes records from the recordsChan, and flushes them to storage periodically. +func (c *consumer) Start(ctx context.Context, recordsChan <-chan []record) func() { + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + flushTicker := time.NewTicker(c.flushInterval) + defer flushTicker.Stop() + for { + select { + case <-flushTicker.C: + level.Info(c.logger).Log("msg", "flushing block") + c.Flush() + case <-ctx.Done(): + level.Info(c.logger).Log("msg", "shutting down consumer") + c.Flush() + return + case records := <-recordsChan: + if err := c.consume(records); err != nil { + level.Error(c.logger).Log("msg", "failed to consume records", "error", err) + return + } + if c.writer.InputSize() > c.maxFlushSize { + level.Info(c.logger).Log("msg", "flushing block due to size limit", "size", humanize.Bytes(uint64(c.writer.InputSize()))) + c.Flush() + } + } + } + }() + return wg.Wait +} + +// consume processes a batch of Kafka records, decoding and storing them +func (c *consumer) consume(records []record) error { + if len(records) == 0 { + return nil + } + var ( + minOffset = int64(math.MaxInt64) + maxOffset = int64(0) + ) + for _, record := range records { + minOffset = min(minOffset, record.offset) + maxOffset = max(maxOffset, record.offset) + } + level.Debug(c.logger).Log("msg", "consuming records", "min_offset", minOffset, "max_offset", maxOffset) + return c.retryWithBackoff(context.Background(), backoff.Config{ + MinBackoff: 250 * time.Millisecond, + MaxBackoff: 2 * time.Second, + MaxRetries: 0, // retry forever + }, func(boff *backoff.Backoff) error { + consumeStart := time.Now() + if err := c.appendRecords(records); err != nil { + level.Error(c.logger).Log( + "msg", "encountered error while ingesting data from Kafka; should retry", + "err", err, + "record_min_offset", minOffset, + "record_max_offset", maxOffset, + "num_retries", boff.NumRetries(), + ) + return err + } + c.lastOffset = maxOffset + c.metrics.currentOffset.Set(float64(c.lastOffset)) + c.metrics.consumeLatency.Observe(time.Since(consumeStart).Seconds()) + return nil + }) +} + +func (c *consumer) appendRecords(records []record) error { + for _, record := range records { + stream, labels, err := c.decoder.Decode(record.content) + if err != nil { + return fmt.Errorf("failed to decode record: %w", err) + } + if len(stream.Entries) == 0 { + continue + } + if len(c.toStore) == 0 { + c.toStore = make([]*logproto.Entry, 0, len(stream.Entries)) + } + c.toStore = c.toStore[:0] + for _, entry := range stream.Entries { + c.toStore = append(c.toStore, &logproto.Entry{ + Timestamp: entry.Timestamp, + Line: entry.Line, + StructuredMetadata: entry.StructuredMetadata, + Parsed: entry.Parsed, + }) + } + c.writer.Append(record.tenantID, stream.Labels, labels, c.toStore, time.Now()) + } + return nil +} + +// Flush writes the accumulated data to storage and updates the metadata store +func (c *consumer) Flush() { + if c.writer.InputSize() == 0 { + return + } + if c.lastOffset == -1 { + return + } + if err := c.retryWithBackoff(context.Background(), backoff.Config{ + MinBackoff: 250 * time.Millisecond, + MaxBackoff: 10 * time.Second, + MaxRetries: 0, // retry forever + }, func(boff *backoff.Backoff) error { + start := time.Now() + c.metrics.flushesTotal.Add(1) + defer func() { c.metrics.flushDuration.Observe(time.Since(start).Seconds()) }() + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + defer cancel() + if err := c.flush(ctx); err != nil { + c.metrics.flushFailuresTotal.Inc() + level.Error(c.logger).Log( + "msg", "failed to flush block", + "error", err, + "num_retries", boff.NumRetries(), + ) + return err + } + c.lastOffset = -1 + return nil + }); err != nil { + level.Error(c.logger).Log("msg", "failed to flush block", "error", err) + } +} + +func (c *consumer) retryWithBackoff(ctx context.Context, cfg backoff.Config, fn func(boff *backoff.Backoff) error) error { + boff := backoff.New(ctx, cfg) + var err error + for boff.Ongoing() { + err = fn(boff) + if err == nil { + return nil + } + boff.Wait() + } + if err != nil { + return err + } + return boff.ErrCause() +} + +func (c *consumer) flush(ctx context.Context) error { + defer c.flushBuf.Reset() + if _, err := c.writer.WriteTo(c.flushBuf); err != nil { + return err + } + + stats := wal.GetSegmentStats(c.writer, time.Now()) + wal.ReportSegmentStats(stats, c.metrics.segmentMetrics) + + id := ulid.MustNew(ulid.Timestamp(time.Now()), rand.Reader).String() + if err := c.storage.PutObject(ctx, fmt.Sprintf(wal.Dir+id), c.flushBuf); err != nil { + return fmt.Errorf("failed to put object to object storage: %w", err) + } + + if _, err := c.metastoreClient.AddBlock(ctx, &metastorepb.AddBlockRequest{ + Block: c.writer.Meta(id), + }); err != nil { + return fmt.Errorf("failed to add block to metastore: %w", err) + } + c.writer.Reset() + if err := c.committer.Commit(ctx, c.lastOffset); err != nil { + return fmt.Errorf("failed to commit offset: %w", err) + } + + return nil +} + +// consumerMetrics holds various Prometheus metrics for monitoring consumer operations +type consumerMetrics struct { + flushesTotal prometheus.Counter + flushFailuresTotal prometheus.Counter + flushDuration prometheus.Histogram + segmentMetrics *wal.SegmentMetrics + consumeLatency prometheus.Histogram + currentOffset prometheus.Gauge +} + +// newConsumerMetrics initializes and returns a new consumerMetrics instance +func newConsumerMetrics(reg prometheus.Registerer) *consumerMetrics { + return &consumerMetrics{ + flushesTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "loki_kafka_ingester_flushes_total", + Help: "The total number of flushes.", + }), + flushFailuresTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "loki_kafka_ingester_flush_failures_total", + Help: "The total number of failed flushes.", + }), + flushDuration: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "loki_kafka_ingester_flush_duration_seconds", + Help: "The flush duration (in seconds).", + Buckets: prometheus.ExponentialBuckets(0.001, 4, 8), + NativeHistogramBucketFactor: 1.1, + }), + consumeLatency: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "loki_ingest_storage_reader_records_batch_process_duration_seconds", + Help: "How long a consumer spent processing a batch of records from Kafka.", + NativeHistogramBucketFactor: 1.1, + }), + segmentMetrics: wal.NewSegmentMetrics(reg), + currentOffset: promauto.With(reg).NewGauge(prometheus.GaugeOpts{ + Name: "loki_kafka_ingester_current_offset", + Help: "The current offset of the Kafka consumer.", + }), + } +} diff --git a/pkg/kafka/ingester/consumer_test.go b/pkg/kafka/ingester/consumer_test.go new file mode 100644 index 000000000000..3f0adcce6247 --- /dev/null +++ b/pkg/kafka/ingester/consumer_test.go @@ -0,0 +1,192 @@ +package ingester + +import ( + "context" + "os" + "strings" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/v3/pkg/ingester-rf1/metastore/metastorepb" + "github.com/grafana/loki/v3/pkg/ingester-rf1/objstore" + "github.com/grafana/loki/v3/pkg/kafka" + "github.com/grafana/loki/v3/pkg/logproto" +) + +type mockCommitter struct { + committed int64 +} + +func newMockCommitter() *mockCommitter { + return &mockCommitter{ + committed: -1, + } +} + +func (m *mockCommitter) Commit(_ context.Context, offset int64) error { + m.committed = offset + return nil +} + +func TestConsumer_PeriodicFlush(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + storage, err := objstore.NewTestStorage(t) + require.NoError(t, err) + + metastore := NewTestMetastore() + reg := prometheus.NewRegistry() + + flushInterval := 100 * time.Millisecond + maxFlushSize := int64(1000) + + committer := &mockCommitter{} + consumerFactory := NewConsumerFactory(metastore, storage, flushInterval, maxFlushSize, log.NewLogfmtLogger(os.Stdout), reg) + consumer, err := consumerFactory(committer) + require.NoError(t, err) + + recordsChan := make(chan []record) + _ = consumer.Start(ctx, recordsChan) + + stream := logproto.Stream{ + Labels: `{__name__="test_metric", label="value1"}`, + Entries: []logproto.Entry{ + {Timestamp: time.Unix(0, 1000), Line: "10.5"}, + }, + } + + encodedRecords, err := kafka.Encode(0, "tenant1", stream, 10<<20) + require.NoError(t, err) + + records := []record{{ + tenantID: "tenant1", + content: encodedRecords[0].Value, + offset: 0, + }} + + recordsChan <- records + + require.Eventually(t, func() bool { + blocks, err := metastore.ListBlocksForQuery(ctx, &metastorepb.ListBlocksForQueryRequest{ + TenantId: "tenant1", + StartTime: 0, + EndTime: 100000, + }) + require.NoError(t, err) + return len(blocks.Blocks) == 1 + }, 5*time.Second, 100*time.Millisecond) + + // Verify committed offset + require.Equal(t, int64(0), committer.committed) +} + +func TestConsumer_ShutdownFlush(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + storage, err := objstore.NewTestStorage(t) + require.NoError(t, err) + + metastore := NewTestMetastore() + reg := prometheus.NewRegistry() + + flushInterval := 1 * time.Hour + maxFlushSize := int64(1000) + + committer := &mockCommitter{} + consumerFactory := NewConsumerFactory(metastore, storage, flushInterval, maxFlushSize, log.NewLogfmtLogger(os.Stdout), reg) + consumer, err := consumerFactory(committer) + require.NoError(t, err) + + recordsChan := make(chan []record) + wait := consumer.Start(ctx, recordsChan) + + stream := logproto.Stream{ + Labels: `{__name__="test_metric", label="value1"}`, + Entries: []logproto.Entry{ + {Timestamp: time.Unix(0, 1000), Line: "10.5"}, + }, + } + + encodedRecords, err := kafka.Encode(0, "tenant1", stream, 10<<20) + require.NoError(t, err) + + records := []record{{ + tenantID: "tenant1", + content: encodedRecords[0].Value, + offset: 0, + }} + + recordsChan <- records + + cancel() + wait() + + blocks, err := metastore.ListBlocksForQuery(ctx, &metastorepb.ListBlocksForQueryRequest{ + TenantId: "tenant1", + StartTime: 0, + EndTime: 100000, + }) + require.NoError(t, err) + require.Equal(t, 1, len(blocks.Blocks)) + + // Verify committed offset + require.Equal(t, int64(0), committer.committed) +} + +func TestConsumer_MaxFlushSize(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + storage, err := objstore.NewTestStorage(t) + require.NoError(t, err) + + metastore := NewTestMetastore() + reg := prometheus.NewRegistry() + + flushInterval := 1 * time.Hour + maxFlushSize := int64(10) + + committer := &mockCommitter{} + consumerFactory := NewConsumerFactory(metastore, storage, flushInterval, maxFlushSize, log.NewLogfmtLogger(os.Stdout), reg) + consumer, err := consumerFactory(committer) + require.NoError(t, err) + + recordsChan := make(chan []record) + _ = consumer.Start(ctx, recordsChan) + + stream := logproto.Stream{ + Labels: `{__name__="test_metric", label="value1"}`, + Entries: []logproto.Entry{ + {Timestamp: time.Unix(0, 1000), Line: strings.Repeat("a", 100)}, + }, + } + + encodedRecords, err := kafka.Encode(0, "tenant1", stream, 10<<20) + require.NoError(t, err) + + records := []record{{ + tenantID: "tenant1", + content: encodedRecords[0].Value, + offset: 0, + }} + + recordsChan <- records + + require.Eventually(t, func() bool { + blocks, err := metastore.ListBlocksForQuery(ctx, &metastorepb.ListBlocksForQueryRequest{ + TenantId: "tenant1", + StartTime: 0, + EndTime: 100000, + }) + require.NoError(t, err) + return len(blocks.Blocks) == 1 + }, 5*time.Second, 100*time.Millisecond) + + require.Equal(t, int64(0), committer.committed) +} diff --git a/pkg/kafka/ingester/ingester.go b/pkg/kafka/ingester/ingester.go new file mode 100644 index 000000000000..56421b1b712d --- /dev/null +++ b/pkg/kafka/ingester/ingester.go @@ -0,0 +1,405 @@ +package ingester + +import ( + "context" + "errors" + "flag" + "fmt" + "net/http" + "regexp" + "strconv" + "strings" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/kv" + "github.com/grafana/dskit/ring" + "github.com/grafana/dskit/services" + "github.com/prometheus/client_golang/prometheus" + "google.golang.org/grpc/health/grpc_health_v1" + + "github.com/grafana/loki/v3/pkg/kafka" + "github.com/grafana/loki/v3/pkg/kafka/ingester/shutdownmarker" + "github.com/grafana/loki/v3/pkg/kafka/partitionring" + util_log "github.com/grafana/loki/v3/pkg/util/log" + + "github.com/grafana/loki/v3/pkg/util" +) + +const ( + RingName = "kafka-ingester" + PartitionRingName = "kafka-partition" +) + +var ( + ingesterIDRegexp = regexp.MustCompile("-([0-9]+)$") + defaultFlushInterval = 15 * time.Second + defaultFlushSize int64 = 300 << 20 // 300 MB +) + +// Config for an ingester. +type Config struct { + Enabled bool `yaml:"enabled" doc:"description=Whether the kafka ingester is enabled."` + LifecyclerConfig ring.LifecyclerConfig `yaml:"lifecycler,omitempty" doc:"description=Configures how the lifecycle of the ingester will operate and where it will register for discovery."` + ShutdownMarkerPath string `yaml:"shutdown_marker_path"` + FlushInterval time.Duration `yaml:"flush_interval" doc:"description=The interval at which the ingester will flush and commit offsets to Kafka. If not set, the default flush interval will be used."` + FlushSize int64 `yaml:"flush_size" doc:"description=The size at which the ingester will flush and commit offsets to Kafka. If not set, the default flush size will be used."` + PartitionRingConfig partitionring.Config `yaml:"partition_ring" category:"experimental"` + KafkaConfig kafka.Config `yaml:"-"` +} + +// RegisterFlags registers the flags. +func (cfg *Config) RegisterFlags(f *flag.FlagSet) { + cfg.LifecyclerConfig.RegisterFlagsWithPrefix("kafka-ingester", f, util_log.Logger) + cfg.PartitionRingConfig.RegisterFlags(f) + f.StringVar(&cfg.ShutdownMarkerPath, "kafka-ingester.shutdown-marker-path", "", "Path where the shutdown marker file is stored. If not set and common.path_prefix is set then common.path_prefix will be used.") + f.BoolVar(&cfg.Enabled, "kafka-ingester.enabled", false, "Whether the Kafka-based ingester path is enabled") + f.DurationVar(&cfg.FlushInterval, "kafka-ingester.flush-interval", defaultFlushInterval, "The interval at which the ingester will flush and commit offsets to Kafka. If not set, the default flush interval will be used.") + f.Int64Var(&cfg.FlushSize, "kafka-ingester.flush-size", defaultFlushSize, "The size at which the ingester will flush and commit offsets to Kafka. If not set, the default flush size will be used.") +} + +func (cfg *Config) Validate() error { + if !cfg.Enabled { + return nil + } + if cfg.FlushInterval <= 0 { + return errors.New("kafka-ingester.flush-interval must be greater than 0") + } + if cfg.LifecyclerConfig.RingConfig.ReplicationFactor != 1 { + cfg.LifecyclerConfig.RingConfig.ReplicationFactor = 1 + level.Warn(util_log.Logger).Log("msg", "kafka-ingester.lifecycler.replication-factor has been set to 1. This is the only supported replication factor for the kafka-ingester.") + } + return nil +} + +type Wrapper interface { + Wrap(wrapped Interface) Interface +} + +// Interface is an interface for the Ingester +type Interface interface { + services.Service + http.Handler + CheckReady(ctx context.Context) error + FlushHandler(w http.ResponseWriter, _ *http.Request) +} + +// Ingester builds chunks for incoming log streams. +type Ingester struct { + services.Service + + cfg Config + logger log.Logger + + metrics *ingesterMetrics + + lifecycler *ring.Lifecycler + lifecyclerWatcher *services.FailureWatcher + ingesterPartitionID int32 + partitionRingLifecycler *ring.PartitionInstanceLifecycler + partitionReader *PartitionReader +} + +// New makes a new Ingester. +func New(cfg Config, + consumerFactory ConsumerFactory, + logger log.Logger, + metricsNamespace string, + registerer prometheus.Registerer, +) (*Ingester, error) { + metrics := newIngesterMetrics(registerer) + + ingesterPartitionID, err := extractIngesterPartitionID(cfg.LifecyclerConfig.ID) + if err != nil { + return nil, fmt.Errorf("calculating ingester partition ID: %w", err) + } + + partitionRingKV := cfg.PartitionRingConfig.KVStore.Mock + if partitionRingKV == nil { + partitionRingKV, err = kv.NewClient(cfg.PartitionRingConfig.KVStore, ring.GetPartitionRingCodec(), kv.RegistererWithKVName(registerer, PartitionRingName+"-lifecycler"), logger) + if err != nil { + return nil, fmt.Errorf("creating KV store for ingester partition ring: %w", err) + } + } + + partitionRingLifecycler := ring.NewPartitionInstanceLifecycler( + cfg.PartitionRingConfig.ToLifecyclerConfig(ingesterPartitionID, cfg.LifecyclerConfig.ID), + PartitionRingName, + PartitionRingName+"-key", + partitionRingKV, + logger, + prometheus.WrapRegistererWithPrefix("loki_", registerer)) + i := &Ingester{ + cfg: cfg, + logger: logger, + ingesterPartitionID: ingesterPartitionID, + partitionRingLifecycler: partitionRingLifecycler, + metrics: metrics, + } + + i.lifecycler, err = ring.NewLifecycler(cfg.LifecyclerConfig, i, RingName, RingName+"-ring", true, logger, prometheus.WrapRegistererWithPrefix(metricsNamespace+"_", registerer)) + if err != nil { + return nil, err + } + i.partitionReader, err = NewPartitionReader(cfg.KafkaConfig, ingesterPartitionID, cfg.LifecyclerConfig.ID, consumerFactory, logger, registerer) + if err != nil { + return nil, err + } + + i.lifecyclerWatcher = services.NewFailureWatcher() + i.lifecyclerWatcher.WatchService(i.lifecycler) + i.lifecyclerWatcher.WatchService(i.partitionRingLifecycler) + i.lifecyclerWatcher.WatchService(i.partitionReader) + + i.Service = services.NewBasicService(i.starting, i.running, i.stopping) + + return i, nil +} + +// ingesterPartitionID returns the partition ID owner the the given ingester. +func extractIngesterPartitionID(ingesterID string) (int32, error) { + if strings.Contains(ingesterID, "local") { + return 0, nil + } + + match := ingesterIDRegexp.FindStringSubmatch(ingesterID) + if len(match) == 0 { + return 0, fmt.Errorf("ingester ID %s doesn't match regular expression %q", ingesterID, ingesterIDRegexp.String()) + } + // Parse the ingester sequence number. + ingesterSeq, err := strconv.Atoi(match[1]) + if err != nil { + return 0, fmt.Errorf("no ingester sequence number in ingester ID %s", ingesterID) + } + + return int32(ingesterSeq), nil +} + +// ServeHTTP implements the pattern ring status page. +func (i *Ingester) ServeHTTP(w http.ResponseWriter, r *http.Request) { + i.lifecycler.ServeHTTP(w, r) +} + +func (i *Ingester) starting(ctx context.Context) (err error) { + defer func() { + if err != nil { + // if starting() fails for any reason (e.g., context canceled), + // the lifecycler must be stopped. + _ = services.StopAndAwaitTerminated(context.Background(), i.lifecycler) + } + }() + + // First of all we have to check if the shutdown marker is set. This needs to be done + // as first thing because, if found, it may change the behaviour of the ingester startup. + if exists, err := shutdownmarker.Exists(shutdownmarker.GetPath(i.cfg.ShutdownMarkerPath)); err != nil { + return fmt.Errorf("failed to check ingester shutdown marker: %w", err) + } else if exists { + level.Info(i.logger).Log("msg", "detected existing shutdown marker, setting unregister and flush on shutdown", "path", shutdownmarker.GetPath(i.cfg.ShutdownMarkerPath)) + i.setPrepareShutdown() + } + + // pass new context to lifecycler, so that it doesn't stop automatically when Ingester's service context is done + err = i.lifecycler.StartAsync(context.Background()) + if err != nil { + return err + } + + err = i.lifecycler.AwaitRunning(ctx) + if err != nil { + return err + } + + err = i.partitionRingLifecycler.StartAsync(context.Background()) + if err != nil { + return err + } + err = i.partitionRingLifecycler.AwaitRunning(ctx) + if err != nil { + return err + } + err = i.partitionReader.StartAsync(context.Background()) + if err != nil { + return err + } + err = i.partitionReader.AwaitRunning(ctx) + if err != nil { + return err + } + + return nil +} + +func (i *Ingester) running(ctx context.Context) error { + var serviceError error + select { + // wait until service is asked to stop + case <-ctx.Done(): + // stop + case err := <-i.lifecyclerWatcher.Chan(): + serviceError = fmt.Errorf("lifecycler failed: %w", err) + } + + return serviceError +} + +// stopping is called when Ingester transitions to Stopping state. +// +// At this point, loop no longer runs, but flushers are still running. +func (i *Ingester) stopping(_ error) error { + var errs util.MultiError + + errs.Add(services.StopAndAwaitTerminated(context.Background(), i.partitionReader)) + errs.Add(services.StopAndAwaitTerminated(context.Background(), i.lifecycler)) + errs.Add(services.StopAndAwaitTerminated(context.Background(), i.partitionRingLifecycler)) + // Remove the shutdown marker if it exists since we are shutting down + shutdownMarkerPath := shutdownmarker.GetPath(i.cfg.ShutdownMarkerPath) + exist, err := shutdownmarker.Exists(shutdownMarkerPath) + if err != nil { + level.Warn(i.logger).Log("msg", "failed to check for prepare-shutdown marker file", "path", shutdownMarkerPath, "err", err) + } else if exist { + if err := shutdownmarker.Remove(shutdownMarkerPath); err != nil { + level.Warn(i.logger).Log("msg", "failed to remove shutdown marker", "path", shutdownMarkerPath, "err", err) + } + } + return errs.Err() +} + +// Watch implements grpc_health_v1.HealthCheck. +func (*Ingester) Watch(*grpc_health_v1.HealthCheckRequest, grpc_health_v1.Health_WatchServer) error { + return nil +} + +func (i *Ingester) PreparePartitionDownscaleHandler(w http.ResponseWriter, r *http.Request) { + logger := log.With(i.logger, "partition", i.ingesterPartitionID) + // Don't allow callers to change the shutdown configuration while we're in the middle + // of starting or shutting down. + if i.State() != services.Running { + w.WriteHeader(http.StatusServiceUnavailable) + return + } + + shutdownMarkerPath := shutdownmarker.GetPath(i.cfg.ShutdownMarkerPath) + exists, err := shutdownmarker.Exists(shutdownMarkerPath) + if err != nil { + level.Error(i.logger).Log("msg", "unable to check for prepare-shutdown marker file", "path", shutdownMarkerPath, "err", err) + w.WriteHeader(http.StatusInternalServerError) + return + } + switch r.Method { + case http.MethodPost: + // It's not allowed to prepare the downscale while in PENDING state. Why? Because if the downscale + // will be later cancelled, we don't know if it was requested in PENDING or ACTIVE state, so we + // don't know to which state reverting back. Given a partition is expected to stay in PENDING state + // for a short period, we simply don't allow this case. + state, _, err := i.partitionRingLifecycler.GetPartitionState(r.Context()) + if err != nil { + level.Error(logger).Log("msg", "failed to check partition state in the ring", "err", err) + w.WriteHeader(http.StatusInternalServerError) + return + } + + if state == ring.PartitionPending { + level.Warn(logger).Log("msg", "received a request to prepare partition for shutdown, but the request can't be satisfied because the partition is in PENDING state") + w.WriteHeader(http.StatusConflict) + return + } + + if err := i.partitionRingLifecycler.ChangePartitionState(r.Context(), ring.PartitionInactive); err != nil { + level.Error(logger).Log("msg", "failed to change partition state to inactive", "err", err) + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + if !exists { + if err := shutdownmarker.Create(shutdownMarkerPath); err != nil { + level.Error(i.logger).Log("msg", "unable to create prepare-shutdown marker file", "path", shutdownMarkerPath, "err", err) + w.WriteHeader(http.StatusInternalServerError) + return + } + } + i.setPrepareShutdown() + + case http.MethodDelete: + state, _, err := i.partitionRingLifecycler.GetPartitionState(r.Context()) + if err != nil { + level.Error(logger).Log("msg", "failed to check partition state in the ring", "err", err) + w.WriteHeader(http.StatusInternalServerError) + return + } + + // If partition is inactive, make it active. We ignore other states Active and especially Pending. + if state == ring.PartitionInactive { + + // We don't switch it back to PENDING state if there are not enough owners because we want to guarantee consistency + // in the read path. If the partition is within the lookback period we need to guarantee that partition will be queried. + // Moving back to PENDING will cause us loosing consistency, because PENDING partitions are not queried by design. + // We could move back to PENDING if there are not enough owners and the partition moved to INACTIVE more than + // "lookback period" ago, but since we delete inactive partitions with no owners that moved to inactive since longer + // than "lookback period" ago, it looks to be an edge case not worth to address. + if err := i.partitionRingLifecycler.ChangePartitionState(r.Context(), ring.PartitionActive); err != nil { + level.Error(logger).Log("msg", "failed to change partition state to active", "err", err) + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + if exists { + if err := shutdownmarker.Remove(shutdownMarkerPath); err != nil { + level.Error(i.logger).Log("msg", "unable to remove prepare-shutdown marker file", "path", shutdownMarkerPath, "err", err) + w.WriteHeader(http.StatusInternalServerError) + return + } + } + i.unsetPrepareShutdown() + } + } + + state, stateTimestamp, err := i.partitionRingLifecycler.GetPartitionState(r.Context()) + if err != nil { + level.Error(logger).Log("msg", "failed to check partition state in the ring", "err", err) + w.WriteHeader(http.StatusInternalServerError) + return + } + + if state == ring.PartitionInactive { + util.WriteJSONResponse(w, map[string]any{"timestamp": stateTimestamp.Unix()}) + } else { + util.WriteJSONResponse(w, map[string]any{"timestamp": 0}) + } +} + +// setPrepareShutdown toggles ingester lifecycler config to prepare for shutdown +func (i *Ingester) setPrepareShutdown() { + i.lifecycler.SetUnregisterOnShutdown(true) + i.lifecycler.SetFlushOnShutdown(true) + i.partitionRingLifecycler.SetCreatePartitionOnStartup(false) + i.partitionRingLifecycler.SetRemoveOwnerOnShutdown(true) + i.metrics.shutdownMarker.Set(1) +} + +func (i *Ingester) unsetPrepareShutdown() { + i.lifecycler.SetUnregisterOnShutdown(i.cfg.LifecyclerConfig.UnregisterOnShutdown) + i.lifecycler.SetFlushOnShutdown(true) + i.partitionRingLifecycler.SetCreatePartitionOnStartup(true) + i.partitionRingLifecycler.SetRemoveOwnerOnShutdown(false) + i.metrics.shutdownMarker.Set(0) +} + +// ReadinessHandler is used to indicate to k8s when the ingesters are ready for +// the addition removal of another ingester. Returns 204 when the ingester is +// ready, 500 otherwise. +func (i *Ingester) CheckReady(ctx context.Context) error { + // todo. + if s := i.State(); s != services.Running && s != services.Stopping { + return fmt.Errorf("ingester not ready: %v", s) + } + return i.lifecycler.CheckReady(ctx) +} + +// Flush implements ring.FlushTransferer +// Flush triggers a flush of all the chunks and closes the flush queues. +// Called from the Lifecycler as part of the ingester shutdown. +func (i *Ingester) Flush() { +} + +func (i *Ingester) TransferOut(_ context.Context) error { + return nil +} diff --git a/pkg/kafka/ingester/ingester_test.go b/pkg/kafka/ingester/ingester_test.go new file mode 100644 index 000000000000..a3bcca72ca3d --- /dev/null +++ b/pkg/kafka/ingester/ingester_test.go @@ -0,0 +1,174 @@ +package ingester + +import ( + "context" + "net/http" + "net/http/httptest" + "testing" + "time" + + "github.com/go-kit/log" + gokitlog "github.com/go-kit/log" + "github.com/grafana/dskit/flagext" + "github.com/grafana/dskit/kv/consul" + "github.com/grafana/dskit/ring" + "github.com/grafana/dskit/services" + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/require" + "google.golang.org/grpc" + + "github.com/grafana/loki/v3/pkg/ingester-rf1/metastore/metastorepb" + "github.com/grafana/loki/v3/pkg/ingester-rf1/objstore" + "github.com/grafana/loki/v3/pkg/util/test" +) + +func TestPreparePartitionDownscaleHandler(t *testing.T) { + cfg := defaultIngesterTestConfig(t) + // start ingester. + storage, err := objstore.NewTestStorage(t) + require.NoError(t, err) + ing, err := New(cfg, + NewConsumerFactory(NewTestMetastore(), storage, cfg.FlushInterval, cfg.FlushSize, gokitlog.NewNopLogger(), prometheus.NewRegistry()), + gokitlog.NewNopLogger(), "test", prometheus.NewRegistry()) + require.NoError(t, err) + err = services.StartAndAwaitRunning(context.Background(), ing) + require.NoError(t, err) + + t.Run("get state", func(t *testing.T) { + w := httptest.NewRecorder() + ing.PreparePartitionDownscaleHandler(w, httptest.NewRequest("GET", "/", nil)) + require.Equal(t, http.StatusOK, w.Code) + require.Equal(t, "{\"timestamp\":0}", w.Body.String()) + }) + t.Run("prepare shutdown pending", func(t *testing.T) { + w := httptest.NewRecorder() + ing.PreparePartitionDownscaleHandler(w, httptest.NewRequest("POST", "/", nil)) + require.Equal(t, http.StatusConflict, w.Code) + }) + t.Run("prepare shutdown and cancel", func(t *testing.T) { + w := httptest.NewRecorder() + test.Poll(t, 5*time.Second, ring.PartitionActive, func() interface{} { + return getState(t, cfg) + }) + ing.PreparePartitionDownscaleHandler(w, httptest.NewRequest("POST", "/", nil)) + require.Equal(t, http.StatusOK, w.Code) + test.Poll(t, 5*time.Second, ring.PartitionInactive, func() interface{} { + return getState(t, cfg) + }) + w2 := httptest.NewRecorder() + ing.PreparePartitionDownscaleHandler(w2, httptest.NewRequest("DELETE", "/", nil)) + require.Equal(t, http.StatusOK, w.Code) + test.Poll(t, 5*time.Second, ring.PartitionActive, func() interface{} { + return getState(t, cfg) + }) + }) + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), ing)) +} + +func getState(t *testing.T, cfg Config) ring.PartitionState { + get, err := cfg.PartitionRingConfig.KVStore.Mock.Get(context.Background(), PartitionRingName+"-key") + require.NoError(t, err) + + ringDesc := ring.GetOrCreatePartitionRingDesc(get) + return ringDesc.Partitions[0].State +} + +// nolint +func defaultIngesterTestConfig(t testing.TB) Config { + kvRing, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { require.NoError(t, closer.Close()) }) + + kvPartitionRing, closerPartitionRing := consul.NewInMemoryClient(ring.GetPartitionRingCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { require.NoError(t, closerPartitionRing.Close()) }) + + cfg := Config{} + flagext.DefaultValues(&cfg) + + cfg.LifecyclerConfig.RingConfig.KVStore.Mock = kvRing + cfg.PartitionRingConfig.KVStore.Mock = kvPartitionRing + cfg.PartitionRingConfig.MinOwnersCount = 1 + cfg.PartitionRingConfig.MinOwnersDuration = 0 + cfg.LifecyclerConfig.RingConfig.ReplicationFactor = 1 + cfg.LifecyclerConfig.NumTokens = 1 + cfg.LifecyclerConfig.ListenPort = 0 + cfg.LifecyclerConfig.Addr = "localhost" + cfg.LifecyclerConfig.ID = "localhost" + cfg.LifecyclerConfig.FinalSleep = 0 + cfg.LifecyclerConfig.MinReadyDuration = 0 + + return cfg +} + +func TestExtractIngesterPartitionID(t *testing.T) { + tests := []struct { + name string + ingesterID string + want int32 + wantErr bool + }{ + { + name: "Valid ingester ID", + ingesterID: "ingester-5", + want: 5, + wantErr: false, + }, + { + name: "Local ingester ID", + ingesterID: "ingester-local", + want: 0, + wantErr: false, + }, + { + name: "Invalid ingester ID format", + ingesterID: "invalid-format", + want: 0, + wantErr: true, + }, + { + name: "Invalid sequence number", + ingesterID: "ingester-abc", + want: 0, + wantErr: true, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := extractIngesterPartitionID(tt.ingesterID) + if (err != nil) != tt.wantErr { + t.Errorf("extractIngesterPartitionID() error = %v, wantErr %v", err, tt.wantErr) + return + } + if got != tt.want { + t.Errorf("extractIngesterPartitionID() = %v, want %v", got, tt.want) + } + }) + } +} + +// TestMetastore is a simple in-memory metastore for testing +type TestMetastore struct { + blocks map[string][]*metastorepb.BlockMeta +} + +func NewTestMetastore() *TestMetastore { + return &TestMetastore{blocks: make(map[string][]*metastorepb.BlockMeta)} +} + +func (m *TestMetastore) ListBlocksForQuery(_ context.Context, req *metastorepb.ListBlocksForQueryRequest, _ ...grpc.CallOption) (*metastorepb.ListBlocksForQueryResponse, error) { + blocks := m.blocks[req.TenantId] + var result []*metastorepb.BlockMeta + for _, block := range blocks { + if block.MinTime <= req.EndTime && block.MaxTime >= req.StartTime { + result = append(result, block) + } + } + return &metastorepb.ListBlocksForQueryResponse{Blocks: result}, nil +} + +func (m *TestMetastore) AddBlock(_ context.Context, in *metastorepb.AddBlockRequest, _ ...grpc.CallOption) (*metastorepb.AddBlockResponse, error) { + for _, stream := range in.Block.TenantStreams { + m.blocks[stream.TenantId] = append(m.blocks[stream.TenantId], in.Block) + } + return &metastorepb.AddBlockResponse{}, nil +} diff --git a/pkg/ingester-kafka/metrics.go b/pkg/kafka/ingester/metrics.go similarity index 62% rename from pkg/ingester-kafka/metrics.go rename to pkg/kafka/ingester/metrics.go index dce284f7c254..e73ee08095c8 100644 --- a/pkg/ingester-kafka/metrics.go +++ b/pkg/kafka/ingester/metrics.go @@ -1,4 +1,4 @@ -package ingesterkafka +package ingester import ( "github.com/prometheus/client_golang/prometheus" @@ -6,15 +6,15 @@ import ( ) type ingesterMetrics struct { - // Shutdown marker for ingester scale down. + // Shutdown marker for ingester scale down shutdownMarker prometheus.Gauge } func newIngesterMetrics(r prometheus.Registerer) *ingesterMetrics { return &ingesterMetrics{ shutdownMarker: promauto.With(r).NewGauge(prometheus.GaugeOpts{ - Name: "loki_ingester_kafka_shutdown_marker", - Help: "1 if prepare shutdown has been called, 0 otherwise.", + Name: "loki_ingester_prepare_shutdown_requested", + Help: "1 if the ingester has been requested to prepare for shutdown via endpoint or marker file.", }), } } diff --git a/pkg/kafka/ingester/partition_committer.go b/pkg/kafka/ingester/partition_committer.go new file mode 100644 index 000000000000..a76e363a64e4 --- /dev/null +++ b/pkg/kafka/ingester/partition_committer.go @@ -0,0 +1,103 @@ +package ingester + +import ( + "context" + "strconv" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/twmb/franz-go/pkg/kadm" + + "github.com/grafana/loki/v3/pkg/kafka" +) + +// partitionCommitter is responsible for committing offsets for a specific Kafka partition +// to the Kafka broker. It also tracks metrics related to the commit process. +type partitionCommitter struct { + commitRequestsTotal prometheus.Counter + commitRequestsLatency prometheus.Histogram + commitFailuresTotal prometheus.Counter + lastCommittedOffset prometheus.Gauge + + logger log.Logger + admClient *kadm.Client + + kafkaCfg kafka.Config + partitionID int32 + consumerGroup string +} + +// newPartitionCommitter creates and initializes a new partitionCommitter. +// It sets up the necessary metrics and initializes the committer with the provided configuration. +func newPartitionCommitter(kafkaCfg kafka.Config, admClient *kadm.Client, partitionID int32, consumerGroup string, logger log.Logger, reg prometheus.Registerer) *partitionCommitter { + c := &partitionCommitter{ + logger: logger, + kafkaCfg: kafkaCfg, + partitionID: partitionID, + consumerGroup: consumerGroup, + admClient: admClient, + commitRequestsTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingest_storage_reader_offset_commit_requests_total", + Help: "Total number of requests issued to commit the last consumed offset (includes both successful and failed requests).", + ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, + }), + commitFailuresTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingest_storage_reader_offset_commit_failures_total", + Help: "Total number of failed requests to commit the last consumed offset.", + ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, + }), + commitRequestsLatency: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "loki_ingest_storage_reader_offset_commit_request_duration_seconds", + Help: "The duration of requests to commit the last consumed offset.", + ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, + NativeHistogramBucketFactor: 1.1, + NativeHistogramMaxBucketNumber: 100, + NativeHistogramMinResetDuration: time.Hour, + Buckets: prometheus.DefBuckets, + }), + lastCommittedOffset: promauto.With(reg).NewGauge(prometheus.GaugeOpts{ + Name: "loki_ingest_storage_reader_last_committed_offset", + Help: "The last consumed offset successfully committed by the partition reader. Set to -1 if not offset has been committed yet.", + ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, + }), + } + + // Initialise the last committed offset metric to -1 to signal no offset has been committed yet (0 is a valid offset). + c.lastCommittedOffset.Set(-1) + + return c +} + +// commit attempts to commit the given offset to Kafka for the partition this committer is responsible for. +// It updates relevant metrics and logs the result of the commit operation. +func (r *partitionCommitter) Commit(ctx context.Context, offset int64) (returnErr error) { + startTime := time.Now() + r.commitRequestsTotal.Inc() + + defer func() { + r.commitRequestsLatency.Observe(time.Since(startTime).Seconds()) + + if returnErr != nil { + level.Error(r.logger).Log("msg", "failed to commit last consumed offset to Kafka", "err", returnErr, "offset", offset) + r.commitFailuresTotal.Inc() + } + }() + + // Commit the last consumed offset. + toCommit := kadm.Offsets{} + toCommit.AddOffset(r.kafkaCfg.Topic, r.partitionID, offset, -1) + committed, err := r.admClient.CommitOffsets(ctx, r.consumerGroup, toCommit) + if err != nil { + return err + } else if !committed.Ok() { + return committed.Error() + } + + committedOffset, _ := committed.Lookup(r.kafkaCfg.Topic, r.partitionID) + level.Debug(r.logger).Log("msg", "last commit offset successfully committed to Kafka", "offset", committedOffset.At) + r.lastCommittedOffset.Set(float64(committedOffset.At)) + return nil +} diff --git a/pkg/kafka/ingester/partition_committer_test.go b/pkg/kafka/ingester/partition_committer_test.go new file mode 100644 index 000000000000..8fb823e3f2ed --- /dev/null +++ b/pkg/kafka/ingester/partition_committer_test.go @@ -0,0 +1,77 @@ +package ingester + +import ( + "context" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/twmb/franz-go/pkg/kadm" + "github.com/twmb/franz-go/plugin/kprom" + + "github.com/prometheus/client_golang/prometheus/testutil" + + "github.com/grafana/loki/v3/pkg/kafka" + "github.com/grafana/loki/v3/pkg/kafka/testkafka" +) + +func TestPartitionCommitter(t *testing.T) { + // Create a test Kafka cluster + numPartitions := int32(3) + topicName := "test-topic" + _, kafkaCfg := testkafka.CreateCluster(t, numPartitions, topicName) + + client, err := kafka.NewReaderClient(kafkaCfg, kprom.NewMetrics("foo"), log.NewNopLogger()) + require.NoError(t, err) + + // Create a Kafka admin client + admClient := kadm.NewClient(client) + defer admClient.Close() + + // Create a partition committer + logger := log.NewNopLogger() + reg := prometheus.NewRegistry() + partitionID := int32(1) + consumerGroup := "test-consumer-group" + committer := newPartitionCommitter(kafkaCfg, admClient, partitionID, consumerGroup, logger, reg) + + // Test committing an offset + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + + testOffset := int64(100) + err = committer.Commit(ctx, testOffset) + require.NoError(t, err) + + // Verify metrics + assert.Equal(t, float64(1), testutil.ToFloat64(committer.commitRequestsTotal)) + assert.Equal(t, float64(0), testutil.ToFloat64(committer.commitFailuresTotal)) + assert.Equal(t, float64(testOffset), testutil.ToFloat64(committer.lastCommittedOffset)) + + // Verify committed offset + offsets, err := admClient.FetchOffsets(context.Background(), consumerGroup) + require.NoError(t, err) + committedOffset, ok := offsets.Lookup(topicName, partitionID) + require.True(t, ok) + assert.Equal(t, testOffset, committedOffset.At) + + // Test committing a new offset + newTestOffset := int64(200) + err = committer.Commit(ctx, newTestOffset) + require.NoError(t, err) + + // Verify updated metrics + assert.Equal(t, float64(2), testutil.ToFloat64(committer.commitRequestsTotal)) + assert.Equal(t, float64(0), testutil.ToFloat64(committer.commitFailuresTotal)) + assert.Equal(t, float64(newTestOffset), testutil.ToFloat64(committer.lastCommittedOffset)) + + // Verify updated committed offset + offsets, err = admClient.FetchOffsets(context.Background(), consumerGroup) + require.NoError(t, err) + committedOffset, ok = offsets.Lookup(topicName, partitionID) + require.True(t, ok) + assert.Equal(t, newTestOffset, committedOffset.At) +} diff --git a/pkg/kafka/ingester/partition_reader.go b/pkg/kafka/ingester/partition_reader.go new file mode 100644 index 000000000000..5ed70412d9e0 --- /dev/null +++ b/pkg/kafka/ingester/partition_reader.go @@ -0,0 +1,269 @@ +package ingester + +import ( + "context" + "fmt" + "math" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/multierror" + "github.com/grafana/dskit/services" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/twmb/franz-go/pkg/kadm" + "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/plugin/kprom" + + "github.com/grafana/loki/v3/pkg/kafka" +) + +// PartitionReader is responsible for reading data from a specific Kafka partition +// and passing it to the consumer for processing. It is a core component of the +// Loki ingester's Kafka-based ingestion pipeline. +type PartitionReader struct { + services.Service + + kafkaCfg kafka.Config + partitionID int32 + consumerGroup string + consumerFactory ConsumerFactory + committer *partitionCommitter + lastProcessedOffset int64 + + client *kgo.Client + logger log.Logger + metrics readerMetrics + reg prometheus.Registerer +} + +type record struct { + // Context holds the tracing (and potentially other) info, that the record was enriched with on fetch from Kafka. + ctx context.Context + tenantID string + content []byte + offset int64 +} + +type ConsumerFactory func(committer Committer) (Consumer, error) + +type Consumer interface { + Start(ctx context.Context, recordsChan <-chan []record) func() +} + +// NewPartitionReader creates and initializes a new PartitionReader. +// It sets up the basic service and initializes the reader with the provided configuration. +func NewPartitionReader( + kafkaCfg kafka.Config, + partitionID int32, + instanceID string, + consumerFactory ConsumerFactory, + logger log.Logger, + reg prometheus.Registerer, +) (*PartitionReader, error) { + r := &PartitionReader{ + kafkaCfg: kafkaCfg, + partitionID: partitionID, + consumerGroup: kafkaCfg.GetConsumerGroup(instanceID, partitionID), + logger: logger, + metrics: newReaderMetrics(reg), + reg: reg, + lastProcessedOffset: -1, + consumerFactory: consumerFactory, + } + r.Service = services.NewBasicService(r.start, r.run, nil) + return r, nil +} + +// start initializes the Kafka client and committer for the PartitionReader. +// This method is called when the PartitionReader service starts. +func (p *PartitionReader) start(_ context.Context) error { + var err error + p.client, err = kafka.NewReaderClient(p.kafkaCfg, p.metrics.kprom, p.logger, + kgo.ConsumePartitions(map[string]map[int32]kgo.Offset{ + p.kafkaCfg.Topic: {p.partitionID: kgo.NewOffset().AtStart()}, + }), + ) + if err != nil { + return errors.Wrap(err, "creating kafka reader client") + } + p.committer = newPartitionCommitter(p.kafkaCfg, kadm.NewClient(p.client), p.partitionID, p.consumerGroup, p.logger, p.reg) + + return nil +} + +// run is the main loop of the PartitionReader. It continuously fetches and processes +// data from Kafka, and send it to the consumer. +func (p *PartitionReader) run(ctx context.Context) error { + level.Info(p.logger).Log("msg", "starting partition reader", "partition", p.partitionID, "consumer_group", p.consumerGroup) + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + consumer, err := p.consumerFactory(p.committer) + if err != nil { + return errors.Wrap(err, "creating consumer") + } + + recordsChan := p.startFetchLoop(ctx) + wait := consumer.Start(ctx, recordsChan) + + wait() + return nil +} + +func (p *PartitionReader) startFetchLoop(ctx context.Context) <-chan []record { + records := make(chan []record) + go func() { + for { + select { + case <-ctx.Done(): + return + default: + records <- p.poll(ctx) + } + } + }() + return records +} + +// logFetchErrors logs any errors encountered during the fetch operation. +func (p *PartitionReader) logFetchErrors(fetches kgo.Fetches) { + mErr := multierror.New() + fetches.EachError(func(topic string, partition int32, err error) { + if errors.Is(err, context.Canceled) { + return + } + + // kgo advises to "restart" the kafka client if the returned error is a kerr.Error. + // Recreating the client would cause duplicate metrics registration, so we don't do it for now. + mErr.Add(fmt.Errorf("topic %q, partition %d: %w", topic, partition, err)) + }) + if len(mErr) == 0 { + return + } + p.metrics.fetchesErrors.Add(float64(len(mErr))) + level.Error(p.logger).Log("msg", "encountered error while fetching", "err", mErr.Err()) +} + +// pollFetches retrieves the next batch of records from Kafka and measures the fetch duration. +func (p *PartitionReader) poll(ctx context.Context) []record { + defer func(start time.Time) { + p.metrics.fetchWaitDuration.Observe(time.Since(start).Seconds()) + }(time.Now()) + fetches := p.client.PollFetches(ctx) + p.recordFetchesMetrics(fetches) + p.logFetchErrors(fetches) + fetches = filterOutErrFetches(fetches) + if fetches.NumRecords() == 0 { + return nil + } + records := make([]record, 0, fetches.NumRecords()) + fetches.EachRecord(func(rec *kgo.Record) { + records = append(records, record{ + // This context carries the tracing data for this individual record; + // kotel populates this data when it fetches the messages. + ctx: rec.Context, + tenantID: string(rec.Key), + content: rec.Value, + offset: rec.Offset, + }) + }) + p.lastProcessedOffset = records[len(records)-1].offset + return records +} + +// recordFetchesMetrics updates various metrics related to the fetch operation. +func (p *PartitionReader) recordFetchesMetrics(fetches kgo.Fetches) { + var ( + now = time.Now() + numRecords = 0 + ) + fetches.EachRecord(func(record *kgo.Record) { + numRecords++ + delay := now.Sub(record.Timestamp).Seconds() + if p.lastProcessedOffset == -1 { + p.metrics.receiveDelayWhenStarting.Observe(delay) + } else { + p.metrics.receiveDelayWhenRunning.Observe(delay) + } + }) + + p.metrics.fetchesTotal.Add(float64(len(fetches))) + p.metrics.recordsPerFetch.Observe(float64(numRecords)) +} + +// filterOutErrFetches removes any fetches that resulted in errors from the provided slice. +func filterOutErrFetches(fetches kgo.Fetches) kgo.Fetches { + filtered := make(kgo.Fetches, 0, len(fetches)) + for i, fetch := range fetches { + if !isErrFetch(fetch) { + filtered = append(filtered, fetches[i]) + } + } + + return filtered +} + +// isErrFetch checks if a given fetch resulted in any errors. +func isErrFetch(fetch kgo.Fetch) bool { + for _, t := range fetch.Topics { + for _, p := range t.Partitions { + if p.Err != nil { + return true + } + } + } + return false +} + +type readerMetrics struct { + receiveDelayWhenStarting prometheus.Observer + receiveDelayWhenRunning prometheus.Observer + recordsPerFetch prometheus.Histogram + fetchesErrors prometheus.Counter + fetchesTotal prometheus.Counter + fetchWaitDuration prometheus.Histogram + // strongConsistencyInstrumentation *StrongReadConsistencyInstrumentation[struct{}] + // lastConsumedOffset prometheus.Gauge + consumeLatency prometheus.Histogram + kprom *kprom.Metrics +} + +// newReaderMetrics initializes and returns a new set of metrics for the PartitionReader. +func newReaderMetrics(reg prometheus.Registerer) readerMetrics { + receiveDelay := promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "loki_ingest_storage_reader_receive_delay_seconds", + Help: "Delay between producing a record and receiving it in the consumer.", + NativeHistogramZeroThreshold: math.Pow(2, -10), // Values below this will be considered to be 0. Equals to 0.0009765625, or about 1ms. + NativeHistogramBucketFactor: 1.2, // We use higher factor (scheme=2) to have wider spread of buckets. + NativeHistogramMaxBucketNumber: 100, + NativeHistogramMinResetDuration: 1 * time.Hour, + Buckets: prometheus.ExponentialBuckets(0.125, 2, 18), // Buckets between 125ms and 9h. + }, []string{"phase"}) + + return readerMetrics{ + receiveDelayWhenStarting: receiveDelay.WithLabelValues("starting"), + receiveDelayWhenRunning: receiveDelay.WithLabelValues("running"), + kprom: kafka.NewReaderClientMetrics("partition-reader", reg), + fetchWaitDuration: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "loki_ingest_storage_reader_records_batch_wait_duration_seconds", + Help: "How long a consumer spent waiting for a batch of records from the Kafka client. If fetching is faster than processing, then this will be close to 0.", + NativeHistogramBucketFactor: 1.1, + }), + recordsPerFetch: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "loki_ingest_storage_reader_records_per_fetch", + Help: "The number of records received by the consumer in a single fetch operation.", + Buckets: prometheus.ExponentialBuckets(1, 2, 15), + }), + fetchesErrors: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingest_storage_reader_fetch_errors_total", + Help: "The number of fetch errors encountered by the consumer.", + }), + fetchesTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingest_storage_reader_fetches_total", + Help: "Total number of Kafka fetches received by the consumer.", + }), + } +} diff --git a/pkg/kafka/ingester/partition_reader_test.go b/pkg/kafka/ingester/partition_reader_test.go new file mode 100644 index 000000000000..2bc6db474765 --- /dev/null +++ b/pkg/kafka/ingester/partition_reader_test.go @@ -0,0 +1,102 @@ +package ingester + +import ( + "context" + "sync" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/services" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/v3/pkg/kafka" + "github.com/grafana/loki/v3/pkg/kafka/testkafka" + "github.com/grafana/loki/v3/pkg/logproto" +) + +type mockConsumer struct { + mock.Mock + recordsChan chan []record + wg sync.WaitGroup +} + +func newMockConsumer() *mockConsumer { + return &mockConsumer{ + recordsChan: make(chan []record, 100), + } +} + +func (m *mockConsumer) Start(ctx context.Context, recordsChan <-chan []record) func() { + m.wg.Add(1) + go func() { + defer m.wg.Done() + for { + select { + case <-ctx.Done(): + return + case records := <-recordsChan: + m.recordsChan <- records + } + } + }() + return m.wg.Wait +} + +func (m *mockConsumer) Flush(ctx context.Context) error { + args := m.Called(ctx) + return args.Error(0) +} + +func TestPartitionReader_BasicFunctionality(t *testing.T) { + _, kafkaCfg := testkafka.CreateCluster(t, 1, "test-topic") + consumer := newMockConsumer() + + consumerFactory := func(committer Committer) (Consumer, error) { + return consumer, nil + } + + partitionReader, err := NewPartitionReader(kafkaCfg, 0, "test-consumer-group", consumerFactory, log.NewNopLogger(), prometheus.NewRegistry()) + require.NoError(t, err) + producer, err := kafka.NewWriterClient(kafkaCfg, 100, log.NewNopLogger(), prometheus.NewRegistry()) + require.NoError(t, err) + + err = services.StartAndAwaitRunning(context.Background(), partitionReader) + require.NoError(t, err) + + stream := logproto.Stream{ + Labels: labels.FromStrings("foo", "bar").String(), + Entries: []logproto.Entry{{Timestamp: time.Now(), Line: "test"}}, + } + + records, err := kafka.Encode(0, "test-tenant", stream, 10<<20) + require.NoError(t, err) + require.Len(t, records, 1) + + producer.ProduceSync(context.Background(), records...) + producer.ProduceSync(context.Background(), records...) + + // Wait for records to be processed + assert.Eventually(t, func() bool { + return len(consumer.recordsChan) == 2 + }, 10*time.Second, 100*time.Millisecond) + + // Verify the records + for i := 0; i < 2; i++ { + select { + case receivedRecords := <-consumer.recordsChan: + require.Len(t, receivedRecords, 1) + assert.Equal(t, "test-tenant", receivedRecords[0].tenantID) + assert.Equal(t, records[0].Value, receivedRecords[0].content) + case <-time.After(1 * time.Second): + t.Fatal("Timeout waiting for records") + } + } + + err = services.StopAndAwaitTerminated(context.Background(), partitionReader) + require.NoError(t, err) +} diff --git a/pkg/kafka/ingester/shutdownmarker/shutdown_marker.go b/pkg/kafka/ingester/shutdownmarker/shutdown_marker.go new file mode 100644 index 000000000000..7d1a4ec2f353 --- /dev/null +++ b/pkg/kafka/ingester/shutdownmarker/shutdown_marker.go @@ -0,0 +1,60 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package shutdownmarker + +import ( + "os" + "path" + "strings" + "time" + + "github.com/grafana/dskit/multierror" + + "github.com/grafana/loki/v3/pkg/util/atomicfs" +) + +const shutdownMarkerFilename = "shutdown-requested.txt" + +// Create writes a marker file on the given path to indicate that a component is +// going to be scaled down in the future. The presence of this file means that a component +// should perform some operations specified by the component itself before being shutdown. +func Create(p string) error { + return atomicfs.CreateFile(p, strings.NewReader(time.Now().UTC().Format(time.RFC3339))) +} + +// Remove removes the shutdown marker file on the given path if it exists. +func Remove(p string) error { + err := os.Remove(p) + if err != nil && !os.IsNotExist(err) { + return err + } + + dir, err := os.OpenFile(path.Dir(p), os.O_RDONLY, 0o777) + if err != nil { + return err + } + + merr := multierror.New() + merr.Add(dir.Sync()) + merr.Add(dir.Close()) + return merr.Err() +} + +// Exists returns true if the shutdown marker file exists on the given path, false otherwise +func Exists(p string) (bool, error) { + s, err := os.Stat(p) + if err != nil && os.IsNotExist(err) { + return false, nil + } + + if err != nil { + return false, err + } + + return s.Mode().IsRegular(), nil +} + +// GetPath returns the absolute path of the shutdown marker file +func GetPath(dirPath string) string { + return path.Join(dirPath, shutdownMarkerFilename) +} diff --git a/pkg/kafka/ingester/shutdownmarker/shutdown_marker_test.go b/pkg/kafka/ingester/shutdownmarker/shutdown_marker_test.go new file mode 100644 index 000000000000..c8e0b851be4e --- /dev/null +++ b/pkg/kafka/ingester/shutdownmarker/shutdown_marker_test.go @@ -0,0 +1,49 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package shutdownmarker + +import ( + "path/filepath" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestShutdownMarker_GetPath(t *testing.T) { + dir := "/a/b/c" + expectedPath := filepath.Join(dir, shutdownMarkerFilename) + require.Equal(t, expectedPath, GetPath(dir)) +} + +func TestShutdownMarker_Create(t *testing.T) { + dir := t.TempDir() + shutdownMarkerPath := GetPath(dir) + exists, err := Exists(shutdownMarkerPath) + require.NoError(t, err) + require.False(t, exists) + + err = Create(shutdownMarkerPath) + require.NoError(t, err) + + exists, err = Exists(shutdownMarkerPath) + require.NoError(t, err) + require.True(t, exists) +} + +func TestShutdownMarker_Remove(t *testing.T) { + dir := t.TempDir() + shutdownMarkerPath := GetPath(dir) + exists, err := Exists(shutdownMarkerPath) + require.NoError(t, err) + require.False(t, exists) + + require.Nil(t, Create(shutdownMarkerPath)) + exists, err = Exists(shutdownMarkerPath) + require.NoError(t, err) + require.True(t, exists) + + require.Nil(t, Remove(shutdownMarkerPath)) + exists, err = Exists(shutdownMarkerPath) + require.NoError(t, err) + require.False(t, exists) +} diff --git a/pkg/kafka/logger.go b/pkg/kafka/logger.go new file mode 100644 index 000000000000..e055094a4163 --- /dev/null +++ b/pkg/kafka/logger.go @@ -0,0 +1,40 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package kafka + +import ( + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/twmb/franz-go/pkg/kgo" +) + +type logger struct { + logger log.Logger +} + +func newLogger(l log.Logger) *logger { + return &logger{ + logger: log.With(l, "component", "kafka_client"), + } +} + +func (l *logger) Level() kgo.LogLevel { + // The Kafka client calls Level() to check whether debug level is enabled or not. + // To keep it simple, we always return Info, so the Kafka client will never try + // to log expensive debug messages. + return kgo.LogLevelInfo +} + +func (l *logger) Log(lev kgo.LogLevel, msg string, keyvals ...any) { + keyvals = append([]any{"msg", msg}, keyvals...) + switch lev { + case kgo.LogLevelDebug: + level.Debug(l.logger).Log(keyvals...) + case kgo.LogLevelInfo: + level.Info(l.logger).Log(keyvals...) + case kgo.LogLevelWarn: + level.Warn(l.logger).Log(keyvals...) + case kgo.LogLevelError: + level.Error(l.logger).Log(keyvals...) + } +} diff --git a/pkg/ingester-kafka/partitionring/partition_ring.go b/pkg/kafka/partitionring/partition_ring.go similarity index 100% rename from pkg/ingester-kafka/partitionring/partition_ring.go rename to pkg/kafka/partitionring/partition_ring.go diff --git a/pkg/kafka/reader_client.go b/pkg/kafka/reader_client.go new file mode 100644 index 000000000000..1b8c6b3bc1dc --- /dev/null +++ b/pkg/kafka/reader_client.go @@ -0,0 +1,44 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package kafka + +import ( + "time" + + "github.com/go-kit/log" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/plugin/kprom" +) + +// NewReaderClient returns the kgo.Client that should be used by the Reader. +func NewReaderClient(cfg Config, metrics *kprom.Metrics, logger log.Logger, opts ...kgo.Opt) (*kgo.Client, error) { + const fetchMaxBytes = 100_000_000 + + opts = append(opts, commonKafkaClientOptions(cfg, metrics, logger)...) + opts = append(opts, + kgo.FetchMinBytes(1), + kgo.FetchMaxBytes(fetchMaxBytes), + kgo.FetchMaxWait(5*time.Second), + kgo.FetchMaxPartitionBytes(50_000_000), + + // BrokerMaxReadBytes sets the maximum response size that can be read from + // Kafka. This is a safety measure to avoid OOMing on invalid responses. + // franz-go recommendation is to set it 2x FetchMaxBytes. + kgo.BrokerMaxReadBytes(2*fetchMaxBytes), + ) + client, err := kgo.NewClient(opts...) + if err != nil { + return nil, errors.Wrap(err, "creating kafka client") + } + + return client, nil +} + +func NewReaderClientMetrics(component string, reg prometheus.Registerer) *kprom.Metrics { + return kprom.NewMetrics("loki_ingest_storage_reader", + kprom.Registerer(prometheus.WrapRegistererWith(prometheus.Labels{"component": component}, reg)), + // Do not export the client ID, because we use it to specify options to the backend. + kprom.FetchAndProduceDetail(kprom.Batches, kprom.Records, kprom.CompressedBytes, kprom.UncompressedBytes)) +} diff --git a/pkg/kafka/tee/tee.go b/pkg/kafka/tee/tee.go new file mode 100644 index 000000000000..2228883efb32 --- /dev/null +++ b/pkg/kafka/tee/tee.go @@ -0,0 +1,174 @@ +package tee + +import ( + "context" + "fmt" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/ring" + "github.com/grafana/dskit/user" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/twmb/franz-go/pkg/kgo" + + "github.com/grafana/loki/v3/pkg/distributor" + "github.com/grafana/loki/v3/pkg/kafka" +) + +const writeTimeout = time.Minute + +// Tee represents a component that duplicates log streams to Kafka. +type Tee struct { + logger log.Logger + producer *kafka.Producer + partitionRing ring.PartitionRingReader + cfg kafka.Config + + ingesterAppends *prometheus.CounterVec + writeLatency prometheus.Histogram + writeBytesTotal prometheus.Counter + recordsPerRequest prometheus.Histogram +} + +// NewTee creates and initializes a new Tee instance. +// +// Parameters: +// - cfg: Kafka configuration +// - metricsNamespace: Namespace for Prometheus metrics +// - registerer: Prometheus registerer for metrics +// - logger: Logger instance +// - partitionRing: Ring for managing partitions +// +// Returns: +// - A new Tee instance and any error encountered during initialization +func NewTee( + cfg kafka.Config, + metricsNamespace string, + registerer prometheus.Registerer, + logger log.Logger, + partitionRing ring.PartitionRingReader, +) (*Tee, error) { + registerer = prometheus.WrapRegistererWithPrefix(metricsNamespace+"_", registerer) + + kafkaClient, err := kafka.NewWriterClient(cfg, 20, logger, registerer) + if err != nil { + return nil, fmt.Errorf("failed to start kafka client: %w", err) + } + producer := kafka.NewProducer(kafkaClient, cfg.ProducerMaxBufferedBytes, + prometheus.WrapRegistererWithPrefix("_kafka_ingester_", registerer)) + + t := &Tee{ + logger: log.With(logger, "component", "kafka-tee"), + ingesterAppends: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{ + Name: "kafka_ingester_appends_total", + Help: "The total number of appends sent to kafka ingest path.", + }, []string{"partition", "status"}), + producer: producer, + partitionRing: partitionRing, + cfg: cfg, + // Metrics. + writeLatency: promauto.With(registerer).NewHistogram(prometheus.HistogramOpts{ + Name: "kafka_ingester_tee_latency_seconds", + Help: "Latency to write an incoming request to the ingest storage.", + NativeHistogramBucketFactor: 1.1, + NativeHistogramMinResetDuration: 1 * time.Hour, + NativeHistogramMaxBucketNumber: 100, + Buckets: prometheus.DefBuckets, + }), + writeBytesTotal: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "kafka_ingester_tee_sent_bytes_total", + Help: "Total number of bytes sent to the ingest storage.", + }), + recordsPerRequest: promauto.With(registerer).NewHistogram(prometheus.HistogramOpts{ + Name: "kafka_ingester_tee_records_per_write_request", + Help: "The number of records a single per-partition write request has been split into.", + Buckets: prometheus.ExponentialBuckets(1, 2, 8), + }), + } + + return t, nil +} + +// Duplicate implements the distributor.Tee interface, which is used to duplicate +// distributor requests to pattern ingesters. It asynchronously sends each stream +// to Kafka. +// +// Parameters: +// - tenant: The tenant identifier +// - streams: A slice of KeyedStream to be duplicated +func (t *Tee) Duplicate(tenant string, streams []distributor.KeyedStream) { + for idx := range streams { + go func(stream distributor.KeyedStream) { + if err := t.sendStream(tenant, stream); err != nil { + level.Error(t.logger).Log("msg", "failed to send stream to kafka", "err", err) + } + }(streams[idx]) + } +} + +func (t *Tee) Close() { + t.producer.Close() +} + +// sendStream sends a single stream to Kafka. +// +// Parameters: +// - tenant: The tenant identifier +// - stream: The KeyedStream to be sent +// +// Returns: +// - An error if the stream couldn't be sent successfully +func (t *Tee) sendStream(tenant string, stream distributor.KeyedStream) error { + if len(stream.Stream.Entries) == 0 { + return nil + } + partitionID, err := t.partitionRing.PartitionRing().ActivePartitionForKey(stream.HashKey) + if err != nil { + t.ingesterAppends.WithLabelValues("partition_unknown", "fail").Inc() + return fmt.Errorf("failed to find active partition for stream: %w", err) + } + + startTime := time.Now() + + records, err := kafka.Encode(partitionID, tenant, stream.Stream, t.cfg.ProducerMaxRecordSizeBytes) + if err != nil { + t.ingesterAppends.WithLabelValues(fmt.Sprintf("partition_%d", partitionID), "fail").Inc() + return fmt.Errorf("failed to marshal write request to records: %w", err) + } + + t.recordsPerRequest.Observe(float64(len(records))) + + ctx, cancel := context.WithTimeout(user.InjectOrgID(context.Background(), tenant), writeTimeout) + defer cancel() + produceResults := t.producer.ProduceSync(ctx, records) + + if count, sizeBytes := successfulProduceRecordsStats(produceResults); count > 0 { + t.writeLatency.Observe(time.Since(startTime).Seconds()) + t.writeBytesTotal.Add(float64(sizeBytes)) + } + + var finalErr error + for _, result := range produceResults { + if result.Err != nil { + t.ingesterAppends.WithLabelValues(fmt.Sprintf("partition_%d", partitionID), "fail").Inc() + finalErr = err + } else { + t.ingesterAppends.WithLabelValues(fmt.Sprintf("partition_%d", partitionID), "success").Inc() + } + } + + return finalErr +} + +func successfulProduceRecordsStats(results kgo.ProduceResults) (count, sizeBytes int) { + for _, res := range results { + if res.Err == nil && res.Record != nil { + count++ + sizeBytes += len(res.Record.Value) + } + } + + return +} diff --git a/pkg/kafka/tee/tee_test.go b/pkg/kafka/tee/tee_test.go new file mode 100644 index 000000000000..2431f42033fc --- /dev/null +++ b/pkg/kafka/tee/tee_test.go @@ -0,0 +1,50 @@ +package tee + +import ( + "os" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/ring" + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/v3/pkg/distributor" + "github.com/grafana/loki/v3/pkg/kafka/testkafka" + + "github.com/grafana/loki/pkg/push" +) + +func TestPushKafkaRecords(t *testing.T) { + _, cfg := testkafka.CreateCluster(t, 1, "topic") + tee, err := NewTee(cfg, "test", prometheus.NewRegistry(), log.NewLogfmtLogger(os.Stdout), newTestPartitionRing()) + require.NoError(t, err) + + err = tee.sendStream("test", distributor.KeyedStream{ + HashKey: 1, + Stream: push.Stream{ + Labels: `{foo="bar"}`, + Entries: []push.Entry{ + {Timestamp: time.Now(), Line: "test"}, + }, + }, + }) + require.NoError(t, err) +} + +type testPartitionRing struct { + partitionRing *ring.PartitionRing +} + +func (t *testPartitionRing) PartitionRing() *ring.PartitionRing { + return t.partitionRing +} + +func newTestPartitionRing() ring.PartitionRingReader { + desc := ring.NewPartitionRingDesc() + desc.AddPartition(0, ring.PartitionActive, time.Now()) + return &testPartitionRing{ + partitionRing: ring.NewPartitionRing(*desc), + } +} diff --git a/pkg/kafka/testkafka/cluster.go b/pkg/kafka/testkafka/cluster.go new file mode 100644 index 000000000000..fc00e7272e7a --- /dev/null +++ b/pkg/kafka/testkafka/cluster.go @@ -0,0 +1,152 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package testkafka + +import ( + "testing" + "time" + + "github.com/grafana/dskit/flagext" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/twmb/franz-go/pkg/kfake" + "github.com/twmb/franz-go/pkg/kmsg" + + "github.com/grafana/loki/v3/pkg/kafka" +) + +// CreateCluster returns a fake Kafka cluster for unit testing. +func CreateCluster(t testing.TB, numPartitions int32, topicName string) (*kfake.Cluster, kafka.Config) { + cluster, addr := CreateClusterWithoutCustomConsumerGroupsSupport(t, numPartitions, topicName) + addSupportForConsumerGroups(t, cluster, topicName, numPartitions) + + return cluster, createTestKafkaConfig(addr, topicName) +} + +func createTestKafkaConfig(clusterAddr, topicName string) kafka.Config { + cfg := kafka.Config{} + flagext.DefaultValues(&cfg) + + cfg.Address = clusterAddr + cfg.Topic = topicName + cfg.WriteTimeout = 2 * time.Second + + return cfg +} + +func CreateClusterWithoutCustomConsumerGroupsSupport(t testing.TB, numPartitions int32, topicName string) (*kfake.Cluster, string) { + cluster, err := kfake.NewCluster(kfake.NumBrokers(1), kfake.SeedTopics(numPartitions, topicName)) + require.NoError(t, err) + t.Cleanup(cluster.Close) + + addrs := cluster.ListenAddrs() + require.Len(t, addrs, 1) + + return cluster, addrs[0] +} + +// addSupportForConsumerGroups adds very bare-bones support for one consumer group. +// It expects that only one partition is consumed at a time. +func addSupportForConsumerGroups(t testing.TB, cluster *kfake.Cluster, topicName string, numPartitions int32) { + committedOffsets := map[string][]int64{} + + ensureConsumerGroupExists := func(consumerGroup string) { + if _, ok := committedOffsets[consumerGroup]; ok { + return + } + committedOffsets[consumerGroup] = make([]int64, numPartitions+1) + + // Initialise the partition offsets with the special value -1 which means "no offset committed". + for i := 0; i < len(committedOffsets[consumerGroup]); i++ { + committedOffsets[consumerGroup][i] = -1 + } + } + + cluster.ControlKey(kmsg.OffsetCommit.Int16(), func(request kmsg.Request) (kmsg.Response, error, bool) { + cluster.KeepControl() + commitR := request.(*kmsg.OffsetCommitRequest) + consumerGroup := commitR.Group + ensureConsumerGroupExists(consumerGroup) + assert.Len(t, commitR.Topics, 1, "test only has support for one topic per request") + topic := commitR.Topics[0] + assert.Equal(t, topicName, topic.Topic) + assert.Len(t, topic.Partitions, 1, "test only has support for one partition per request") + + partitionID := topic.Partitions[0].Partition + committedOffsets[consumerGroup][partitionID] = topic.Partitions[0].Offset + + resp := request.ResponseKind().(*kmsg.OffsetCommitResponse) + resp.Default() + resp.Topics = []kmsg.OffsetCommitResponseTopic{ + { + Topic: topicName, + Partitions: []kmsg.OffsetCommitResponseTopicPartition{{Partition: partitionID}}, + }, + } + + return resp, nil, true + }) + + cluster.ControlKey(kmsg.OffsetFetch.Int16(), func(kreq kmsg.Request) (kmsg.Response, error, bool) { + cluster.KeepControl() + req := kreq.(*kmsg.OffsetFetchRequest) + assert.Len(t, req.Groups, 1, "test only has support for one consumer group per request") + consumerGroup := req.Groups[0].Group + ensureConsumerGroupExists(consumerGroup) + + const allPartitions = -1 + var partitionID int32 + + if len(req.Groups[0].Topics) == 0 { + // An empty request means fetch all topic-partitions for this group. + partitionID = allPartitions + } else { + partitionID = req.Groups[0].Topics[0].Partitions[0] + assert.Len(t, req.Groups[0], 1, "test only has support for one partition per request") + assert.Len(t, req.Groups[0].Topics[0].Partitions, 1, "test only has support for one partition per request") + } + + // Prepare the list of partitions for which the offset has been committed. + // This mimics the real Kafka behaviour. + var partitionsResp []kmsg.OffsetFetchResponseGroupTopicPartition + if partitionID == allPartitions { + for i := int32(1); i < numPartitions+1; i++ { + if committedOffsets[consumerGroup][i] >= 0 { + partitionsResp = append(partitionsResp, kmsg.OffsetFetchResponseGroupTopicPartition{ + Partition: i, + Offset: committedOffsets[consumerGroup][i], + }) + } + } + } else { + if committedOffsets[consumerGroup][partitionID] >= 0 { + partitionsResp = append(partitionsResp, kmsg.OffsetFetchResponseGroupTopicPartition{ + Partition: partitionID, + Offset: committedOffsets[consumerGroup][partitionID], + }) + } + } + + // Prepare the list topics for which there are some committed offsets. + // This mimics the real Kafka behaviour. + var topicsResp []kmsg.OffsetFetchResponseGroupTopic + if len(partitionsResp) > 0 { + topicsResp = []kmsg.OffsetFetchResponseGroupTopic{ + { + Topic: topicName, + Partitions: partitionsResp, + }, + } + } + + resp := kreq.ResponseKind().(*kmsg.OffsetFetchResponse) + resp.Default() + resp.Groups = []kmsg.OffsetFetchResponseGroup{ + { + Group: consumerGroup, + Topics: topicsResp, + }, + } + return resp, nil, true + }) +} diff --git a/pkg/kafka/testkafka/message.go b/pkg/kafka/testkafka/message.go new file mode 100644 index 000000000000..cb7fd50a3f50 --- /dev/null +++ b/pkg/kafka/testkafka/message.go @@ -0,0 +1,26 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package testkafka + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// CreateProduceResponseError returns a kmsg.ProduceResponse containing err for the input topic and partition. +func CreateProduceResponseError(version int16, topic string, partition int32, err *kerr.Error) *kmsg.ProduceResponse { + return &kmsg.ProduceResponse{ + Version: version, + Topics: []kmsg.ProduceResponseTopic{ + { + Topic: topic, + Partitions: []kmsg.ProduceResponseTopicPartition{ + { + Partition: partition, + ErrorCode: err.Code, + }, + }, + }, + }, + } +} diff --git a/pkg/kafka/writer_client.go b/pkg/kafka/writer_client.go new file mode 100644 index 000000000000..bb6fbb8082c5 --- /dev/null +++ b/pkg/kafka/writer_client.go @@ -0,0 +1,322 @@ +package kafka + +import ( + "context" + "errors" + "math" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/pkg/kmsg" + "github.com/twmb/franz-go/plugin/kotel" + "github.com/twmb/franz-go/plugin/kprom" + "go.opentelemetry.io/otel/propagation" + "go.opentelemetry.io/otel/trace" + "go.uber.org/atomic" +) + +// NewWriterClient returns the kgo.Client that should be used by the Writer. +// +// The input prometheus.Registerer must be wrapped with a prefix (the names of metrics +// registered don't have a prefix). +func NewWriterClient(kafkaCfg Config, maxInflightProduceRequests int, logger log.Logger, reg prometheus.Registerer) (*kgo.Client, error) { + // Do not export the client ID, because we use it to specify options to the backend. + metrics := kprom.NewMetrics( + "", // No prefix. We expect the input prometheus.Registered to be wrapped with a prefix. + kprom.Registerer(reg), + kprom.FetchAndProduceDetail(kprom.Batches, kprom.Records, kprom.CompressedBytes, kprom.UncompressedBytes)) + + opts := append( + commonKafkaClientOptions(kafkaCfg, metrics, logger), + kgo.RequiredAcks(kgo.AllISRAcks()), + kgo.DefaultProduceTopic(kafkaCfg.Topic), + + // We set the partition field in each record. + kgo.RecordPartitioner(kgo.ManualPartitioner()), + + // Set the upper bounds the size of a record batch. + kgo.ProducerBatchMaxBytes(producerBatchMaxBytes), + + // By default, the Kafka client allows 1 Produce in-flight request per broker. Disabling write idempotency + // (which we don't need), we can increase the max number of in-flight Produce requests per broker. A higher + // number of in-flight requests, in addition to short buffering ("linger") in client side before firing the + // next Produce request allows us to reduce the end-to-end latency. + // + // The result of the multiplication of producer linger and max in-flight requests should match the maximum + // Produce latency expected by the Kafka backend in a steady state. For example, 50ms * 20 requests = 1s, + // which means the Kafka client will keep issuing a Produce request every 50ms as far as the Kafka backend + // doesn't take longer than 1s to process them (if it takes longer, the client will buffer data and stop + // issuing new Produce requests until some previous ones complete). + kgo.DisableIdempotentWrite(), + kgo.ProducerLinger(50*time.Millisecond), + kgo.MaxProduceRequestsInflightPerBroker(maxInflightProduceRequests), + + // Unlimited number of Produce retries but a deadline on the max time a record can take to be delivered. + // With the default config it would retry infinitely. + // + // Details of the involved timeouts: + // - RecordDeliveryTimeout: how long a Kafka client Produce() call can take for a given record. The overhead + // timeout is NOT applied. + // - ProduceRequestTimeout: how long to wait for the response to the Produce request (the Kafka protocol message) + // after being sent on the network. The actual timeout is increased by the configured overhead. + // + // When a Produce request to Kafka fail, the client will retry up until the RecordDeliveryTimeout is reached. + // Once the timeout is reached, the Produce request will fail and all other buffered requests in the client + // (for the same partition) will fail too. See kgo.RecordDeliveryTimeout() documentation for more info. + kgo.RecordRetries(math.MaxInt64), + kgo.RecordDeliveryTimeout(kafkaCfg.WriteTimeout), + kgo.ProduceRequestTimeout(kafkaCfg.WriteTimeout), + kgo.RequestTimeoutOverhead(writerRequestTimeoutOverhead), + + // Unlimited number of buffered records because we limit on bytes in Writer. The reason why we don't use + // kgo.MaxBufferedBytes() is because it suffers a deadlock issue: + // https://github.com/twmb/franz-go/issues/777 + kgo.MaxBufferedRecords(math.MaxInt), // Use a high value to set it as unlimited, because the client doesn't support "0 as unlimited". + kgo.MaxBufferedBytes(0), + ) + + return kgo.NewClient(opts...) +} + +type onlySampledTraces struct { + propagation.TextMapPropagator +} + +func (o onlySampledTraces) Inject(ctx context.Context, carrier propagation.TextMapCarrier) { + sc := trace.SpanContextFromContext(ctx) + if !sc.IsSampled() { + return + } + o.TextMapPropagator.Inject(ctx, carrier) +} + +func commonKafkaClientOptions(cfg Config, metrics *kprom.Metrics, logger log.Logger) []kgo.Opt { + opts := []kgo.Opt{ + kgo.ClientID(cfg.ClientID), + kgo.SeedBrokers(cfg.Address), + kgo.DialTimeout(cfg.DialTimeout), + + // A cluster metadata update is a request sent to a broker and getting back the map of partitions and + // the leader broker for each partition. The cluster metadata can be updated (a) periodically or + // (b) when some events occur (e.g. backoff due to errors). + // + // MetadataMinAge() sets the minimum time between two cluster metadata updates due to events. + // MetadataMaxAge() sets how frequently the periodic update should occur. + // + // It's important to note that the periodic update is also used to discover new brokers (e.g. during a + // rolling update or after a scale up). For this reason, it's important to run the update frequently. + // + // The other two side effects of frequently updating the cluster metadata: + // 1. The "metadata" request may be expensive to run on the Kafka backend. + // 2. If the backend returns each time a different authoritative owner for a partition, then each time + // the cluster metadata is updated the Kafka client will create a new connection for each partition, + // leading to a high connections churn rate. + // + // We currently set min and max age to the same value to have constant load on the Kafka backend: regardless + // there are errors or not, the metadata requests frequency doesn't change. + kgo.MetadataMinAge(10 * time.Second), + kgo.MetadataMaxAge(10 * time.Second), + + kgo.WithLogger(newLogger(logger)), + + kgo.RetryTimeoutFn(func(key int16) time.Duration { + switch key { + case ((*kmsg.ListOffsetsRequest)(nil)).Key(): + return cfg.LastProducedOffsetRetryTimeout + } + + // 30s is the default timeout in the Kafka client. + return 30 * time.Second + }), + } + + if cfg.AutoCreateTopicEnabled { + opts = append(opts, kgo.AllowAutoTopicCreation()) + } + + tracer := kotel.NewTracer( + kotel.TracerPropagator(propagation.NewCompositeTextMapPropagator(onlySampledTraces{propagation.TraceContext{}})), + ) + opts = append(opts, kgo.WithHooks(kotel.NewKotel(kotel.WithTracer(tracer)).Hooks()...)) + + if metrics != nil { + opts = append(opts, kgo.WithHooks(metrics)) + } + + return opts +} + +// Producer is a kgo.Client wrapper exposing some higher level features and metrics useful for producers. +type Producer struct { + *kgo.Client + + closeOnce *sync.Once + closed chan struct{} + + // Keep track of Kafka records size (bytes) currently in-flight in the Kafka client. + // This counter is used to implement a limit on the max buffered bytes. + bufferedBytes *atomic.Int64 + + // The max buffered bytes allowed. Once this limit is reached, produce requests fail. + maxBufferedBytes int64 + + // Custom metrics. + bufferedProduceBytes prometheus.Summary + bufferedProduceBytesLimit prometheus.Gauge + produceRequestsTotal prometheus.Counter + produceFailuresTotal *prometheus.CounterVec +} + +// NewProducer returns a new KafkaProducer. +// +// The input prometheus.Registerer must be wrapped with a prefix (the names of metrics +// registered don't have a prefix). +func NewProducer(client *kgo.Client, maxBufferedBytes int64, reg prometheus.Registerer) *Producer { + producer := &Producer{ + Client: client, + closeOnce: &sync.Once{}, + closed: make(chan struct{}), + bufferedBytes: atomic.NewInt64(0), + maxBufferedBytes: maxBufferedBytes, + + // Metrics. + bufferedProduceBytes: promauto.With(reg).NewSummary( + prometheus.SummaryOpts{ + Name: "buffered_produce_bytes", + Help: "The buffered produce records in bytes. Quantile buckets keep track of buffered records size over the last 60s.", + Objectives: map[float64]float64{0.5: 0.05, 0.99: 0.001, 1: 0.001}, + MaxAge: time.Minute, + AgeBuckets: 6, + }), + bufferedProduceBytesLimit: promauto.With(reg).NewGauge( + prometheus.GaugeOpts{ + Name: "buffered_produce_bytes_limit", + Help: "The bytes limit on buffered produce records. Produce requests fail once this limit is reached.", + }), + produceRequestsTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "produce_requests_total", + Help: "Total number of produce requests issued to Kafka.", + }), + produceFailuresTotal: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "produce_failures_total", + Help: "Total number of failed produce requests issued to Kafka.", + }, []string{"reason"}), + } + + producer.bufferedProduceBytesLimit.Set(float64(maxBufferedBytes)) + + go producer.updateMetricsLoop() + + return producer +} + +func (c *Producer) Close() { + c.closeOnce.Do(func() { + close(c.closed) + }) + + c.Client.Close() +} + +func (c *Producer) updateMetricsLoop() { + // We observe buffered produce bytes and at regular intervals, to have a good + // approximation of the peak value reached over the observation period. + ticker := time.NewTicker(250 * time.Millisecond) + + for { + select { + case <-ticker.C: + c.bufferedProduceBytes.Observe(float64(c.Client.BufferedProduceBytes())) + + case <-c.closed: + return + } + } +} + +// ProduceSync produces records to Kafka and returns once all records have been successfully committed, +// or an error occurred. +// +// This function honors the configure max buffered bytes and refuse to produce a record, returnin kgo.ErrMaxBuffered, +// if the configured limit is reached. +func (c *Producer) ProduceSync(ctx context.Context, records []*kgo.Record) kgo.ProduceResults { + var ( + remaining = atomic.NewInt64(int64(len(records))) + done = make(chan struct{}) + resMx sync.Mutex + res = make(kgo.ProduceResults, 0, len(records)) + ) + + c.produceRequestsTotal.Add(float64(len(records))) + + onProduceDone := func(r *kgo.Record, err error) { + if c.maxBufferedBytes > 0 { + c.bufferedBytes.Add(-int64(len(r.Value))) + } + + resMx.Lock() + res = append(res, kgo.ProduceResult{Record: r, Err: err}) + resMx.Unlock() + + if err != nil { + c.produceFailuresTotal.WithLabelValues(produceErrReason(err)).Inc() + } + + // In case of error we'll wait for all responses anyway before returning from produceSync(). + // It allows us to keep code easier, given we don't expect this function to be frequently + // called with multiple records. + if remaining.Dec() == 0 { + close(done) + } + } + + for _, record := range records { + // Fast fail if the Kafka client buffer is full. Buffered bytes counter is decreased onProducerDone(). + if c.maxBufferedBytes > 0 && c.bufferedBytes.Add(int64(len(record.Value))) > c.maxBufferedBytes { + onProduceDone(record, kgo.ErrMaxBuffered) + continue + } + + // We use a new context to avoid that other Produce() may be cancelled when this call's context is + // canceled. It's important to note that cancelling the context passed to Produce() doesn't actually + // prevent the data to be sent over the wire (because it's never removed from the buffer) but in some + // cases may cause all requests to fail with context cancelled. + // + // Produce() may theoretically block if the buffer is full, but we configure the Kafka client with + // unlimited buffer because we implement the buffer limit ourselves (see maxBufferedBytes). This means + // Produce() should never block for us in practice. + c.Client.Produce(context.WithoutCancel(ctx), record, onProduceDone) + } + + // Wait for a response or until the context has done. + select { + case <-ctx.Done(): + return kgo.ProduceResults{{Err: context.Cause(ctx)}} + case <-done: + // Once we're done, it's guaranteed that no more results will be appended, so we can safely return it. + return res + } +} + +func produceErrReason(err error) string { + if errors.Is(err, context.DeadlineExceeded) || errors.Is(err, kgo.ErrRecordTimeout) { + return "timeout" + } + if errors.Is(err, kgo.ErrMaxBuffered) { + return "buffer-full" + } + if errors.Is(err, kerr.MessageTooLarge) { + return "record-too-large" + } + if errors.Is(err, context.Canceled) { + // This should never happen because we don't cancel produce requests, however we + // check this error anyway to detect if something unexpected happened. + return "canceled" + } + return "other" +} diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index fa1b6cbfe410..5107bf9ee765 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -39,14 +39,13 @@ import ( "github.com/grafana/loki/v3/pkg/distributor" "github.com/grafana/loki/v3/pkg/indexgateway" "github.com/grafana/loki/v3/pkg/ingester" - ingesterkafka "github.com/grafana/loki/v3/pkg/ingester-kafka" - "github.com/grafana/loki/v3/pkg/ingester-kafka/kafka" - "github.com/grafana/loki/v3/pkg/ingester-kafka/partitionring" ingester_rf1 "github.com/grafana/loki/v3/pkg/ingester-rf1" "github.com/grafana/loki/v3/pkg/ingester-rf1/metastore" metastoreclient "github.com/grafana/loki/v3/pkg/ingester-rf1/metastore/client" "github.com/grafana/loki/v3/pkg/ingester-rf1/metastore/health" ingester_client "github.com/grafana/loki/v3/pkg/ingester/client" + "github.com/grafana/loki/v3/pkg/kafka" + ingester_kafka "github.com/grafana/loki/v3/pkg/kafka/ingester" "github.com/grafana/loki/v3/pkg/loghttp/push" "github.com/grafana/loki/v3/pkg/loki/common" "github.com/grafana/loki/v3/pkg/lokifrontend" @@ -114,9 +113,8 @@ type Config struct { MemberlistKV memberlist.KVConfig `yaml:"memberlist"` Metastore metastore.Config `yaml:"metastore,omitempty"` MetastoreClient metastoreclient.Config `yaml:"metastore_client"` - PartitionRingConfig partitionring.Config `yaml:"partition_ring,omitempty" category:"experimental"` KafkaConfig kafka.Config `yaml:"kafka_config,omitempty" category:"experimental"` - KafkaIngester ingesterkafka.Config `yaml:"kafka_ingester,omitempty" category:"experimental"` + KafkaIngester ingester_kafka.Config `yaml:"kafka_ingester,omitempty" category:"experimental"` RuntimeConfig runtimeconfig.Config `yaml:"runtime_config,omitempty"` OperationalConfig runtime.Config `yaml:"operational_config,omitempty"` @@ -199,7 +197,6 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) { c.Profiling.RegisterFlags(f) c.Metastore.RegisterFlags(f) c.MetastoreClient.RegisterFlags(f) - c.PartitionRingConfig.RegisterFlags(f) c.KafkaConfig.RegisterFlags(f) c.KafkaIngester.RegisterFlags(f) } @@ -227,6 +224,8 @@ func (c *Config) registerServerFlagsWithChangedDefaultValues(fs *flag.FlagSet) { case "pattern-ingester.distributor.replication-factor": _ = f.Value.Set("1") + case "kafka-ingester.distributor.replication-factor": + _ = f.Value.Set("1") } fs.Var(f.Value, f.Name, f.Usage) @@ -305,6 +304,14 @@ func (c *Config) Validate() error { if err := c.Pattern.Validate(); err != nil { errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid pattern_ingester config")) } + if c.KafkaIngester.Enabled { + if err := c.KafkaConfig.Validate(); err != nil { + errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid kafka_config config")) + } + if err := c.KafkaIngester.Validate(); err != nil { + errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid kafka_ingester config")) + } + } errs = append(errs, validateSchemaValues(c)...) errs = append(errs, ValidateConfigCompatibility(*c)...) @@ -384,7 +391,7 @@ type Loki struct { MetastoreClient *metastoreclient.Client partitionRingWatcher *ring.PartitionRingWatcher partitionRing *ring.PartitionInstanceRing - kafkaIngester *ingesterkafka.Ingester + kafkaIngester *ingester_kafka.Ingester ClientMetrics storage.ClientMetrics deleteClientMetrics *deletion.DeleteRequestClientMetrics @@ -732,7 +739,7 @@ func (t *Loki) setupModuleManager() error { Distributor: {Ring, Server, Overrides, TenantConfigs, PatternRingClient, PatternIngesterTee, IngesterRF1RingClient, Analytics, PartitionRing}, Store: {Overrides, IndexGatewayRing}, IngesterRF1: {Store, Server, MemberlistKV, TenantConfigs, MetastoreClient, Analytics, PartitionRing}, - IngesterKafka: {PartitionRing}, + IngesterKafka: {Store, Server, MemberlistKV, TenantConfigs, MetastoreClient, Analytics, PartitionRing}, Ingester: {Store, Server, MemberlistKV, TenantConfigs, Analytics}, Querier: {Store, Ring, Server, IngesterQuerier, PatternRingClient, MetastoreClient, Overrides, Analytics, CacheGenerationLoader, QuerySchedulerRing}, QueryFrontendTripperware: {Server, Overrides, TenantConfigs}, diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index b08f829bb915..20d744b0f620 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -47,14 +47,14 @@ import ( "github.com/grafana/loki/v3/pkg/distributor" "github.com/grafana/loki/v3/pkg/indexgateway" "github.com/grafana/loki/v3/pkg/ingester" - ingesterkafka "github.com/grafana/loki/v3/pkg/ingester-kafka" - "github.com/grafana/loki/v3/pkg/ingester-kafka/kafka" ingester_rf1 "github.com/grafana/loki/v3/pkg/ingester-rf1" "github.com/grafana/loki/v3/pkg/ingester-rf1/metastore" metastoreclient "github.com/grafana/loki/v3/pkg/ingester-rf1/metastore/client" "github.com/grafana/loki/v3/pkg/ingester-rf1/metastore/health" "github.com/grafana/loki/v3/pkg/ingester-rf1/metastore/metastorepb" "github.com/grafana/loki/v3/pkg/ingester-rf1/objstore" + ingesterkafka "github.com/grafana/loki/v3/pkg/kafka/ingester" + kafka_tee "github.com/grafana/loki/v3/pkg/kafka/tee" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql" "github.com/grafana/loki/v3/pkg/logqlmodel/stats" @@ -342,7 +342,7 @@ func (t *Loki) initDistributor() (services.Service, error) { t.Tee = distributor.WrapTee(t.Tee, rf1Tee) } if t.Cfg.KafkaIngester.Enabled { - kafkaTee, err := kafka.NewTee(t.Cfg.KafkaConfig, t.Cfg.MetricsNamespace, prometheus.DefaultRegisterer, util_log.Logger, t.partitionRing) + kafkaTee, err := kafka_tee.NewTee(t.Cfg.KafkaConfig, t.Cfg.MetricsNamespace, prometheus.DefaultRegisterer, util_log.Logger, t.partitionRing) if err != nil { return nil, err } @@ -653,7 +653,7 @@ func (t *Loki) initKafkaIngester() (_ services.Service, err error) { if !t.Cfg.KafkaIngester.Enabled { return nil, nil } - + t.Cfg.KafkaIngester.KafkaConfig = t.Cfg.KafkaConfig logger := log.With(util_log.Logger, "component", "ingester-kafka") t.Cfg.KafkaIngester.LifecyclerConfig.ListenPort = t.Cfg.Server.GRPCListenPort @@ -661,28 +661,26 @@ func (t *Loki) initKafkaIngester() (_ services.Service, err error) { t.Cfg.KafkaIngester.ShutdownMarkerPath = t.Cfg.Common.PathPrefix } if t.Cfg.KafkaIngester.ShutdownMarkerPath == "" { - level.Warn(util_log.Logger).Log("msg", "The config setting shutdown marker path is not set. The /ingester/prepare_shutdown endpoint won't work") + return nil, errors.New("the config setting shutdown marker path is not set. The /ingester/prepare-partition-downscale endpoint won't work") } - - t.kafkaIngester, err = ingesterkafka.New(t.Cfg.KafkaIngester, prometheus.DefaultRegisterer, t.Cfg.MetricsNamespace, logger) + storage, err := objstore.New(t.Cfg.SchemaConfig.Configs, t.Cfg.StorageConfig, t.ClientMetrics) if err != nil { - fmt.Println("Error initializing ingester rf1", err) - return + return nil, err } - // Not enabled for kafka ingester yet - // fmt.Println("registered GRPC") - // logproto.RegisterPusherRF1Server(t.Server.GRPC, t.IngesterRF1) + consumerFactory := ingesterkafka.NewConsumerFactory(t.MetastoreClient, storage, t.Cfg.KafkaIngester.FlushInterval, t.Cfg.KafkaIngester.FlushSize, logger, prometheus.DefaultRegisterer) + t.kafkaIngester, err = ingesterkafka.New(t.Cfg.KafkaIngester, consumerFactory, logger, t.Cfg.MetricsNamespace, prometheus.DefaultRegisterer) + if err != nil { + return nil, err + } httpMiddleware := middleware.Merge( serverutil.RecoveryHTTPMiddleware, ) - t.Server.HTTP.Methods("POST", "GET", "DELETE").Path("/ingester-rf1/prepare_shutdown").Handler( - httpMiddleware.Wrap(http.HandlerFunc(t.kafkaIngester.PrepareShutdown)), - ) - t.Server.HTTP.Methods("POST", "GET").Path("/ingester-rf1/shutdown").Handler( - httpMiddleware.Wrap(http.HandlerFunc(t.kafkaIngester.ShutdownHandler)), + t.Server.HTTP.Methods("POST", "GET", "DELETE").Path("/ingester/prepare-partition-downscale").Handler( + httpMiddleware.Wrap(http.HandlerFunc(t.kafkaIngester.PreparePartitionDownscaleHandler)), ) + return t.kafkaIngester, nil } @@ -1482,6 +1480,7 @@ func (t *Loki) initMemberlistKV() (services.Service, error) { t.Cfg.MemberlistKV.Codecs = []codec.Codec{ ring.GetCodec(), analytics.JSONCodec, + ring.GetPartitionRingCodec(), } dnsProviderReg := prometheus.WrapRegistererWithPrefix( @@ -1503,6 +1502,8 @@ func (t *Loki) initMemberlistKV() (services.Service, error) { t.Cfg.Ruler.Ring.KVStore.MemberlistKV = t.MemberlistKV.GetMemberlistKV t.Cfg.Pattern.LifecyclerConfig.RingConfig.KVStore.MemberlistKV = t.MemberlistKV.GetMemberlistKV t.Cfg.IngesterRF1.LifecyclerConfig.RingConfig.KVStore.MemberlistKV = t.MemberlistKV.GetMemberlistKV + t.Cfg.KafkaIngester.PartitionRingConfig.KVStore.MemberlistKV = t.MemberlistKV.GetMemberlistKV + t.Cfg.KafkaIngester.LifecyclerConfig.RingConfig.KVStore.MemberlistKV = t.MemberlistKV.GetMemberlistKV t.Server.HTTP.Handle("/memberlist", t.MemberlistKV) if t.Cfg.InternalServer.Enable { @@ -1867,7 +1868,7 @@ func (t *Loki) initAnalytics() (services.Service, error) { } func (t *Loki) initMetastore() (services.Service, error) { - if !t.Cfg.IngesterRF1.Enabled { + if !t.Cfg.IngesterRF1.Enabled && !t.Cfg.KafkaIngester.Enabled { return nil, nil } if t.Cfg.isTarget(All) { @@ -1884,7 +1885,7 @@ func (t *Loki) initMetastore() (services.Service, error) { } func (t *Loki) initMetastoreClient() (services.Service, error) { - if !t.Cfg.IngesterRF1.Enabled && !t.Cfg.QuerierRF1.Enabled { + if !t.Cfg.IngesterRF1.Enabled && !t.Cfg.QuerierRF1.Enabled && !t.Cfg.KafkaIngester.Enabled { return nil, nil } mc, err := metastoreclient.New(t.Cfg.MetastoreClient, prometheus.DefaultRegisterer) @@ -1901,8 +1902,7 @@ func (t *Loki) initPartitionRing() (services.Service, error) { return nil, nil } - // TODO: New config? - kvClient, err := kv.NewClient(t.Cfg.KafkaIngester.LifecyclerConfig.RingConfig.KVStore, ring.GetPartitionRingCodec(), kv.RegistererWithKVName(prometheus.DefaultRegisterer, ingesterkafka.PartitionRingName+"-watcher"), util_log.Logger) + kvClient, err := kv.NewClient(t.Cfg.KafkaIngester.PartitionRingConfig.KVStore, ring.GetPartitionRingCodec(), kv.RegistererWithKVName(prometheus.DefaultRegisterer, ingesterkafka.PartitionRingName+"-watcher"), util_log.Logger) if err != nil { return nil, fmt.Errorf("creating KV store for partitions ring watcher: %w", err) } diff --git a/pkg/storage/wal/segment.go b/pkg/storage/wal/segment.go index 93b824bbcb70..83841aab1697 100644 --- a/pkg/storage/wal/segment.go +++ b/pkg/storage/wal/segment.go @@ -177,7 +177,7 @@ func (b *SegmentWriter) Append(tenantID, labelsString string, lbls labels.Labels b.lastAppend = now for _, e := range entries { - b.inputSize.Add(int64(len(e.Line))) + b.inputSize.Add(int64(len(e.Line))) // todo(cyriltovena): should add the size of structured metadata } id := streamID{labels: labelsString, tenant: tenantID} s := b.getOrCreateStream(id, lbls) diff --git a/pkg/util/atomicfs/fsync.go b/pkg/util/atomicfs/fsync.go new file mode 100644 index 000000000000..5d85865c495c --- /dev/null +++ b/pkg/util/atomicfs/fsync.go @@ -0,0 +1,101 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package atomicfs + +import ( + "io" + "os" + "path/filepath" + + "github.com/grafana/dskit/multierror" +) + +// Create creates a new file at a temporary path that will be renamed to the +// supplied path on close from a temporary file in the same directory, ensuring +// all data and the containing directory have been fsynced to disk. +func Create(path string) (*File, error) { + // We rename from a temporary file in the same directory to because rename + // can only operate on two files that are on the same filesystem. Creating + // a temporary file in the same directory is an easy way to guarantee that. + final := filepath.Clean(path) + tmp := tempPath(final) + + file, err := os.Create(tmp) + if err != nil { + return nil, err + } + + return &File{ + File: file, + finalPath: final, + }, nil +} + +// tempPath returns a path for the temporary version of a file. This function exists +// to ensure the logic here stays in sync with unit tests that check for this file being +// cleaned up. +func tempPath(final string) string { + return final + ".tmp" +} + +// File is a wrapper around an os.File instance that uses a temporary file for writes +// that is renamed to its final path when Close is called. The Close method will also +// ensure that all data from the file has been fsynced as well as the containing +// directory. If the temporary file cannot be renamed or fsynced on Close, it is +// removed. +type File struct { + *os.File + finalPath string +} + +func (a *File) Close() error { + cleanup := true + defer func() { + if cleanup { + _ = os.Remove(a.File.Name()) + } + }() + + merr := multierror.New() + merr.Add(a.File.Sync()) + merr.Add(a.File.Close()) + if err := merr.Err(); err != nil { + return err + } + + if err := os.Rename(a.File.Name(), a.finalPath); err != nil { + return err + } + + cleanup = false + // After writing the file and calling fsync on it, fsync the containing directory + // to ensure the directory entry is persisted to disk. + // + // From https://man7.org/linux/man-pages/man2/fsync.2.html + // > Calling fsync() does not necessarily ensure that the entry in the + // > directory containing the file has also reached disk. For that an + // > explicit fsync() on a file descriptor for the directory is also + // > needed. + dir, err := os.Open(filepath.Dir(a.finalPath)) + if err != nil { + return err + } + + merr.Add(dir.Sync()) + merr.Add(dir.Close()) + return merr.Err() +} + +// CreateFile safely writes the contents of data to filePath, ensuring that all data +// has been fsynced as well as the containing directory of the file. +func CreateFile(filePath string, data io.Reader) error { + f, err := Create(filePath) + if err != nil { + return err + } + + _, err = io.Copy(f, data) + merr := multierror.New(err) + merr.Add(f.Close()) + return merr.Err() +} diff --git a/pkg/util/atomicfs/fsync_test.go b/pkg/util/atomicfs/fsync_test.go new file mode 100644 index 000000000000..65933b3d9c16 --- /dev/null +++ b/pkg/util/atomicfs/fsync_test.go @@ -0,0 +1,84 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package atomicfs + +import ( + "os" + "path/filepath" + "strings" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestCreateFile(t *testing.T) { + path := filepath.Join(t.TempDir(), "TestCreateFile") + require.NoError(t, CreateFile(path, strings.NewReader("test"))) + + // Ensure the temporary file created by CreateFile has been removed. + _, err := os.Stat(tempPath(path)) + require.ErrorIs(t, err, os.ErrNotExist) + + // Ensure the directory entry for the file exists. + entries, err := os.ReadDir(filepath.Dir(path)) + require.NoError(t, err) + requireContainsFile(t, entries, path) + + // Check the contents of the file. + contents, err := os.ReadFile(path) + require.NoError(t, err) + require.Equal(t, "test", string(contents)) +} + +func TestCreate(t *testing.T) { + t.Run("success", func(t *testing.T) { + path := filepath.Join(t.TempDir(), "TestCreate") + f, err := Create(path) + require.NoError(t, err) + + _, err = f.WriteString("test") + require.NoError(t, err) + require.NoError(t, f.Close()) + + // Ensure the directory entry for the file exists. + entries, err := os.ReadDir(filepath.Dir(path)) + require.NoError(t, err) + requireContainsFile(t, entries, path) + + // Check the contents of the file. + contents, err := os.ReadFile(path) + require.NoError(t, err) + require.Equal(t, "test", string(contents)) + }) + + t.Run("duplicate close", func(t *testing.T) { + path := filepath.Join(t.TempDir(), "TestCreate") + f, err := Create(path) + require.NoError(t, err) + + _, err = f.WriteString("test") + require.NoError(t, err) + require.NoError(t, f.Close()) + + // File has already been closed, our attempt to fsync and close again should fail. + require.ErrorIs(t, f.Close(), os.ErrClosed) + + // Original file _should not_ have been modified by trying to close again. + contents, err := os.ReadFile(path) + require.NoError(t, err) + require.Equal(t, "test", string(contents)) + }) + +} + +func requireContainsFile(t *testing.T, entries []os.DirEntry, path string) { + name := filepath.Base(path) + + for _, entry := range entries { + if entry.Name() == name { + return + } + } + + t.Fatalf("expected to find %s in %+v", name, entries) +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/LICENSE b/vendor/github.com/twmb/franz-go/pkg/kadm/LICENSE new file mode 100644 index 000000000000..36e18034325d --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/LICENSE @@ -0,0 +1,24 @@ +Copyright 2020, Travis Bischel. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of the library nor the + names of its contributors may be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY +DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/acls.go b/vendor/github.com/twmb/franz-go/pkg/kadm/acls.go new file mode 100644 index 000000000000..62676b5b8c07 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/acls.go @@ -0,0 +1,1117 @@ +package kadm + +import ( + "context" + "fmt" + "strings" + "sync" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// ACLBuilder is a builder that is used for batch creating / listing / deleting +// ACLS. +// +// An ACL consists of five components: +// +// - the user (principal) +// - the host the user runs on +// - what resource to access (topic name, group id, etc.) +// - the operation (read, write) +// - whether to allow or deny the above +// +// This builder allows for adding the above five components in batches and then +// creating, listing, or deleting a batch of ACLs in one go. This builder +// merges the fifth component (allowing or denying) into allowing principals +// and hosts and denying principals and hosts. The builder must always have an +// Allow or Deny. For creating, the host is optional and defaults to the +// wildcard * that allows or denies all hosts. For listing / deleting, the host +// is also required (specifying no hosts matches all hosts, but you must +// specify this). +// +// Building works on a multiplying factor: every user, every host, every +// resource, and every operation is combined (principals * hosts * resources * +// operations). +// +// With the Kafka simple authorizer (and most reimplementations), all +// principals are required to have the "User:" prefix. The PrefixUserExcept +// function can be used to easily add the "User:" prefix if missing. +// +// The full set of operations and which requests require what operations is +// described in a large doc comment on the ACLOperation type. +// +// Lastly, resources to access / deny access to can be created / matched based +// on literal (exact) names, or on prefix names, or more. See the ACLPattern +// docs for more information. +type ACLBuilder struct { + any []string + anyResource bool + topics []string + anyTopic bool + groups []string + anyGroup bool + anyCluster bool + txnIDs []string + anyTxn bool + tokens []string + anyToken bool + + allow []string + anyAllow bool + allowHosts []string + anyAllowHosts bool + deny []string + anyDeny bool + denyHosts []string + anyDenyHosts bool + + ops []ACLOperation + + pattern ACLPattern +} + +// PrefixUser prefixes all allowed and denied principals with "User:". +func (b *ACLBuilder) PrefixUser() { + b.PrefixUserExcept() +} + +// PrefixUserExcept prefixes all allowed and denied principals with "User:", +// unless they have any of the given except prefixes. +func (b *ACLBuilder) PrefixUserExcept(except ...string) { + replace := func(u string) string { + if !strings.HasPrefix(u, "User:") { + for _, e := range except { + if strings.HasPrefix(u, e) { + return u + } + } + return "User:" + u + } + return u + } + + for i, u := range b.allow { + b.allow[i] = replace(u) + } + for i, u := range b.deny { + b.deny[i] = replace(u) + } +} + +// NewACLs returns a new ACL builder. +func NewACLs() *ACLBuilder { + return new(ACLBuilder) +} + +// AnyResource lists & deletes ACLs of any type matching the given names +// (pending other filters). If no names are given, this matches all names. +// +// This returns the input pointer. +// +// This function does nothing for creating. +func (b *ACLBuilder) AnyResource(name ...string) *ACLBuilder { + b.any = name + if len(name) == 0 { + b.anyResource = true + } + return b +} + +// Topics lists/deletes/creates ACLs of resource type "topic" for the given +// topics. +// +// This returns the input pointer. +// +// For listing or deleting, if this is provided no topics, all "topic" resource +// type ACLs are matched. For creating, if no topics are provided, this +// function does nothing. +func (b *ACLBuilder) Topics(t ...string) *ACLBuilder { + b.topics = t + if len(t) == 0 { + b.anyTopic = true + } + return b +} + +// MaybeTopics is the same as Topics, but does not match all topics if none are +// provided. +func (b *ACLBuilder) MaybeTopics(t ...string) *ACLBuilder { b.topics = t; return b } + +// Groups lists/deletes/creates ACLs of resource type "group" for the given +// groups. +// +// This returns the input pointer. +// +// For listing or deleting, if this is provided no groups, all "group" resource +// type ACLs are matched. For creating, if no groups are provided, this +// function does nothing. +func (b *ACLBuilder) Groups(g ...string) *ACLBuilder { + b.groups = g + if len(g) == 0 { + b.anyGroup = true + } + return b +} + +// MaybeGroups is the same as Groups, but does not match all groups if none are +// provided. +func (b *ACLBuilder) MaybeGroups(g ...string) *ACLBuilder { b.groups = g; return b } + +// Clusters lists/deletes/creates ACLs of resource type "cluster". +// +// This returns the input pointer. +// +// There is only one type of cluster in Kafka, "kafka-cluster". Opting in to +// listing or deleting by cluster inherently matches all ACLS of resource type +// cluster. For creating, this function allows for creating cluster ACLs. +func (b *ACLBuilder) Clusters() *ACLBuilder { + b.anyCluster = true + return b +} + +// MaybeClusters is the same as Clusters, but only matches clusters if c is +// true. +func (b *ACLBuilder) MaybeClusters(c bool) *ACLBuilder { b.anyCluster = c; return b } + +// TransactionalIDs lists/deletes/creates ACLs of resource type +// "transactional_id" for the given transactional IDs. +// +// This returns the input pointer. +// +// For listing or deleting, if this is provided no IDs, all "transactional_id" +// resource type ACLs matched. For creating, if no IDs are provided, this +// function does nothing. +func (b *ACLBuilder) TransactionalIDs(x ...string) *ACLBuilder { + b.txnIDs = x + if len(x) == 0 { + b.anyTxn = true + } + return b +} + +// MaybeTransactionalIDs is the same as TransactionalIDs, but does not match +// all transactional ID's if none are provided. +func (b *ACLBuilder) MaybeTransactionalIDs(x ...string) *ACLBuilder { b.txnIDs = x; return b } + +// DelegationTokens lists/deletes/creates ACLs of resource type +// "delegation_token" for the given delegation tokens. +// +// This returns the input pointer. +// +// For listing or deleting, if this is provided no tokens, all +// "delegation_token" resource type ACLs are matched. For creating, if no +// tokens are provided, this function does nothing. +func (b *ACLBuilder) DelegationTokens(t ...string) *ACLBuilder { + b.tokens = t + if len(t) == 0 { + b.anyToken = true + } + return b +} + +// MaybeDelegationTokens is the same as DelegationTokens, but does not match +// all tokens if none are provided. +func (b *ACLBuilder) MaybeDelegationTokens(t ...string) *ACLBuilder { b.tokens = t; return b } + +// Allow sets the principals to add allow permissions for. For listing and +// deleting, you must also use AllowHosts. +// +// This returns the input pointer. +// +// For creating, if this is not paired with AllowHosts, the user will have +// access to all hosts (the wildcard *). +// +// For listing & deleting, if the principals are empty, this matches any user. +func (b *ACLBuilder) Allow(principals ...string) *ACLBuilder { + b.allow = principals + if len(principals) == 0 { + b.anyAllow = true + } + return b +} + +// MaybeAllow is the same as Allow, but does not match all allowed principals +// if none are provided. +func (b *ACLBuilder) MaybeAllow(principals ...string) *ACLBuilder { b.allow = principals; return b } + +// AllowHosts sets the hosts to add allow permissions for. If using this, you +// must also use Allow. +// +// This returns the input pointer. +// +// For creating, if this is empty, the user will have access to all hosts (the +// wildcard *) and this function is actually not necessary. +// +// For listing & deleting, if the hosts are empty, this matches any host. +func (b *ACLBuilder) AllowHosts(hosts ...string) *ACLBuilder { + b.allowHosts = hosts + if len(hosts) == 0 { + b.anyAllowHosts = true + } + return b +} + +// MaybeAllowHosts is the same as AllowHosts, but does not match all allowed +// hosts if none are provided. +func (b *ACLBuilder) MaybeAllowHosts(hosts ...string) *ACLBuilder { b.allowHosts = hosts; return b } + +// Deny sets the principals to add deny permissions for. For listing and +// deleting, you must also use DenyHosts. +// +// This returns the input pointer. +// +// For creating, if this is not paired with DenyHosts, the user will be denied +// access to all hosts (the wildcard *). +// +// For listing & deleting, if the principals are empty, this matches any user. +func (b *ACLBuilder) Deny(principals ...string) *ACLBuilder { + b.deny = principals + if len(principals) == 0 { + b.anyDeny = true + } + return b +} + +// MaybeDeny is the same as Deny, but does not match all denied principals if +// none are provided. +func (b *ACLBuilder) MaybeDeny(principals ...string) *ACLBuilder { b.deny = principals; return b } + +// DenyHosts sets the hosts to add deny permissions for. If using this, you +// must also use Deny. +// +// This returns the input pointer. +// +// For creating, if this is empty, the user will be denied access to all hosts +// (the wildcard *) and this function is actually not necessary. +// +// For listing & deleting, if the hosts are empty, this matches any host. +func (b *ACLBuilder) DenyHosts(hosts ...string) *ACLBuilder { + b.denyHosts = hosts + if len(hosts) == 0 { + b.anyDenyHosts = true + } + return b +} + +// MaybeDenyHosts is the same as DenyHosts, but does not match all denied +// hosts if none are provided. +func (b *ACLBuilder) MaybeDenyHosts(hosts ...string) *ACLBuilder { b.denyHosts = hosts; return b } + +// ACLOperation is a type alias for kmsg.ACLOperation, which is an enum +// containing all Kafka ACL operations and has helper functions. +// +// Kafka requests require the following operations (broker <=> broker ACLs +// elided): +// +// PRODUCING/CONSUMING +// =================== +// Produce WRITE on TOPIC for topics +// WRITE on TRANSACTIONAL_ID for txn id (if transactionally producing) +// +// Fetch READ on TOPIC for topics +// +// ListOffsets DESCRIBE on TOPIC for topics +// +// Metadata DESCRIBE on TOPIC for topics +// CREATE on CLUSTER for kafka-cluster (if automatically creating new topics) +// CREATE on TOPIC for topics (if automatically creating new topics) +// +// OffsetForLeaderEpoch DESCRIBE on TOPIC for topics +// +// GROUPS +// ====== +// FindCoordinator DESCRIBE on GROUP for group (if finding group coordinator) +// DESCRIBE on TRANSACTIONAL_ID for id (if finding transactiona coordinator) +// +// OffsetCommit READ on GROUP for group +// READ on TOPIC for topics +// +// OffsetFetch DESCRIBE on GROUP for group +// DESCRIBE on TOPIC for topics +// +// OffsetDelete DELETE on GROUP For group +// READ on TOPIC for topics +// +// JoinGroup READ on GROUP for group +// Heartbeat READ on GROUP for group +// LeaveGroup READ on GROUP for group +// SyncGroup READ on GROUP for group +// +// DescribeGroup DESCRIBE on GROUP for groups +// +// ListGroups DESCRIBE on GROUP for groups +// or, DESCRIBE on CLUSTER for kafka-cluster +// +// DeleteGroups DELETE on GROUP for groups +// +// TRANSACTIONS (including FindCoordinator above) +// ============ +// InitProducerID WRITE on TRANSACTIONAL_ID for id, if using transactions +// or, IDEMPOTENT_WRITE on CLUSTER for kafka-cluster, if pre Kafka 3.0 +// or, WRITE on TOPIC for any topic, if Kafka 3.0+ +// +// AddPartitionsToTxn WRITE on TRANSACTIONAL_ID for id +// WRITE on TOPIC for topics +// +// AddOffsetsToTxn WRITE on TRANSACTIONAL_ID for id +// READ on GROUP for group +// +// EndTxn WRITE on TRANSACTIONAL_ID for id +// +// TxnOffsetCommit WRITE on TRANSACTIONAL_ID for id +// READ on GROUP for group +// READ on TOPIC for topics +// +// TOPIC ADMIN +// =========== +// CreateTopics CREATE on CLUSTER for kafka-cluster +// CREATE on TOPIC for topics +// DESCRIBE_CONFIGS on TOPIC for topics, for returning topic configs on create +// +// CreatePartitions ALTER on TOPIC for topics +// +// DeleteTopics DELETE on TOPIC for topics +// DESCRIBE on TOPIC for topics, if deleting by topic id (in addition to prior ACL) +// +// DeleteRecords DELETE on TOPIC for topics +// +// CONFIG ADMIN +// ============ +// DescribeConfigs DESCRIBE_CONFIGS on CLUSTER for kafka-cluster, for broker or broker-logger describing +// DESCRIBE_CONFIGS on TOPIC for topics, for topic describing +// +// AlterConfigs ALTER_CONFIGS on CLUSTER for kafka-cluster, for broker altering +// ALTER_CONFIGS on TOPIC for topics, for topic altering +// +// IncrementalAlterConfigs ALTER_CONFIGS on CLUSTER for kafka-cluster, for broker or broker-logger altering +// ALTER_CONFIGS on TOPIC for topics, for topic altering +// +// +// MISC ADMIN +// ========== +// AlterReplicaLogDirs ALTER on CLUSTER for kafka-cluster +// DescribeLogDirs DESCRIBE on CLUSTER for kafka-cluster +// +// AlterPartitionAssignments ALTER on CLUSTER for kafka-cluster +// ListPartitionReassignments DESCRIBE on CLUSTER for kafka-cluster +// +// DescribeDelegationTokens DESCRIBE on DELEGATION_TOKEN for id +// +// ElectLeaders ALTER on CLUSTER for kafka-cluster +// +// DescribeClientQuotas DESCRIBE_CONFIGS on CLUSTER for kafka-cluster +// AlterClientQuotas ALTER_CONFIGS on CLUSTER for kafka-cluster +// +// DescribeUserScramCredentials DESCRIBE on CLUSTER for kafka-cluster +// AlterUserScramCredentials ALTER on CLUSTER for kafka-cluster +// +// UpdateFeatures ALTER on CLUSTER for kafka-cluster +// +// DescribeCluster DESCRIBE on CLUSTER for kafka-cluster +// +// DescribeProducerIDs READ on TOPIC for topics +// DescribeTransactions DESCRIBE on TRANSACTIONAL_ID for ids +// DESCRIBE on TOPIC for topics +// ListTransactions DESCRIBE on TRANSACTIONAL_ID for ids +type ACLOperation = kmsg.ACLOperation + +const ( + // OpUnknown is returned for unknown operations. + OpUnknown ACLOperation = kmsg.ACLOperationUnknown + + // OpAny, used for listing and deleting, matches any operation. + OpAny ACLOperation = kmsg.ACLOperationAny + + // OpAll is a shortcut for allowing / denying all operations. + OpAll ACLOperation = kmsg.ACLOperationAll + + // OpRead is the READ operation. + OpRead ACLOperation = kmsg.ACLOperationRead + + // OpWrite is the WRITE operation. + OpWrite ACLOperation = kmsg.ACLOperationWrite + + // OpCreate is the CREATE operation. + OpCreate ACLOperation = kmsg.ACLOperationCreate + + // OpDelete is the DELETE operation. + OpDelete ACLOperation = kmsg.ACLOperationDelete + + // OpAlter is the ALTER operation. + OpAlter ACLOperation = kmsg.ACLOperationAlter + + // OpDescribe is the DESCRIBE operation. + OpDescribe ACLOperation = kmsg.ACLOperationDescribe + + // OpClusterAction is the CLUSTER_ACTION operation. This operation is + // used for any broker<=>broker communication and is not needed by + // clients. + OpClusterAction ACLOperation = kmsg.ACLOperationClusterAction + + // OpDescribeConfigs is the DESCRIBE_CONFIGS operation. + OpDescribeConfigs ACLOperation = kmsg.ACLOperationDescribeConfigs + + // OpAlterConfigs is the ALTER_CONFIGS operation. + OpAlterConfigs ACLOperation = kmsg.ACLOperationAlterConfigs + + // OpIdempotentWrite is the IDEMPOTENT_WRITE operation. As of Kafka + // 3.0+, this has been deprecated and replaced by the ability to WRITE + // on any topic. + OpIdempotentWrite ACLOperation = kmsg.ACLOperationIdempotentWrite +) + +// Operations sets operations to allow or deny. Passing no operations defaults +// to OpAny. +// +// This returns the input pointer. +// +// For creating, OpAny returns an error, for it is strictly used for filters +// (listing & deleting). +func (b *ACLBuilder) Operations(operations ...ACLOperation) *ACLBuilder { + b.ops = operations + if len(operations) == 0 { + b.ops = []ACLOperation{OpAny} + } + return b +} + +// MaybeOperations is the same as Operations, but does not match all operations +// if none are provided. +func (b *ACLBuilder) MaybeOperations(operations ...ACLOperation) *ACLBuilder { + if len(operations) > 0 { + b.Operations(operations...) + } + return b +} + +// ACLPattern is a type alias for kmsg.ACLResourcePatternType, which is an enum +// containing all Kafka ACL resource pattern options. +// +// Creating/listing/deleting ACLs works on a resource name basis: every ACL +// created has a name, and every ACL filtered for listing / deleting matches by +// name. The name by default is "literal", meaning created ACLs will have the +// exact name, and matched ACLs must match completely. +// +// Prefixed names allow for creating an ACL that matches any prefix: principals +// foo-bar and foo-baz both have the prefix "foo-", meaning a READ on TOPIC for +// User:foo- with prefix pattern will allow both of those principals to read +// the topic. +// +// Any and match are used for listing and deleting. Any will match any name, be +// it literal or prefix or a wildcard name. There is no need for specifying +// topics, groups, etc. when using any resource pattern. +// +// Alternatively, match requires a name, but it matches any literal name (exact +// match), any prefix, and any wildcard. +type ACLPattern = kmsg.ACLResourcePatternType + +const ( + // ACLPatternUnknown is returned for unknown patterns. + ACLPatternUnknown ACLPattern = kmsg.ACLResourcePatternTypeUnknown + + // ACLPatternAny is the ANY resource pattern. + ACLPatternAny ACLPattern = kmsg.ACLResourcePatternTypeAny + + // ACLPatternMatch is the MATCH resource pattern. + ACLPatternMatch ACLPattern = kmsg.ACLResourcePatternTypeMatch + + // ACLPatternLiteral is the LITERAL resource pattern, the default. + ACLPatternLiteral ACLPattern = kmsg.ACLResourcePatternTypeLiteral + + // ACLPatternPrefixed is the PREFIXED resource pattern. + ACLPatternPrefixed ACLPattern = kmsg.ACLResourcePatternTypePrefixed +) + +// ResourcePatternType sets the pattern type to use when creating or filtering +// ACL resource names, overriding the default of LITERAL. +// +// This returns the input pointer. +// +// For creating, only LITERAL and PREFIXED are supported. +func (b *ACLBuilder) ResourcePatternType(pattern ACLPattern) *ACLBuilder { + b.pattern = pattern + return b +} + +// ValidateCreate returns an error if the builder is invalid for creating ACLs. +func (b *ACLBuilder) ValidateCreate() error { + for _, op := range b.ops { + switch op { + case OpAny, OpUnknown: + return fmt.Errorf("invalid operation %s for creating ACLs", op) + } + } + + switch b.pattern { + case ACLPatternLiteral, ACLPatternPrefixed: + default: + return fmt.Errorf("invalid acl resource pattern %s for creating ACLs", b.pattern) + } + + if len(b.allowHosts) != 0 && len(b.allow) == 0 { + return fmt.Errorf("invalid allow hosts with no allow principals") + } + if len(b.denyHosts) != 0 && len(b.deny) == 0 { + return fmt.Errorf("invalid deny hosts with no deny principals") + } + return nil +} + +// ValidateDelete is an alias for ValidateFilter. +func (b *ACLBuilder) ValidateDelete() error { return b.ValidateFilter() } + +// ValidateDescribe is an alias for ValidateFilter. +func (b *ACLBuilder) ValidateDescribe() error { return b.ValidateFilter() } + +// ValidateFilter returns an error if the builder is invalid for deleting or +// describing ACLs (which both operate on a filter basis). +func (b *ACLBuilder) ValidateFilter() error { + if len(b.allowHosts) != 0 && len(b.allow) == 0 && !b.anyAllow { + return fmt.Errorf("invalid allow hosts with no allow principals") + } + if len(b.allow) != 0 && len(b.allowHosts) == 0 && !b.anyAllowHosts { + return fmt.Errorf("invalid allow principals with no allow hosts") + } + if len(b.denyHosts) != 0 && len(b.deny) == 0 && !b.anyDeny { + return fmt.Errorf("invalid deny hosts with no deny principals") + } + if len(b.deny) != 0 && len(b.denyHosts) == 0 && !b.anyDenyHosts { + return fmt.Errorf("invalid deny principals with no deny hosts") + } + return nil +} + +// HasAnyFilter returns whether any field in this builder is opted into "any", +// meaning a wide glob. This would be if you used Topics with no topics, and so +// on. This function can be used to detect if you accidentally opted into a +// non-specific ACL. +// +// The evaluated fields are: resources, principals/hosts, a single OpAny +// operation, and an Any pattern. +func (b *ACLBuilder) HasAnyFilter() bool { + return b.anyResource || + b.anyTopic || + b.anyGroup || + b.anyTxn || + b.anyToken || + b.anyAllow || + b.anyAllowHosts || + b.anyDeny || + b.anyDenyHosts || + b.hasOpAny() || + b.pattern == ACLPatternAny +} + +func (b *ACLBuilder) hasOpAny() bool { + for _, op := range b.ops { + if op == OpAny { + return true + } + } + return false +} + +// HasResource returns true if the builder has a non-empty resource (topic, +// group, ...), or if any resource has "any" set to true. +func (b *ACLBuilder) HasResource() bool { + l := len(b.any) + + len(b.topics) + + len(b.groups) + + len(b.txnIDs) + + len(b.tokens) + return l > 0 || + b.anyResource || + b.anyTopic || + b.anyGroup || + b.anyCluster || + b.anyTxn || + b.anyToken +} + +// HasPrincipals returns if any allow or deny principals have been set, or if +// their "any" field is true. +func (b *ACLBuilder) HasPrincipals() bool { + return len(b.allow) > 0 || + b.anyAllow || + len(b.deny) > 0 || + b.anyDeny +} + +// HasHosts returns if any allow or deny hosts have been set, or if their "any" +// field is true. +func (b *ACLBuilder) HasHosts() bool { + return len(b.allowHosts) > 0 || + b.anyAllowHosts || + len(b.denyHosts) > 0 || + b.anyDenyHosts +} + +func (b *ACLBuilder) dup() *ACLBuilder { // shallow copy + d := *b + return &d +} + +// CreateACLsResult is a result for an individual ACL creation. +type CreateACLsResult struct { + Principal string + Host string + + Type kmsg.ACLResourceType // Type is the type of resource this is. + Name string // Name is the name of the resource allowed / denied. + Pattern ACLPattern // Pattern is the name pattern. + Operation ACLOperation // Operation is the operation allowed / denied. + Permission kmsg.ACLPermissionType // Permission is whether this is allowed / denied. + + Err error // Err is the error for this ACL creation. + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// CreateACLsResults contains all results to created ACLs. +type CreateACLsResults []CreateACLsResult + +// CreateACLs creates a batch of ACLs using the ACL builder, validating the +// input before issuing the CreateACLs request. +// +// If the input is invalid, or if the response fails, or if the response does +// not contain as many ACLs as we issued in our create request, this returns an +// error. +func (cl *Client) CreateACLs(ctx context.Context, b *ACLBuilder) (CreateACLsResults, error) { + if err := b.ValidateCreate(); err != nil { + return nil, err + } + if len(b.allow) != 0 && len(b.allowHosts) == 0 { + b.allowHosts = []string{"*"} + } + if len(b.deny) != 0 && len(b.denyHosts) == 0 { + b.denyHosts = []string{"*"} + } + + var clusters []string + if b.anyCluster { + clusters = []string{"kafka-cluster"} + } + + req := kmsg.NewPtrCreateACLsRequest() + for _, typeNames := range []struct { + t kmsg.ACLResourceType + names []string + }{ + {kmsg.ACLResourceTypeTopic, b.topics}, + {kmsg.ACLResourceTypeGroup, b.groups}, + {kmsg.ACLResourceTypeCluster, clusters}, + {kmsg.ACLResourceTypeTransactionalId, b.txnIDs}, + {kmsg.ACLResourceTypeDelegationToken, b.tokens}, + } { + for _, name := range typeNames.names { + for _, op := range b.ops { + for _, perm := range []struct { + principals []string + hosts []string + permType kmsg.ACLPermissionType + }{ + {b.allow, b.allowHosts, kmsg.ACLPermissionTypeAllow}, + {b.deny, b.denyHosts, kmsg.ACLPermissionTypeDeny}, + } { + for _, principal := range perm.principals { + for _, host := range perm.hosts { + c := kmsg.NewCreateACLsRequestCreation() + c.ResourceType = typeNames.t + c.ResourceName = name + c.ResourcePatternType = b.pattern + c.Operation = op + c.Principal = principal + c.Host = host + c.PermissionType = perm.permType + req.Creations = append(req.Creations, c) + } + } + } + } + } + } + + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + + if len(resp.Results) != len(req.Creations) { + return nil, fmt.Errorf("received %d results to %d creations", len(resp.Results), len(req.Creations)) + } + + var rs CreateACLsResults + for i, r := range resp.Results { + c := &req.Creations[i] + rs = append(rs, CreateACLsResult{ + Principal: c.Principal, + Host: c.Host, + + Type: c.ResourceType, + Name: c.ResourceName, + Pattern: c.ResourcePatternType, + Operation: c.Operation, + Permission: c.PermissionType, + + Err: kerr.ErrorForCode(r.ErrorCode), + ErrMessage: unptrStr(r.ErrorMessage), + }) + } + + return rs, nil +} + +// DeletedACL an ACL that was deleted. +type DeletedACL struct { + Principal string // Principal is this deleted ACL's principal. + Host string // Host is this deleted ACL's host. + + Type kmsg.ACLResourceType // Type is this deleted ACL's resource type. + Name string // Name is this deleted ACL's resource name. + Pattern ACLPattern // Pattern is this deleted ACL's resource name pattern. + Operation ACLOperation // Operation is this deleted ACL's operation. + Permission kmsg.ACLPermissionType // Permission this deleted ACLs permission. + + Err error // Err is non-nil if this match has an error. + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// DeletedACLs contains ACLs that were deleted from a single delete filter. +type DeletedACLs []DeletedACL + +// DeleteACLsResult contains the input used for a delete ACL filter, and the +// deletes that the filter matched or the error for this filter. +// +// All fields but Deleted and Err are set from the request input. The response +// sets either Deleted (potentially to nothing if the filter matched nothing) +// or Err. +type DeleteACLsResult struct { + Principal *string // Principal is the optional user that was used in this filter. + Host *string // Host is the optional host that was used in this filter. + + Type kmsg.ACLResourceType // Type is the type of resource used for this filter. + Name *string // Name is the name of the resource used for this filter. + Pattern ACLPattern // Pattern is the name pattern used for this filter. + Operation ACLOperation // Operation is the operation used for this filter. + Permission kmsg.ACLPermissionType // Permission is permission used for this filter. + + Deleted DeletedACLs // Deleted contains all ACLs this delete filter matched. + + Err error // Err is non-nil if this filter has an error. + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// DeleteACLsResults contains all results to deleted ACLs. +type DeleteACLsResults []DeleteACLsResult + +// DeleteACLs deletes a batch of ACLs using the ACL builder, validating the +// input before issuing the DeleteACLs request. +// +// If the input is invalid, or if the response fails, or if the response does +// not contain as many ACL results as we issued in our delete request, this +// returns an error. +// +// Deleting ACLs works on a filter basis: a single filter can match many ACLs. +// For example, deleting with operation ANY matches any operation. For safety / +// verification purposes, you an DescribeACLs with the same builder first to +// see what would be deleted. +func (cl *Client) DeleteACLs(ctx context.Context, b *ACLBuilder) (DeleteACLsResults, error) { + dels, _, err := createDelDescACL(b) + if err != nil { + return nil, err + } + + req := kmsg.NewPtrDeleteACLsRequest() + req.Filters = dels + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + if len(resp.Results) != len(req.Filters) { + return nil, fmt.Errorf("received %d results to %d filters", len(resp.Results), len(req.Filters)) + } + + var rs DeleteACLsResults + for i, r := range resp.Results { + f := &req.Filters[i] + var ms DeletedACLs + for _, m := range r.MatchingACLs { + ms = append(ms, DeletedACL{ + Principal: m.Principal, + Host: m.Host, + Type: m.ResourceType, + Name: m.ResourceName, + Pattern: m.ResourcePatternType, + Operation: m.Operation, + Permission: m.PermissionType, + Err: kerr.ErrorForCode(m.ErrorCode), + ErrMessage: unptrStr(m.ErrorMessage), + }) + } + rs = append(rs, DeleteACLsResult{ + Principal: f.Principal, + Host: f.Host, + Type: f.ResourceType, + Name: f.ResourceName, + Pattern: f.ResourcePatternType, + Operation: f.Operation, + Permission: f.PermissionType, + Deleted: ms, + Err: kerr.ErrorForCode(r.ErrorCode), + ErrMessage: unptrStr(r.ErrorMessage), + }) + } + return rs, nil +} + +// DescribedACL is an ACL that was described. +type DescribedACL struct { + Principal string // Principal is this described ACL's principal. + Host string // Host is this described ACL's host. + + Type kmsg.ACLResourceType // Type is this described ACL's resource type. + Name string // Name is this described ACL's resource name. + Pattern ACLPattern // Pattern is this described ACL's resource name pattern. + Operation ACLOperation // Operation is this described ACL's operation. + Permission kmsg.ACLPermissionType // Permission this described ACLs permission. +} + +// DescribedACLs contains ACLs that were described from a single describe +// filter. +type DescribedACLs []DescribedACL + +// DescribeACLsResults contains the input used for a describe ACL filter, and +// the describes that the filter matched or the error for this filter. +// +// All fields but Described and Err are set from the request input. The +// response sets either Described (potentially to nothing if the filter matched +// nothing) or Err. +type DescribeACLsResult struct { + Principal *string // Principal is the optional user that was used in this filter. + Host *string // Host is the optional host that was used in this filter. + + Type kmsg.ACLResourceType // Type is the type of resource used for this filter. + Name *string // Name is the name of the resource used for this filter. + Pattern ACLPattern // Pattern is the name pattern used for this filter. + Operation ACLOperation // Operation is the operation used for this filter. + Permission kmsg.ACLPermissionType // Permission is permission used for this filter. + + Described DescribedACLs // Described contains all ACLs this describe filter matched. + + Err error // Err is non-nil if this filter has an error. + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// DescribeACLsResults contains all results to described ACLs. +type DescribeACLsResults []DescribeACLsResult + +// DescribeACLs describes a batch of ACLs using the ACL builder, validating the +// input before issuing DescribeACLs requests. +// +// If the input is invalid, or if any response fails, this returns an error. +// +// Listing ACLs works on a filter basis: a single filter can match many ACLs. +// For example, describing with operation ANY matches any operation. Under the +// hood, this method issues one describe request per filter, because describing +// ACLs does not work on a batch basis (unlike creating & deleting). The return +// of this function can be used to see what would be deleted given the same +// builder input. +func (cl *Client) DescribeACLs(ctx context.Context, b *ACLBuilder) (DescribeACLsResults, error) { + _, descs, err := createDelDescACL(b) + if err != nil { + return nil, err + } + + var ( + ictx, cancel = context.WithCancel(ctx) + mu sync.Mutex + wg sync.WaitGroup + firstErr error + resps = make([]*kmsg.DescribeACLsResponse, len(descs)) + ) + defer cancel() + for i := range descs { + req := descs[i] // each req is unique per loop, we are not reusing req, this is safe + myIdx := i + wg.Add(1) + go func() { + defer wg.Done() + resp, err := req.RequestWith(ictx, cl.cl) + resps[myIdx] = resp + if err == nil { + return + } + cancel() + mu.Lock() + defer mu.Unlock() + if firstErr == nil { // keep the first err + firstErr = err + } + }() + } + wg.Wait() + if firstErr != nil { + return nil, firstErr + } + + var rs DescribeACLsResults + for i, r := range resps { + f := descs[i] + var ds DescribedACLs + for _, resource := range r.Resources { + for _, acl := range resource.ACLs { + ds = append(ds, DescribedACL{ + Principal: acl.Principal, + Host: acl.Host, + Type: resource.ResourceType, + Name: resource.ResourceName, + Pattern: resource.ResourcePatternType, + Operation: acl.Operation, + Permission: acl.PermissionType, + }) + } + } + rs = append(rs, DescribeACLsResult{ + Principal: f.Principal, + Host: f.Host, + Type: f.ResourceType, + Name: f.ResourceName, + Pattern: f.ResourcePatternType, + Operation: f.Operation, + Permission: f.PermissionType, + Described: ds, + Err: kerr.ErrorForCode(r.ErrorCode), + ErrMessage: unptrStr(r.ErrorMessage), + }) + } + return rs, nil +} + +var sliceAny = []string{"any"} + +func createDelDescACL(b *ACLBuilder) ([]kmsg.DeleteACLsRequestFilter, []*kmsg.DescribeACLsRequest, error) { + if err := b.ValidateFilter(); err != nil { + return nil, nil, err + } + + // As a special shortcut, if we have any allow and deny principals and + // hosts, we collapse these into one "any" group. The anyAny and + // anyAnyHosts vars are used in our looping below, and if we do this, + // we dup and set all the relevant fields to false to not expand them + // in our loops. + var anyAny, anyAnyHosts bool + if b.anyAllow && b.anyDeny && b.anyAllowHosts && b.anyDenyHosts { + anyAny = true + anyAnyHosts = true + + b = b.dup() + b.allow = nil + b.allowHosts = nil + b.deny = nil + b.denyHosts = nil + b.anyAllow = false + b.anyAllowHosts = false + b.anyDeny = false + b.anyDenyHosts = false + } + + var clusters []string + if b.anyCluster { + clusters = []string{"kafka-cluster"} + } + var deletions []kmsg.DeleteACLsRequestFilter + var describes []*kmsg.DescribeACLsRequest + for _, typeNames := range []struct { + t kmsg.ACLResourceType + names []string + any bool + }{ + {kmsg.ACLResourceTypeAny, b.any, b.anyResource}, + {kmsg.ACLResourceTypeTopic, b.topics, b.anyTopic}, + {kmsg.ACLResourceTypeGroup, b.groups, b.anyGroup}, + {kmsg.ACLResourceTypeCluster, clusters, b.anyCluster}, + {kmsg.ACLResourceTypeTransactionalId, b.txnIDs, b.anyTxn}, + {kmsg.ACLResourceTypeDelegationToken, b.tokens, b.anyToken}, + } { + if typeNames.any { + typeNames.names = sliceAny + } + for _, name := range typeNames.names { + for _, op := range b.ops { + for _, perm := range []struct { + principals []string + anyPrincipal bool + hosts []string + anyHost bool + permType kmsg.ACLPermissionType + }{ + { + b.allow, + b.anyAllow, + b.allowHosts, + b.anyAllowHosts, + kmsg.ACLPermissionTypeAllow, + }, + { + b.deny, + b.anyDeny, + b.denyHosts, + b.anyDenyHosts, + kmsg.ACLPermissionTypeDeny, + }, + { + nil, + anyAny, + nil, + anyAnyHosts, + kmsg.ACLPermissionTypeAny, + }, + } { + if perm.anyPrincipal { + perm.principals = sliceAny + } + if perm.anyHost { + perm.hosts = sliceAny + } + for _, principal := range perm.principals { + for _, host := range perm.hosts { + deletion := kmsg.NewDeleteACLsRequestFilter() + describe := kmsg.NewPtrDescribeACLsRequest() + + deletion.ResourceType = typeNames.t + describe.ResourceType = typeNames.t + + if !typeNames.any { + deletion.ResourceName = kmsg.StringPtr(name) + describe.ResourceName = kmsg.StringPtr(name) + } + + deletion.ResourcePatternType = b.pattern + describe.ResourcePatternType = b.pattern + + deletion.Operation = op + describe.Operation = op + + if !perm.anyPrincipal { + deletion.Principal = kmsg.StringPtr(principal) + describe.Principal = kmsg.StringPtr(principal) + } + + if !perm.anyHost { + deletion.Host = kmsg.StringPtr(host) + describe.Host = kmsg.StringPtr(host) + } + + deletion.PermissionType = perm.permType + describe.PermissionType = perm.permType + + deletions = append(deletions, deletion) + describes = append(describes, describe) + } + } + } + } + } + } + return deletions, describes, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/configs.go b/vendor/github.com/twmb/franz-go/pkg/kadm/configs.go new file mode 100644 index 000000000000..e6e37245f7a6 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/configs.go @@ -0,0 +1,417 @@ +package kadm + +import ( + "context" + "strconv" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// ConfigSynonym is a fallback value for a config. +type ConfigSynonym struct { + Key string // Key is the fallback config name. + Value *string // Value is the fallback config value, if any (sensitive is elided). + Source kmsg.ConfigSource // Source is where this config synonym is defined from. +} + +// Config is a configuration for a resource (topic, broker) +type Config struct { + Key string // Key is the config name. + Value *string // Value is the config value, if any. + Sensitive bool // Sensitive is if this config is sensitive (if so, Value is nil). + Source kmsg.ConfigSource // Source is where this config is defined from. + + // Synonyms contains fallback key/value pairs for this same + // configuration key in order or preference. That is, if a config entry + // is both dynamically defined and has a default value as well, the top + // level config will be the dynamic value, while the synonym will be + // the default. + Synonyms []ConfigSynonym +} + +// MaybeValue returns the config's value if it is non-nil, otherwise an empty +// string. +func (c *Config) MaybeValue() string { + if c.Value != nil { + return *c.Value + } + return "" +} + +// ResourceConfig contains the configuration values for a resource (topic, +// broker, broker logger). +type ResourceConfig struct { + Name string // Name is the name of this resource. + Configs []Config // Configs are the configs for this topic. + Err error // Err is any error preventing configs from loading (likely, an unknown topic). + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// ResourceConfigs contains the configuration values for many resources. +type ResourceConfigs []ResourceConfig + +// On calls fn for the response config if it exists, returning the config and +// the error returned from fn. If fn is nil, this simply returns the config. +// +// The fn is given a copy of the config. This function returns the copy as +// well; any modifications within fn are modifications on the returned copy. +// +// If the resource does not exist, this returns kerr.UnknownTopicOrPartition. +func (rs ResourceConfigs) On(name string, fn func(*ResourceConfig) error) (ResourceConfig, error) { + for _, r := range rs { + if r.Name == name { + if fn == nil { + return r, nil + } + return r, fn(&r) + } + } + return ResourceConfig{}, kerr.UnknownTopicOrPartition +} + +// DescribeTopicConfigs returns the configuration for the requested topics. +// +// This may return *ShardErrors. +func (cl *Client) DescribeTopicConfigs( + ctx context.Context, + topics ...string, +) (ResourceConfigs, error) { + if len(topics) == 0 { + return nil, nil + } + return cl.describeConfigs(ctx, kmsg.ConfigResourceTypeTopic, topics) +} + +// DescribeBrokerConfigs returns configuration for the requested brokers. If no +// brokers are requested, a single request is issued and any broker in the +// cluster replies with the cluster-level dynamic config values. +// +// This may return *ShardErrors. +func (cl *Client) DescribeBrokerConfigs( + ctx context.Context, + brokers ...int32, +) (ResourceConfigs, error) { + var names []string + if len(brokers) == 0 { + names = append(names, "") + } + for _, b := range brokers { + names = append(names, strconv.Itoa(int(b))) + } + return cl.describeConfigs(ctx, kmsg.ConfigResourceTypeBroker, names) +} + +func (cl *Client) describeConfigs( + ctx context.Context, + kind kmsg.ConfigResourceType, + names []string, +) (ResourceConfigs, error) { + req := kmsg.NewPtrDescribeConfigsRequest() + req.IncludeSynonyms = true + for _, name := range names { + rr := kmsg.NewDescribeConfigsRequestResource() + rr.ResourceName = name + rr.ResourceType = kind + req.Resources = append(req.Resources, rr) + } + shards := cl.cl.RequestSharded(ctx, req) + + var configs []ResourceConfig + return configs, shardErrEach(req, shards, func(kr kmsg.Response) error { + resp := kr.(*kmsg.DescribeConfigsResponse) + for _, r := range resp.Resources { + if err := maybeAuthErr(r.ErrorCode); err != nil { + return err + } + rc := ResourceConfig{ + Name: r.ResourceName, + Err: kerr.ErrorForCode(r.ErrorCode), + ErrMessage: unptrStr(r.ErrorMessage), + } + for _, c := range r.Configs { + rcv := Config{ + Key: c.Name, + Value: c.Value, + Sensitive: c.IsSensitive, + Source: c.Source, + } + for _, syn := range c.ConfigSynonyms { + rcv.Synonyms = append(rcv.Synonyms, ConfigSynonym{ + Key: syn.Name, + Value: syn.Value, + Source: syn.Source, + }) + } + rc.Configs = append(rc.Configs, rcv) + } + configs = append(configs, rc) // we are not storing in a map, no existence-check possible + } + return nil + }) +} + +// IncrementalOp is a typed int8 that is used for incrementally updating +// configuration keys for topics and brokers. +type IncrementalOp int8 + +const ( + // SetConfig is an incremental operation to set an individual config + // key. + SetConfig IncrementalOp = iota + + // DeleteConfig is an incremental operation to delete an individual + // config key. + DeleteConfig + + // AppendConfig is an incremental operation to append a value to a + // config key that is a list type. + AppendConfig + + // SubtractConfig is an incremental operation to remove a value from a + // config key that is a list type. + SubtractConfig +) + +// AlterConfig is an individual key/value operation to perform when altering +// configs. +// +// This package includes a StringPtr function to aid in building config values. +type AlterConfig struct { + Op IncrementalOp // Op is the incremental alter operation to perform. This is ignored for State alter functions. + Name string // Name is the name of the config to alter. + Value *string // Value is the value to use when altering, if any. +} + +// AlteredConfigsResponse contains the response for an individual alteration. +type AlterConfigsResponse struct { + Name string // Name is the name of this resource (topic name or broker number). + Err error // Err is non-nil if the config could not be altered. + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// AlterConfigsResponses contains responses for many alterations. +type AlterConfigsResponses []AlterConfigsResponse + +// On calls fn for the response name if it exists, returning the response and +// the error returned from fn. If fn is nil, this simply returns the response. +// +// The fn is given a copy of the response. This function returns the copy as +// well; any modifications within fn are modifications on the returned copy. +// +// If the resource does not exist, this returns kerr.UnknownTopicOrPartition. +func (rs AlterConfigsResponses) On(name string, fn func(*AlterConfigsResponse) error) (AlterConfigsResponse, error) { + for _, r := range rs { + if r.Name == name { + if fn == nil { + return r, nil + } + return r, fn(&r) + } + } + return AlterConfigsResponse{}, kerr.UnknownTopicOrPartition +} + +// AlterTopicConfigs incrementally alters topic configuration values. +// +// This method requires talking to a cluster that supports +// IncrementalAlterConfigs (officially introduced in Kafka v2.3, but many +// broker reimplementations support this request even if they do not support +// all other requests from Kafka v2.3). +// +// If you want to alter the entire configs state using the older AlterConfigs +// request, use AlterTopicConfigsState. +// +// This may return *ShardErrors. You may consider checking +// ValidateAlterTopicConfigs before using this method. +func (cl *Client) AlterTopicConfigs(ctx context.Context, configs []AlterConfig, topics ...string) (AlterConfigsResponses, error) { + return cl.alterConfigs(ctx, false, configs, kmsg.ConfigResourceTypeTopic, topics) +} + +// ValidateAlterTopicConfigs validates an incremental alter config for the given +// topics. +// +// This returns exactly what AlterTopicConfigs returns, but does not actually +// alter configurations. +func (cl *Client) ValidateAlterTopicConfigs(ctx context.Context, configs []AlterConfig, topics ...string) (AlterConfigsResponses, error) { + return cl.alterConfigs(ctx, true, configs, kmsg.ConfigResourceTypeTopic, topics) +} + +// AlterBrokerConfigs incrementally alters broker configuration values. If +// brokers are specified, this updates each specific broker. If no brokers are +// specified, this updates whole-cluster broker configuration values. +// +// This method requires talking to a cluster that supports +// IncrementalAlterConfigs (officially introduced in Kafka v2.3, but many +// broker reimplementations support this request even if they do not support +// all other requests from Kafka v2.3). +// +// If you want to alter the entire configs state using the older AlterConfigs +// request, use AlterBrokerConfigsState. +// +// This may return *ShardErrors. You may consider checking +// ValidateAlterBrokerConfigs before using this method. +func (cl *Client) AlterBrokerConfigs(ctx context.Context, configs []AlterConfig, brokers ...int32) (AlterConfigsResponses, error) { + var names []string + if len(brokers) == 0 { + names = append(names, "") + } + for _, broker := range brokers { + names = append(names, strconv.Itoa(int(broker))) + } + return cl.alterConfigs(ctx, false, configs, kmsg.ConfigResourceTypeBroker, names) +} + +// ValidateAlterBrokerConfigs validates an incremental alter config for the given +// brokers. +// +// This returns exactly what AlterBrokerConfigs returns, but does not actually +// alter configurations. +func (cl *Client) ValidateAlterBrokerConfigs(ctx context.Context, configs []AlterConfig, brokers ...int32) (AlterConfigsResponses, error) { + var names []string + if len(brokers) == 0 { + names = append(names, "") + } + for _, broker := range brokers { + names = append(names, strconv.Itoa(int(broker))) + } + return cl.alterConfigs(ctx, true, configs, kmsg.ConfigResourceTypeBroker, names) +} + +func (cl *Client) alterConfigs( + ctx context.Context, + dry bool, + configs []AlterConfig, + kind kmsg.ConfigResourceType, + names []string, +) (AlterConfigsResponses, error) { + req := kmsg.NewPtrIncrementalAlterConfigsRequest() + req.ValidateOnly = dry + for _, name := range names { + rr := kmsg.NewIncrementalAlterConfigsRequestResource() + rr.ResourceType = kind + rr.ResourceName = name + for _, config := range configs { + rc := kmsg.NewIncrementalAlterConfigsRequestResourceConfig() + rc.Name = config.Name + rc.Value = config.Value + switch config.Op { + case SetConfig: + rc.Op = kmsg.IncrementalAlterConfigOpSet + case DeleteConfig: + rc.Op = kmsg.IncrementalAlterConfigOpDelete + case AppendConfig: + rc.Op = kmsg.IncrementalAlterConfigOpAppend + case SubtractConfig: + rc.Op = kmsg.IncrementalAlterConfigOpSubtract + } + rr.Configs = append(rr.Configs, rc) + } + req.Resources = append(req.Resources, rr) + } + + shards := cl.cl.RequestSharded(ctx, req) + + var rs []AlterConfigsResponse + return rs, shardErrEach(req, shards, func(kr kmsg.Response) error { + resp := kr.(*kmsg.IncrementalAlterConfigsResponse) + for _, r := range resp.Resources { + rs = append(rs, AlterConfigsResponse{ // we are not storing in a map, no existence check possible + Name: r.ResourceName, + Err: kerr.ErrorForCode(r.ErrorCode), + ErrMessage: unptrStr(r.ErrorMessage), + }) + } + return nil + }) +} + +// AlterTopicConfigsState alters the full state of topic configurations. +// All prior configuration is lost. +// +// This may return *ShardErrors. You may consider checking +// ValidateAlterTopicConfigs before using this method. +func (cl *Client) AlterTopicConfigsState(ctx context.Context, configs []AlterConfig, topics ...string) (AlterConfigsResponses, error) { + return cl.alterConfigsState(ctx, false, configs, kmsg.ConfigResourceTypeTopic, topics) +} + +// ValidateAlterTopicConfigs validates an AlterTopicConfigsState for the given +// topics. +// +// This returns exactly what AlterTopicConfigsState returns, but does not +// actually alter configurations. +func (cl *Client) ValidateAlterTopicConfigsState(ctx context.Context, configs []AlterConfig, topics ...string) (AlterConfigsResponses, error) { + return cl.alterConfigsState(ctx, true, configs, kmsg.ConfigResourceTypeTopic, topics) +} + +// AlterBrokerConfigs alters the full state of broker configurations. If +// broker are specified, this updates each specific broker. If no brokers are +// specified, this updates whole-cluster broker configuration values. +// All prior configuration is lost. +// +// This may return *ShardErrors. You may consider checking +// ValidateAlterBrokerConfigs before using this method. +func (cl *Client) AlterBrokerConfigsState(ctx context.Context, configs []AlterConfig, brokers ...int32) (AlterConfigsResponses, error) { + var names []string + if len(brokers) == 0 { + names = append(names, "") + } + for _, broker := range brokers { + names = append(names, strconv.Itoa(int(broker))) + } + return cl.alterConfigsState(ctx, false, configs, kmsg.ConfigResourceTypeBroker, names) +} + +// ValidateAlterBrokerConfigs validates an AlterBrokerconfigsState for the +// given brokers. +// +// This returns exactly what AlterBrokerConfigs returns, but does not actually +// alter configurations. +func (cl *Client) ValidateAlterBrokerConfigsState(ctx context.Context, configs []AlterConfig, brokers ...int32) (AlterConfigsResponses, error) { + var names []string + if len(brokers) == 0 { + names = append(names, "") + } + for _, broker := range brokers { + names = append(names, strconv.Itoa(int(broker))) + } + return cl.alterConfigsState(ctx, true, configs, kmsg.ConfigResourceTypeBroker, names) +} + +func (cl *Client) alterConfigsState( + ctx context.Context, + dry bool, + configs []AlterConfig, + kind kmsg.ConfigResourceType, + names []string, +) (AlterConfigsResponses, error) { + req := kmsg.NewPtrAlterConfigsRequest() + req.ValidateOnly = dry + for _, name := range names { + rr := kmsg.NewAlterConfigsRequestResource() + rr.ResourceType = kind + rr.ResourceName = name + for _, config := range configs { + rc := kmsg.NewAlterConfigsRequestResourceConfig() + rc.Name = config.Name + rc.Value = config.Value + rr.Configs = append(rr.Configs, rc) + } + req.Resources = append(req.Resources, rr) + } + + shards := cl.cl.RequestSharded(ctx, req) + + var rs []AlterConfigsResponse + return rs, shardErrEach(req, shards, func(kr kmsg.Response) error { + resp := kr.(*kmsg.AlterConfigsResponse) + for _, r := range resp.Resources { + rs = append(rs, AlterConfigsResponse{ // we are not storing in a map, no existence check possible + Name: r.ResourceName, + Err: kerr.ErrorForCode(r.ErrorCode), + ErrMessage: unptrStr(r.ErrorMessage), + }) + } + return nil + }) +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/dtoken.go b/vendor/github.com/twmb/franz-go/pkg/kadm/dtoken.go new file mode 100644 index 000000000000..7591cf43a326 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/dtoken.go @@ -0,0 +1,229 @@ +package kadm + +import ( + "context" + "time" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// Principal is a principal that owns or renews a delegation token. This is the +// same as an ACL's principal, but rather than being a single string, the type +// and name are split into two fields. +type Principal struct { + Type string // Type is the type of a principal owner or renewer. If empty, this defaults to "User". + Name string // Name is the name of a principal owner or renewer. +} + +// DelegationToken contains information about a delegation token. +type DelegationToken struct { + // Owner is the owner of the delegation token. + Owner Principal + // TokenRequesterPrincipal is the principal of the creator of the + // token. This exists for v3+, where you can override the owner. + // For prior than v3, this is just the Owner. + TokenRequesterPrincipal Principal + // IssueTimestamp is timestamp the delegation token creation request + // is received within the broker. + IssueTimestamp time.Time + // ExpiryTimestamp is the timestamp the delegation token will expire. + // This field is: + // min(MaxTimestamp, IssueTimestamp+delegation.token.expiry.time.ms) + // where the default expiry is 24hr. + ExpiryTimestamp time.Time + // MaxTimestamp is the timestamp past which the delegation token cannot + // be renewed. This is either the requested MaxLifetime, or the + // broker's delegation.token.max.lifetime.ms which is 7d by default. + MaxTimestamp time.Time + // TokenID is the username of this token for use in authorization. + TokenID string + // HMAC is the password of this token for use for in authorization. + HMAC []byte + // Renewers is the list of principals that can renew this token in + // addition to the owner (which always can). + Renewers []Principal +} + +// DelegationTokens contains a list of delegation tokens. +type DelegationTokens []DelegationToken + +// CreateDelegationToken is a create delegation token request, allowing you to +// create scoped tokens with the same ACLs as the creator. This allows you to +// more easily manage authorization for a wide array of clients. All delegation +// tokens use SCRAM-SHA-256 SASL for authorization. +type CreateDelegationToken struct { + // Owner overrides the owner of the token from the principal issuing + // the request to the principal in this field. This allows a superuser + // to create tokens without requiring individual user credentials, and + // for a superuser to run clients on behalf of another user. These + // fields require Kafka 3.3+; see KIP-373 for more details. + Owner *Principal + // Renewers is a list of principals that can renew the delegation + // token in addition to the owner of the token. This list does not + // include the owner. + Renewers []Principal + // MaxLifetime is how long the delegation token is valid for. + // If -1, the default is the server's delegation.token.max.lifetime.ms, + // which is by default 7d. + MaxLifetime time.Duration +} + +// CreateDelegationToken creates a delegation token, which is a scoped +// SCRAM-SHA-256 username and password. +// +// Creating delegation tokens allows for an (ideally) quicker and easier method +// of enabling authorization for a wide array of clients. Rather than having to +// manage many passwords external to Kafka, you only need to manage a few +// accounts and use those to create delegation tokens per client. +// +// Note that delegation tokens inherit the same ACLs as the user creating the +// token. Thus, if you want to properly scope ACLs, you should not create +// delegation tokens with admin accounts. +// +// This can return *AuthError. +func (cl *Client) CreateDelegationToken(ctx context.Context, d CreateDelegationToken) (DelegationToken, error) { + req := kmsg.NewPtrCreateDelegationTokenRequest() + if d.Owner != nil { + req.OwnerPrincipalType = &d.Owner.Type + req.OwnerPrincipalName = &d.Owner.Name + } + for _, renewer := range d.Renewers { + rr := kmsg.NewCreateDelegationTokenRequestRenewer() + rr.PrincipalType = renewer.Type + rr.PrincipalName = renewer.Name + req.Renewers = append(req.Renewers, rr) + } + req.MaxLifetimeMillis = d.MaxLifetime.Milliseconds() + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return DelegationToken{}, err + } + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return DelegationToken{}, err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return DelegationToken{}, err + } + + t := DelegationToken{ + Owner: Principal{ + Type: resp.PrincipalType, + Name: resp.PrincipalName, + }, + TokenRequesterPrincipal: Principal{ + Type: resp.TokenRequesterPrincipalType, + Name: resp.TokenRequesterPrincipalName, + }, + IssueTimestamp: time.UnixMilli(resp.IssueTimestamp).UTC(), + ExpiryTimestamp: time.UnixMilli(resp.ExpiryTimestamp).UTC(), + MaxTimestamp: time.UnixMilli(resp.MaxTimestamp).UTC(), + TokenID: resp.TokenID, + HMAC: resp.HMAC, + Renewers: append([]Principal(nil), d.Renewers...), + } + if resp.Version < 3 { + t.TokenRequesterPrincipal = t.Owner + } + return t, nil +} + +// RenewDelegationToken renews a delegation token that has not yet hit its max +// timestamp and returns the new expiry timestamp. +// +// This can return *AuthError. +func (cl *Client) RenewDelegationToken(ctx context.Context, hmac []byte, renewTime time.Duration) (expiryTimestamp time.Time, err error) { + req := kmsg.NewPtrRenewDelegationTokenRequest() + req.HMAC = hmac + req.RenewTimeMillis = renewTime.Milliseconds() + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return time.Time{}, err + } + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return time.Time{}, err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return time.Time{}, err + } + return time.UnixMilli(resp.ExpiryTimestamp).UTC(), nil +} + +// ExpireDelegationToken changes a delegation token's expiry timestamp and +// returns the new expiry timestamp, which is min(now+expiry, maxTimestamp). +// This request can be used to force tokens to expire quickly, or to give +// tokens a grace period before expiry. Using an expiry of -1 expires the token +// immediately. +// +// This can return *AuthError. +func (cl *Client) ExpireDelegationToken(ctx context.Context, hmac []byte, expiry time.Duration) (expiryTimestamp time.Time, err error) { + req := kmsg.NewPtrExpireDelegationTokenRequest() + req.HMAC = hmac + req.ExpiryPeriodMillis = expiry.Milliseconds() + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return time.Time{}, err + } + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return time.Time{}, err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return time.Time{}, err + } + return time.UnixMilli(resp.ExpiryTimestamp).UTC(), nil +} + +// DescribeDelegationTokens describes delegation tokens. This returns either +// all delegation tokens, or returns only tokens with owners in the requested +// owners list. +// +// This can return *AuthError. +func (cl *Client) DescribeDelegationTokens(ctx context.Context, owners ...Principal) (DelegationTokens, error) { + req := kmsg.NewPtrDescribeDelegationTokenRequest() + for _, owner := range owners { + ro := kmsg.NewDescribeDelegationTokenRequestOwner() + ro.PrincipalType = owner.Type + ro.PrincipalName = owner.Name + req.Owners = append(req.Owners, ro) + } + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return nil, err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return nil, err + } + + var ts DelegationTokens + for _, d := range resp.TokenDetails { + t := DelegationToken{ + Owner: Principal{ + Type: d.PrincipalType, + Name: d.PrincipalName, + }, + TokenRequesterPrincipal: Principal{ + Type: d.TokenRequesterPrincipalType, + Name: d.TokenRequesterPrincipalName, + }, + IssueTimestamp: time.UnixMilli(d.IssueTimestamp).UTC(), + ExpiryTimestamp: time.UnixMilli(d.ExpiryTimestamp).UTC(), + MaxTimestamp: time.UnixMilli(d.MaxTimestamp).UTC(), + TokenID: d.TokenID, + HMAC: d.HMAC, + } + if resp.Version < 3 { + t.TokenRequesterPrincipal = t.Owner + } + for _, r := range d.Renewers { + t.Renewers = append(t.Renewers, Principal{ + Type: r.PrincipalType, + Name: r.PrincipalName, + }) + } + ts = append(ts, t) + } + return ts, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/errors.go b/vendor/github.com/twmb/franz-go/pkg/kadm/errors.go new file mode 100644 index 000000000000..878e62af93b0 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/errors.go @@ -0,0 +1,134 @@ +package kadm + +import ( + "errors" + "fmt" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// AuthError can be returned from requests for resources that you are not +// authorized for. +type AuthError struct { + Err error // Err is the inner *kerr.Error authorization error. +} + +func (a *AuthError) Error() string { return a.Err.Error() } +func (a *AuthError) Unwrap() error { return a.Err } +func (a *AuthError) Is(err error) bool { return a.Err == err } + +func maybeAuthErr(code int16) error { + switch err := kerr.ErrorForCode(code); err { + case kerr.ClusterAuthorizationFailed, + kerr.TopicAuthorizationFailed, + kerr.GroupAuthorizationFailed, + kerr.TransactionalIDAuthorizationFailed, + kerr.DelegationTokenAuthorizationFailed: + return &AuthError{err} + } + return nil +} + +// ShardError is a piece of a request that failed. See ShardErrors for more +// detail. +type ShardError struct { + Req kmsg.Request // Req is a piece of the original request. + Err error // Err is the error that resulted in this request failing. + + // Broker, if non-nil, is the broker this request was meant to be + // issued to. If the NodeID is -1, then this piece of the request + // failed before being mapped to a broker. + Broker BrokerDetail +} + +// ShardErrors contains each individual error shard of a request. +// +// Under the hood, some requests to Kafka need to be mapped to brokers, split, +// and sent to many brokers. The kgo.Client handles this all internally, but +// returns the individual pieces that were requested as "shards". Internally, +// each of these pieces can also fail, and they can all fail uniquely. +// +// The kadm package takes one further step and hides the failing pieces into +// one meta error, the ShardErrors. Methods in this package that can return +// this meta error are documented; if desired, you can use errors.As to check +// and unwrap any ShardErrors return. +// +// If a request returns ShardErrors, it is possible that some aspects of the +// request were still successful. You can check ShardErrors.AllFailed as a +// shortcut for whether any of the response is usable or not. +type ShardErrors struct { + Name string // Name is the name of the request these shard errors are for. + AllFailed bool // AllFailed indicates if the original request was entirely unsuccessful. + Errs []ShardError // Errs contains all individual shard errors. +} + +func shardErrEach(req kmsg.Request, shards []kgo.ResponseShard, fn func(kmsg.Response) error) error { + return shardErrEachBroker(req, shards, func(_ BrokerDetail, resp kmsg.Response) error { + return fn(resp) + }) +} + +func shardErrEachBroker(req kmsg.Request, shards []kgo.ResponseShard, fn func(BrokerDetail, kmsg.Response) error) error { + se := ShardErrors{ + Name: kmsg.NameForKey(req.Key()), + } + var ae *AuthError + for _, shard := range shards { + if shard.Err != nil { + se.Errs = append(se.Errs, ShardError{ + Req: shard.Req, + Err: shard.Err, + Broker: shard.Meta, + }) + continue + } + if err := fn(shard.Meta, shard.Resp); errors.As(err, &ae) { + return ae + } + } + se.AllFailed = len(shards) == len(se.Errs) + return se.into() +} + +func (se *ShardErrors) into() error { + if se == nil || len(se.Errs) == 0 { + return nil + } + return se +} + +// Merges two shard errors; the input errors should come from the same request. +func mergeShardErrs(e1, e2 error) error { + var se1, se2 *ShardErrors + if !errors.As(e1, &se1) { + return e2 + } + if !errors.As(e2, &se2) { + return e1 + } + se1.Errs = append(se1.Errs, se2.Errs...) + se1.AllFailed = se1.AllFailed && se2.AllFailed + return se1 +} + +// Error returns an error indicating the name of the request that failed, the +// number of separate errors, and the first error. +func (e *ShardErrors) Error() string { + if len(e.Errs) == 0 { + return "INVALID: ShardErrors contains no errors!" + } + return fmt.Sprintf("request %s has %d separate shard errors, first: %s", e.Name, len(e.Errs), e.Errs[0].Err) +} + +// Unwrap returns the underlying errors. +func (e *ShardErrors) Unwrap() []error { + unwrapped := make([]error, 0, len(e.Errs)) + + for _, shardErr := range e.Errs { + unwrapped = append(unwrapped, shardErr.Err) + } + + return unwrapped +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/groups.go b/vendor/github.com/twmb/franz-go/pkg/kadm/groups.go new file mode 100644 index 000000000000..9f1fc60b5a53 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/groups.go @@ -0,0 +1,1841 @@ +package kadm + +import ( + "context" + "errors" + "fmt" + "sort" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// GroupMemberMetadata is the metadata that a client sent in a JoinGroup request. +// This can have one of three types: +// +// *kmsg.ConsumerMemberMetadata, if the group's ProtocolType is "consumer" +// *kmsg.ConnectMemberMetadata, if the group's ProtocolType is "connect" +// []byte, if the group's ProtocolType is unknown +type GroupMemberMetadata struct{ i any } + +// AsConsumer returns the metadata as a ConsumerMemberMetadata if possible. +func (m GroupMemberMetadata) AsConsumer() (*kmsg.ConsumerMemberMetadata, bool) { + c, ok := m.i.(*kmsg.ConsumerMemberMetadata) + return c, ok +} + +// AsConnect returns the metadata as ConnectMemberMetadata if possible. +func (m GroupMemberMetadata) AsConnect() (*kmsg.ConnectMemberMetadata, bool) { + c, ok := m.i.(*kmsg.ConnectMemberMetadata) + return c, ok +} + +// Raw returns the metadata as a raw byte slice, if it is neither of consumer +// type nor connect type. +func (m GroupMemberMetadata) Raw() ([]byte, bool) { + c, ok := m.i.([]byte) + return c, ok +} + +// GroupMemberAssignment is the assignment that a leader sent / a member +// received in a SyncGroup request. This can have one of three types: +// +// *kmsg.ConsumerMemberAssignment, if the group's ProtocolType is "consumer" +// *kmsg.ConnectMemberAssignment, if the group's ProtocolType is "connect" +// []byte, if the group's ProtocolType is unknown +type GroupMemberAssignment struct{ i any } + +// AsConsumer returns the assignment as a ConsumerMemberAssignment if possible. +func (m GroupMemberAssignment) AsConsumer() (*kmsg.ConsumerMemberAssignment, bool) { + c, ok := m.i.(*kmsg.ConsumerMemberAssignment) + return c, ok +} + +// AsConnect returns the assignment as ConnectMemberAssignment if possible. +func (m GroupMemberAssignment) AsConnect() (*kmsg.ConnectMemberAssignment, bool) { + c, ok := m.i.(*kmsg.ConnectMemberAssignment) + return c, ok +} + +// Raw returns the assignment as a raw byte slice, if it is neither of consumer +// type nor connect type. +func (m GroupMemberAssignment) Raw() ([]byte, bool) { + c, ok := m.i.([]byte) + return c, ok +} + +// DescribedGroupMember is the detail of an individual group member as returned +// by a describe groups response. +type DescribedGroupMember struct { + MemberID string // MemberID is the Kafka assigned member ID of this group member. + InstanceID *string // InstanceID is a potential user assigned instance ID of this group member (KIP-345). + ClientID string // ClientID is the Kafka client given ClientID of this group member. + ClientHost string // ClientHost is the host this member is running on. + + Join GroupMemberMetadata // Join is what this member sent in its join group request; what it wants to consume. + Assigned GroupMemberAssignment // Assigned is what this member was assigned to consume by the leader. +} + +// AssignedPartitions returns the set of unique topics and partitions that are +// assigned across all members in this group. +// +// This function is only relevant if the group is of type "consumer". +func (d *DescribedGroup) AssignedPartitions() TopicsSet { + s := make(TopicsSet) + for _, m := range d.Members { + if c, ok := m.Assigned.AsConsumer(); ok { + for _, t := range c.Topics { + s.Add(t.Topic, t.Partitions...) + } + } + } + return s +} + +// DescribedGroup contains data from a describe groups response for a single +// group. +type DescribedGroup struct { + Group string // Group is the name of the described group. + + Coordinator BrokerDetail // Coordinator is the coordinator broker for this group. + State string // State is the state this group is in (Empty, Dead, Stable, etc.). + ProtocolType string // ProtocolType is the type of protocol the group is using, "consumer" for normal consumers, "connect" for Kafka connect. + Protocol string // Protocol is the partition assignor strategy this group is using. + Members []DescribedGroupMember // Members contains the members of this group sorted first by InstanceID, or if nil, by MemberID. + + Err error // Err is non-nil if the group could not be described. +} + +// DescribedGroups contains data for multiple groups from a describe groups +// response. +type DescribedGroups map[string]DescribedGroup + +// AssignedPartitions returns the set of unique topics and partitions that are +// assigned across all members in all groups. This is the all-group analogue to +// DescribedGroup.AssignedPartitions. +// +// This function is only relevant for groups of type "consumer". +func (ds DescribedGroups) AssignedPartitions() TopicsSet { + s := make(TopicsSet) + for _, g := range ds { + for _, m := range g.Members { + if c, ok := m.Assigned.AsConsumer(); ok { + for _, t := range c.Topics { + s.Add(t.Topic, t.Partitions...) + } + } + } + } + return s +} + +// Sorted returns all groups sorted by group name. +func (ds DescribedGroups) Sorted() []DescribedGroup { + s := make([]DescribedGroup, 0, len(ds)) + for _, d := range ds { + s = append(s, d) + } + sort.Slice(s, func(i, j int) bool { return s[i].Group < s[j].Group }) + return s +} + +// On calls fn for the group if it exists, returning the group and the error +// returned from fn. If fn is nil, this simply returns the group. +// +// The fn is given a shallow copy of the group. This function returns the copy +// as well; any modifications within fn are modifications on the returned copy. +// Modifications on a described group's inner fields are persisted to the +// original map (because slices are pointers). +// +// If the group does not exist, this returns kerr.GroupIDNotFound. +func (rs DescribedGroups) On(group string, fn func(*DescribedGroup) error) (DescribedGroup, error) { + if len(rs) > 0 { + r, ok := rs[group] + if ok { + if fn == nil { + return r, nil + } + return r, fn(&r) + } + } + return DescribedGroup{}, kerr.GroupIDNotFound +} + +// Error iterates over all groups and returns the first error encountered, if +// any. +func (ds DescribedGroups) Error() error { + for _, d := range ds { + if d.Err != nil { + return d.Err + } + } + return nil +} + +// Topics returns a sorted list of all group names. +func (ds DescribedGroups) Names() []string { + all := make([]string, 0, len(ds)) + for g := range ds { + all = append(all, g) + } + sort.Strings(all) + return all +} + +// ListedGroup contains data from a list groups response for a single group. +type ListedGroup struct { + Coordinator int32 // Coordinator is the node ID of the coordinator for this group. + Group string // Group is the name of this group. + ProtocolType string // ProtocolType is the type of protocol the group is using, "consumer" for normal consumers, "connect" for Kafka connect. + State string // State is the state this group is in (Empty, Dead, Stable, etc.; only if talking to Kafka 2.6+). +} + +// ListedGroups contains information from a list groups response. +type ListedGroups map[string]ListedGroup + +// Sorted returns all groups sorted by group name. +func (ls ListedGroups) Sorted() []ListedGroup { + s := make([]ListedGroup, 0, len(ls)) + for _, l := range ls { + s = append(s, l) + } + sort.Slice(s, func(i, j int) bool { return s[i].Group < s[j].Group }) + return s +} + +// Groups returns a sorted list of all group names. +func (ls ListedGroups) Groups() []string { + all := make([]string, 0, len(ls)) + for g := range ls { + all = append(all, g) + } + sort.Strings(all) + return all +} + +// ListGroups returns all groups in the cluster. If you are talking to Kafka +// 2.6+, filter states can be used to return groups only in the requested +// states. By default, this returns all groups. In almost all cases, +// DescribeGroups is more useful. +// +// This may return *ShardErrors or *AuthError. +func (cl *Client) ListGroups(ctx context.Context, filterStates ...string) (ListedGroups, error) { + req := kmsg.NewPtrListGroupsRequest() + req.StatesFilter = append(req.StatesFilter, filterStates...) + shards := cl.cl.RequestSharded(ctx, req) + list := make(ListedGroups) + return list, shardErrEachBroker(req, shards, func(b BrokerDetail, kr kmsg.Response) error { + resp := kr.(*kmsg.ListGroupsResponse) + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return err + } + for _, g := range resp.Groups { + list[g.Group] = ListedGroup{ // group only lives on one broker, no need to exist-check + Coordinator: b.NodeID, + Group: g.Group, + ProtocolType: g.ProtocolType, + State: g.GroupState, + } + } + return nil + }) +} + +// DescribeGroups describes either all groups specified, or all groups in the +// cluster if none are specified. +// +// This may return *ShardErrors or *AuthError. +// +// If no groups are specified and this method first lists groups, and list +// groups returns a *ShardErrors, this function describes all successfully +// listed groups and appends the list shard errors to any describe shard +// errors. +// +// If only one group is described, there will be at most one request issued, +// and there is no need to deeply inspect the error. +func (cl *Client) DescribeGroups(ctx context.Context, groups ...string) (DescribedGroups, error) { + var seList *ShardErrors + if len(groups) == 0 { + listed, err := cl.ListGroups(ctx) + switch { + case err == nil: + case errors.As(err, &seList): + default: + return nil, err + } + groups = listed.Groups() + if len(groups) == 0 { + return nil, err + } + } + + req := kmsg.NewPtrDescribeGroupsRequest() + req.Groups = groups + + shards := cl.cl.RequestSharded(ctx, req) + described := make(DescribedGroups) + err := shardErrEachBroker(req, shards, func(b BrokerDetail, kr kmsg.Response) error { + resp := kr.(*kmsg.DescribeGroupsResponse) + for _, rg := range resp.Groups { + if err := maybeAuthErr(rg.ErrorCode); err != nil { + return err + } + g := DescribedGroup{ + Group: rg.Group, + Coordinator: b, + State: rg.State, + ProtocolType: rg.ProtocolType, + Protocol: rg.Protocol, + Err: kerr.ErrorForCode(rg.ErrorCode), + } + for _, rm := range rg.Members { + gm := DescribedGroupMember{ + MemberID: rm.MemberID, + InstanceID: rm.InstanceID, + ClientID: rm.ClientID, + ClientHost: rm.ClientHost, + } + + var mi, ai any + switch g.ProtocolType { + case "consumer": + m := new(kmsg.ConsumerMemberMetadata) + a := new(kmsg.ConsumerMemberAssignment) + + m.ReadFrom(rm.ProtocolMetadata) + a.ReadFrom(rm.MemberAssignment) + + mi, ai = m, a + case "connect": + m := new(kmsg.ConnectMemberMetadata) + a := new(kmsg.ConnectMemberAssignment) + + m.ReadFrom(rm.ProtocolMetadata) + a.ReadFrom(rm.MemberAssignment) + + mi, ai = m, a + default: + mi, ai = rm.ProtocolMetadata, rm.MemberAssignment + } + + gm.Join = GroupMemberMetadata{mi} + gm.Assigned = GroupMemberAssignment{ai} + g.Members = append(g.Members, gm) + } + sort.Slice(g.Members, func(i, j int) bool { + if g.Members[i].InstanceID != nil { + if g.Members[j].InstanceID == nil { + return true + } + return *g.Members[i].InstanceID < *g.Members[j].InstanceID + } + if g.Members[j].InstanceID != nil { + return false + } + return g.Members[i].MemberID < g.Members[j].MemberID + }) + described[g.Group] = g // group only lives on one broker, no need to exist-check + } + return nil + }) + + var seDesc *ShardErrors + switch { + case err == nil: + return described, seList.into() + case errors.As(err, &seDesc): + if seList != nil { + seDesc.Errs = append(seList.Errs, seDesc.Errs...) + } + return described, seDesc.into() + default: + return nil, err + } +} + +// DeleteGroupResponse contains the response for an individual deleted group. +type DeleteGroupResponse struct { + Group string // Group is the group this response is for. + Err error // Err is non-nil if the group failed to be deleted. +} + +// DeleteGroupResponses contains per-group responses to deleted groups. +type DeleteGroupResponses map[string]DeleteGroupResponse + +// Sorted returns all deleted group responses sorted by group name. +func (ds DeleteGroupResponses) Sorted() []DeleteGroupResponse { + s := make([]DeleteGroupResponse, 0, len(ds)) + for _, d := range ds { + s = append(s, d) + } + sort.Slice(s, func(i, j int) bool { return s[i].Group < s[j].Group }) + return s +} + +// On calls fn for the response group if it exists, returning the response and +// the error returned from fn. If fn is nil, this simply returns the group. +// +// The fn is given a copy of the response. This function returns the copy as +// well; any modifications within fn are modifications on the returned copy. +// +// If the group does not exist, this returns kerr.GroupIDNotFound. +func (rs DeleteGroupResponses) On(group string, fn func(*DeleteGroupResponse) error) (DeleteGroupResponse, error) { + if len(rs) > 0 { + r, ok := rs[group] + if ok { + if fn == nil { + return r, nil + } + return r, fn(&r) + } + } + return DeleteGroupResponse{}, kerr.GroupIDNotFound +} + +// Error iterates over all groups and returns the first error encountered, if +// any. +func (rs DeleteGroupResponses) Error() error { + for _, r := range rs { + if r.Err != nil { + return r.Err + } + } + return nil +} + +// DeleteGroup deletes the specified group. This is similar to DeleteGroups, +// but returns the kerr.ErrorForCode(response.ErrorCode) if the request/response +// is successful. +func (cl *Client) DeleteGroup(ctx context.Context, group string) (DeleteGroupResponse, error) { + rs, err := cl.DeleteGroups(ctx, group) + if err != nil { + return DeleteGroupResponse{}, err + } + g, exists := rs[group] + if !exists { + return DeleteGroupResponse{}, errors.New("requested group was not part of the delete group response") + } + return g, g.Err +} + +// DeleteGroups deletes all groups specified. +// +// The purpose of this request is to allow operators a way to delete groups +// after Kafka 1.1, which removed RetentionTimeMillis from offset commits. See +// KIP-229 for more details. +// +// This may return *ShardErrors. This does not return on authorization +// failures, instead, authorization failures are included in the responses. +func (cl *Client) DeleteGroups(ctx context.Context, groups ...string) (DeleteGroupResponses, error) { + if len(groups) == 0 { + return nil, nil + } + req := kmsg.NewPtrDeleteGroupsRequest() + req.Groups = append(req.Groups, groups...) + shards := cl.cl.RequestSharded(ctx, req) + + rs := make(map[string]DeleteGroupResponse) + return rs, shardErrEach(req, shards, func(kr kmsg.Response) error { + resp := kr.(*kmsg.DeleteGroupsResponse) + for _, g := range resp.Groups { + rs[g.Group] = DeleteGroupResponse{ // group is always on one broker, no need to exist-check + Group: g.Group, + Err: kerr.ErrorForCode(g.ErrorCode), + } + } + return nil + }) +} + +// LeaveGroupBuilder helps build a leave group request, rather than having +// a function signature (string, string, ...string). +// +// All functions on this type accept and return the same pointer, allowing +// for easy build-and-use usage. +type LeaveGroupBuilder struct { + group string + reason *string + instanceIDs []*string +} + +// LeaveGroup returns a LeaveGroupBuilder for the input group. +func LeaveGroup(group string) *LeaveGroupBuilder { + return &LeaveGroupBuilder{ + group: group, + } +} + +// Reason attaches a reason to all members in the leave group request. +// This requires Kafka 3.2+. +func (b *LeaveGroupBuilder) Reason(reason string) *LeaveGroupBuilder { + b.reason = StringPtr(reason) + return b +} + +// InstanceIDs are members to remove from a group. +func (b *LeaveGroupBuilder) InstanceIDs(ids ...string) *LeaveGroupBuilder { + for _, id := range ids { + if id != "" { + b.instanceIDs = append(b.instanceIDs, StringPtr(id)) + } + } + return b +} + +// LeaveGroupResponse contains the response for an individual instance ID that +// left a group. +type LeaveGroupResponse struct { + Group string // Group is the group that was left. + InstanceID string // InstanceID is the instance ID that left the group. + MemberID string // MemberID is the member ID that left the group. + Err error // Err is non-nil if this member did not exist. +} + +// LeaveGroupResponses contains responses for each member of a leave group +// request. The map key is the instance ID that was removed from the group. +type LeaveGroupResponses map[string]LeaveGroupResponse + +// Sorted returns all removed group members by instance ID. +func (ls LeaveGroupResponses) Sorted() []LeaveGroupResponse { + s := make([]LeaveGroupResponse, 0, len(ls)) + for _, l := range ls { + s = append(s, l) + } + sort.Slice(s, func(i, j int) bool { return s[i].InstanceID < s[j].InstanceID }) + return s +} + +// EachError calls fn for every removed member that has a non-nil error. +func (ls LeaveGroupResponses) EachError(fn func(l LeaveGroupResponse)) { + for _, l := range ls { + if l.Err != nil { + fn(l) + } + } +} + +// Each calls fn for every removed member. +func (ls LeaveGroupResponses) Each(fn func(l LeaveGroupResponse)) { + for _, l := range ls { + fn(l) + } +} + +// Error iterates over all removed members and returns the first error +// encountered, if any. +func (ls LeaveGroupResponses) Error() error { + for _, l := range ls { + if l.Err != nil { + return l.Err + } + } + return nil +} + +// Ok returns true if there are no errors. This is a shortcut for ls.Error() == +// nil. +func (ls LeaveGroupResponses) Ok() bool { + return ls.Error() == nil +} + +// LeaveGroup causes instance IDs to leave a group. +// +// This function allows manually removing members using instance IDs from a +// group, which allows for fast scale down / host replacement (see KIP-345 for +// more detail). This returns an *AuthErr if the use is not authorized to +// remove members from groups. +func (cl *Client) LeaveGroup(ctx context.Context, b *LeaveGroupBuilder) (LeaveGroupResponses, error) { + if b == nil || len(b.instanceIDs) == 0 { + return nil, nil + } + req := kmsg.NewPtrLeaveGroupRequest() + req.Group = b.group + for _, id := range b.instanceIDs { + m := kmsg.NewLeaveGroupRequestMember() + id := id + m.InstanceID = id + m.Reason = b.reason + req.Members = append(req.Members, m) + } + + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return nil, err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return nil, err + } + + resps := make(LeaveGroupResponses) + for _, m := range resp.Members { + if m.InstanceID == nil { + continue // highly unexpected, buggy kafka + } + resps[*m.InstanceID] = LeaveGroupResponse{ + Group: b.group, + MemberID: m.MemberID, + InstanceID: *m.InstanceID, + Err: kerr.ErrorForCode(resp.ErrorCode), + } + } + return resps, err +} + +// OffsetResponse contains the response for an individual offset for offset +// methods. +type OffsetResponse struct { + Offset + Err error // Err is non-nil if the offset operation failed. +} + +// OffsetResponses contains per-partition responses to offset methods. +type OffsetResponses map[string]map[int32]OffsetResponse + +// Lookup returns the offset at t and p and whether it exists. +func (os OffsetResponses) Lookup(t string, p int32) (OffsetResponse, bool) { + if len(os) == 0 { + return OffsetResponse{}, false + } + ps := os[t] + if len(ps) == 0 { + return OffsetResponse{}, false + } + o, exists := ps[p] + return o, exists +} + +// Keep filters the responses to only keep the input offsets. +func (os OffsetResponses) Keep(o Offsets) { + os.DeleteFunc(func(r OffsetResponse) bool { + if len(o) == 0 { + return true // keep nothing, delete + } + ot := o[r.Topic] + if ot == nil { + return true // topic missing, delete + } + _, ok := ot[r.Partition] + return !ok // does not exist, delete + }) +} + +// Offsets returns these offset responses as offsets. +func (os OffsetResponses) Offsets() Offsets { + i := make(Offsets) + os.Each(func(o OffsetResponse) { + i.Add(o.Offset) + }) + return i +} + +// KOffsets returns these offset responses as a kgo offset map. +func (os OffsetResponses) KOffsets() map[string]map[int32]kgo.Offset { + return os.Offsets().KOffsets() +} + +// DeleteFunc keeps only the offsets for which fn returns true. +func (os OffsetResponses) KeepFunc(fn func(OffsetResponse) bool) { + for t, ps := range os { + for p, o := range ps { + if !fn(o) { + delete(ps, p) + } + } + if len(ps) == 0 { + delete(os, t) + } + } +} + +// DeleteFunc deletes any offset for which fn returns true. +func (os OffsetResponses) DeleteFunc(fn func(OffsetResponse) bool) { + os.KeepFunc(func(o OffsetResponse) bool { return !fn(o) }) +} + +// Add adds an offset for a given topic/partition to this OffsetResponses map +// (even if it exists). +func (os *OffsetResponses) Add(o OffsetResponse) { + if *os == nil { + *os = make(map[string]map[int32]OffsetResponse) + } + ot := (*os)[o.Topic] + if ot == nil { + ot = make(map[int32]OffsetResponse) + (*os)[o.Topic] = ot + } + ot[o.Partition] = o +} + +// EachError calls fn for every offset that as a non-nil error. +func (os OffsetResponses) EachError(fn func(o OffsetResponse)) { + for _, ps := range os { + for _, o := range ps { + if o.Err != nil { + fn(o) + } + } + } +} + +// Sorted returns the responses sorted by topic and partition. +func (os OffsetResponses) Sorted() []OffsetResponse { + var s []OffsetResponse + os.Each(func(o OffsetResponse) { s = append(s, o) }) + sort.Slice(s, func(i, j int) bool { + return s[i].Topic < s[j].Topic || + s[i].Topic == s[j].Topic && s[i].Partition < s[j].Partition + }) + return s +} + +// Each calls fn for every offset. +func (os OffsetResponses) Each(fn func(OffsetResponse)) { + for _, ps := range os { + for _, o := range ps { + fn(o) + } + } +} + +// Partitions returns the set of unique topics and partitions in these offsets. +func (os OffsetResponses) Partitions() TopicsSet { + s := make(TopicsSet) + os.Each(func(o OffsetResponse) { + s.Add(o.Topic, o.Partition) + }) + return s +} + +// Error iterates over all offsets and returns the first error encountered, if +// any. This can be used to check if an operation was entirely successful or +// not. +// +// Note that offset operations can be partially successful. For example, some +// offsets could succeed in an offset commit while others fail (maybe one topic +// does not exist for some reason, or you are not authorized for one topic). If +// this is something you need to worry about, you may need to check all offsets +// manually. +func (os OffsetResponses) Error() error { + for _, ps := range os { + for _, o := range ps { + if o.Err != nil { + return o.Err + } + } + } + return nil +} + +// Ok returns true if there are no errors. This is a shortcut for os.Error() == +// nil. +func (os OffsetResponses) Ok() bool { + return os.Error() == nil +} + +// CommitOffsets issues an offset commit request for the input offsets. +// +// This function can be used to manually commit offsets when directly consuming +// partitions outside of an actual consumer group. For example, if you assign +// partitions manually, but want still use Kafka to checkpoint what you have +// consumed, you can manually issue an offset commit request with this method. +// +// This does not return on authorization failures, instead, authorization +// failures are included in the responses. +func (cl *Client) CommitOffsets(ctx context.Context, group string, os Offsets) (OffsetResponses, error) { + req := kmsg.NewPtrOffsetCommitRequest() + req.Group = group + for t, ps := range os { + rt := kmsg.NewOffsetCommitRequestTopic() + rt.Topic = t + for p, o := range ps { + rp := kmsg.NewOffsetCommitRequestTopicPartition() + rp.Partition = p + rp.Offset = o.At + rp.LeaderEpoch = o.LeaderEpoch + if len(o.Metadata) > 0 { + rp.Metadata = kmsg.StringPtr(o.Metadata) + } + rt.Partitions = append(rt.Partitions, rp) + } + req.Topics = append(req.Topics, rt) + } + + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + + rs := make(OffsetResponses) + for _, t := range resp.Topics { + rt := make(map[int32]OffsetResponse) + rs[t.Topic] = rt + for _, p := range t.Partitions { + rt[p.Partition] = OffsetResponse{ + Offset: os[t.Topic][p.Partition], + Err: kerr.ErrorForCode(p.ErrorCode), + } + } + } + + for t, ps := range os { + respt := rs[t] + if respt == nil { + respt = make(map[int32]OffsetResponse) + rs[t] = respt + } + for p, o := range ps { + if _, exists := respt[p]; exists { + continue + } + respt[p] = OffsetResponse{ + Offset: o, + Err: errOffsetCommitMissing, + } + } + } + + return rs, nil +} + +var errOffsetCommitMissing = errors.New("partition missing in commit response") + +// CommitAllOffsets is identical to CommitOffsets, but returns an error if the +// offset commit was successful, but some offset within the commit failed to be +// committed. +// +// This is a shortcut function provided to avoid checking two errors, but you +// must be careful with this if partially successful commits can be a problem +// for you. +func (cl *Client) CommitAllOffsets(ctx context.Context, group string, os Offsets) error { + commits, err := cl.CommitOffsets(ctx, group, os) + if err != nil { + return err + } + return commits.Error() +} + +// FetchOffsets issues an offset fetch requests for all topics and partitions +// in the group. Because Kafka returns only partitions you are authorized to +// fetch, this only returns an auth error if you are not authorized to describe +// the group at all. +// +// This method requires talking to Kafka v0.11+. +func (cl *Client) FetchOffsets(ctx context.Context, group string) (OffsetResponses, error) { + req := kmsg.NewPtrOffsetFetchRequest() + req.Group = group + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return nil, err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return nil, err + } + rs := make(OffsetResponses) + for _, t := range resp.Topics { + rt := make(map[int32]OffsetResponse) + rs[t.Topic] = rt + for _, p := range t.Partitions { + if err := maybeAuthErr(p.ErrorCode); err != nil { + return nil, err + } + var meta string + if p.Metadata != nil { + meta = *p.Metadata + } + rt[p.Partition] = OffsetResponse{ + Offset: Offset{ + Topic: t.Topic, + Partition: p.Partition, + At: p.Offset, + LeaderEpoch: p.LeaderEpoch, + Metadata: meta, + }, + Err: kerr.ErrorForCode(p.ErrorCode), + } + } + } + return rs, nil +} + +// FetchAllGroupTopics is a kadm "internal" topic name that can be used in +// [FetchOffsetsForTopics]. By default, [FetchOffsetsForTopics] only returns +// topics that are explicitly requested. Other topics that may be committed to +// in the group are not returned. Using FetchAllRequestedTopics switches the +// behavior to return the union of all committed topics and all requested +// topics. +const FetchAllGroupTopics = "|fetch-all-group-topics|" + +// FetchOffsetsForTopics is a helper function that returns the currently +// committed offsets for the given group, as well as default -1 offsets for any +// topic/partition that does not yet have a commit. +// +// If any partition fetched or listed has an error, this function returns an +// error. The returned offset responses are ready to be used or converted +// directly to pure offsets with `Into`, and again into kgo offsets with +// another `Into`. +// +// By default, this function returns offsets for only the requested topics. You +// can use the special "topic" [FetchAllGroupTopics] to return all committed-to +// topics in addition to all requested topics. +func (cl *Client) FetchOffsetsForTopics(ctx context.Context, group string, topics ...string) (OffsetResponses, error) { + os := make(Offsets) + + var all bool + keept := topics[:0] + for _, topic := range topics { + if topic == FetchAllGroupTopics { + all = true + continue + } + keept = append(keept, topic) + } + topics = keept + + if !all && len(topics) == 0 { + return make(OffsetResponses), nil + } + + // We have to request metadata to learn all partitions in all the + // topics. The default returned offset for all partitions is filled in + // to be -1. + if len(topics) > 0 { + listed, err := cl.ListTopics(ctx, topics...) + if err != nil { + return nil, fmt.Errorf("unable to list topics: %w", err) + } + + for _, topic := range topics { + t := listed[topic] + if t.Err != nil { + return nil, fmt.Errorf("unable to describe topics, topic err: %w", t.Err) + } + for _, p := range t.Partitions { + os.AddOffset(topic, p.Partition, -1, -1) + } + } + } + + resps, err := cl.FetchOffsets(ctx, group) + if err != nil { + return nil, fmt.Errorf("unable to fetch offsets: %w", err) + } + if err := resps.Error(); err != nil { + return nil, fmt.Errorf("offset fetches had a load error, first error: %w", err) + } + + // For any topic (and any partition) we explicitly asked for, if the + // partition does not exist in the response, we fill the default -1 + // from above. + os.Each(func(o Offset) { + if _, ok := resps.Lookup(o.Topic, o.Partition); !ok { + resps.Add(OffsetResponse{Offset: o}) + } + }) + + // If we are not requesting all group offsets, then we strip any topic + // that was not explicitly requested. + if !all { + tset := make(map[string]struct{}) + for _, t := range topics { + tset[t] = struct{}{} + } + for t := range resps { + if _, ok := tset[t]; !ok { + delete(resps, t) + } + } + } + return resps, nil +} + +// FetchOffsetsResponse contains a fetch offsets response for a single group. +type FetchOffsetsResponse struct { + Group string // Group is the offsets these fetches correspond to. + Fetched OffsetResponses // Fetched contains offsets fetched for this group, if any. + Err error // Err contains any error preventing offsets from being fetched. +} + +// CommittedPartitions returns the set of unique topics and partitions that +// have been committed to in this group. +func (r FetchOffsetsResponse) CommittedPartitions() TopicsSet { + return r.Fetched.Partitions() +} + +// FetchOFfsetsResponses contains responses for many fetch offsets requests. +type FetchOffsetsResponses map[string]FetchOffsetsResponse + +// EachError calls fn for every response that as a non-nil error. +func (rs FetchOffsetsResponses) EachError(fn func(FetchOffsetsResponse)) { + for _, r := range rs { + if r.Err != nil { + fn(r) + } + } +} + +// AllFailed returns whether all fetch offsets requests failed. +func (rs FetchOffsetsResponses) AllFailed() bool { + var n int + rs.EachError(func(FetchOffsetsResponse) { n++ }) + return len(rs) > 0 && n == len(rs) +} + +// CommittedPartitions returns the set of unique topics and partitions that +// have been committed to across all members in all responses. This is the +// all-group analogue to FetchOffsetsResponse.CommittedPartitions. +func (rs FetchOffsetsResponses) CommittedPartitions() TopicsSet { + s := make(TopicsSet) + for _, r := range rs { + s.Merge(r.CommittedPartitions()) + } + return s +} + +// On calls fn for the response group if it exists, returning the response and +// the error returned from fn. If fn is nil, this simply returns the group. +// +// The fn is given a copy of the response. This function returns the copy as +// well; any modifications within fn are modifications on the returned copy. +// +// If the group does not exist, this returns kerr.GroupIDNotFound. +func (rs FetchOffsetsResponses) On(group string, fn func(*FetchOffsetsResponse) error) (FetchOffsetsResponse, error) { + if len(rs) > 0 { + r, ok := rs[group] + if ok { + if fn == nil { + return r, nil + } + return r, fn(&r) + } + } + return FetchOffsetsResponse{}, kerr.GroupIDNotFound +} + +// Error iterates over all responses and returns the first error encountered, +// if any. +func (rs FetchOffsetsResponses) Error() error { + for _, r := range rs { + if r.Err != nil { + return r.Err + } + } + return nil +} + +// FetchManyOffsets issues a fetch offsets requests for each group specified. +// +// This function is a batch version of FetchOffsets. FetchOffsets and +// CommitOffsets are important to provide as simple APIs for users that manage +// group offsets outside of a consumer group. Each individual group may have an +// auth error. +func (cl *Client) FetchManyOffsets(ctx context.Context, groups ...string) FetchOffsetsResponses { + fetched := make(FetchOffsetsResponses) + if len(groups) == 0 { + return fetched + } + + req := kmsg.NewPtrOffsetFetchRequest() + for _, group := range groups { + rg := kmsg.NewOffsetFetchRequestGroup() + rg.Group = group + req.Groups = append(req.Groups, rg) + } + + groupErr := func(g string, err error) { + fetched[g] = FetchOffsetsResponse{ + Group: g, + Err: err, + } + } + allGroupsErr := func(req *kmsg.OffsetFetchRequest, err error) { + for _, g := range req.Groups { + groupErr(g.Group, err) + } + } + + shards := cl.cl.RequestSharded(ctx, req) + for _, shard := range shards { + req := shard.Req.(*kmsg.OffsetFetchRequest) + if shard.Err != nil { + allGroupsErr(req, shard.Err) + continue + } + resp := shard.Resp.(*kmsg.OffsetFetchResponse) + if err := maybeAuthErr(resp.ErrorCode); err != nil { + allGroupsErr(req, err) + continue + } + for _, g := range resp.Groups { + if err := maybeAuthErr(g.ErrorCode); err != nil { + groupErr(g.Group, err) + continue + } + rs := make(OffsetResponses) + fg := FetchOffsetsResponse{ + Group: g.Group, + Fetched: rs, + Err: kerr.ErrorForCode(g.ErrorCode), + } + fetched[g.Group] = fg // group coordinator owns all of a group, no need to check existence + for _, t := range g.Topics { + rt := make(map[int32]OffsetResponse) + rs[t.Topic] = rt + for _, p := range t.Partitions { + var meta string + if p.Metadata != nil { + meta = *p.Metadata + } + rt[p.Partition] = OffsetResponse{ + Offset: Offset{ + Topic: t.Topic, + Partition: p.Partition, + At: p.Offset, + LeaderEpoch: p.LeaderEpoch, + Metadata: meta, + }, + Err: kerr.ErrorForCode(p.ErrorCode), + } + } + } + } + } + return fetched +} + +// DeleteOffsetsResponses contains the per topic, per partition errors. If an +// offset deletion for a partition was successful, the error will be nil. +type DeleteOffsetsResponses map[string]map[int32]error + +// Lookup returns the response at t and p and whether it exists. +func (ds DeleteOffsetsResponses) Lookup(t string, p int32) (error, bool) { + if len(ds) == 0 { + return nil, false + } + ps := ds[t] + if len(ps) == 0 { + return nil, false + } + r, exists := ps[p] + return r, exists +} + +// EachError calls fn for every partition that as a non-nil deletion error. +func (ds DeleteOffsetsResponses) EachError(fn func(string, int32, error)) { + for t, ps := range ds { + for p, err := range ps { + if err != nil { + fn(t, p, err) + } + } + } +} + +// Error iterates over all responses and returns the first error encountered, +// if any. +func (ds DeleteOffsetsResponses) Error() error { + for _, ps := range ds { + for _, err := range ps { + if err != nil { + return err + } + } + } + return nil +} + +// DeleteOffsets deletes offsets for the given group. +// +// Originally, offset commits were persisted in Kafka for some retention time. +// This posed problematic for infrequently committing consumers, so the +// retention time concept was removed in Kafka v2.1 in favor of deleting +// offsets for a group only when the group became empty. However, if a group +// stops consuming from a topic, then the offsets will persist and lag +// monitoring for the group will notice an ever increasing amount of lag for +// these no-longer-consumed topics. Thus, Kafka v2.4 introduced an OffsetDelete +// request to allow admins to manually delete offsets for no longer consumed +// topics. +// +// This method requires talking to Kafka v2.4+. This returns an *AuthErr if the +// user is not authorized to delete offsets in the group at all. This does not +// return on per-topic authorization failures, instead, per-topic authorization +// failures are included in the responses. +func (cl *Client) DeleteOffsets(ctx context.Context, group string, s TopicsSet) (DeleteOffsetsResponses, error) { + if len(s) == 0 { + return nil, nil + } + + req := kmsg.NewPtrOffsetDeleteRequest() + req.Group = group + for t, ps := range s { + rt := kmsg.NewOffsetDeleteRequestTopic() + rt.Topic = t + for p := range ps { + rp := kmsg.NewOffsetDeleteRequestTopicPartition() + rp.Partition = p + rt.Partitions = append(rt.Partitions, rp) + } + req.Topics = append(req.Topics, rt) + } + + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return nil, err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return nil, err + } + + r := make(DeleteOffsetsResponses) + for _, t := range resp.Topics { + rt := make(map[int32]error) + r[t.Topic] = rt + for _, p := range t.Partitions { + rt[p.Partition] = kerr.ErrorForCode(p.ErrorCode) + } + } + return r, nil +} + +// GroupMemberLag is the lag between a group member's current offset commit and +// the current end offset. +// +// If either the offset commits have load errors, or the listed end offsets +// have load errors, the Lag field will be -1 and the Err field will be set (to +// the first of either the commit error, or else the list error). +// +// If the group is in the Empty state, lag is calculated for all partitions in +// a topic, but the member is nil. The calculate function assumes that any +// assigned topic is meant to be entirely consumed. If the group is Empty and +// topics could not be listed, some partitions may be missing. +type GroupMemberLag struct { + // Member is a reference to the group member consuming this partition. + // If the group is in state Empty, the member will be nil. + Member *DescribedGroupMember + Topic string // Topic is the topic this lag is for. + Partition int32 // Partition is the partition this lag is for. + + Commit Offset // Commit is this member's current offset commit. + Start ListedOffset // Start is a reference to the start of this partition, if provided. Start offsets are optional; if not provided, Start.Err is a non-nil error saying this partition is missing from list offsets. This is always present if lag is calculated via Client.Lag. + End ListedOffset // End is a reference to the end offset of this partition. + Lag int64 // Lag is how far behind this member is, or -1 if there is a commit error or list offset error. + + Err error // Err is either the commit error, or the list end offsets error, or nil. +} + +// IsEmpty returns if the this lag is for a group in the Empty state. +func (g *GroupMemberLag) IsEmpty() bool { return g.Member == nil } + +// GroupLag is the per-topic, per-partition lag of members in a group. +type GroupLag map[string]map[int32]GroupMemberLag + +// Lookup returns the lag at t and p and whether it exists. +func (l GroupLag) Lookup(t string, p int32) (GroupMemberLag, bool) { + if len(l) == 0 { + return GroupMemberLag{}, false + } + ps := l[t] + if len(ps) == 0 { + return GroupMemberLag{}, false + } + m, exists := ps[p] + return m, exists +} + +// Sorted returns the per-topic, per-partition lag by member sorted in order by +// topic then partition. +func (l GroupLag) Sorted() []GroupMemberLag { + var all []GroupMemberLag + for _, ps := range l { + for _, l := range ps { + all = append(all, l) + } + } + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + if l.Topic < r.Topic { + return true + } + if l.Topic > r.Topic { + return false + } + return l.Partition < r.Partition + }) + return all +} + +// IsEmpty returns if the group is empty. +func (l GroupLag) IsEmpty() bool { + for _, ps := range l { + for _, m := range ps { + return m.IsEmpty() + } + } + return false +} + +// Total returns the total lag across all topics. +func (l GroupLag) Total() int64 { + var tot int64 + for _, tl := range l.TotalByTopic() { + tot += tl.Lag + } + return tot +} + +// TotalByTopic returns the total lag for each topic. +func (l GroupLag) TotalByTopic() GroupTopicsLag { + m := make(map[string]TopicLag) + for t, ps := range l { + mt := TopicLag{ + Topic: t, + } + for _, l := range ps { + if l.Lag > 0 { + mt.Lag += l.Lag + } + } + m[t] = mt + } + return m +} + +// GroupTopicsLag is the total lag per topic within a group. +type GroupTopicsLag map[string]TopicLag + +// TopicLag is the lag for an individual topic within a group. +type TopicLag struct { + Topic string + Lag int64 +} + +// Sorted returns the per-topic lag, sorted by topic. +func (l GroupTopicsLag) Sorted() []TopicLag { + var all []TopicLag + for _, tl := range l { + all = append(all, tl) + } + sort.Slice(all, func(i, j int) bool { + return all[i].Topic < all[j].Topic + }) + return all +} + +// DescribedGroupLag contains a described group and its lag, or the errors that +// prevent the lag from being calculated. +type DescribedGroupLag struct { + Group string // Group is the group name. + + Coordinator BrokerDetail // Coordinator is the coordinator broker for this group. + State string // State is the state this group is in (Empty, Dead, Stable, etc.). + ProtocolType string // ProtocolType is the type of protocol the group is using, "consumer" for normal consumers, "connect" for Kafka connect. + Protocol string // Protocol is the partition assignor strategy this group is using. + Members []DescribedGroupMember // Members contains the members of this group sorted first by InstanceID, or if nil, by MemberID. + Lag GroupLag // Lag is the lag for the group. + + DescribeErr error // DescribeErr is the error returned from describing the group, if any. + FetchErr error // FetchErr is the error returned from fetching offsets, if any. +} + +// Err returns the first of DescribeErr or FetchErr that is non-nil. +func (l *DescribedGroupLag) Error() error { + if l.DescribeErr != nil { + return l.DescribeErr + } + return l.FetchErr +} + +// DescribedGroupLags is a map of group names to the described group with its +// lag, or error for those groups. +type DescribedGroupLags map[string]DescribedGroupLag + +// Sorted returns all lags sorted by group name. +func (ls DescribedGroupLags) Sorted() []DescribedGroupLag { + s := make([]DescribedGroupLag, 0, len(ls)) + for _, l := range ls { + s = append(s, l) + } + sort.Slice(s, func(i, j int) bool { return s[i].Group < s[j].Group }) + return s +} + +// EachError calls fn for every group that has a non-nil error. +func (ls DescribedGroupLags) EachError(fn func(l DescribedGroupLag)) { + for _, l := range ls { + if l.Error() != nil { + fn(l) + } + } +} + +// Each calls fn for every group. +func (ls DescribedGroupLags) Each(fn func(l DescribedGroupLag)) { + for _, l := range ls { + fn(l) + } +} + +// Error iterates over all groups and returns the first error encountered, if +// any. +func (ls DescribedGroupLags) Error() error { + for _, l := range ls { + if l.Error() != nil { + return l.Error() + } + } + return nil +} + +// Ok returns true if there are no errors. This is a shortcut for ls.Error() == +// nil. +func (ls DescribedGroupLags) Ok() bool { + return ls.Error() == nil +} + +// Lag returns the lag for all input groups. This function is a shortcut for +// the steps required to use CalculateGroupLagWithStartOffsets properly, with +// some opinionated choices for error handling since calculating lag is +// multi-request process. If a group cannot be described or the offsets cannot +// be fetched, an error is returned for the group. If any topic cannot have its +// end offsets listed, the lag for the partition has a corresponding error. If +// any request fails with an auth error, this returns *AuthError. +func (cl *Client) Lag(ctx context.Context, groups ...string) (DescribedGroupLags, error) { + set := make(map[string]struct{}, len(groups)) + for _, g := range groups { + set[g] = struct{}{} + } + rem := func() []string { + groups = groups[:0] + for g := range set { + groups = append(groups, g) + } + return groups + } + lags := make(DescribedGroupLags) + + described, err := cl.DescribeGroups(ctx, rem()...) + // For auth err: always return. + // For shard errors, if we had some partial success, then we continue + // to the rest of the logic in this function. + // If every shard failed, or on all other errors, we return. + var ae *AuthError + var se *ShardErrors + switch { + case errors.As(err, &ae): + return nil, err + case errors.As(err, &se) && !se.AllFailed: + for _, se := range se.Errs { + for _, g := range se.Req.(*kmsg.DescribeGroupsRequest).Groups { + lags[g] = DescribedGroupLag{ + Group: g, + Coordinator: se.Broker, + DescribeErr: se.Err, + } + delete(set, g) + } + } + case err != nil: + return nil, err + } + for _, g := range described { + lags[g.Group] = DescribedGroupLag{ + Group: g.Group, + Coordinator: g.Coordinator, + State: g.State, + ProtocolType: g.ProtocolType, + Protocol: g.Protocol, + Members: g.Members, + DescribeErr: g.Err, + } + if g.Err != nil { + delete(set, g.Group) + continue + } + + // If the input set of groups is empty, DescribeGroups returns all groups. + // We add to `set` here so that the Lag function itself can calculate + // lag for all groups. + set[g.Group] = struct{}{} + } + if len(set) == 0 { + return lags, nil + } + + // Same thought here. For auth errors, we always return. + // If a group offset fetch failed, we delete it from described + // because we cannot calculate lag for it. + fetched := cl.FetchManyOffsets(ctx, rem()...) + for _, r := range fetched { + switch { + case errors.As(r.Err, &ae): + return nil, err + case r.Err != nil: + l := lags[r.Group] + l.FetchErr = r.Err + lags[r.Group] = l + delete(set, r.Group) + delete(described, r.Group) + } + } + if len(set) == 0 { + return lags, nil + } + + // We have to list the start & end offset for all assigned and + // committed partitions. + var startOffsets, endOffsets ListedOffsets + listPartitions := described.AssignedPartitions() + listPartitions.Merge(fetched.CommittedPartitions()) + if topics := listPartitions.Topics(); len(topics) > 0 { + for _, list := range []struct { + fn func(context.Context, ...string) (ListedOffsets, error) + dst *ListedOffsets + }{ + {cl.ListStartOffsets, &startOffsets}, + {cl.ListEndOffsets, &endOffsets}, + } { + listed, err := list.fn(ctx, topics...) + *list.dst = listed + // As above: return on auth error. If there are shard errors, + // the topics will be missing in the response and then + // CalculateGroupLag will return UnknownTopicOrPartition. + switch { + case errors.As(err, &ae): + return nil, err + case errors.As(err, &se): + // do nothing: these show up as errListMissing + case err != nil: + return nil, err + } + // For anything that lists with a single -1 partition, the + // topic does not exist. We add an UnknownTopicOrPartition + // error for all partitions that were committed to, so that + // this shows up in the lag output as UnknownTopicOrPartition + // rather than errListMissing. + for t, ps := range listed { + if len(ps) != 1 { + continue + } + if _, ok := ps[-1]; !ok { + continue + } + delete(ps, -1) + for p := range listPartitions[t] { + ps[p] = ListedOffset{ + Topic: t, + Partition: p, + Err: kerr.UnknownTopicOrPartition, + } + } + } + } + } + + for _, g := range described { + l := lags[g.Group] + l.Lag = CalculateGroupLagWithStartOffsets(g, fetched[g.Group].Fetched, startOffsets, endOffsets) + lags[g.Group] = l + } + return lags, nil +} + +var noOffsets = make(ListedOffsets) + +// CalculateGroupLagWithStartOffsets returns the per-partition lag of all +// members in a group. This function slightly expands on [CalculateGroupLag] to +// handle calculating lag for partitions that (1) have no commits AND (2) have +// some segments deleted (cleanup.policy=delete) such that the log start offset +// is non-zero. +// +// As an example, if a group is consuming a partition with log end offset 30 +// and log start offset 10 and has not yet committed to the group, this +// function can correctly tell you that the lag is 20, whereas +// CalculateGroupLag would tell you the lag is 30. +// +// This function accepts 'nil' for startOffsets, which will result in the same +// behavior as CalculateGroupLag. This function is useful if you have +// infrequently committing groups against topics that have segments being +// deleted. +func CalculateGroupLagWithStartOffsets( + group DescribedGroup, + commit OffsetResponses, + startOffsets ListedOffsets, + endOffsets ListedOffsets, +) GroupLag { + if commit == nil { // avoid panics below + commit = make(OffsetResponses) + } + if startOffsets == nil { + startOffsets = noOffsets + } + if endOffsets == nil { + endOffsets = noOffsets + } + if group.State == "Empty" { + return calculateEmptyLag(commit, startOffsets, endOffsets) + } + + l := make(map[string]map[int32]GroupMemberLag) + for mi, m := range group.Members { + c, ok := m.Assigned.AsConsumer() + if !ok { + continue + } + for _, t := range c.Topics { + lt := l[t.Topic] + if lt == nil { + lt = make(map[int32]GroupMemberLag) + l[t.Topic] = lt + } + + tcommit := commit[t.Topic] + tstart := startOffsets[t.Topic] + tend := endOffsets[t.Topic] + for _, p := range t.Partitions { + var ( + pcommit = OffsetResponse{Offset: Offset{ + Topic: t.Topic, + Partition: p, + At: -1, + }} + pend = ListedOffset{ + Topic: t.Topic, + Partition: p, + Err: errListMissing, + } + pstart = pend + perr error + ) + + if tcommit != nil { + if pcommitActual, ok := tcommit[p]; ok { + pcommit = pcommitActual + } + } + perr = errListMissing + if tend != nil { + if pendActual, ok := tend[p]; ok { + pend = pendActual + perr = nil + } + } + if perr == nil { + if perr = pcommit.Err; perr == nil { + perr = pend.Err + } + } + if tstart != nil { + if pstartActual, ok := tstart[p]; ok { + pstart = pstartActual + } + } + + lag := int64(-1) + if perr == nil { + lag = pend.Offset + if pstart.Err == nil { + lag = pend.Offset - pstart.Offset + } + if pcommit.At >= 0 { + lag = pend.Offset - pcommit.At + } + // It is possible for a commit to be after the + // end, in which case we will round to 0. We do + // this check here to also handle a potential non-commit + // weird pend < pstart scenario where a segment + // was deleted between listing offsets. + if lag < 0 { + lag = 0 + } + } + + lt[p] = GroupMemberLag{ + Member: &group.Members[mi], + Topic: t.Topic, + Partition: p, + Commit: pcommit.Offset, + Start: pstart, + End: pend, + Lag: lag, + Err: perr, + } + + } + } + } + + return l +} + +// CalculateGroupLag returns the per-partition lag of all members in a group. +// The input to this method is the returns from the following methods (make +// sure to check shard errors): +// +// // Note that FetchOffsets exists to fetch only one group's offsets, +// // but some of the code below slightly changes. +// groups := DescribeGroups(ctx, group) +// commits := FetchManyOffsets(ctx, group) +// var endOffsets ListedOffsets +// listPartitions := described.AssignedPartitions() +// listPartitions.Merge(commits.CommittedPartitions() +// if topics := listPartitions.Topics(); len(topics) > 0 { +// endOffsets = ListEndOffsets(ctx, listPartitions.Topics()) +// } +// for _, group := range groups { +// lag := CalculateGroupLag(group, commits[group.Group].Fetched, endOffsets) +// } +// +// If assigned partitions are missing in the listed end offsets, the partition +// will have an error indicating it is missing. A missing topic or partition in +// the commits is assumed to be nothing committing yet. +func CalculateGroupLag( + group DescribedGroup, + commit OffsetResponses, + endOffsets ListedOffsets, +) GroupLag { + return CalculateGroupLagWithStartOffsets(group, commit, nil, endOffsets) +} + +func calculateEmptyLag(commit OffsetResponses, startOffsets, endOffsets ListedOffsets) GroupLag { + l := make(map[string]map[int32]GroupMemberLag) + for t, ps := range commit { + lt := l[t] + if lt == nil { + lt = make(map[int32]GroupMemberLag) + l[t] = lt + } + tstart := startOffsets[t] + tend := endOffsets[t] + for p, pcommit := range ps { + var ( + pend = ListedOffset{ + Topic: t, + Partition: p, + Err: errListMissing, + } + pstart = pend + perr error + ) + + // In order of priority, perr (the error on the Lag + // calculation) is non-nil if: + // + // * The topic is missing from end ListOffsets + // * The partition is missing from end ListOffsets + // * OffsetFetch has an error on the partition + // * ListOffsets has an error on the partition + // + // If we have no error, then we can calculate lag. + // We *do* allow an error on start ListedOffsets; + // if there are no start offsets or the start offset + // has an error, it is not used for lag calculation. + perr = errListMissing + if tend != nil { + if pendActual, ok := tend[p]; ok { + pend = pendActual + perr = nil + } + } + if perr == nil { + if perr = pcommit.Err; perr == nil { + perr = pend.Err + } + } + if tstart != nil { + if pstartActual, ok := tstart[p]; ok { + pstart = pstartActual + } + } + + lag := int64(-1) + if perr == nil { + lag = pend.Offset + if pstart.Err == nil { + lag = pend.Offset - pstart.Offset + } + if pcommit.At >= 0 { + lag = pend.Offset - pcommit.At + } + if lag < 0 { + lag = 0 + } + } + + lt[p] = GroupMemberLag{ + Topic: t, + Partition: p, + Commit: pcommit.Offset, + Start: pstart, + End: pend, + Lag: lag, + Err: perr, + } + } + } + + // Now we look at all topics that we calculated lag for, and check out + // the partitions we listed. If those partitions are missing from the + // lag calculations above, the partitions were not committed to and we + // count that as entirely lagging. + for t, lt := range l { + tstart := startOffsets[t] + tend := endOffsets[t] + for p, pend := range tend { + if _, ok := lt[p]; ok { + continue + } + pcommit := Offset{ + Topic: t, + Partition: p, + At: -1, + LeaderEpoch: -1, + } + perr := pend.Err + lag := int64(-1) + if perr == nil { + lag = pend.Offset + } + pstart := ListedOffset{ + Topic: t, + Partition: p, + Err: errListMissing, + } + if tstart != nil { + if pstartActual, ok := tstart[p]; ok { + pstart = pstartActual + if pstart.Err == nil { + lag = pend.Offset - pstart.Offset + if lag < 0 { + lag = 0 + } + } + } + } + lt[p] = GroupMemberLag{ + Topic: t, + Partition: p, + Commit: pcommit, + Start: pstart, + End: pend, + Lag: lag, + Err: perr, + } + } + } + + return l +} + +var errListMissing = errors.New("missing from list offsets") diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/kadm.go b/vendor/github.com/twmb/franz-go/pkg/kadm/kadm.go new file mode 100644 index 000000000000..432fc4c76a3c --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/kadm.go @@ -0,0 +1,655 @@ +// Package kadm provides a helper Kafka admin client around a *kgo.Client. +// +// This package is meant to cover the common use cases for dropping into an +// "admin" like interface for Kafka. As with any admin client, this package +// must make opinionated decisions on what to provide and what to hide. The +// underlying Kafka protocol gives more detailed information in responses, or +// allows more fine tuning in requests, but most of the time, these details are +// unnecessary. +// +// By virtue of making opinionated decisions, this package cannot satisfy every +// need for requests and responses. If you need more control than this admin +// client provides, you can use the kmsg package directly. +// +// This package contains a lot of types, but the main two types type to know +// are Client and ShardErrors. Every other type is used for inputs or outputs +// to methods on the client. +// +// The Client type is a simple small wrapper around a *kgo.Client that exists +// solely to namespace methods. The ShardErrors type is a bit more complicated. +// When issuing requests, under the hood some of these requests actually need +// to be mapped to brokers and split, issuing different pieces of the input +// request to different brokers. The *kgo.Client handles this all internally, +// but (if using RequestSharded as directed), returns each response to each of +// these split requests individually. Each response can fail or be successful. +// This package goes one step further and merges these failures into one meta +// failure, ShardErrors. Any function that returns ShardErrors is documented as +// such, and if a function returns a non-nil ShardErrors, it is possible that +// the returned data is actually valid and usable. If you care to, you can log +// / react to the partial failures and continue using the partial successful +// result. This is in contrast to other clients, which either require to to +// request individual brokers directly, or they completely hide individual +// failures, or they completely fail on any individual failure. +// +// For methods that list or describe things, this package often completely +// fails responses on auth failures. If you use a method that accepts two +// topics, one that you are authorized to and one that you are not, you will +// not receive a partial successful response. Instead, you will receive an +// AuthError. Methods that do *not* fail on auth errors are explicitly +// documented as such. +// +// Users may often find it easy to work with lists of topics or partitions. +// Rather than needing to build deeply nested maps directly, this package has a +// few helper types that are worth knowing: +// +// TopicsList - a slice of topics and their partitions +// TopicsSet - a set of topics, each containing a set of partitions +// Partitions - a slice of partitions +// OffsetsList - a slice of offsets +// Offsets - a map of offsets +// +// These types are meant to be easy to build and use, and can be used as the +// starting point for other types. +// +// Many functions in this package are variadic and return either a map or a +// list of responses, and you may only use one element as input and are only +// interested in one element of output. This package provides the following +// functions to help: +// +// Any(map) +// AnyE(map, err) +// First(slice) +// FirstE(slice, err) +// +// The intended use case of these is something like `kadm.AnyE(kadm.CreateTopics(..., "my-one-topic"))`, +// such that you can immediately get the response for the one topic you are +// creating. +package kadm + +import ( + "errors" + "regexp" + "runtime/debug" + "sort" + "sync" + + "github.com/twmb/franz-go/pkg/kgo" +) + +func unptrStr(s *string) string { + if s == nil { + return "" + } + return *s +} + +var ( + reVersion *regexp.Regexp + reVersionOnce sync.Once +) + +// Copied from kgo, but we use the kadm package version. +func softwareVersion() string { + info, ok := debug.ReadBuildInfo() + if ok { + reVersionOnce.Do(func() { reVersion = regexp.MustCompile(`^[a-zA-Z0-9](?:[a-zA-Z0-9.-]*[a-zA-Z0-9])?$`) }) + for _, dep := range info.Deps { + if dep.Path == "github.com/twmb/franz-go/pkg/kadm" { + if reVersion.MatchString(dep.Version) { + return dep.Version + } + } + } + } + return "unknown" +} + +// Client is an admin client. +// +// This is a simple wrapper around a *kgo.Client to provide helper admin methods. +type Client struct { + cl *kgo.Client + + timeoutMillis int32 +} + +// NewClient returns an admin client. +func NewClient(cl *kgo.Client) *Client { + return &Client{cl, 15000} // 15s timeout default, matching kmsg +} + +// NewOptClient returns a new client directly from kgo options. This is a +// wrapper around creating a new *kgo.Client and then creating an admin client. +func NewOptClient(opts ...kgo.Opt) (*Client, error) { + cl, err := kgo.NewClient(opts...) + if err != nil { + return nil, err + } + return NewClient(cl), nil +} + +// Close closes the underlying *kgo.Client. +func (cl *Client) Close() { + cl.cl.Close() +} + +// SetTimeoutMillis sets the timeout to use for requests that have a timeout, +// overriding the default of 15,000 (15s). +// +// Not all requests have timeouts. Most requests are expected to return +// immediately or are expected to deliberately hang. The following requests +// have timeout fields: +// +// Produce +// CreateTopics +// DeleteTopics +// DeleteRecords +// CreatePartitions +// ElectLeaders +// AlterPartitionAssignments +// ListPartitionReassignments +// UpdateFeatures +// +// Not all requests above are supported in the admin API. +func (cl *Client) SetTimeoutMillis(millis int32) { + cl.timeoutMillis = millis +} + +// StringPtr is a shortcut function to aid building configs for creating or +// altering topics. +func StringPtr(s string) *string { + return &s +} + +// BrokerDetail is a type alias for kgo.BrokerMetadata. +type BrokerDetail = kgo.BrokerMetadata + +// BrokerDetails contains the details for many brokers. +type BrokerDetails []BrokerDetail + +// NodeIDs returns the IDs of all nodes. +func (ds BrokerDetails) NodeIDs() []int32 { + var all []int32 + for _, d := range ds { + all = append(all, d.NodeID) + } + return int32s(all) +} + +// Partition is a partition for a topic. +type Partition struct { + Topic string // Topic is the topic for this partition. + Partition int32 // Partition is this partition's number. +} + +// Offset is an offset for a topic. +type Offset struct { + Topic string + Partition int32 + At int64 // Offset is the partition to set. + LeaderEpoch int32 // LeaderEpoch is the broker leader epoch of the record at this offset. + Metadata string // Metadata, if non-empty, is used for offset commits. +} + +// Partitions wraps many partitions. +type Partitions []Partition + +// TopicsSet returns these partitions as TopicsSet. +func (ps Partitions) TopicsSet() TopicsSet { + s := make(TopicsSet) + for _, p := range ps { + s.Add(p.Topic, p.Partition) + } + return s +} + +// TopicsList returns these partitions as sorted TopicsList. +func (ps Partitions) TopicsList() TopicsList { + return ps.TopicsSet().Sorted() +} + +// OffsetsList wraps many offsets and is a helper for building Offsets. +type OffsetsList []Offset + +// Offsets returns this list as the non-list Offsets. All fields in each +// Offset must be set properly. +func (l OffsetsList) Offsets() Offsets { + os := make(Offsets) + for _, o := range l { + os.Add(o) + } + return os +} + +// KOffsets returns this list as a kgo offset map. +func (l OffsetsList) KOffsets() map[string]map[int32]kgo.Offset { + return l.Offsets().KOffsets() +} + +// Offsets wraps many offsets and is the type used for offset functions. +type Offsets map[string]map[int32]Offset + +// Lookup returns the offset at t and p and whether it exists. +func (os Offsets) Lookup(t string, p int32) (Offset, bool) { + if len(os) == 0 { + return Offset{}, false + } + ps := os[t] + if len(ps) == 0 { + return Offset{}, false + } + o, exists := ps[p] + return o, exists +} + +// Add adds an offset for a given topic/partition to this Offsets map. +// +// If the partition already exists, the offset is only added if: +// +// - the new leader epoch is higher than the old, or +// - the leader epochs equal, and the new offset is higher than the old +// +// If you would like to add offsets forcefully no matter what, use the Delete +// method before this. +func (os *Offsets) Add(o Offset) { + if *os == nil { + *os = make(map[string]map[int32]Offset) + } + ot := (*os)[o.Topic] + if ot == nil { + ot = make(map[int32]Offset) + (*os)[o.Topic] = ot + } + + prior, exists := ot[o.Partition] + if !exists || prior.LeaderEpoch < o.LeaderEpoch || + prior.LeaderEpoch == o.LeaderEpoch && prior.At < o.At { + ot[o.Partition] = o + } +} + +// Delete removes any offset at topic t and partition p. +func (os Offsets) Delete(t string, p int32) { + if os == nil { + return + } + ot := os[t] + if ot == nil { + return + } + delete(ot, p) + if len(ot) == 0 { + delete(os, t) + } +} + +// AddOffset is a helper to add an offset for a given topic and partition. The +// leader epoch field must be -1 if you do not know the leader epoch or if +// you do not have an offset yet. +func (os *Offsets) AddOffset(t string, p int32, o int64, leaderEpoch int32) { + os.Add(Offset{ + Topic: t, + Partition: p, + At: o, + LeaderEpoch: leaderEpoch, + }) +} + +// KeepFunc calls fn for every offset, keeping the offset if fn returns true. +func (os Offsets) KeepFunc(fn func(o Offset) bool) { + for t, ps := range os { + for p, o := range ps { + if !fn(o) { + delete(ps, p) + } + } + if len(ps) == 0 { + delete(os, t) + } + } +} + +// DeleteFunc calls fn for every offset, deleting the offset if fn returns +// true. +func (os Offsets) DeleteFunc(fn func(o Offset) bool) { + os.KeepFunc(func(o Offset) bool { return !fn(o) }) +} + +// Topics returns the set of topics and partitions currently used in these +// offsets. +func (os Offsets) TopicsSet() TopicsSet { + s := make(TopicsSet) + os.Each(func(o Offset) { s.Add(o.Topic, o.Partition) }) + return s +} + +// Each calls fn for each offset in these offsets. +func (os Offsets) Each(fn func(Offset)) { + for _, ps := range os { + for _, o := range ps { + fn(o) + } + } +} + +// KOffsets returns these offsets as a kgo offset map. +func (os Offsets) KOffsets() map[string]map[int32]kgo.Offset { + tskgo := make(map[string]map[int32]kgo.Offset) + for t, ps := range os { + pskgo := make(map[int32]kgo.Offset) + for p, o := range ps { + pskgo[p] = kgo.NewOffset(). + At(o.At). + WithEpoch(o.LeaderEpoch) + } + tskgo[t] = pskgo + } + return tskgo +} + +// Sorted returns the offsets sorted by topic and partition. +func (os Offsets) Sorted() []Offset { + var s []Offset + os.Each(func(o Offset) { s = append(s, o) }) + sort.Slice(s, func(i, j int) bool { + return s[i].Topic < s[j].Topic || + s[i].Topic == s[j].Topic && s[i].Partition < s[j].Partition + }) + return s +} + +// OffsetsFromFetches returns Offsets for the final record in any partition in +// the fetches. This is a helper to enable committing an entire returned batch. +// +// This function looks at only the last record per partition, assuming that the +// last record is the highest offset (which is the behavior returned by kgo's +// Poll functions). The returned offsets are one past the offset contained in +// the records. +func OffsetsFromFetches(fs kgo.Fetches) Offsets { + os := make(Offsets) + fs.EachPartition(func(p kgo.FetchTopicPartition) { + if len(p.Records) == 0 { + return + } + r := p.Records[len(p.Records)-1] + os.AddOffset(r.Topic, r.Partition, r.Offset+1, r.LeaderEpoch) + }) + return os +} + +// OffsetsFromRecords returns offsets for all given records, using the highest +// offset per partition. The returned offsets are one past the offset contained +// in the records. +func OffsetsFromRecords(rs ...kgo.Record) Offsets { + os := make(Offsets) + for _, r := range rs { + os.AddOffset(r.Topic, r.Partition, r.Offset+1, r.LeaderEpoch) + } + return os +} + +// TopicsSet is a set of topics and, per topic, a set of partitions. +// +// All methods provided for TopicsSet are safe to use on a nil (default) set. +type TopicsSet map[string]map[int32]struct{} + +// Lookup returns whether the topic and partition exists. +func (s TopicsSet) Lookup(t string, p int32) bool { + if len(s) == 0 { + return false + } + ps := s[t] + if len(ps) == 0 { + return false + } + _, exists := ps[p] + return exists +} + +// Each calls fn for each topic / partition in the topics set. +func (s TopicsSet) Each(fn func(t string, p int32)) { + for t, ps := range s { + for p := range ps { + fn(t, p) + } + } +} + +// EachPartitions calls fn for each topic and its partitions in the topics set. +func (s TopicsSet) EachPartitions(fn func(t string, ps []int32)) { + for t, ps := range s { + sliced := make([]int32, 0, len(ps)) + for p := range ps { + sliced = append(sliced, p) + } + fn(t, sliced) + } +} + +// EmptyTopics returns all topics with no partitions. +func (s TopicsSet) EmptyTopics() []string { + var e []string + for t, ps := range s { + if len(ps) == 0 { + e = append(e, t) + } + } + return e +} + +// Add adds partitions for a topic to the topics set. If no partitions are +// added, this still creates the topic. +func (s *TopicsSet) Add(t string, ps ...int32) { + if *s == nil { + *s = make(map[string]map[int32]struct{}) + } + existing := (*s)[t] + if existing == nil { + existing = make(map[int32]struct{}, len(ps)) + (*s)[t] = existing + } + for _, p := range ps { + existing[p] = struct{}{} + } +} + +// Delete removes partitions from a topic from the topics set. If the topic +// ends up with no partitions, the topic is removed from the set. +func (s TopicsSet) Delete(t string, ps ...int32) { + if s == nil || len(ps) == 0 { + return + } + existing := s[t] + if existing == nil { + return + } + for _, p := range ps { + delete(existing, p) + } + if len(existing) == 0 { + delete(s, t) + } +} + +// Topics returns all topics in this set in sorted order. +func (s TopicsSet) Topics() []string { + ts := make([]string, 0, len(s)) + for t := range s { + ts = append(ts, t) + } + sort.Strings(ts) + return ts +} + +// Merge merges another topic set into this one. +func (s TopicsSet) Merge(other TopicsSet) { + for t, ps := range other { + for p := range ps { + s.Add(t, p) + } + } +} + +// IntoList returns this set as a list. +func (s TopicsSet) IntoList() TopicsList { + l := make(TopicsList, 0, len(s)) + for t, ps := range s { + lps := make([]int32, 0, len(ps)) + for p := range ps { + lps = append(lps, p) + } + l = append(l, TopicPartitions{ + Topic: t, + Partitions: lps, + }) + } + return l +} + +// Sorted returns this set as a list in topic-sorted order, with each topic +// having sorted partitions. +func (s TopicsSet) Sorted() TopicsList { + l := make(TopicsList, 0, len(s)) + for t, ps := range s { + tps := TopicPartitions{ + Topic: t, + Partitions: make([]int32, 0, len(ps)), + } + for p := range ps { + tps.Partitions = append(tps.Partitions, p) + } + tps.Partitions = int32s(tps.Partitions) + l = append(l, tps) + } + sort.Slice(l, func(i, j int) bool { return l[i].Topic < l[j].Topic }) + return l +} + +// TopicPartitions is a topic and partitions. +type TopicPartitions struct { + Topic string + Partitions []int32 +} + +// TopicsList is a list of topics and partitions. +type TopicsList []TopicPartitions + +// Each calls fn for each topic / partition in the topics list. +func (l TopicsList) Each(fn func(t string, p int32)) { + for _, t := range l { + for _, p := range t.Partitions { + fn(t.Topic, p) + } + } +} + +// EachPartitions calls fn for each topic and its partitions in the topics +// list. +func (l TopicsList) EachPartitions(fn func(t string, ps []int32)) { + for _, t := range l { + fn(t.Topic, t.Partitions) + } +} + +// EmptyTopics returns all topics with no partitions. +func (l TopicsList) EmptyTopics() []string { + var e []string + for _, t := range l { + if len(t.Partitions) == 0 { + e = append(e, t.Topic) + } + } + return e +} + +// Topics returns all topics in this set in sorted order. +func (l TopicsList) Topics() []string { + ts := make([]string, 0, len(l)) + for _, t := range l { + ts = append(ts, t.Topic) + } + sort.Strings(ts) + return ts +} + +// IntoSet returns this list as a set. +func (l TopicsList) IntoSet() TopicsSet { + s := make(TopicsSet) + for _, t := range l { + s.Add(t.Topic, t.Partitions...) + } + return s +} + +// First returns the first element of the input slice and whether it exists. +// This is the non-error-accepting equivalent of FirstE. +// +// Many client methods in kadm accept a variadic amount of input arguments and +// return either a slice or a map of responses, but you often use the method +// with only one argument. This function can help extract the one response you +// are interested in. +func First[S ~[]T, T any](s S) (T, bool) { + if len(s) == 0 { + var t T + return t, false + } + return s[0], true +} + +// Any returns the first range element of the input map and whether it exists. +// This is the non-error-accepting equivalent of AnyE. +// +// Many client methods in kadm accept a variadic amount of input arguments and +// return either a slice or a map of responses, but you often use the method +// with only one argument. This function can help extract the one response you +// are interested in. +func Any[M ~map[K]V, K comparable, V any](m M) (V, bool) { + for _, v := range m { + return v, true + } + var v V + return v, false +} + +// ErrEmpty is returned from FirstE or AnyE if the input is empty. +var ErrEmpty = errors.New("empty") + +// FirstE returns the first element of the input slice, or the input error +// if it is non-nil. If the error is nil but the slice is empty, this returns +// ErrEmpty. This is the error-accepting equivalent of First. +// +// Many client methods in kadm accept a variadic amount of input arguments and +// return either a slice or a map of responses, but you often use the method +// with only one argument. This function can help extract the one response you +// are interested in. +func FirstE[S ~[]T, T any](s S, err error) (T, error) { + if err != nil { + var t T + return t, err + } + if len(s) == 0 { + var t T + return t, ErrEmpty + } + return s[0], err +} + +// AnyE returns the first range element of the input map, or the input error if +// it is non-nil. If the error is nil but the map is empty, this returns +// ErrEmpty. This is the error-accepting equivalent of Any. +// +// Many client methods in kadm accept a variadic amount of input arguments and +// return either a slice or a map of responses, but you often use the method +// with only one argument. This function can help extract the one response you +// are interested in. +func AnyE[M ~map[K]V, K comparable, V any](m M, err error) (V, error) { + if err != nil { + var v V + return v, err + } + for _, v := range m { + return v, nil + } + var v V + return v, ErrEmpty +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/logdirs.go b/vendor/github.com/twmb/franz-go/pkg/kadm/logdirs.go new file mode 100644 index 000000000000..c1487cbcce75 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/logdirs.go @@ -0,0 +1,592 @@ +package kadm + +import ( + "context" + "sort" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// AlterReplicaLogDirsReq is the input for a request to alter replica log +// directories. The key is the directory that all topics and partitions in +// the topic set will move to. +type AlterReplicaLogDirsReq map[string]TopicsSet + +// Add merges the input topic set into the given directory. +func (r *AlterReplicaLogDirsReq) Add(d string, s TopicsSet) { + if *r == nil { + *r = make(map[string]TopicsSet) + } + existing := (*r)[d] + if existing == nil { + existing = make(TopicsSet) + (*r)[d] = existing + } + existing.Merge(s) +} + +func (r AlterReplicaLogDirsReq) req() *kmsg.AlterReplicaLogDirsRequest { + req := kmsg.NewPtrAlterReplicaLogDirsRequest() + for dir, ts := range r { + rd := kmsg.NewAlterReplicaLogDirsRequestDir() + rd.Dir = dir + for t, ps := range ts { + rt := kmsg.NewAlterReplicaLogDirsRequestDirTopic() + rt.Topic = t + for p := range ps { + rt.Partitions = append(rt.Partitions, p) + } + rd.Topics = append(rd.Topics, rt) + } + req.Dirs = append(req.Dirs, rd) + } + return req +} + +func (r AlterReplicaLogDirsReq) dirfor(t string, p int32) string { + for d, dts := range r { + if dts == nil { + continue + } + dtps, ok := dts[t] // does this dir contain this topic? + if !ok { + continue + } + if _, ok = dtps[p]; !ok { // does this topic in this dir contain this partition? + continue + } + return d // yes + } + return "" +} + +// AlterAllReplicaLogDirsResponses contains per-broker responses to altered +// partition directories. +type AlterAllReplicaLogDirsResponses map[int32]AlterReplicaLogDirsResponses + +// Sorted returns the responses sorted by broker, topic, and partition. +func (rs AlterAllReplicaLogDirsResponses) Sorted() []AlterReplicaLogDirsResponse { + var all []AlterReplicaLogDirsResponse + rs.Each(func(r AlterReplicaLogDirsResponse) { + all = append(all, r) + }) + sort.Slice(all, func(i, j int) bool { return all[i].Less(all[j]) }) + return all +} + +// Each calls fn for every response. +func (rs AlterAllReplicaLogDirsResponses) Each(fn func(AlterReplicaLogDirsResponse)) { + for _, ts := range rs { + ts.Each(fn) + } +} + +// AlterReplicaLogDirsResponses contains responses to altered partition +// directories for a single broker. +type AlterReplicaLogDirsResponses map[string]map[int32]AlterReplicaLogDirsResponse + +// Sorted returns the responses sorted by topic and partition. +func (rs AlterReplicaLogDirsResponses) Sorted() []AlterReplicaLogDirsResponse { + var all []AlterReplicaLogDirsResponse + rs.Each(func(r AlterReplicaLogDirsResponse) { + all = append(all, r) + }) + sort.Slice(all, func(i, j int) bool { return all[i].Less(all[j]) }) + return all +} + +// Each calls fn for every response. +func (rs AlterReplicaLogDirsResponses) Each(fn func(AlterReplicaLogDirsResponse)) { + for _, ps := range rs { + for _, r := range ps { + fn(r) + } + } +} + +// AlterReplicaLogDirsResponse contains a the response for an individual +// altered partition directory. +type AlterReplicaLogDirsResponse struct { + Broker int32 // Broker is the broker this response came from. + Dir string // Dir is the directory this partition was requested to be moved to. + Topic string // Topic is the topic for this partition. + Partition int32 // Partition is the partition that was moved. + Err error // Err is non-nil if this move had an error. +} + +// Less returns if the response is less than the other by broker, dir, topic, +// and partition. +func (a AlterReplicaLogDirsResponse) Less(other AlterReplicaLogDirsResponse) bool { + if a.Broker < other.Broker { + return true + } + if a.Broker > other.Broker { + return false + } + if a.Dir < other.Dir { + return true + } + if a.Dir > other.Dir { + return false + } + if a.Topic < other.Topic { + return true + } + if a.Topic > other.Topic { + return false + } + return a.Partition < other.Partition +} + +func newAlterLogDirsResp(node int32, req AlterReplicaLogDirsReq, resp *kmsg.AlterReplicaLogDirsResponse) AlterReplicaLogDirsResponses { + a := make(AlterReplicaLogDirsResponses) + for _, kt := range resp.Topics { + ps := make(map[int32]AlterReplicaLogDirsResponse) + a[kt.Topic] = ps + for _, kp := range kt.Partitions { + ps[kp.Partition] = AlterReplicaLogDirsResponse{ + Broker: node, + Dir: req.dirfor(kt.Topic, kp.Partition), + Topic: kt.Topic, + Partition: kp.Partition, + Err: kerr.ErrorForCode(kp.ErrorCode), + } + } + } + return a +} + +// AlterAllReplicaLogDirs alters the log directories for the input topic +// partitions, moving each partition to the requested directory. This function +// moves all replicas on any broker. +// +// This may return *ShardErrors. +func (cl *Client) AlterAllReplicaLogDirs(ctx context.Context, alter AlterReplicaLogDirsReq) (AlterAllReplicaLogDirsResponses, error) { + if len(alter) == 0 { + return make(AlterAllReplicaLogDirsResponses), nil + } + req := alter.req() + shards := cl.cl.RequestSharded(ctx, req) + resps := make(AlterAllReplicaLogDirsResponses) + return resps, shardErrEachBroker(req, shards, func(b BrokerDetail, kr kmsg.Response) error { + resp := kr.(*kmsg.AlterReplicaLogDirsResponse) + resps[b.NodeID] = newAlterLogDirsResp(b.NodeID, alter, resp) // one node ID, no need to unique-check + return nil + }) +} + +// AlterBrokerReplicaLogDirs alters the log directories for the input topic on the +// given broker, moving each partition to the requested directory. +func (cl *Client) AlterBrokerReplicaLogDirs(ctx context.Context, broker int32, alter AlterReplicaLogDirsReq) (AlterReplicaLogDirsResponses, error) { + if len(alter) == 0 { + return make(AlterReplicaLogDirsResponses), nil + } + b := cl.cl.Broker(int(broker)) + kresp, err := b.RetriableRequest(ctx, alter.req()) + if err != nil { + return nil, err + } + resp := kresp.(*kmsg.AlterReplicaLogDirsResponse) + return newAlterLogDirsResp(broker, alter, resp), nil +} + +func describeLogDirsReq(s TopicsSet) *kmsg.DescribeLogDirsRequest { + req := kmsg.NewPtrDescribeLogDirsRequest() + for t, ps := range s { + rt := kmsg.NewDescribeLogDirsRequestTopic() + rt.Topic = t + for p := range ps { + rt.Partitions = append(rt.Partitions, p) + } + req.Topics = append(req.Topics, rt) + } + return req +} + +// DescribedAllLogDirs contains per-broker responses to described log +// directories. +type DescribedAllLogDirs map[int32]DescribedLogDirs + +// Sorted returns each log directory sorted by broker, then by directory. +func (ds DescribedAllLogDirs) Sorted() []DescribedLogDir { + var all []DescribedLogDir + ds.Each(func(d DescribedLogDir) { + all = append(all, d) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Broker < r.Broker || l.Broker == r.Broker && l.Dir < r.Dir + }) + return all +} + +// Each calls fn for every described log dir in all responses. +func (ds DescribedAllLogDirs) Each(fn func(DescribedLogDir)) { + for _, bds := range ds { + bds.Each(fn) + } +} + +// DescribedLogDirs contains per-directory responses to described log +// directories for a single broker. +type DescribedLogDirs map[string]DescribedLogDir + +// Lookup returns the described partition if it exists. +func (ds DescribedLogDirs) Lookup(d, t string, p int32) (DescribedLogDirPartition, bool) { + dir, exists := ds[d] + if !exists { + return DescribedLogDirPartition{}, false + } + ps, exists := dir.Topics[t] + if !exists { + return DescribedLogDirPartition{}, false + } + dp, exists := ps[p] + if !exists { + return DescribedLogDirPartition{}, false + } + return dp, true +} + +// LookupPartition returns the described partition if it exists in any +// directory. Brokers should only have one replica of a partition, so this +// should always find at most one partition. +func (ds DescribedLogDirs) LookupPartition(t string, p int32) (DescribedLogDirPartition, bool) { + for _, dir := range ds { + ps, exists := dir.Topics[t] + if !exists { + continue + } + dp, exists := ps[p] + if !exists { + continue + } + return dp, true + } + return DescribedLogDirPartition{}, false +} + +// Size returns the total size of all directories. +func (ds DescribedLogDirs) Size() int64 { + var tot int64 + ds.EachPartition(func(d DescribedLogDirPartition) { + tot += d.Size + }) + return tot +} + +// Error iterates over all directories and returns the first error encounted, +// if any. This can be used to check if describing was entirely successful or +// not. +func (ds DescribedLogDirs) Error() error { + for _, d := range ds { + if d.Err != nil { + return d.Err + } + } + return nil +} + +// Ok returns true if there are no errors. This is a shortcut for ds.Error() == +// nil. +func (ds DescribedLogDirs) Ok() bool { + return ds.Error() == nil +} + +// Sorted returns all directories sorted by dir. +func (ds DescribedLogDirs) Sorted() []DescribedLogDir { + var all []DescribedLogDir + ds.Each(func(d DescribedLogDir) { + all = append(all, d) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Broker < r.Broker || l.Broker == r.Broker && l.Dir < r.Dir + }) + return all +} + +// SortedPartitions returns all partitions sorted by dir, then topic, then +// partition. +func (ds DescribedLogDirs) SortedPartitions() []DescribedLogDirPartition { + var all []DescribedLogDirPartition + ds.EachPartition(func(d DescribedLogDirPartition) { + all = append(all, d) + }) + sort.Slice(all, func(i, j int) bool { return all[i].Less(all[j]) }) + return all +} + +// SortedBySize returns all directories sorted from smallest total directory +// size to largest. +func (ds DescribedLogDirs) SortedBySize() []DescribedLogDir { + var all []DescribedLogDir + ds.Each(func(d DescribedLogDir) { + all = append(all, d) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + ls, rs := l.Size(), r.Size() + return ls < rs || ls == rs && + (l.Broker < r.Broker || l.Broker == r.Broker && + l.Dir < r.Dir) + }) + return all +} + +// SortedPartitionsBySize returns all partitions across all directories sorted +// by smallest to largest, falling back to by broker, dir, topic, and +// partition. +func (ds DescribedLogDirs) SortedPartitionsBySize() []DescribedLogDirPartition { + var all []DescribedLogDirPartition + ds.EachPartition(func(d DescribedLogDirPartition) { + all = append(all, d) + }) + sort.Slice(all, func(i, j int) bool { return all[i].LessBySize(all[j]) }) + return all +} + +// SmallestPartitionBySize returns the smallest partition by directory size, or +// no partition if there are no partitions. +func (ds DescribedLogDirs) SmallestPartitionBySize() (DescribedLogDirPartition, bool) { + sorted := ds.SortedPartitionsBySize() + if len(sorted) == 0 { + return DescribedLogDirPartition{}, false + } + return sorted[0], true +} + +// LargestPartitionBySize returns the largest partition by directory size, or +// no partition if there are no partitions. +func (ds DescribedLogDirs) LargestPartitionBySize() (DescribedLogDirPartition, bool) { + sorted := ds.SortedPartitionsBySize() + if len(sorted) == 0 { + return DescribedLogDirPartition{}, false + } + return sorted[len(sorted)-1], true +} + +// Each calls fn for each log directory. +func (ds DescribedLogDirs) Each(fn func(DescribedLogDir)) { + for _, d := range ds { + fn(d) + } +} + +// Each calls fn for each partition in any directory. +func (ds DescribedLogDirs) EachPartition(fn func(d DescribedLogDirPartition)) { + for _, d := range ds { + d.Topics.Each(fn) + } +} + +// EachError calls fn for every directory that has a non-nil error. +func (ds DescribedLogDirs) EachError(fn func(DescribedLogDir)) { + for _, d := range ds { + if d.Err != nil { + fn(d) + } + } +} + +// DescribedLogDir is a described log directory. +type DescribedLogDir struct { + Broker int32 // Broker is the broker being described. + Dir string // Dir is the described directory. + Topics DescribedLogDirTopics // Partitions are the partitions in this directory. + Err error // Err is non-nil if this directory could not be described. +} + +// Size returns the total size of all partitions in this directory. This is +// a shortcut for .Topics.Size(). +func (ds DescribedLogDir) Size() int64 { + return ds.Topics.Size() +} + +// DescribedLogDirTopics contains per-partition described log directories. +type DescribedLogDirTopics map[string]map[int32]DescribedLogDirPartition + +// Lookup returns the described partition if it exists. +func (ds DescribedLogDirTopics) Lookup(t string, p int32) (DescribedLogDirPartition, bool) { + ps, exists := ds[t] + if !exists { + return DescribedLogDirPartition{}, false + } + d, exists := ps[p] + return d, exists +} + +// Size returns the total size of all partitions in this directory. +func (ds DescribedLogDirTopics) Size() int64 { + var tot int64 + ds.Each(func(d DescribedLogDirPartition) { + tot += d.Size + }) + return tot +} + +// Sorted returns all partitions sorted by topic then partition. +func (ds DescribedLogDirTopics) Sorted() []DescribedLogDirPartition { + var all []DescribedLogDirPartition + ds.Each(func(d DescribedLogDirPartition) { + all = append(all, d) + }) + sort.Slice(all, func(i, j int) bool { return all[i].Less(all[j]) }) + return all +} + +// SortedBySize returns all partitions sorted by smallest size to largest. If +// partitions are of equal size, the sorting is topic then partition. +func (ds DescribedLogDirTopics) SortedBySize() []DescribedLogDirPartition { + var all []DescribedLogDirPartition + ds.Each(func(d DescribedLogDirPartition) { + all = append(all, d) + }) + sort.Slice(all, func(i, j int) bool { return all[i].LessBySize(all[j]) }) + return all +} + +// Each calls fn for every partition. +func (ds DescribedLogDirTopics) Each(fn func(p DescribedLogDirPartition)) { + for _, ps := range ds { + for _, d := range ps { + fn(d) + } + } +} + +// DescribedLogDirPartition is the information for a single partitions described +// log directory. +type DescribedLogDirPartition struct { + Broker int32 // Broker is the broker this partition is on. + Dir string // Dir is the directory this partition lives in. + Topic string // Topic is the topic for this partition. + Partition int32 // Partition is this partition. + Size int64 // Size is the total size of the log segments of this partition, in bytes. + + // OffsetLag is how far behind the log end offset this partition is. + // The math is: + // + // if IsFuture { + // logEndOffset - futureLogEndOffset + // } else { + // max(highWaterMark - logEndOffset) + // } + // + OffsetLag int64 + // IsFuture is true if this replica was created by an + // AlterReplicaLogDirsRequest and will replace the current log of the + // replica in the future. + IsFuture bool +} + +// Less returns if one dir partition is less than the other, by dir, topic, +// partition, and size. +func (p DescribedLogDirPartition) Less(other DescribedLogDirPartition) bool { + if p.Broker < other.Broker { + return true + } + if p.Broker > other.Broker { + return false + } + if p.Dir < other.Dir { + return true + } + if p.Dir > other.Dir { + return false + } + if p.Topic < other.Topic { + return true + } + if p.Topic > other.Topic { + return false + } + if p.Partition < other.Partition { + return true + } + if p.Partition > other.Partition { + return false + } + return p.Size < other.Size +} + +// LessBySize returns if one dir partition is less than the other by size, +// otherwise by normal Less semantics. +func (p DescribedLogDirPartition) LessBySize(other DescribedLogDirPartition) bool { + if p.Size < other.Size { + return true + } + return p.Less(other) +} + +func newDescribeLogDirsResp(node int32, resp *kmsg.DescribeLogDirsResponse) DescribedLogDirs { + ds := make(DescribedLogDirs) + for _, rd := range resp.Dirs { + d := DescribedLogDir{ + Broker: node, + Dir: rd.Dir, + Topics: make(DescribedLogDirTopics), + Err: kerr.ErrorForCode(rd.ErrorCode), + } + for _, rt := range rd.Topics { + t := make(map[int32]DescribedLogDirPartition) + d.Topics[rt.Topic] = t + for _, rp := range rt.Partitions { + t[rp.Partition] = DescribedLogDirPartition{ + Broker: node, + Dir: rd.Dir, + Topic: rt.Topic, + Partition: rp.Partition, + Size: rp.Size, + OffsetLag: rp.OffsetLag, + IsFuture: rp.IsFuture, + } + } + } + ds[rd.Dir] = d + } + return ds +} + +// DescribeAllLogDirs describes the log directores for every input topic +// partition on every broker. If the input set is nil, this describes all log +// directories. +// +// This may return *ShardErrors. +func (cl *Client) DescribeAllLogDirs(ctx context.Context, s TopicsSet) (DescribedAllLogDirs, error) { + req := describeLogDirsReq(s) + shards := cl.cl.RequestSharded(ctx, req) + resps := make(DescribedAllLogDirs) + return resps, shardErrEachBroker(req, shards, func(b BrokerDetail, kr kmsg.Response) error { + resp := kr.(*kmsg.DescribeLogDirsResponse) + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return err + } + resps[b.NodeID] = newDescribeLogDirsResp(b.NodeID, resp) // one node ID, no need to unique-check + return nil + }) +} + +// DescribeBrokerLogDirs describes the log directories for the input topic +// partitions on the given broker. If the input set is nil, this describes all +// log directories. +func (cl *Client) DescribeBrokerLogDirs(ctx context.Context, broker int32, s TopicsSet) (DescribedLogDirs, error) { + req := describeLogDirsReq(s) + b := cl.cl.Broker(int(broker)) + kresp, err := b.RetriableRequest(ctx, req) + if err != nil { + return nil, err + } + resp := kresp.(*kmsg.DescribeLogDirsResponse) + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return nil, err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return nil, err + } + return newDescribeLogDirsResp(broker, resp), nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/metadata.go b/vendor/github.com/twmb/franz-go/pkg/kadm/metadata.go new file mode 100644 index 000000000000..f2797186ee62 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/metadata.go @@ -0,0 +1,518 @@ +package kadm + +import ( + "bytes" + "context" + "encoding/base64" + "fmt" + "sort" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// TopicID is the 16 byte underlying topic ID. +type TopicID [16]byte + +// String returns the topic ID encoded as base64. +func (t TopicID) String() string { return base64.StdEncoding.EncodeToString(t[:]) } + +// MarshalJSON returns the topic ID encoded as quoted base64. +func (t TopicID) MarshalJSON() ([]byte, error) { return []byte(`"` + t.String() + `"`), nil } + +// Less returns if this ID is less than the other, byte by byte. +func (t TopicID) Less(other TopicID) bool { + return bytes.Compare(t[:], other[:]) == -1 +} + +// PartitionDetail is the detail of a partition as returned by a metadata +// response. If the partition fails to load / has an error, then only the +// partition number itself and the Err fields will be set. +type PartitionDetail struct { + Topic string // Topic is the topic this partition belongs to. + Partition int32 // Partition is the partition number these details are for. + + Leader int32 // Leader is the broker leader, if there is one, otherwise -1. + LeaderEpoch int32 // LeaderEpoch is the leader's current epoch. + Replicas []int32 // Replicas is the list of replicas. + ISR []int32 // ISR is the list of in sync replicas. + OfflineReplicas []int32 // OfflineReplicas is the list of offline replicas. + + Err error // Err is non-nil if the partition currently has a load error. +} + +// PartitionDetails contains details for partitions as returned by a metadata +// response. +type PartitionDetails map[int32]PartitionDetail + +// Sorted returns the partitions in sorted order. +func (ds PartitionDetails) Sorted() []PartitionDetail { + s := make([]PartitionDetail, 0, len(ds)) + for _, d := range ds { + s = append(s, d) + } + sort.Slice(s, func(i, j int) bool { return s[i].Partition < s[j].Partition }) + return s +} + +// Numbers returns a sorted list of all partition numbers. +func (ds PartitionDetails) Numbers() []int32 { + all := make([]int32, 0, len(ds)) + for p := range ds { + all = append(all, p) + } + return int32s(all) +} + +// NumReplicas returns the number of replicas for these partitions +// +// It is assumed that all partitions have the same number of replicas, so this +// simply returns the number of replicas in the first encountered partition. +func (ds PartitionDetails) NumReplicas() int { + for _, p := range ds { + return len(p.Replicas) + } + return 0 +} + +// TopicDetail is the detail of a topic as returned by a metadata response. If +// the topic fails to load / has an error, then there will be no partitions. +type TopicDetail struct { + Topic string // Topic is the topic these details are for. + + ID TopicID // TopicID is the topic's ID, or all 0 if the broker does not support IDs. + IsInternal bool // IsInternal is whether the topic is an internal topic. + Partitions PartitionDetails // Partitions contains details about the topic's partitions. + + Err error // Err is non-nil if the topic could not be loaded. +} + +// TopicDetails contains details for topics as returned by a metadata response. +type TopicDetails map[string]TopicDetail + +// Topics returns a sorted list of all topic names. +func (ds TopicDetails) Names() []string { + all := make([]string, 0, len(ds)) + for t := range ds { + all = append(all, t) + } + sort.Strings(all) + return all +} + +// Sorted returns all topics in sorted order. +func (ds TopicDetails) Sorted() []TopicDetail { + s := make([]TopicDetail, 0, len(ds)) + for _, d := range ds { + s = append(s, d) + } + sort.Slice(s, func(i, j int) bool { + if s[i].Topic == "" { + if s[j].Topic == "" { + return bytes.Compare(s[i].ID[:], s[j].ID[:]) == -1 + } + return true + } + if s[j].Topic == "" { + return false + } + return s[i].Topic < s[j].Topic + }) + return s +} + +// Has returns whether the topic details has the given topic and, if so, that +// the topic's load error is not an unknown topic error. +func (ds TopicDetails) Has(topic string) bool { + d, ok := ds[topic] + return ok && d.Err != kerr.UnknownTopicOrPartition +} + +// FilterInternal deletes any internal topics from this set of topic details. +func (ds TopicDetails) FilterInternal() { + for t, d := range ds { + if d.IsInternal { + delete(ds, t) + } + } +} + +// EachPartition calls fn for every partition in all topics. +func (ds TopicDetails) EachPartition(fn func(PartitionDetail)) { + for _, td := range ds { + for _, d := range td.Partitions { + fn(d) + } + } +} + +// EachError calls fn for each topic that could not be loaded. +func (ds TopicDetails) EachError(fn func(TopicDetail)) { + for _, td := range ds { + if td.Err != nil { + fn(td) + } + } +} + +// Error iterates over all topic details and returns the first error +// encountered, if any. +func (ds TopicDetails) Error() error { + for _, t := range ds { + if t.Err != nil { + return t.Err + } + } + return nil +} + +// TopicsSet returns the topics and partitions as a set. +func (ds TopicDetails) TopicsSet() TopicsSet { + var s TopicsSet + ds.EachPartition(func(d PartitionDetail) { + s.Add(d.Topic, d.Partition) + }) + return s +} + +// TopicsList returns the topics and partitions as a list. +func (ds TopicDetails) TopicsList() TopicsList { + return ds.TopicsSet().Sorted() +} + +// Metadata is the data from a metadata response. +type Metadata struct { + Cluster string // Cluster is the cluster name, if any. + Controller int32 // Controller is the node ID of the controller broker, if available, otherwise -1. + Brokers BrokerDetails // Brokers contains broker details, sorted by default. + Topics TopicDetails // Topics contains topic details. +} + +func int32s(is []int32) []int32 { + sort.Slice(is, func(i, j int) bool { return is[i] < is[j] }) + return is +} + +// ListBrokers issues a metadata request and returns BrokerDetails. This +// returns an error if the request fails to be issued, or an *AuthError. +func (cl *Client) ListBrokers(ctx context.Context) (BrokerDetails, error) { + m, err := cl.Metadata(ctx) + if err != nil { + return nil, err + } + return m.Brokers, nil +} + +// BrokerMetadata issues a metadata request and returns it, and does not ask +// for any topics. +// +// This returns an error if the request fails to be issued, or an *AuthErr. +func (cl *Client) BrokerMetadata(ctx context.Context) (Metadata, error) { + return cl.metadata(ctx, true, nil) +} + +// Metadata issues a metadata request and returns it. Specific topics to +// describe can be passed as additional arguments. If no topics are specified, +// all topics are requested. +// +// This returns an error if the request fails to be issued, or an *AuthErr. +func (cl *Client) Metadata( + ctx context.Context, + topics ...string, +) (Metadata, error) { + return cl.metadata(ctx, false, topics) +} + +func (cl *Client) metadata(ctx context.Context, noTopics bool, topics []string) (Metadata, error) { + req := kmsg.NewPtrMetadataRequest() + for _, t := range topics { + rt := kmsg.NewMetadataRequestTopic() + rt.Topic = kmsg.StringPtr(t) + req.Topics = append(req.Topics, rt) + } + if noTopics { + req.Topics = []kmsg.MetadataRequestTopic{} + } + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return Metadata{}, err + } + + tds := make(map[string]TopicDetail, len(resp.Topics)) + for _, t := range resp.Topics { + if err := maybeAuthErr(t.ErrorCode); err != nil { + return Metadata{}, err + } + td := TopicDetail{ + ID: t.TopicID, + Partitions: make(map[int32]PartitionDetail), + IsInternal: t.IsInternal, + Err: kerr.ErrorForCode(t.ErrorCode), + } + if t.Topic != nil { + td.Topic = *t.Topic + } + for _, p := range t.Partitions { + td.Partitions[p.Partition] = PartitionDetail{ + Topic: td.Topic, + Partition: p.Partition, + + Leader: p.Leader, + LeaderEpoch: p.LeaderEpoch, + Replicas: p.Replicas, + ISR: p.ISR, + OfflineReplicas: p.OfflineReplicas, + + Err: kerr.ErrorForCode(p.ErrorCode), + } + } + tds[*t.Topic] = td + } + + m := Metadata{ + Controller: resp.ControllerID, + Topics: tds, + } + if resp.ClusterID != nil { + m.Cluster = *resp.ClusterID + } + + for _, b := range resp.Brokers { + m.Brokers = append(m.Brokers, kgo.BrokerMetadata{ + NodeID: b.NodeID, + Host: b.Host, + Port: b.Port, + Rack: b.Rack, + }) + } + sort.Slice(m.Brokers, func(i, j int) bool { return m.Brokers[i].NodeID < m.Brokers[j].NodeID }) + + if len(topics) > 0 && len(m.Topics) != len(topics) { + return Metadata{}, fmt.Errorf("metadata returned only %d topics of %d requested", len(m.Topics), len(topics)) + } + + return m, nil +} + +// ListedOffset contains record offset information. +type ListedOffset struct { + Topic string // Topic is the topic this offset is for. + Partition int32 // Partition is the partition this offset is for. + + Timestamp int64 // Timestamp is the millisecond of the offset if listing after a time, otherwise -1. + Offset int64 // Offset is the record offset, or -1 if one could not be found. + LeaderEpoch int32 // LeaderEpoch is the leader epoch at this offset, if any, otherwise -1. + + Err error // Err is non-nil if the partition has a load error. +} + +// ListedOffsets contains per-partition record offset information that is +// returned from any of the List.*Offsets functions. +type ListedOffsets map[string]map[int32]ListedOffset + +// Lookup returns the offset at t and p and whether it exists. +func (l ListedOffsets) Lookup(t string, p int32) (ListedOffset, bool) { + if len(l) == 0 { + return ListedOffset{}, false + } + ps := l[t] + if len(ps) == 0 { + return ListedOffset{}, false + } + o, exists := ps[p] + return o, exists +} + +// Each calls fn for each listed offset. +func (l ListedOffsets) Each(fn func(ListedOffset)) { + for _, ps := range l { + for _, o := range ps { + fn(o) + } + } +} + +// Error iterates over all offsets and returns the first error encountered, if +// any. This can be to check if a listing was entirely successful or not. +// +// Note that offset listing can be partially successful. For example, some +// offsets could succeed to be listed, while other could fail (maybe one +// partition is offline). If this is something you need to worry about, you may +// need to check all offsets manually. +func (l ListedOffsets) Error() error { + for _, ps := range l { + for _, o := range ps { + if o.Err != nil { + return o.Err + } + } + } + return nil +} + +// Offsets returns these listed offsets as offsets. +func (l ListedOffsets) Offsets() Offsets { + o := make(Offsets) + l.Each(func(l ListedOffset) { + o.Add(Offset{ + Topic: l.Topic, + Partition: l.Partition, + At: l.Offset, + LeaderEpoch: l.LeaderEpoch, + }) + }) + return o +} + +// KOffsets returns these listed offsets as a kgo offset map. +func (l ListedOffsets) KOffsets() map[string]map[int32]kgo.Offset { + return l.Offsets().KOffsets() +} + +// ListStartOffsets returns the start (oldest) offsets for each partition in +// each requested topic. In Kafka terms, this returns the log start offset. If +// no topics are specified, all topics are listed. If a requested topic does +// not exist, no offsets for it are listed and it is not present in the +// response. +// +// If any topics being listed do not exist, a special -1 partition is added +// to the response with the expected error code kerr.UnknownTopicOrPartition. +// +// This may return *ShardErrors. +func (cl *Client) ListStartOffsets(ctx context.Context, topics ...string) (ListedOffsets, error) { + return cl.listOffsets(ctx, 0, -2, topics) +} + +// ListEndOffsets returns the end (newest) offsets for each partition in each +// requested topic. In Kafka terms, this returns high watermarks. If no topics +// are specified, all topics are listed. If a requested topic does not exist, +// no offsets for it are listed and it is not present in the response. +// +// If any topics being listed do not exist, a special -1 partition is added +// to the response with the expected error code kerr.UnknownTopicOrPartition. +// +// This may return *ShardErrors. +func (cl *Client) ListEndOffsets(ctx context.Context, topics ...string) (ListedOffsets, error) { + return cl.listOffsets(ctx, 0, -1, topics) +} + +// ListCommittedOffsets returns newest committed offsets for each partition in +// each requested topic. A committed offset may be slightly less than the +// latest offset. In Kafka terms, committed means the last stable offset, and +// newest means the high watermark. Record offsets in active, uncommitted +// transactions will not be returned. If no topics are specified, all topics +// are listed. If a requested topic does not exist, no offsets for it are +// listed and it is not present in the response. +// +// If any topics being listed do not exist, a special -1 partition is added +// to the response with the expected error code kerr.UnknownTopicOrPartition. +// +// This may return *ShardErrors. +func (cl *Client) ListCommittedOffsets(ctx context.Context, topics ...string) (ListedOffsets, error) { + return cl.listOffsets(ctx, 1, -1, topics) +} + +// ListOffsetsAfterMilli returns the first offsets after the requested +// millisecond timestamp. Unlike listing start/end/committed offsets, offsets +// returned from this function also include the timestamp of the offset. If no +// topics are specified, all topics are listed. If a partition has no offsets +// after the requested millisecond, the offset will be the current end offset. +// If a requested topic does not exist, no offsets for it are listed and it is +// not present in the response. +// +// If any topics being listed do not exist, a special -1 partition is added +// to the response with the expected error code kerr.UnknownTopicOrPartition. +// +// This may return *ShardErrors. +func (cl *Client) ListOffsetsAfterMilli(ctx context.Context, millisecond int64, topics ...string) (ListedOffsets, error) { + return cl.listOffsets(ctx, 0, millisecond, topics) +} + +func (cl *Client) listOffsets(ctx context.Context, isolation int8, timestamp int64, topics []string) (ListedOffsets, error) { + tds, err := cl.ListTopics(ctx, topics...) + if err != nil { + return nil, err + } + + // If we request with timestamps, we may request twice: once for after + // timestamps, and once for any -1 (and no error) offsets where the + // timestamp is in the future. + list := make(ListedOffsets) + + for _, td := range tds { + if td.Err != nil { + list[td.Topic] = map[int32]ListedOffset{ + -1: { + Topic: td.Topic, + Partition: -1, + Err: td.Err, + }, + } + } + } + rerequest := make(map[string][]int32) + shardfn := func(kr kmsg.Response) error { + resp := kr.(*kmsg.ListOffsetsResponse) + for _, t := range resp.Topics { + lt, ok := list[t.Topic] + if !ok { + lt = make(map[int32]ListedOffset) + list[t.Topic] = lt + } + for _, p := range t.Partitions { + if err := maybeAuthErr(p.ErrorCode); err != nil { + return err + } + lt[p.Partition] = ListedOffset{ + Topic: t.Topic, + Partition: p.Partition, + Timestamp: p.Timestamp, + Offset: p.Offset, + LeaderEpoch: p.LeaderEpoch, + Err: kerr.ErrorForCode(p.ErrorCode), + } + if timestamp != -1 && p.Offset == -1 && p.ErrorCode == 0 { + rerequest[t.Topic] = append(rerequest[t.Topic], p.Partition) + } + } + } + return nil + } + + req := kmsg.NewPtrListOffsetsRequest() + req.IsolationLevel = isolation + for t, td := range tds { + rt := kmsg.NewListOffsetsRequestTopic() + if td.Err != nil { + continue + } + rt.Topic = t + for p := range td.Partitions { + rp := kmsg.NewListOffsetsRequestTopicPartition() + rp.Partition = p + rp.Timestamp = timestamp + rt.Partitions = append(rt.Partitions, rp) + } + req.Topics = append(req.Topics, rt) + } + shards := cl.cl.RequestSharded(ctx, req) + err = shardErrEach(req, shards, shardfn) + if len(rerequest) > 0 { + req.Topics = req.Topics[:0] + for t, ps := range rerequest { + rt := kmsg.NewListOffsetsRequestTopic() + rt.Topic = t + for _, p := range ps { + rp := kmsg.NewListOffsetsRequestTopicPartition() + rp.Partition = p + rp.Timestamp = -1 // we always list end offsets when rerequesting + rt.Partitions = append(rt.Partitions, rp) + } + req.Topics = append(req.Topics, rt) + } + shards = cl.cl.RequestSharded(ctx, req) + err = mergeShardErrs(err, shardErrEach(req, shards, shardfn)) + } + return list, err +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/misc.go b/vendor/github.com/twmb/franz-go/pkg/kadm/misc.go new file mode 100644 index 000000000000..05add5415351 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/misc.go @@ -0,0 +1,959 @@ +package kadm + +import ( + "context" + "crypto/rand" + "crypto/sha256" + "crypto/sha512" + "errors" + "fmt" + "sort" + "strings" + "sync" + + "golang.org/x/crypto/pbkdf2" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" + "github.com/twmb/franz-go/pkg/kversion" +) + +// ErrAndMessage is returned as the error from requests that were successfully +// responded to, but the response indicates failure with a message. +type ErrAndMessage struct { + Err error // Err is the response ErrorCode. + ErrMessage string // Message is the response ErrorMessage. +} + +func (e *ErrAndMessage) Error() string { + var ke *kerr.Error + if errors.As(e.Err, &ke) && e.ErrMessage != "" { + return ke.Message + ": " + e.ErrMessage + } + return e.Err.Error() +} + +func (e *ErrAndMessage) Unwrap() error { + return e.Err +} + +// FindCoordinatorResponse contains information for the coordinator for a group +// or transactional ID. +type FindCoordinatorResponse struct { + Name string // Name is the coordinator key this response is for. + NodeID int32 // NodeID is the node ID of the coordinator for this key. + Host string // Host is the host of the coordinator for this key. + Port int32 // Port is the port of the coordinator for this key. + Err error // Err is any error encountered when requesting the coordinator. + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// FindCoordinatorResponses contains responses to finding coordinators for +// groups or transactions. +type FindCoordinatorResponses map[string]FindCoordinatorResponse + +// AllFailed returns whether all responses are errored. +func (rs FindCoordinatorResponses) AllFailed() bool { + var n int + rs.EachError(func(FindCoordinatorResponse) { n++ }) + return len(rs) > 0 && n == len(rs) +} + +// Sorted returns all coordinator responses sorted by name. +func (rs FindCoordinatorResponses) Sorted() []FindCoordinatorResponse { + s := make([]FindCoordinatorResponse, 0, len(rs)) + for _, r := range rs { + s = append(s, r) + } + sort.Slice(s, func(i, j int) bool { return s[i].Name < s[j].Name }) + return s +} + +// EachError calls fn for every response that has a non-nil error. +func (rs FindCoordinatorResponses) EachError(fn func(FindCoordinatorResponse)) { + for _, r := range rs { + if r.Err != nil { + fn(r) + } + } +} + +// Each calls fn for every response. +func (rs FindCoordinatorResponses) Each(fn func(FindCoordinatorResponse)) { + for _, r := range rs { + fn(r) + } +} + +// Error iterates over all responses and returns the first error encountered, +// if any. +func (rs FindCoordinatorResponses) Error() error { + for _, r := range rs { + if r.Err != nil { + return r.Err + } + } + return nil +} + +// Ok returns true if there are no errors. This is a shortcut for rs.Error() == +// nil. +func (rs FindCoordinatorResponses) Ok() bool { + return rs.Error() == nil +} + +// FindGroupCoordinators returns the coordinator for all requested group names. +// +// This may return *ShardErrors or *AuthError. +func (cl *Client) FindGroupCoordinators(ctx context.Context, groups ...string) FindCoordinatorResponses { + return cl.findCoordinators(ctx, 0, groups...) +} + +// FindTxnCoordinators returns the coordinator for all requested transactional +// IDs. +// +// This may return *ShardErrors or *AuthError. +func (cl *Client) FindTxnCoordinators(ctx context.Context, txnIDs ...string) FindCoordinatorResponses { + return cl.findCoordinators(ctx, 1, txnIDs...) +} + +func (cl *Client) findCoordinators(ctx context.Context, kind int8, names ...string) FindCoordinatorResponses { + resps := make(FindCoordinatorResponses) + if len(names) == 0 { + return resps + } + + req := kmsg.NewPtrFindCoordinatorRequest() + req.CoordinatorType = kind + req.CoordinatorKeys = names + + keyErr := func(k string, err error) { + resps[k] = FindCoordinatorResponse{ + Name: k, + Err: err, + } + } + allKeysErr := func(req *kmsg.FindCoordinatorRequest, err error) { + for _, k := range req.CoordinatorKeys { + keyErr(k, err) + } + } + + shards := cl.cl.RequestSharded(ctx, req) + for _, shard := range shards { + req := shard.Req.(*kmsg.FindCoordinatorRequest) + if shard.Err != nil { + allKeysErr(req, shard.Err) + continue + } + resp := shard.Resp.(*kmsg.FindCoordinatorResponse) + if err := maybeAuthErr(resp.ErrorCode); err != nil { + allKeysErr(req, err) + continue + } + for _, c := range resp.Coordinators { + if err := maybeAuthErr(c.ErrorCode); err != nil { + keyErr(c.Key, err) + continue + } + resps[c.Key] = FindCoordinatorResponse{ // key is always on one broker, no need to check existence + Name: c.Key, + NodeID: c.NodeID, + Host: c.Host, + Port: c.Port, + Err: kerr.ErrorForCode(c.ErrorCode), + ErrMessage: unptrStr(c.ErrorMessage), + } + } + } + return resps +} + +type minmax struct { + min, max int16 +} + +// BrokerApiVersions contains the API versions for a single broker. +type BrokerApiVersions struct { + NodeID int32 // NodeID is the node this API versions response is for. + + raw *kmsg.ApiVersionsResponse + keyVersions map[int16]minmax + + Err error // Err is non-nil if the API versions request failed. +} + +// Raw returns the raw API versions response. +func (v *BrokerApiVersions) Raw() *kmsg.ApiVersionsResponse { + return v.raw +} + +// KeyVersions returns the broker's min and max version for an API key and +// whether this broker supports the request. +func (v *BrokerApiVersions) KeyVersions(key int16) (min, max int16, exists bool) { + vs, exists := v.keyVersions[key] + return vs.min, vs.max, exists +} + +// KeyVersions returns the broker's min version for an API key and whether this +// broker supports the request. +func (v *BrokerApiVersions) KeyMinVersion(key int16) (min int16, exists bool) { + min, _, exists = v.KeyVersions(key) + return min, exists +} + +// KeyVersions returns the broker's max version for an API key and whether this +// broker supports the request. +func (v *BrokerApiVersions) KeyMaxVersion(key int16) (max int16, exists bool) { + _, max, exists = v.KeyVersions(key) + return max, exists +} + +// EachKeySorted calls fn for every API key in the broker response, from the +// smallest API key to the largest. +func (v *BrokerApiVersions) EachKeySorted(fn func(key, min, max int16)) { + type kmm struct { + k, min, max int16 + } + kmms := make([]kmm, 0, len(v.keyVersions)) + for key, minmax := range v.keyVersions { + kmms = append(kmms, kmm{key, minmax.min, minmax.max}) + } + sort.Slice(kmms, func(i, j int) bool { return kmms[i].k < kmms[j].k }) + for _, kmm := range kmms { + fn(kmm.k, kmm.min, kmm.max) + } +} + +// VersionGuess returns the best guess of Kafka that this broker is. This is a +// shorcut for: +// +// kversion.FromApiVersionsResponse(v.Raw()).VersionGuess(opt...) +// +// Check the kversion.VersionGuess API docs for more details. +func (v *BrokerApiVersions) VersionGuess(opt ...kversion.VersionGuessOpt) string { + return kversion.FromApiVersionsResponse(v.raw).VersionGuess(opt...) +} + +// BrokerApiVersions contains API versions for all brokers that are reachable +// from a metadata response. +type BrokersApiVersions map[int32]BrokerApiVersions + +// Sorted returns all broker responses sorted by node ID. +func (vs BrokersApiVersions) Sorted() []BrokerApiVersions { + s := make([]BrokerApiVersions, 0, len(vs)) + for _, v := range vs { + s = append(s, v) + } + sort.Slice(s, func(i, j int) bool { return s[i].NodeID < s[j].NodeID }) + return s +} + +// Each calls fn for every broker response. +func (vs BrokersApiVersions) Each(fn func(BrokerApiVersions)) { + for _, v := range vs { + fn(v) + } +} + +// ApiVersions queries every broker in a metadata response for their API +// versions. This returns an error only if the metadata request fails. +func (cl *Client) ApiVersions(ctx context.Context) (BrokersApiVersions, error) { + m, err := cl.BrokerMetadata(ctx) + if err != nil { + return nil, err + } + + var mu sync.Mutex + var wg sync.WaitGroup + vs := make(BrokersApiVersions, len(m.Brokers)) + for _, n := range m.Brokers.NodeIDs() { + n := n + wg.Add(1) + go func() { + defer wg.Done() + req := kmsg.NewPtrApiVersionsRequest() + req.ClientSoftwareName = "kadm" + req.ClientSoftwareVersion = softwareVersion() + v := BrokerApiVersions{NodeID: n, keyVersions: make(map[int16]minmax)} + v.raw, v.Err = req.RequestWith(ctx, cl.cl.Broker(int(n))) + + mu.Lock() + defer mu.Unlock() + defer func() { vs[n] = v }() + if v.Err != nil { + return + } + + v.Err = kerr.ErrorForCode(v.raw.ErrorCode) + for _, k := range v.raw.ApiKeys { + v.keyVersions[k.ApiKey] = minmax{ + min: k.MinVersion, + max: k.MaxVersion, + } + } + }() + } + wg.Wait() + + return vs, nil +} + +// ClientQuotaEntityComponent is a quota entity component. +type ClientQuotaEntityComponent struct { + Type string // Type is the entity type ("user", "client-id", "ip"). + Name *string // Name is the entity name, or null if the default. +} + +// String returns key=value, or key= if value is nil. +func (d ClientQuotaEntityComponent) String() string { + if d.Name == nil { + return d.Type + "=" + } + return fmt.Sprintf("%s=%s", d.Type, *d.Name) +} + +// ClientQuotaEntity contains the components that make up a single entity. +type ClientQuotaEntity []ClientQuotaEntityComponent + +// String returns {key=value, key=value}, joining all entities with a ", " and +// wrapping in braces. +func (ds ClientQuotaEntity) String() string { + var ss []string + for _, d := range ds { + ss = append(ss, d.String()) + } + return "{" + strings.Join(ss, ", ") + "}" +} + +// ClientQuotaValue is a quota name and value. +type ClientQuotaValue struct { + Key string // Key is the quota configuration key. + Value float64 // Value is the quota configuration value. +} + +// String returns key=value. +func (d ClientQuotaValue) String() string { + return fmt.Sprintf("%s=%f", d.Key, d.Value) +} + +// ClientQuotaValues contains all client quota values. +type ClientQuotaValues []ClientQuotaValue + +// QuotasMatchType specifies how to match a described client quota entity. +// +// 0 means to match the name exactly: user=foo will only match components of +// entity type "user" and entity name "foo". +// +// 1 means to match the default of the name: entity type "user" with a default +// match will return the default quotas for user entities. +// +// 2 means to match any name: entity type "user" with any matching will return +// both names and defaults. +type QuotasMatchType = kmsg.QuotasMatchType + +// DescribeClientQuotaComponent is an input entity component to describing +// client quotas: we define the type of quota ("client-id", "user"), how to +// match, and the match name if needed. +type DescribeClientQuotaComponent struct { + Type string // Type is the type of entity component to describe ("user", "client-id", "ip"). + MatchName *string // MatchName is the name to match again; this is only needed when MatchType is 0 (exact). + MatchType QuotasMatchType // MatchType is how to match an entity. +} + +// DescribedClientQuota contains a described quota. A single quota is made up +// of multiple entities and multiple values, for example, "user=foo" is one +// component of the entity, and "client-id=bar" is another. +type DescribedClientQuota struct { + Entity ClientQuotaEntity // Entity is the entity of this described client quota. + Values ClientQuotaValues // Values contains the quota valies for this entity. +} + +// DescribedClientQuota contains client quotas that were described. +type DescribedClientQuotas []DescribedClientQuota + +// DescribeClientQuotas describes client quotas. If strict is true, the +// response includes only the requested components. +func (cl *Client) DescribeClientQuotas(ctx context.Context, strict bool, entityComponents []DescribeClientQuotaComponent) (DescribedClientQuotas, error) { + req := kmsg.NewPtrDescribeClientQuotasRequest() + req.Strict = strict + for _, entity := range entityComponents { + rc := kmsg.NewDescribeClientQuotasRequestComponent() + rc.EntityType = entity.Type + rc.Match = entity.MatchName + rc.MatchType = entity.MatchType + req.Components = append(req.Components, rc) + } + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return nil, err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return nil, &ErrAndMessage{err, unptrStr(resp.ErrorMessage)} + } + var qs DescribedClientQuotas + for _, entry := range resp.Entries { + var q DescribedClientQuota + for _, e := range entry.Entity { + q.Entity = append(q.Entity, ClientQuotaEntityComponent{ + Type: e.Type, + Name: e.Name, + }) + } + for _, v := range entry.Values { + q.Values = append(q.Values, ClientQuotaValue{ + Key: v.Key, + Value: v.Value, + }) + } + qs = append(qs, q) + } + return qs, nil +} + +// AlterClientQuotaOp sets or remove a client quota. +type AlterClientQuotaOp struct { + Key string // Key is the quota configuration key to set or remove. + Value float64 // Value is the quota configuration value to set or remove. + Remove bool // Remove, if true, removes this quota rather than sets it. +} + +// AlterClientQuotaEntry pairs an entity with quotas to set or remove. +type AlterClientQuotaEntry struct { + Entity ClientQuotaEntity // Entity is the entity to alter quotas for. + Ops []AlterClientQuotaOp // Ops are quotas to set or remove. +} + +// AlteredClientQuota is the result for a single entity that was altered. +type AlteredClientQuota struct { + Entity ClientQuotaEntity // Entity is the entity this result is for. + Err error // Err is non-nil if the alter operation on this entity failed. + ErrMessage string // ErrMessage is an optional additional message on error. +} + +// AlteredClientQuotas contains results for all altered entities. +type AlteredClientQuotas []AlteredClientQuota + +// AlterClientQuotas alters quotas for the input entries. You may consider +// checking ValidateAlterClientQuotas before using this method. +func (cl *Client) AlterClientQuotas(ctx context.Context, entries []AlterClientQuotaEntry) (AlteredClientQuotas, error) { + return cl.alterClientQuotas(ctx, false, entries) +} + +// ValidateAlterClientQuotas validates an alter client quota request. This +// returns exactly what AlterClientQuotas returns, but does not actually alter +// quotas. +func (cl *Client) ValidateAlterClientQuotas(ctx context.Context, entries []AlterClientQuotaEntry) (AlteredClientQuotas, error) { + return cl.alterClientQuotas(ctx, true, entries) +} + +func (cl *Client) alterClientQuotas(ctx context.Context, validate bool, entries []AlterClientQuotaEntry) (AlteredClientQuotas, error) { + req := kmsg.NewPtrAlterClientQuotasRequest() + req.ValidateOnly = validate + for _, entry := range entries { + re := kmsg.NewAlterClientQuotasRequestEntry() + for _, c := range entry.Entity { + rec := kmsg.NewAlterClientQuotasRequestEntryEntity() + rec.Type = c.Type + rec.Name = c.Name + re.Entity = append(re.Entity, rec) + } + for _, op := range entry.Ops { + reo := kmsg.NewAlterClientQuotasRequestEntryOp() + reo.Key = op.Key + reo.Value = op.Value + reo.Remove = op.Remove + re.Ops = append(re.Ops, reo) + } + req.Entries = append(req.Entries, re) + } + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + var as AlteredClientQuotas + for _, entry := range resp.Entries { + var e ClientQuotaEntity + for _, c := range entry.Entity { + e = append(e, ClientQuotaEntityComponent{ + Type: c.Type, + Name: c.Name, + }) + } + a := AlteredClientQuota{ + Entity: e, + Err: kerr.ErrorForCode(entry.ErrorCode), + ErrMessage: unptrStr(entry.ErrorMessage), + } + as = append(as, a) + } + return as, nil +} + +// ScramMechanism is a SCRAM mechanism. +type ScramMechanism int8 + +const ( + // ScramSha256 represents the SCRAM-SHA-256 mechanism. + ScramSha256 ScramMechanism = 1 + // ScramSha512 represents the SCRAM-SHA-512 mechanism. + ScramSha512 ScramMechanism = 2 +) + +// String returns either SCRAM-SHA-256, SCRAM-SHA-512, or UNKNOWN. +func (s ScramMechanism) String() string { + switch s { + case ScramSha256: + return "SCRAM-SHA-256" + case ScramSha512: + return "SCRAM-SHA-512" + default: + return "UNKNOWN" + } +} + +// CredInfo contains the SCRAM mechanism and iterations for a password. +type CredInfo struct { + // Mechanism is the SCRAM mechanism a password exists for. This is 0 + // for UNKNOWN, 1 for SCRAM-SHA-256, and 2 for SCRAM-SHA-512. + Mechanism ScramMechanism + // Iterations is the number of SCRAM iterations for this password. + Iterations int32 +} + +// String returns MECHANISM=iterations={c.Iterations}. +func (c CredInfo) String() string { + return fmt.Sprintf("%s=iterations=%d", c.Mechanism, c.Iterations) +} + +// DescribedUserSCRAM contains a user, the SCRAM mechanisms that the user has +// passwords for, and if describing the user SCRAM credentials errored. +type DescribedUserSCRAM struct { + User string // User is the user this described user credential is for. + CredInfos []CredInfo // CredInfos contains SCRAM mechanisms the user has passwords for. + Err error // Err is any error encountered when describing the user. + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// DescribedUserSCRAMs contains described user SCRAM credentials keyed by user. +type DescribedUserSCRAMs map[string]DescribedUserSCRAM + +// Sorted returns the described user credentials ordered by user. +func (ds DescribedUserSCRAMs) Sorted() []DescribedUserSCRAM { + s := make([]DescribedUserSCRAM, 0, len(ds)) + for _, d := range ds { + s = append(s, d) + } + sort.Slice(s, func(i, j int) bool { return s[i].User < s[j].User }) + return s +} + +// AllFailed returns whether all described user credentials are errored. +func (ds DescribedUserSCRAMs) AllFailed() bool { + var n int + ds.EachError(func(DescribedUserSCRAM) { n++ }) + return len(ds) > 0 && n == len(ds) +} + +// EachError calls fn for every described user that has a non-nil error. +func (ds DescribedUserSCRAMs) EachError(fn func(DescribedUserSCRAM)) { + for _, d := range ds { + if d.Err != nil { + fn(d) + } + } +} + +// Each calls fn for every described user. +func (ds DescribedUserSCRAMs) Each(fn func(DescribedUserSCRAM)) { + for _, d := range ds { + fn(d) + } +} + +// Error iterates over all described users and returns the first error +// encountered, if any. +func (ds DescribedUserSCRAMs) Error() error { + for _, d := range ds { + if d.Err != nil { + return d.Err + } + } + return nil +} + +// Ok returns true if there are no errors. This is a shortcut for rs.Error() == +// nil. +func (ds DescribedUserSCRAMs) Ok() bool { + return ds.Error() == nil +} + +// DescribeUserSCRAMs returns a small bit of information about all users in the +// input request that have SCRAM passwords configured. No users requests all +// users. +func (cl *Client) DescribeUserSCRAMs(ctx context.Context, users ...string) (DescribedUserSCRAMs, error) { + req := kmsg.NewPtrDescribeUserSCRAMCredentialsRequest() + for _, u := range users { + ru := kmsg.NewDescribeUserSCRAMCredentialsRequestUser() + ru.Name = u + req.Users = append(req.Users, ru) + } + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return nil, err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return nil, err + } + rs := make(DescribedUserSCRAMs) + for _, res := range resp.Results { + r := DescribedUserSCRAM{ + User: res.User, + Err: kerr.ErrorForCode(res.ErrorCode), + ErrMessage: unptrStr(res.ErrorMessage), + } + for _, i := range res.CredentialInfos { + r.CredInfos = append(r.CredInfos, CredInfo{ + Mechanism: ScramMechanism(i.Mechanism), + Iterations: i.Iterations, + }) + } + rs[r.User] = r + } + return rs, nil +} + +// DeleteSCRAM deletes a password with the given mechanism for the user. +type DeleteSCRAM struct { + User string // User is the username to match for deletion. + Mechanism ScramMechanism // Mechanism is the mechanism to match to delete a password for. +} + +// UpsertSCRAM either updates or creates (inserts) a new password for a user. +// There are two ways to specify a password: either with the Password field +// directly, or by specifying both Salt and SaltedPassword. If you specify just +// a password, this package generates a 24 byte salt and uses pbkdf2 to create +// the salted password. +type UpsertSCRAM struct { + User string // User is the username to use. + Mechanism ScramMechanism // Mechanism is the mechanism to use. + Iterations int32 // Iterations is the SCRAM iterations to use; must be between 4096 and 16384. + Password string // Password is the password to salt and convert to a salted password. Requires Salt and SaltedPassword to be empty. + Salt []byte // Salt must be paired with SaltedPassword and requires Password to be empty. + SaltedPassword []byte // SaltedPassword must be paired with Salt and requires Password to be empty. +} + +// AlteredUserSCRAM is the result of an alter operation. +type AlteredUserSCRAM struct { + User string // User is the username that was altered. + Err error // Err is any error encountered when altering the user. + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// AlteredUserSCRAMs contains altered user SCRAM credentials keyed by user. +type AlteredUserSCRAMs map[string]AlteredUserSCRAM + +// Sorted returns the altered user credentials ordered by user. +func (as AlteredUserSCRAMs) Sorted() []AlteredUserSCRAM { + s := make([]AlteredUserSCRAM, 0, len(as)) + for _, a := range as { + s = append(s, a) + } + sort.Slice(s, func(i, j int) bool { return s[i].User < s[j].User }) + return s +} + +// AllFailed returns whether all altered user credentials are errored. +func (as AlteredUserSCRAMs) AllFailed() bool { + var n int + as.EachError(func(AlteredUserSCRAM) { n++ }) + return len(as) > 0 && n == len(as) +} + +// EachError calls fn for every altered user that has a non-nil error. +func (as AlteredUserSCRAMs) EachError(fn func(AlteredUserSCRAM)) { + for _, a := range as { + if a.Err != nil { + fn(a) + } + } +} + +// Each calls fn for every altered user. +func (as AlteredUserSCRAMs) Each(fn func(AlteredUserSCRAM)) { + for _, a := range as { + fn(a) + } +} + +// Error iterates over all altered users and returns the first error +// encountered, if any. +func (as AlteredUserSCRAMs) Error() error { + for _, a := range as { + if a.Err != nil { + return a.Err + } + } + return nil +} + +// Ok returns true if there are no errors. This is a shortcut for rs.Error() == +// nil. +func (as AlteredUserSCRAMs) Ok() bool { + return as.Error() == nil +} + +// AlterUserSCRAMs deletes, updates, or creates (inserts) user SCRAM +// credentials. Note that a username can only appear once across both upserts +// and deletes. This modifies elements of the upsert slice that need to have a +// salted password generated. +func (cl *Client) AlterUserSCRAMs(ctx context.Context, del []DeleteSCRAM, upsert []UpsertSCRAM) (AlteredUserSCRAMs, error) { + for i, u := range upsert { + if u.Password != "" { + if len(u.Salt) > 0 || len(u.SaltedPassword) > 0 { + return nil, fmt.Errorf("user %s: cannot specify both a password and a salt / salted password", u.User) + } + u.Salt = make([]byte, 24) + if _, err := rand.Read(u.Salt); err != nil { + return nil, fmt.Errorf("user %s: unable to generate salt: %v", u.User, err) + } + switch u.Mechanism { + case ScramSha256: + u.SaltedPassword = pbkdf2.Key([]byte(u.Password), u.Salt, int(u.Iterations), sha256.Size, sha256.New) + case ScramSha512: + u.SaltedPassword = pbkdf2.Key([]byte(u.Password), u.Salt, int(u.Iterations), sha512.Size, sha512.New) + default: + return nil, fmt.Errorf("user %s: unknown mechanism, unable to generate password", u.User) + } + upsert[i] = u + } else { + if len(u.Salt) == 0 || len(u.SaltedPassword) == 0 { + return nil, fmt.Errorf("user %s: must specify either a password or a salt and salted password", u.User) + } + } + } + + req := kmsg.NewPtrAlterUserSCRAMCredentialsRequest() + for _, d := range del { + rd := kmsg.NewAlterUserSCRAMCredentialsRequestDeletion() + rd.Name = d.User + rd.Mechanism = int8(d.Mechanism) + req.Deletions = append(req.Deletions, rd) + } + for _, u := range upsert { + ru := kmsg.NewAlterUserSCRAMCredentialsRequestUpsertion() + ru.Name = u.User + ru.Mechanism = int8(u.Mechanism) + ru.Iterations = u.Iterations + ru.Salt = u.Salt + ru.SaltedPassword = u.SaltedPassword + req.Upsertions = append(req.Upsertions, ru) + } + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + rs := make(AlteredUserSCRAMs) + for _, res := range resp.Results { + if err := maybeAuthErr(res.ErrorCode); err != nil { + return nil, err + } + r := AlteredUserSCRAM{ + User: res.User, + Err: kerr.ErrorForCode(res.ErrorCode), + ErrMessage: unptrStr(res.ErrorMessage), + } + rs[r.User] = r + } + return rs, nil +} + +// ElectLeadersHow is how partition leaders should be elected. +type ElectLeadersHow int8 + +const ( + // ElectPreferredReplica elects the preferred replica for a partition. + ElectPreferredReplica ElectLeadersHow = 0 + // ElectLiveReplica elects the first life replica if there are no + // in-sync replicas (i.e., this is unclean leader election). + ElectLiveReplica ElectLeadersHow = 1 +) + +// ElectLeadersResult is the result for a single partition in an elect leaders +// request. +type ElectLeadersResult struct { + Topic string // Topic is the topic this result is for. + Partition int32 // Partition is the partition this result is for. + How ElectLeadersHow // How is the type of election that was performed. + Err error // Err is non-nil if electing this partition's leader failed, such as the partition not existing or the preferred leader is not available and you used ElectPreferredReplica. + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// ElectLeadersResults contains per-topic, per-partition results for an elect +// leaders request. +type ElectLeadersResults map[string]map[int32]ElectLeadersResult + +// ElectLeaders elects leaders for partitions. This request was added in Kafka +// 2.2 to replace the previously-ZooKeeper-only option of triggering leader +// elections. See KIP-183 for more details. +// +// Kafka 2.4 introduced the ability to use unclean leader election. If you use +// unclean leader election on a Kafka 2.2 or 2.3 cluster, the client will +// instead fall back to preferred replica (clean) leader election. You can +// check the result's How function (or field) to see. +// +// If s is nil, this will elect leaders for all partitions. +// +// This will return *AuthError if you do not have ALTER on CLUSTER for +// kafka-cluster. +func (cl *Client) ElectLeaders(ctx context.Context, how ElectLeadersHow, s TopicsSet) (ElectLeadersResults, error) { + req := kmsg.NewPtrElectLeadersRequest() + req.ElectionType = int8(how) + for _, t := range s.IntoList() { + rt := kmsg.NewElectLeadersRequestTopic() + rt.Topic = t.Topic + rt.Partitions = t.Partitions + req.Topics = append(req.Topics, rt) + } + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return nil, err + } + if resp.Version == 0 { // v0 does not have the election type field + how = ElectPreferredReplica + } + rs := make(ElectLeadersResults) + for _, t := range resp.Topics { + rt := make(map[int32]ElectLeadersResult) + rs[t.Topic] = rt + for _, p := range t.Partitions { + if err := maybeAuthErr(p.ErrorCode); err != nil { + return nil, err // v0 has no top-level err + } + rt[p.Partition] = ElectLeadersResult{ + Topic: t.Topic, + Partition: p.Partition, + How: how, + Err: kerr.ErrorForCode(p.ErrorCode), + ErrMessage: unptrStr(p.ErrorMessage), + } + } + } + return rs, nil +} + +// OffsetForLeaderEpochRequest contains topics, partitions, and leader epochs +// to request offsets for in an OffsetForLeaderEpoch. +type OffsetForLeaderEpochRequest map[string]map[int32]int32 + +// Add adds a topic, partition, and leader epoch to the request. +func (l *OffsetForLeaderEpochRequest) Add(topic string, partition, leaderEpoch int32) { + if *l == nil { + *l = make(map[string]map[int32]int32) + } + t := (*l)[topic] + if t == nil { + t = make(map[int32]int32) + (*l)[topic] = t + } + t[partition] = leaderEpoch +} + +// OffsetForLeaderEpoch contains a response for a single partition in an +// OffsetForLeaderEpoch request. +type OffsetForLeaderEpoch struct { + NodeID int32 // NodeID is the node that is the leader of this topic / partition. + Topic string // Topic is the topic this leader epoch response is for. + Partition int32 // Partition is the partition this leader epoch response is for. + + // LeaderEpoch is either + // + // 1) -1, if the requested LeaderEpoch is unknown. + // + // 2) Less than the requested LeaderEpoch, if the requested LeaderEpoch + // exists but has no records in it. For example, epoch 1 had end offset + // 37, then epoch 2 and 3 had no records: if you request LeaderEpoch 3, + // this will return LeaderEpoch 1 with EndOffset 37. + // + // 3) Equal to the requested LeaderEpoch, if the requested LeaderEpoch + // is equal to or less than the current epoch for the partition. + LeaderEpoch int32 + + // EndOffset is either + // + // 1) The LogEndOffset, if the broker has the same LeaderEpoch as the + // request. + // + // 2) the beginning offset of the next LeaderEpoch, if the broker has a + // higher LeaderEpoch. + // + // The second option allows the user to detect data loss: if the + // consumer consumed past the EndOffset that is returned, then the + // consumer should reset to the returned offset and the consumer knows + // that everything from the returned offset to the requested offset was + // lost. + EndOffset int64 + + // Err is non-nil if this partition had a response error. + Err error +} + +// OffsetsForLeaderEpochs contains responses for partitions in a +// OffsetForLeaderEpochRequest. +type OffsetsForLeaderEpochs map[string]map[int32]OffsetForLeaderEpoch + +// OffsetForLeaderEpoch requests end offsets for the requested leader epoch in +// partitions in the request. This is a relatively advanced and client internal +// request, for more details, see the doc comments on the OffsetForLeaderEpoch +// type. +// +// This may return *ShardErrors or *AuthError. +func (cl *Client) OffetForLeaderEpoch(ctx context.Context, r OffsetForLeaderEpochRequest) (OffsetsForLeaderEpochs, error) { + req := kmsg.NewPtrOffsetForLeaderEpochRequest() + for t, ps := range r { + rt := kmsg.NewOffsetForLeaderEpochRequestTopic() + rt.Topic = t + for p, e := range ps { + rp := kmsg.NewOffsetForLeaderEpochRequestTopicPartition() + rp.Partition = p + rp.LeaderEpoch = e + rt.Partitions = append(rt.Partitions, rp) + } + req.Topics = append(req.Topics, rt) + } + shards := cl.cl.RequestSharded(ctx, req) + ls := make(OffsetsForLeaderEpochs) + return ls, shardErrEachBroker(req, shards, func(b BrokerDetail, kr kmsg.Response) error { + resp := kr.(*kmsg.OffsetForLeaderEpochResponse) + for _, rt := range resp.Topics { + lps, exists := ls[rt.Topic] + if !exists { // topic partitions could be spread around brokers, need to check existence + lps = make(map[int32]OffsetForLeaderEpoch) + ls[rt.Topic] = lps + } + for _, rp := range rt.Partitions { + if err := maybeAuthErr(rp.ErrorCode); err != nil { + return err + } + lps[rp.Partition] = OffsetForLeaderEpoch{ // one partition globally, no need to exist check + NodeID: b.NodeID, + Topic: rt.Topic, + Partition: rp.Partition, + LeaderEpoch: rp.LeaderEpoch, + EndOffset: rp.EndOffset, + Err: kerr.ErrorForCode(rp.ErrorCode), + } + } + } + return nil + }) +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/partas.go b/vendor/github.com/twmb/franz-go/pkg/kadm/partas.go new file mode 100644 index 000000000000..84b67790d884 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/partas.go @@ -0,0 +1,208 @@ +package kadm + +import ( + "context" + "sort" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// AlterPartitionAssignmentsReq is the input for a request to alter partition +// assignments. The keys are topics and partitions, and the final slice +// corresponds to brokers that replicas will be assigneed to. If the brokers +// for a given partition are null, the request will *cancel* any active +// reassignment for that partition. +type AlterPartitionAssignmentsReq map[string]map[int32][]int32 + +// Assign specifies brokers that a partition should be placed on. Using null +// for the brokers cancels a pending reassignment of the parititon. +func (r *AlterPartitionAssignmentsReq) Assign(t string, p int32, brokers []int32) { + if *r == nil { + *r = make(map[string]map[int32][]int32) + } + ps := (*r)[t] + if ps == nil { + ps = make(map[int32][]int32) + (*r)[t] = ps + } + ps[p] = brokers +} + +// CancelAssign cancels a reassignment of the given partition. +func (r *AlterPartitionAssignmentsReq) CancelAssign(t string, p int32) { + r.Assign(t, p, nil) +} + +// AlterPartitionAssignmentsResponse contains a response for an individual +// partition that was assigned. +type AlterPartitionAssignmentsResponse struct { + Topic string // Topic is the topic that was assigned. + Partition int32 // Partition is the partition that was assigned. + Err error // Err is non-nil if this assignment errored. + ErrMessage string // ErrMessage is an optional additional message on error. +} + +// AlterPartitionAssignmentsResponses contains responses to all partitions in an +// alter assignment request. +type AlterPartitionAssignmentsResponses map[string]map[int32]AlterPartitionAssignmentsResponse + +// Sorted returns the responses sorted by topic and partition. +func (rs AlterPartitionAssignmentsResponses) Sorted() []AlterPartitionAssignmentsResponse { + var all []AlterPartitionAssignmentsResponse + rs.Each(func(r AlterPartitionAssignmentsResponse) { + all = append(all, r) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Topic < r.Topic || l.Topic == r.Topic && l.Partition < r.Partition + }) + return all +} + +// Each calls fn for every response. +func (rs AlterPartitionAssignmentsResponses) Each(fn func(AlterPartitionAssignmentsResponse)) { + for _, ps := range rs { + for _, r := range ps { + fn(r) + } + } +} + +// Error returns the first error in the responses, if any. +func (rs AlterPartitionAssignmentsResponses) Error() error { + for _, ps := range rs { + for _, r := range ps { + if r.Err != nil { + return r.Err + } + } + } + return nil +} + +// AlterPartitionAssignments alters partition assignments for the requested +// partitions, returning an error if the response could not be issued or if +// you do not have permissions. +func (cl *Client) AlterPartitionAssignments(ctx context.Context, req AlterPartitionAssignmentsReq) (AlterPartitionAssignmentsResponses, error) { + if len(req) == 0 { + return make(AlterPartitionAssignmentsResponses), nil + } + + kreq := kmsg.NewPtrAlterPartitionAssignmentsRequest() + kreq.TimeoutMillis = cl.timeoutMillis + for t, ps := range req { + rt := kmsg.NewAlterPartitionAssignmentsRequestTopic() + rt.Topic = t + for p, bs := range ps { + rp := kmsg.NewAlterPartitionAssignmentsRequestTopicPartition() + rp.Partition = p + rp.Replicas = bs + rt.Partitions = append(rt.Partitions, rp) + } + kreq.Topics = append(kreq.Topics, rt) + } + + kresp, err := kreq.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + if err = kerr.ErrorForCode(kresp.ErrorCode); err != nil { + return nil, &ErrAndMessage{err, unptrStr(kresp.ErrorMessage)} + } + + a := make(AlterPartitionAssignmentsResponses) + for _, kt := range kresp.Topics { + ps := make(map[int32]AlterPartitionAssignmentsResponse) + a[kt.Topic] = ps + for _, kp := range kt.Partitions { + ps[kp.Partition] = AlterPartitionAssignmentsResponse{ + Topic: kt.Topic, + Partition: kp.Partition, + Err: kerr.ErrorForCode(kp.ErrorCode), + ErrMessage: unptrStr(kp.ErrorMessage), + } + } + } + return a, nil +} + +// ListPartitionReassignmentsResponse contains a response for an individual +// partition that was listed. +type ListPartitionReassignmentsResponse struct { + Topic string // Topic is the topic that was listed. + Partition int32 // Partition is the partition that was listed. + Replicas []int32 // Replicas are the partition's current replicas. + AddingReplicas []int32 // AddingReplicas are replicas currently being added to the partition. + RemovingReplicas []int32 // RemovingReplicas are replicas currently being removed from the partition. +} + +// ListPartitionReassignmentsResponses contains responses to all partitions in +// a list reassignment request. +type ListPartitionReassignmentsResponses map[string]map[int32]ListPartitionReassignmentsResponse + +// Sorted returns the responses sorted by topic and partition. +func (rs ListPartitionReassignmentsResponses) Sorted() []ListPartitionReassignmentsResponse { + var all []ListPartitionReassignmentsResponse + rs.Each(func(r ListPartitionReassignmentsResponse) { + all = append(all, r) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Topic < r.Topic || l.Topic == r.Topic && l.Partition < r.Partition + }) + return all +} + +// Each calls fn for every response. +func (rs ListPartitionReassignmentsResponses) Each(fn func(ListPartitionReassignmentsResponse)) { + for _, ps := range rs { + for _, r := range ps { + fn(r) + } + } +} + +// ListPartitionReassignments lists the state of any active reassignments for +// all requested partitions, returning an error if the response could not be +// issued or if you do not have permissions. +func (cl *Client) ListPartitionReassignments(ctx context.Context, s TopicsSet) (ListPartitionReassignmentsResponses, error) { + if len(s) == 0 { + return make(ListPartitionReassignmentsResponses), nil + } + + kreq := kmsg.NewPtrListPartitionReassignmentsRequest() + kreq.TimeoutMillis = cl.timeoutMillis + for t, ps := range s { + rt := kmsg.NewListPartitionReassignmentsRequestTopic() + rt.Topic = t + for p := range ps { + rt.Partitions = append(rt.Partitions, p) + } + kreq.Topics = append(kreq.Topics, rt) + } + + kresp, err := kreq.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + if err = kerr.ErrorForCode(kresp.ErrorCode); err != nil { + return nil, &ErrAndMessage{err, unptrStr(kresp.ErrorMessage)} + } + + a := make(ListPartitionReassignmentsResponses) + for _, kt := range kresp.Topics { + ps := make(map[int32]ListPartitionReassignmentsResponse) + a[kt.Topic] = ps + for _, kp := range kt.Partitions { + ps[kp.Partition] = ListPartitionReassignmentsResponse{ + Topic: kt.Topic, + Partition: kp.Partition, + Replicas: kp.Replicas, + AddingReplicas: kp.AddingReplicas, + RemovingReplicas: kp.RemovingReplicas, + } + } + } + return a, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/topics.go b/vendor/github.com/twmb/franz-go/pkg/kadm/topics.go new file mode 100644 index 000000000000..408506e5df4f --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/topics.go @@ -0,0 +1,645 @@ +package kadm + +import ( + "context" + "errors" + "sort" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// ListTopics issues a metadata request and returns TopicDetails. Specific +// topics to describe can be passed as additional arguments. If no topics are +// specified, all topics are requested. Internal topics are not returned unless +// specifically requested. To see all topics including internal topics, use +// ListTopicsWithInternal. +// +// This returns an error if the request fails to be issued, or an *AuthError. +func (cl *Client) ListTopics( + ctx context.Context, + topics ...string, +) (TopicDetails, error) { + t, err := cl.ListTopicsWithInternal(ctx, topics...) + if err != nil { + return nil, err + } + t.FilterInternal() + return t, nil +} + +// ListTopicsWithInternal is the same as ListTopics, but does not filter +// internal topics before returning. +func (cl *Client) ListTopicsWithInternal( + ctx context.Context, + topics ...string, +) (TopicDetails, error) { + m, err := cl.Metadata(ctx, topics...) + if err != nil { + return nil, err + } + return m.Topics, nil +} + +// CreateTopicResponse contains the response for an individual created topic. +type CreateTopicResponse struct { + Topic string // Topic is the topic that was created. + ID TopicID // ID is the topic ID for this topic, if talking to Kafka v2.8+. + Err error // Err is any error preventing this topic from being created. + ErrMessage string // ErrMessage a potential extra message describing any error. + NumPartitions int32 // NumPartitions is the number of partitions in the response, if talking to Kafka v2.4+. + ReplicationFactor int16 // ReplicationFactor is how many replicas every partition has for this topic, if talking to Kafka 2.4+. + Configs map[string]Config // Configs contains the topic configuration (minus config synonyms), if talking to Kafka 2.4+. +} + +// CreateTopicRepsonses contains per-topic responses for created topics. +type CreateTopicResponses map[string]CreateTopicResponse + +// Sorted returns all create topic responses sorted first by topic ID, then by +// topic name. +func (rs CreateTopicResponses) Sorted() []CreateTopicResponse { + s := make([]CreateTopicResponse, 0, len(rs)) + for _, d := range rs { + s = append(s, d) + } + sort.Slice(s, func(i, j int) bool { + l, r := s[i], s[j] + if l.ID.Less(r.ID) { + return true + } + return l.Topic < r.Topic + }) + return s +} + +// On calls fn for the response topic if it exists, returning the response and +// the error returned from fn. If fn is nil, this simply returns the response. +// +// The fn is given a copy of the response. This function returns the copy as +// well; any modifications within fn are modifications on the returned copy. +// +// If the topic does not exist, this returns kerr.UnknownTopicOrPartition. +func (rs CreateTopicResponses) On(topic string, fn func(*CreateTopicResponse) error) (CreateTopicResponse, error) { + if len(rs) > 0 { + r, ok := rs[topic] + if ok { + if fn == nil { + return r, nil + } + return r, fn(&r) + } + } + return CreateTopicResponse{}, kerr.UnknownTopicOrPartition +} + +// Error iterates over all responses and returns the first error +// encountered, if any. +func (rs CreateTopicResponses) Error() error { + for _, r := range rs { + if r.Err != nil { + return r.Err + } + } + return nil +} + +// CreateTopic issues a create topics request with the given partitions, +// replication factor, and (optional) configs for the given topic name. +// This is similar to CreateTopics, but returns the kerr.ErrorForCode(response.ErrorCode) +// if the request/response is successful. +func (cl *Client) CreateTopic( + ctx context.Context, + partitions int32, + replicationFactor int16, + configs map[string]*string, + topic string, +) (CreateTopicResponse, error) { + createTopicResponse, err := cl.CreateTopics( + ctx, + partitions, + replicationFactor, + configs, + topic, + ) + if err != nil { + return CreateTopicResponse{}, err + } + + response, exists := createTopicResponse[topic] + if !exists { + return CreateTopicResponse{}, errors.New("requested topic was not part of create topic response") + } + + return response, response.Err +} + +// CreateTopics issues a create topics request with the given partitions, +// replication factor, and (optional) configs for every topic. Under the hood, +// this uses the default 15s request timeout and lets Kafka choose where to +// place partitions. +// +// Version 4 of the underlying create topic request was introduced in Kafka 2.4 +// and brought client support for creation defaults. If talking to a 2.4+ +// cluster, you can use -1 for partitions and replicationFactor to use broker +// defaults. +// +// This package includes a StringPtr function to aid in building config values. +// +// This does not return an error on authorization failures, instead, +// authorization failures are included in the responses. This only returns an +// error if the request fails to be issued. You may consider checking +// ValidateCreateTopics before using this method. +func (cl *Client) CreateTopics( + ctx context.Context, + partitions int32, + replicationFactor int16, + configs map[string]*string, + topics ...string, +) (CreateTopicResponses, error) { + return cl.createTopics(ctx, false, partitions, replicationFactor, configs, topics) +} + +// ValidateCreateTopics validates a create topics request with the given +// partitions, replication factor, and (optional) configs for every topic. +// +// This package includes a StringPtr function to aid in building config values. +// +// This uses the same logic as CreateTopics, but with the request's +// ValidateOnly field set to true. The response is the same response you would +// receive from CreateTopics, but no topics are actually created. +func (cl *Client) ValidateCreateTopics( + ctx context.Context, + partitions int32, + replicationFactor int16, + configs map[string]*string, + topics ...string, +) (CreateTopicResponses, error) { + return cl.createTopics(ctx, true, partitions, replicationFactor, configs, topics) +} + +func (cl *Client) createTopics(ctx context.Context, dry bool, p int32, rf int16, configs map[string]*string, topics []string) (CreateTopicResponses, error) { + if len(topics) == 0 { + return make(CreateTopicResponses), nil + } + + req := kmsg.NewCreateTopicsRequest() + req.TimeoutMillis = cl.timeoutMillis + req.ValidateOnly = dry + for _, t := range topics { + rt := kmsg.NewCreateTopicsRequestTopic() + rt.Topic = t + rt.NumPartitions = p + rt.ReplicationFactor = rf + for k, v := range configs { + rc := kmsg.NewCreateTopicsRequestTopicConfig() + rc.Name = k + rc.Value = v + rt.Configs = append(rt.Configs, rc) + } + req.Topics = append(req.Topics, rt) + } + + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + + rs := make(CreateTopicResponses) + for _, t := range resp.Topics { + rt := CreateTopicResponse{ + Topic: t.Topic, + ID: t.TopicID, + Err: kerr.ErrorForCode(t.ErrorCode), + ErrMessage: unptrStr(t.ErrorMessage), + NumPartitions: t.NumPartitions, + ReplicationFactor: t.ReplicationFactor, + Configs: make(map[string]Config), + } + for _, c := range t.Configs { + rt.Configs[c.Name] = Config{ + Key: c.Name, + Value: c.Value, + Source: kmsg.ConfigSource(c.Source), + Sensitive: c.IsSensitive, + } + } + rs[t.Topic] = rt + } + return rs, nil +} + +// DeleteTopicResponse contains the response for an individual deleted topic. +type DeleteTopicResponse struct { + Topic string // Topic is the topic that was deleted, if not using topic IDs. + ID TopicID // ID is the topic ID for this topic, if talking to Kafka v2.8+ and using topic IDs. + Err error // Err is any error preventing this topic from being deleted. + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// DeleteTopicResponses contains per-topic responses for deleted topics. +type DeleteTopicResponses map[string]DeleteTopicResponse + +// Sorted returns all delete topic responses sorted first by topic ID, then by +// topic name. +func (rs DeleteTopicResponses) Sorted() []DeleteTopicResponse { + s := make([]DeleteTopicResponse, 0, len(rs)) + for _, d := range rs { + s = append(s, d) + } + sort.Slice(s, func(i, j int) bool { + l, r := s[i], s[j] + if l.ID.Less(r.ID) { + return true + } + return l.Topic < r.Topic + }) + return s +} + +// On calls fn for the response topic if it exists, returning the response and +// the error returned from fn. If fn is nil, this simply returns the response. +// +// The fn is given a copy of the response. This function returns the copy as +// well; any modifications within fn are modifications on the returned copy. +// +// If the topic does not exist, this returns kerr.UnknownTopicOrPartition. +func (rs DeleteTopicResponses) On(topic string, fn func(*DeleteTopicResponse) error) (DeleteTopicResponse, error) { + if len(rs) > 0 { + r, ok := rs[topic] + if ok { + if fn == nil { + return r, nil + } + return r, fn(&r) + } + } + return DeleteTopicResponse{}, kerr.UnknownTopicOrPartition +} + +// Error iterates over all responses and returns the first error +// encountered, if any. +func (rs DeleteTopicResponses) Error() error { + for _, r := range rs { + if r.Err != nil { + return r.Err + } + } + return nil +} + +// DeleteTopic issues a delete topic request for the given topic name with a +// (by default) 15s timeout. This is similar to DeleteTopics, but returns the +// kerr.ErrorForCode(response.ErrorCode) if the request/response is successful. +func (cl *Client) DeleteTopic(ctx context.Context, topic string) (DeleteTopicResponse, error) { + rs, err := cl.DeleteTopics(ctx, topic) + if err != nil { + return DeleteTopicResponse{}, err + } + r, exists := rs[topic] + if !exists { + return DeleteTopicResponse{}, errors.New("requested topic was not part of delete topic response") + } + return r, r.Err +} + +// DeleteTopics issues a delete topics request for the given topic names with a +// (by default) 15s timeout. +// +// This does not return an error on authorization failures, instead, +// authorization failures are included in the responses. This only returns an +// error if the request fails to be issued. +func (cl *Client) DeleteTopics(ctx context.Context, topics ...string) (DeleteTopicResponses, error) { + if len(topics) == 0 { + return make(DeleteTopicResponses), nil + } + + req := kmsg.NewDeleteTopicsRequest() + req.TimeoutMillis = cl.timeoutMillis + req.TopicNames = topics + for _, t := range topics { + rt := kmsg.NewDeleteTopicsRequestTopic() + rt.Topic = kmsg.StringPtr(t) + req.Topics = append(req.Topics, rt) + } + + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + + rs := make(DeleteTopicResponses) + for _, t := range resp.Topics { + // A valid Kafka will return non-nil topics here, because we + // are deleting by topic name, not ID. We still check to be + // sure, but multiple invalid (nil) topics will collide. + var topic string + if t.Topic != nil { + topic = *t.Topic + } + rs[topic] = DeleteTopicResponse{ + Topic: topic, + ID: t.TopicID, + Err: kerr.ErrorForCode(t.ErrorCode), + ErrMessage: unptrStr(t.ErrorMessage), + } + } + return rs, nil +} + +// DeleteRecordsResponse contains the response for an individual partition from +// a delete records request. +type DeleteRecordsResponse struct { + Topic string // Topic is the topic this response is for. + Partition int32 // Partition is the partition this response is for. + LowWatermark int64 // LowWatermark is the new earliest / start offset for this partition if the request was successful. + Err error // Err is any error preventing the delete records request from being successful for this partition. +} + +// DeleteRecordsResponses contains per-partition responses to a delete records request. +type DeleteRecordsResponses map[string]map[int32]DeleteRecordsResponse + +// Lookup returns the response at t and p and whether it exists. +func (ds DeleteRecordsResponses) Lookup(t string, p int32) (DeleteRecordsResponse, bool) { + if len(ds) == 0 { + return DeleteRecordsResponse{}, false + } + ps := ds[t] + if len(ps) == 0 { + return DeleteRecordsResponse{}, false + } + r, exists := ps[p] + return r, exists +} + +// Each calls fn for every delete records response. +func (ds DeleteRecordsResponses) Each(fn func(DeleteRecordsResponse)) { + for _, ps := range ds { + for _, d := range ps { + fn(d) + } + } +} + +// Sorted returns all delete records responses sorted first by topic, then by +// partition. +func (rs DeleteRecordsResponses) Sorted() []DeleteRecordsResponse { + var s []DeleteRecordsResponse + for _, ps := range rs { + for _, d := range ps { + s = append(s, d) + } + } + sort.Slice(s, func(i, j int) bool { + l, r := s[i], s[j] + if l.Topic < r.Topic { + return true + } + if l.Topic > r.Topic { + return false + } + return l.Partition < r.Partition + }) + return s +} + +// On calls fn for the response topic/partition if it exists, returning the +// response and the error returned from fn. If fn is nil, this simply returns +// the response. +// +// The fn is given a copy of the response. This function returns the copy as +// well; any modifications within fn are modifications on the returned copy. +// +// If the topic or partition does not exist, this returns +// kerr.UnknownTopicOrPartition. +func (rs DeleteRecordsResponses) On(topic string, partition int32, fn func(*DeleteRecordsResponse) error) (DeleteRecordsResponse, error) { + if len(rs) > 0 { + t, ok := rs[topic] + if ok { + p, ok := t[partition] + if ok { + if fn == nil { + return p, nil + } + return p, fn(&p) + } + } + } + return DeleteRecordsResponse{}, kerr.UnknownTopicOrPartition +} + +// Error iterates over all responses and returns the first error +// encountered, if any. +func (rs DeleteRecordsResponses) Error() error { + for _, ps := range rs { + for _, r := range ps { + if r.Err != nil { + return r.Err + } + } + } + return nil +} + +// DeleteRecords issues a delete records request for the given offsets. Per +// offset, only the Offset field needs to be set. +// +// To delete records, Kafka sets the LogStartOffset for partitions to the +// requested offset. All segments whose max partition is before the requested +// offset are deleted, and any records within the segment before the requested +// offset can no longer be read. +// +// This does not return an error on authorization failures, instead, +// authorization failures are included in the responses. +// +// This may return *ShardErrors. +func (cl *Client) DeleteRecords(ctx context.Context, os Offsets) (DeleteRecordsResponses, error) { + if len(os) == 0 { + return make(DeleteRecordsResponses), nil + } + + req := kmsg.NewPtrDeleteRecordsRequest() + req.TimeoutMillis = cl.timeoutMillis + for t, ps := range os { + rt := kmsg.NewDeleteRecordsRequestTopic() + rt.Topic = t + for p, o := range ps { + rp := kmsg.NewDeleteRecordsRequestTopicPartition() + rp.Partition = p + rp.Offset = o.At + rt.Partitions = append(rt.Partitions, rp) + } + req.Topics = append(req.Topics, rt) + } + + shards := cl.cl.RequestSharded(ctx, req) + rs := make(DeleteRecordsResponses) + return rs, shardErrEach(req, shards, func(kr kmsg.Response) error { + resp := kr.(*kmsg.DeleteRecordsResponse) + for _, t := range resp.Topics { + rt, exists := rs[t.Topic] + if !exists { // topic could be spread around brokers, we need to check existence + rt = make(map[int32]DeleteRecordsResponse) + rs[t.Topic] = rt + } + for _, p := range t.Partitions { + rt[p.Partition] = DeleteRecordsResponse{ + Topic: t.Topic, + Partition: p.Partition, + LowWatermark: p.LowWatermark, + Err: kerr.ErrorForCode(p.ErrorCode), + } + } + } + return nil + }) +} + +// CreatePartitionsResponse contains the response for an individual topic from +// a create partitions request. +type CreatePartitionsResponse struct { + Topic string // Topic is the topic this response is for. + Err error // Err is non-nil if partitions were unable to be added to this topic. + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// CreatePartitionsResponses contains per-topic responses for a create +// partitions request. +type CreatePartitionsResponses map[string]CreatePartitionsResponse + +// Sorted returns all create partitions responses sorted by topic. +func (rs CreatePartitionsResponses) Sorted() []CreatePartitionsResponse { + var s []CreatePartitionsResponse + for _, r := range rs { + s = append(s, r) + } + sort.Slice(s, func(i, j int) bool { return s[i].Topic < s[j].Topic }) + return s +} + +// On calls fn for the response topic if it exists, returning the response and +// the error returned from fn. If fn is nil, this simply returns the response. +// +// The fn is given a copy of the response. This function returns the copy as +// well; any modifications within fn are modifications on the returned copy. +// +// If the topic does not exist, this returns kerr.UnknownTopicOrPartition. +func (rs CreatePartitionsResponses) On(topic string, fn func(*CreatePartitionsResponse) error) (CreatePartitionsResponse, error) { + if len(rs) > 0 { + r, ok := rs[topic] + if ok { + if fn == nil { + return r, nil + } + return r, fn(&r) + } + } + return CreatePartitionsResponse{}, kerr.UnknownTopicOrPartition +} + +// Error iterates over all responses and returns the first error +// encountered, if any. +func (rs CreatePartitionsResponses) Error() error { + for _, r := range rs { + if r.Err != nil { + return r.Err + } + } + return nil +} + +// CreatePartitions issues a create partitions request for the given topics, +// adding "add" partitions to each topic. This request lets Kafka choose where +// the new partitions should be. +// +// This does not return an error on authorization failures for the create +// partitions request itself, instead, authorization failures are included in +// the responses. Before adding partitions, this request must issue a metadata +// request to learn the current count of partitions. If that fails, this +// returns the metadata request error. If you already know the final amount of +// partitions you want, you can use UpdatePartitions to set the count directly +// (rather than adding to the current count). You may consider checking +// ValidateCreatePartitions before using this method. +func (cl *Client) CreatePartitions(ctx context.Context, add int, topics ...string) (CreatePartitionsResponses, error) { + return cl.createPartitions(ctx, false, add, -1, topics) +} + +// UpdatePartitions issues a create partitions request for the given topics, +// setting the final partition count to "set" for each topic. This request lets +// Kafka choose where the new partitions should be. +// +// This does not return an error on authorization failures for the create +// partitions request itself, instead, authorization failures are included in +// the responses. Unlike CreatePartitions, this request uses your "set" value +// to set the new final count of partitions. "set" must be equal to or larger +// than the current count of partitions in the topic. All topics will have the +// same final count of partitions (unlike CreatePartitions, which allows you to +// add a specific count of partitions to topics that have a different amount of +// current partitions). You may consider checking ValidateUpdatePartitions +// before using this method. +func (cl *Client) UpdatePartitions(ctx context.Context, set int, topics ...string) (CreatePartitionsResponses, error) { + return cl.createPartitions(ctx, false, -1, set, topics) +} + +// ValidateCreatePartitions validates a create partitions request for adding +// "add" partitions to the given topics. +// +// This uses the same logic as CreatePartitions, but with the request's +// ValidateOnly field set to true. The response is the same response you would +// receive from CreatePartitions, but no partitions are actually added. +func (cl *Client) ValidateCreatePartitions(ctx context.Context, add int, topics ...string) (CreatePartitionsResponses, error) { + return cl.createPartitions(ctx, true, add, -1, topics) +} + +// ValidateUpdatePartitions validates a create partitions request for setting +// the partition count on the given topics to "set". +// +// This uses the same logic as UpdatePartitions, but with the request's +// ValidateOnly field set to true. The response is the same response you would +// receive from UpdatePartitions, but no partitions are actually added. +func (cl *Client) ValidateUpdatePartitions(ctx context.Context, set int, topics ...string) (CreatePartitionsResponses, error) { + return cl.createPartitions(ctx, true, -1, set, topics) +} + +func (cl *Client) createPartitions(ctx context.Context, dry bool, add, set int, topics []string) (CreatePartitionsResponses, error) { + if len(topics) == 0 { + return make(CreatePartitionsResponses), nil + } + + var td TopicDetails + var err error + if add != -1 { + td, err = cl.ListTopics(ctx, topics...) + if err != nil { + return nil, err + } + } + + req := kmsg.NewCreatePartitionsRequest() + req.TimeoutMillis = cl.timeoutMillis + req.ValidateOnly = dry + for _, t := range topics { + rt := kmsg.NewCreatePartitionsRequestTopic() + rt.Topic = t + if add == -1 { + rt.Count = int32(set) + } else { + rt.Count = int32(len(td[t].Partitions) + add) + } + req.Topics = append(req.Topics, rt) + } + + resp, err := req.RequestWith(ctx, cl.cl) + if err != nil { + return nil, err + } + + rs := make(CreatePartitionsResponses) + for _, t := range resp.Topics { + rs[t.Topic] = CreatePartitionsResponse{ + Topic: t.Topic, + Err: kerr.ErrorForCode(t.ErrorCode), + ErrMessage: unptrStr(t.ErrorMessage), + } + } + return rs, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kadm/txn.go b/vendor/github.com/twmb/franz-go/pkg/kadm/txn.go new file mode 100644 index 000000000000..2b8ccbe2e7a1 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kadm/txn.go @@ -0,0 +1,761 @@ +package kadm + +import ( + "context" + "errors" + "sort" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// DescribedProducer contains the state of a transactional producer's last +// produce. +type DescribedProducer struct { + Leader int32 // Leader is the leader broker for this topic / partition. + Topic string // Topic is the topic being produced to. + Partition int32 // Partition is the partition being produced to. + ProducerID int64 // ProducerID is the producer ID that produced. + ProducerEpoch int16 // ProducerEpoch is the epoch that produced. + LastSequence int32 // LastSequence is the last sequence number the producer produced. + LastTimestamp int64 // LastTimestamp is the last time this producer produced. + CoordinatorEpoch int32 // CoordinatorEpoch is the epoch of the transactional coordinator for the last produce. + CurrentTxnStartOffset int64 // CurrentTxnStartOffset is the first offset in the transaction. +} + +// Less returns whether the left described producer is less than the right, +// in order of: +// +// - Topic +// - Partition +// - ProducerID +// - ProducerEpoch +// - LastTimestamp +// - LastSequence +func (l *DescribedProducer) Less(r *DescribedProducer) bool { + if l.Topic < r.Topic { + return true + } + if l.Topic > r.Topic { + return false + } + if l.Partition < r.Partition { + return true + } + if l.Partition > r.Partition { + return false + } + if l.ProducerID < r.ProducerID { + return true + } + if l.ProducerID > r.ProducerID { + return false + } + if l.ProducerEpoch < r.ProducerEpoch { + return true + } + if l.ProducerEpoch > r.ProducerEpoch { + return false + } + if l.LastTimestamp < r.LastTimestamp { + return true + } + if l.LastTimestamp > r.LastTimestamp { + return false + } + return l.LastSequence < r.LastSequence +} + +// DescribedProducers maps producer IDs to the full described producer. +type DescribedProducers map[int64]DescribedProducer + +// Sorted returns the described producers sorted by topic, partition, and +// producer ID. +func (ds DescribedProducers) Sorted() []DescribedProducer { + var all []DescribedProducer + for _, d := range ds { + all = append(all, d) + } + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Topic < r.Topic || l.Topic == r.Topic && (l.Partition < r.Partition || l.Partition == r.Partition && l.ProducerID < r.ProducerID) + }) + return all +} + +// Each calls fn for each described producer. +func (ds DescribedProducers) Each(fn func(DescribedProducer)) { + for _, d := range ds { + fn(d) + } +} + +// DescribedProducersPartition is a partition whose producer's were described. +type DescribedProducersPartition struct { + Leader int32 // Leader is the leader broker for this topic / partition. + Topic string // Topic is the topic whose producer's were described. + Partition int32 // Partition is the partition whose producer's were described. + ActiveProducers DescribedProducers // ActiveProducers are producer's actively transactionally producing to this partition. + Err error // Err is non-nil if describing this partition failed. + ErrMessage string // ErrMessage a potential extra message describing any error. +} + +// DescribedProducersPartitions contains partitions whose producer's were described. +type DescribedProducersPartitions map[int32]DescribedProducersPartition + +// Sorted returns the described partitions sorted by topic and partition. +func (ds DescribedProducersPartitions) Sorted() []DescribedProducersPartition { + var all []DescribedProducersPartition + for _, d := range ds { + all = append(all, d) + } + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Topic < r.Topic || l.Topic == r.Topic && l.Partition < r.Partition + }) + return all +} + +// SortedProducer returns all producers sorted first by partition, then by producer ID. +func (ds DescribedProducersPartitions) SortedProducers() []DescribedProducer { + var all []DescribedProducer + ds.EachProducer(func(d DescribedProducer) { + all = append(all, d) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Topic < r.Topic || l.Topic == r.Topic && (l.Partition < r.Partition || l.Partition == r.Partition && l.ProducerID < r.ProducerID) + }) + return all +} + +// Each calls fn for each partition. +func (ds DescribedProducersPartitions) Each(fn func(DescribedProducersPartition)) { + for _, d := range ds { + fn(d) + } +} + +// EachProducer calls fn for each producer in all partitions. +func (ds DescribedProducersPartitions) EachProducer(fn func(DescribedProducer)) { + for _, d := range ds { + for _, p := range d.ActiveProducers { + fn(p) + } + } +} + +// DescribedProducersTopic contains topic partitions whose producer's were described. +type DescribedProducersTopic struct { + Topic string // Topic is the topic whose producer's were described. + Partitions DescribedProducersPartitions // Partitions are partitions whose producer's were described. +} + +// DescribedProducersTopics contains topics whose producer's were described. +type DescribedProducersTopics map[string]DescribedProducersTopic + +// Sorted returns the described topics sorted by topic. +func (ds DescribedProducersTopics) Sorted() []DescribedProducersTopic { + var all []DescribedProducersTopic + ds.Each(func(d DescribedProducersTopic) { + all = append(all, d) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Topic < r.Topic + }) + return all +} + +// Sorted returns the described partitions sorted by topic and partition. +func (ds DescribedProducersTopics) SortedPartitions() []DescribedProducersPartition { + var all []DescribedProducersPartition + ds.EachPartition(func(d DescribedProducersPartition) { + all = append(all, d) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Topic < r.Topic || l.Topic == r.Topic && l.Partition < r.Partition + }) + return all +} + +// SortedProducer returns all producers sorted first by partition, then by producer ID. +func (ds DescribedProducersTopics) SortedProducers() []DescribedProducer { + var all []DescribedProducer + ds.EachProducer(func(d DescribedProducer) { + all = append(all, d) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Topic < r.Topic || l.Topic == r.Topic && (l.Partition < r.Partition || l.Partition == r.Partition && l.ProducerID < r.ProducerID) + }) + return all +} + +// Each calls fn for every topic. +func (ds DescribedProducersTopics) Each(fn func(DescribedProducersTopic)) { + for _, d := range ds { + fn(d) + } +} + +// EachPartitions calls fn for all topic partitions. +func (ds DescribedProducersTopics) EachPartition(fn func(DescribedProducersPartition)) { + for _, d := range ds { + for _, p := range d.Partitions { + fn(p) + } + } +} + +// EachProducer calls fn for each producer in all topics and partitions. +func (ds DescribedProducersTopics) EachProducer(fn func(DescribedProducer)) { + for _, d := range ds { + for _, p := range d.Partitions { + for _, b := range p.ActiveProducers { + fn(b) + } + } + } +} + +// DescribeProducers describes all producers that are transactionally producing +// to the requested topic set. This request can be used to detect hanging +// transactions or other transaction related problems. If the input set is +// empty, this requests data for all partitions. +// +// This may return *ShardErrors or *AuthError. +func (cl *Client) DescribeProducers(ctx context.Context, s TopicsSet) (DescribedProducersTopics, error) { + if len(s) == 0 { + m, err := cl.Metadata(ctx) + if err != nil { + return nil, err + } + s = m.Topics.TopicsSet() + } else if e := s.EmptyTopics(); len(e) > 0 { + m, err := cl.Metadata(ctx, e...) + if err != nil { + return nil, err + } + for t, ps := range m.Topics.TopicsSet() { + s[t] = ps + } + } + + req := kmsg.NewPtrDescribeProducersRequest() + for _, t := range s.IntoList() { + rt := kmsg.NewDescribeProducersRequestTopic() + rt.Topic = t.Topic + rt.Partitions = t.Partitions + req.Topics = append(req.Topics, rt) + } + shards := cl.cl.RequestSharded(ctx, req) + dts := make(DescribedProducersTopics) + return dts, shardErrEachBroker(req, shards, func(b BrokerDetail, kr kmsg.Response) error { + resp := kr.(*kmsg.DescribeProducersResponse) + for _, rt := range resp.Topics { + dt, exists := dts[rt.Topic] + if !exists { // topic could be spread around brokers, we need to check existence + dt = DescribedProducersTopic{ + Topic: rt.Topic, + Partitions: make(DescribedProducersPartitions), + } + dts[rt.Topic] = dt + } + dps := dt.Partitions + for _, rp := range rt.Partitions { + if err := maybeAuthErr(rp.ErrorCode); err != nil { + return err + } + drs := make(DescribedProducers) + dp := DescribedProducersPartition{ + Leader: b.NodeID, + Topic: rt.Topic, + Partition: rp.Partition, + ActiveProducers: drs, + Err: kerr.ErrorForCode(rp.ErrorCode), + ErrMessage: unptrStr(rp.ErrorMessage), + } + dps[rp.Partition] = dp // one partition globally, no need to exist-check + for _, rr := range rp.ActiveProducers { + dr := DescribedProducer{ + Leader: b.NodeID, + Topic: rt.Topic, + Partition: rp.Partition, + ProducerID: rr.ProducerID, + ProducerEpoch: int16(rr.ProducerEpoch), + LastSequence: rr.LastSequence, + LastTimestamp: rr.LastTimestamp, + CoordinatorEpoch: rr.CoordinatorEpoch, + CurrentTxnStartOffset: rr.CurrentTxnStartOffset, + } + drs[dr.ProducerID] = dr + } + } + } + return nil + }) +} + +// DescribedTransaction contains data from a describe transactions response for +// a single transactional ID. +type DescribedTransaction struct { + Coordinator int32 // Coordinator is the coordinator broker for this transactional ID. + TxnID string // TxnID is the name of this transactional ID. + State string // State is the state this transaction is in (Empty, Ongoing, PrepareCommit, PrepareAbort, CompleteCommit, CompleteAbort, Dead, PrepareEpochFence). + TimeoutMillis int32 // TimeoutMillis is the timeout of this transaction in milliseconds. + StartTimestamp int64 // StartTimestamp is millisecond when this transaction started. + ProducerID int64 // ProducerID is the ID in use by the transactional ID. + ProducerEpoch int16 // ProducerEpoch is the epoch associated with the produce rID. + + // Topics is the set of partitions in the transaction, if active. When + // preparing to commit or abort, this includes only partitions which do + // not have markers. This does not include topics the user is not + // authorized to describe. + Topics TopicsSet + + Err error // Err is non-nil if the transaction could not be described. +} + +// DescribedTransactions contains information from a describe transactions +// response. +type DescribedTransactions map[string]DescribedTransaction + +// Sorted returns all described transactions sorted by transactional ID. +func (ds DescribedTransactions) Sorted() []DescribedTransaction { + s := make([]DescribedTransaction, 0, len(ds)) + for _, d := range ds { + s = append(s, d) + } + sort.Slice(s, func(i, j int) bool { return s[i].TxnID < s[j].TxnID }) + return s +} + +// Each calls fn for each described transaction. +func (ds DescribedTransactions) Each(fn func(DescribedTransaction)) { + for _, d := range ds { + fn(d) + } +} + +// On calls fn for the transactional ID if it exists, returning the transaction +// and the error returned from fn. If fn is nil, this simply returns the +// transaction. +// +// The fn is given a shallow copy of the transaction. This function returns the +// copy as well; any modifications within fn are modifications on the returned +// copy. Modifications on a described transaction's inner fields are persisted +// to the original map (because slices are pointers). +// +// If the transaction does not exist, this returns +// kerr.TransactionalIDNotFound. +func (rs DescribedTransactions) On(txnID string, fn func(*DescribedTransaction) error) (DescribedTransaction, error) { + if len(rs) > 0 { + r, ok := rs[txnID] + if ok { + if fn == nil { + return r, nil + } + return r, fn(&r) + } + } + return DescribedTransaction{}, kerr.TransactionalIDNotFound +} + +// TransactionalIDs returns a sorted list of all transactional IDs. +func (ds DescribedTransactions) TransactionalIDs() []string { + all := make([]string, 0, len(ds)) + for t := range ds { + all = append(all, t) + } + sort.Strings(all) + return all +} + +// DescribeTransactions describes either all transactional IDs specified, or +// all transactional IDs in the cluster if none are specified. +// +// This may return *ShardErrors or *AuthError. +// +// If no transactional IDs are specified and this method first lists +// transactional IDs, and listing IDs returns a *ShardErrors, this function +// describes all successfully listed IDs and appends the list shard errors to +// any describe shard errors. +// +// If only one ID is described, there will be at most one request issued and +// there is no need to deeply inspect the error. +func (cl *Client) DescribeTransactions(ctx context.Context, txnIDs ...string) (DescribedTransactions, error) { + var seList *ShardErrors + if len(txnIDs) == 0 { + listed, err := cl.ListTransactions(ctx, nil, nil) + switch { + case err == nil: + case errors.As(err, &seList): + default: + return nil, err + } + txnIDs = listed.TransactionalIDs() + if len(txnIDs) == 0 { + return nil, err + } + } + + req := kmsg.NewPtrDescribeTransactionsRequest() + req.TransactionalIDs = txnIDs + + shards := cl.cl.RequestSharded(ctx, req) + described := make(DescribedTransactions) + err := shardErrEachBroker(req, shards, func(b BrokerDetail, kr kmsg.Response) error { + resp := kr.(*kmsg.DescribeTransactionsResponse) + for _, rt := range resp.TransactionStates { + if err := maybeAuthErr(rt.ErrorCode); err != nil { + return err + } + t := DescribedTransaction{ + Coordinator: b.NodeID, + TxnID: rt.TransactionalID, + State: rt.State, + TimeoutMillis: rt.TimeoutMillis, + StartTimestamp: rt.StartTimestamp, + ProducerID: rt.ProducerID, + ProducerEpoch: rt.ProducerEpoch, + Err: kerr.ErrorForCode(rt.ErrorCode), + } + for _, rtt := range rt.Topics { + t.Topics.Add(rtt.Topic, rtt.Partitions...) + } + described[t.TxnID] = t // txnID lives on one coordinator, no need to exist-check + } + return nil + }) + + var seDesc *ShardErrors + switch { + case err == nil: + return described, seList.into() + case errors.As(err, &seDesc): + if seList != nil { + seDesc.Errs = append(seList.Errs, seDesc.Errs...) + } + return described, seDesc.into() + default: + return nil, err + } +} + +// ListedTransaction contains data from a list transactions response for a +// single transactional ID. +type ListedTransaction struct { + Coordinator int32 // Coordinator the coordinator broker for this transactional ID. + TxnID string // TxnID is the name of this transactional ID. + ProducerID int64 // ProducerID is the producer ID for this transaction. + State string // State is the state this transaction is in (Empty, Ongoing, PrepareCommit, PrepareAbort, CompleteCommit, CompleteAbort, Dead, PrepareEpochFence). +} + +// ListedTransactions contains information from a list transactions response. +type ListedTransactions map[string]ListedTransaction + +// Sorted returns all transactions sorted by transactional ID. +func (ls ListedTransactions) Sorted() []ListedTransaction { + s := make([]ListedTransaction, 0, len(ls)) + for _, l := range ls { + s = append(s, l) + } + sort.Slice(s, func(i, j int) bool { return s[i].TxnID < s[j].TxnID }) + return s +} + +// Each calls fn for each listed transaction. +func (ls ListedTransactions) Each(fn func(ListedTransaction)) { + for _, l := range ls { + fn(l) + } +} + +// TransactionalIDs returns a sorted list of all transactional IDs. +func (ls ListedTransactions) TransactionalIDs() []string { + all := make([]string, 0, len(ls)) + for t := range ls { + all = append(all, t) + } + sort.Strings(all) + return all +} + +// ListTransactions returns all transactions and their states in the cluster. +// Filter states can be used to return transactions only in the requested +// states. By default, this returns all transactions you have DESCRIBE access +// to. Producer IDs can be specified to filter for transactions from the given +// producer. +// +// This may return *ShardErrors or *AuthError. +func (cl *Client) ListTransactions(ctx context.Context, producerIDs []int64, filterStates []string) (ListedTransactions, error) { + req := kmsg.NewPtrListTransactionsRequest() + req.ProducerIDFilters = producerIDs + req.StateFilters = filterStates + shards := cl.cl.RequestSharded(ctx, req) + list := make(ListedTransactions) + return list, shardErrEachBroker(req, shards, func(b BrokerDetail, kr kmsg.Response) error { + resp := kr.(*kmsg.ListTransactionsResponse) + if err := maybeAuthErr(resp.ErrorCode); err != nil { + return err + } + if err := kerr.ErrorForCode(resp.ErrorCode); err != nil { + return err + } + for _, t := range resp.TransactionStates { + list[t.TransactionalID] = ListedTransaction{ // txnID lives on one coordinator, no need to exist-check + Coordinator: b.NodeID, + TxnID: t.TransactionalID, + ProducerID: t.ProducerID, + State: t.TransactionState, + } + } + return nil + }) +} + +// TxnMarkers marks the end of a partition: the producer ID / epoch doing the +// writing, whether this is a commit, the coordinator epoch of the broker we +// are writing to (for fencing), and the topics and partitions that we are +// writing this abort or commit for. +// +// This is a very low level admin request and should likely be built from data +// in a DescribeProducers response. See KIP-664 if you are trying to use this. +type TxnMarkers struct { + ProducerID int64 // ProducerID is the ID to write markers for. + ProducerEpoch int16 // ProducerEpoch is the epoch to write markers for. + Commit bool // Commit is true if we are committing, false if we are aborting. + CoordinatorEpoch int32 // CoordinatorEpoch is the epoch of the transactional coordinator we are writing to; this is used for fencing. + Topics TopicsSet // Topics are topics and partitions to write markers for. +} + +// TxnMarkersPartitionResponse is a response to a topic's partition within a +// single marker written. +type TxnMarkersPartitionResponse struct { + NodeID int32 // NodeID is the node that this marker was written to. + ProducerID int64 // ProducerID corresponds to the PID in the write marker request. + Topic string // Topic is the topic being responded to. + Partition int32 // Partition is the partition being responded to. + Err error // Err is non-nil if the WriteTxnMarkers request for this pid/topic/partition failed. +} + +// TxnMarkersPartitionResponses contains per-partition responses to a +// WriteTxnMarkers request. +type TxnMarkersPartitionResponses map[int32]TxnMarkersPartitionResponse + +// Sorted returns all partitions sorted by partition. +func (ps TxnMarkersPartitionResponses) Sorted() []TxnMarkersPartitionResponse { + var all []TxnMarkersPartitionResponse + ps.Each(func(p TxnMarkersPartitionResponse) { + all = append(all, p) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Partition < r.Partition + }) + return all +} + +// Each calls fn for each partition. +func (ps TxnMarkersPartitionResponses) Each(fn func(TxnMarkersPartitionResponse)) { + for _, p := range ps { + fn(p) + } +} + +// TxnMarkersTopicResponse is a response to a topic within a single marker +// written. +type TxnMarkersTopicResponse struct { + ProducerID int64 // ProducerID corresponds to the PID in the write marker request. + Topic string // Topic is the topic being responded to. + Partitions TxnMarkersPartitionResponses // Partitions are the responses for partitions in this marker. +} + +// TxnMarkersTopicResponses contains per-topic responses to a WriteTxnMarkers +// request. +type TxnMarkersTopicResponses map[string]TxnMarkersTopicResponse + +// Sorted returns all topics sorted by topic. +func (ts TxnMarkersTopicResponses) Sorted() []TxnMarkersTopicResponse { + var all []TxnMarkersTopicResponse + ts.Each(func(t TxnMarkersTopicResponse) { + all = append(all, t) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Topic < r.Topic + }) + return all +} + +// SortedPartitions returns all topics sorted by topic then partition. +func (ts TxnMarkersTopicResponses) SortedPartitions() []TxnMarkersPartitionResponse { + var all []TxnMarkersPartitionResponse + ts.EachPartition(func(p TxnMarkersPartitionResponse) { + all = append(all, p) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.Topic < r.Topic || l.Topic == r.Topic && l.Partition < r.Partition + }) + return all +} + +// Each calls fn for each topic. +func (ts TxnMarkersTopicResponses) Each(fn func(TxnMarkersTopicResponse)) { + for _, t := range ts { + fn(t) + } +} + +// EachPartition calls fn for every partition in all topics. +func (ts TxnMarkersTopicResponses) EachPartition(fn func(TxnMarkersPartitionResponse)) { + for _, t := range ts { + for _, p := range t.Partitions { + fn(p) + } + } +} + +// TxnMarkersResponse is a response for a single marker written. +type TxnMarkersResponse struct { + ProducerID int64 // ProducerID corresponds to the PID in the write marker request. + Topics TxnMarkersTopicResponses // Topics contains the topics that markers were written for, for this ProducerID. +} + +// TxnMarkersResponse contains per-partition-ID responses to a WriteTxnMarkers +// request. +type TxnMarkersResponses map[int64]TxnMarkersResponse + +// Sorted returns all markers sorted by producer ID. +func (ms TxnMarkersResponses) Sorted() []TxnMarkersResponse { + var all []TxnMarkersResponse + ms.Each(func(m TxnMarkersResponse) { + all = append(all, m) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.ProducerID < r.ProducerID + }) + return all +} + +// SortedTopics returns all marker topics sorted by producer ID then topic. +func (ms TxnMarkersResponses) SortedTopics() []TxnMarkersTopicResponse { + var all []TxnMarkersTopicResponse + ms.EachTopic(func(t TxnMarkersTopicResponse) { + all = append(all, t) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.ProducerID < r.ProducerID || l.ProducerID == r.ProducerID && l.Topic < r.Topic + }) + return all +} + +// SortedPartitions returns all marker topic partitions sorted by producer ID +// then topic then partition. +func (ms TxnMarkersResponses) SortedPartitions() []TxnMarkersPartitionResponse { + var all []TxnMarkersPartitionResponse + ms.EachPartition(func(p TxnMarkersPartitionResponse) { + all = append(all, p) + }) + sort.Slice(all, func(i, j int) bool { + l, r := all[i], all[j] + return l.ProducerID < r.ProducerID || l.ProducerID == r.ProducerID && l.Topic < r.Topic || l.Topic == r.Topic && l.Partition < r.Partition + }) + return all +} + +// Each calls fn for each marker response. +func (ms TxnMarkersResponses) Each(fn func(TxnMarkersResponse)) { + for _, m := range ms { + fn(m) + } +} + +// EachTopic calls fn for every topic in all marker responses. +func (ms TxnMarkersResponses) EachTopic(fn func(TxnMarkersTopicResponse)) { + for _, m := range ms { + for _, t := range m.Topics { + fn(t) + } + } +} + +// EachPartition calls fn for every partition in all topics in all marker +// responses. +func (ms TxnMarkersResponses) EachPartition(fn func(TxnMarkersPartitionResponse)) { + for _, m := range ms { + for _, t := range m.Topics { + for _, p := range t.Partitions { + fn(p) + } + } + } +} + +// WriteTxnMarkers writes transaction markers to brokers. This is an advanced +// admin way to close out open transactions. See KIP-664 for more details. +// +// This may return *ShardErrors or *AuthError. +func (cl *Client) WriteTxnMarkers(ctx context.Context, markers ...TxnMarkers) (TxnMarkersResponses, error) { + req := kmsg.NewPtrWriteTxnMarkersRequest() + for _, m := range markers { + rm := kmsg.NewWriteTxnMarkersRequestMarker() + rm.ProducerID = m.ProducerID + rm.ProducerEpoch = m.ProducerEpoch + rm.Committed = m.Commit + rm.CoordinatorEpoch = m.CoordinatorEpoch + for t, ps := range m.Topics { + rt := kmsg.NewWriteTxnMarkersRequestMarkerTopic() + rt.Topic = t + for p := range ps { + rt.Partitions = append(rt.Partitions, p) + } + rm.Topics = append(rm.Topics, rt) + } + req.Markers = append(req.Markers, rm) + } + shards := cl.cl.RequestSharded(ctx, req) + rs := make(TxnMarkersResponses) + return rs, shardErrEachBroker(req, shards, func(b BrokerDetail, kr kmsg.Response) error { + resp := kr.(*kmsg.WriteTxnMarkersResponse) + for _, rm := range resp.Markers { + m, exists := rs[rm.ProducerID] // partitions are spread around, our marker could be split: we need to check existence + if !exists { + m = TxnMarkersResponse{ + ProducerID: rm.ProducerID, + Topics: make(TxnMarkersTopicResponses), + } + rs[rm.ProducerID] = m + } + for _, rt := range rm.Topics { + t, exists := m.Topics[rt.Topic] + if !exists { // same thought + t = TxnMarkersTopicResponse{ + ProducerID: rm.ProducerID, + Topic: rt.Topic, + Partitions: make(TxnMarkersPartitionResponses), + } + m.Topics[rt.Topic] = t + } + for _, rp := range rt.Partitions { + if err := maybeAuthErr(rp.ErrorCode); err != nil { + return err + } + t.Partitions[rp.Partition] = TxnMarkersPartitionResponse{ // one partition globally, no need to exist-check + NodeID: b.NodeID, + ProducerID: rm.ProducerID, + Topic: rt.Topic, + Partition: rp.Partition, + Err: kerr.ErrorForCode(rp.ErrorCode), + } + } + } + } + return nil + }) +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/00_produce.go b/vendor/github.com/twmb/franz-go/pkg/kfake/00_produce.go new file mode 100644 index 000000000000..9aeb7668130a --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/00_produce.go @@ -0,0 +1,180 @@ +package kfake + +import ( + "hash/crc32" + "net" + "strconv" + "time" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// TODO +// * Leaders +// * Support txns +// * Multiple batches in one produce +// * Compact + +func init() { regKey(0, 3, 10) } + +func (c *Cluster) handleProduce(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + var ( + req = kreq.(*kmsg.ProduceRequest) + resp = req.ResponseKind().(*kmsg.ProduceResponse) + tdone = make(map[string][]kmsg.ProduceResponseTopicPartition) + ) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + donep := func(t string, p kmsg.ProduceRequestTopicPartition, errCode int16) *kmsg.ProduceResponseTopicPartition { + sp := kmsg.NewProduceResponseTopicPartition() + sp.Partition = p.Partition + sp.ErrorCode = errCode + ps := tdone[t] + ps = append(ps, sp) + tdone[t] = ps + return &ps[len(ps)-1] + } + donet := func(t kmsg.ProduceRequestTopic, errCode int16) { + for _, p := range t.Partitions { + donep(t.Topic, p, errCode) + } + } + donets := func(errCode int16) { + for _, t := range req.Topics { + donet(t, errCode) + } + } + var includeBrokers bool + toresp := func() kmsg.Response { + for topic, partitions := range tdone { + st := kmsg.NewProduceResponseTopic() + st.Topic = topic + st.Partitions = partitions + resp.Topics = append(resp.Topics, st) + } + if includeBrokers { + for _, b := range c.bs { + sb := kmsg.NewProduceResponseBroker() + h, p, _ := net.SplitHostPort(b.ln.Addr().String()) + p32, _ := strconv.Atoi(p) + sb.NodeID = b.node + sb.Host = h + sb.Port = int32(p32) + resp.Brokers = append(resp.Brokers, sb) + } + } + return resp + } + + if req.TransactionID != nil { + donets(kerr.TransactionalIDAuthorizationFailed.Code) + return toresp(), nil + } + switch req.Acks { + case -1, 0, 1: + default: + donets(kerr.InvalidRequiredAcks.Code) + return toresp(), nil + } + + now := time.Now().UnixMilli() + for _, rt := range req.Topics { + for _, rp := range rt.Partitions { + pd, ok := c.data.tps.getp(rt.Topic, rp.Partition) + if !ok { + donep(rt.Topic, rp, kerr.UnknownTopicOrPartition.Code) + continue + } + if pd.leader != b { + p := donep(rt.Topic, rp, kerr.NotLeaderForPartition.Code) + p.CurrentLeader.LeaderID = pd.leader.node + p.CurrentLeader.LeaderEpoch = pd.epoch + includeBrokers = true + continue + } + + var b kmsg.RecordBatch + if err := b.ReadFrom(rp.Records); err != nil { + donep(rt.Topic, rp, kerr.CorruptMessage.Code) + continue + } + if b.FirstOffset != 0 { + donep(rt.Topic, rp, kerr.CorruptMessage.Code) + continue + } + if int(b.Length) != len(rp.Records)-12 { + donep(rt.Topic, rp, kerr.CorruptMessage.Code) + continue + } + if b.PartitionLeaderEpoch != -1 { + donep(rt.Topic, rp, kerr.CorruptMessage.Code) + continue + } + if b.Magic != 2 { + donep(rt.Topic, rp, kerr.CorruptMessage.Code) + continue + } + if b.CRC != int32(crc32.Checksum(rp.Records[21:], crc32c)) { // crc starts at byte 21 + donep(rt.Topic, rp, kerr.CorruptMessage.Code) + continue + } + attrs := uint16(b.Attributes) + if attrs&0x0007 > 4 { + donep(rt.Topic, rp, kerr.CorruptMessage.Code) + continue + } + logAppendTime := int64(-1) + if attrs&0x0008 > 0 { + b.FirstTimestamp = now + b.MaxTimestamp = now + logAppendTime = now + } + if attrs&0xfff0 != 0 { // TODO txn bit + donep(rt.Topic, rp, kerr.CorruptMessage.Code) + continue + } + if b.LastOffsetDelta != b.NumRecords-1 { + donep(rt.Topic, rp, kerr.CorruptMessage.Code) + continue + } + + seqs, epoch := c.pids.get(b.ProducerID, b.ProducerEpoch, rt.Topic, rp.Partition) + if be := b.ProducerEpoch; be != -1 { + if be < epoch { + donep(rt.Topic, rp, kerr.FencedLeaderEpoch.Code) + continue + } else if be > epoch { + donep(rt.Topic, rp, kerr.UnknownLeaderEpoch.Code) + continue + } + } + ok, dup := seqs.pushAndValidate(b.FirstSequence, b.NumRecords) + if !ok { + donep(rt.Topic, rp, kerr.OutOfOrderSequenceNumber.Code) + continue + } + if dup { + donep(rt.Topic, rp, 0) + continue + } + baseOffset := pd.highWatermark + lso := pd.logStartOffset + pd.pushBatch(len(rp.Records), b) + sp := donep(rt.Topic, rp, 0) + sp.BaseOffset = baseOffset + sp.LogAppendTime = logAppendTime + sp.LogStartOffset = lso + } + } + + if req.Acks == 0 { + return nil, nil + } + return toresp(), nil +} + +var crc32c = crc32.MakeTable(crc32.Castagnoli) diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/01_fetch.go b/vendor/github.com/twmb/franz-go/pkg/kfake/01_fetch.go new file mode 100644 index 000000000000..53f9a8e6ee96 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/01_fetch.go @@ -0,0 +1,236 @@ +package kfake + +import ( + "net" + "strconv" + "sync" + "time" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// Behavior: +// +// * If topic does not exist, we hang +// * Topic created while waiting is not returned in final response +// * If any partition is on a different broker, we return immediately +// * Out of range fetch causes early return +// * Raw bytes of batch counts against wait bytes + +func init() { regKey(1, 4, 16) } + +func (c *Cluster) handleFetch(creq *clientReq, w *watchFetch) (kmsg.Response, error) { + var ( + req = creq.kreq.(*kmsg.FetchRequest) + resp = req.ResponseKind().(*kmsg.FetchResponse) + ) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + var ( + nbytes int + returnEarly bool + needp tps[int] + ) + if w == nil { + out: + for i, rt := range req.Topics { + if req.Version >= 13 { + rt.Topic = c.data.id2t[rt.TopicID] + req.Topics[i].Topic = rt.Topic + } + t, ok := c.data.tps.gett(rt.Topic) + if !ok { + continue + } + for _, rp := range rt.Partitions { + pd, ok := t[rp.Partition] + if !ok || pd.createdAt.After(creq.at) { + continue + } + if pd.leader != creq.cc.b { + returnEarly = true // NotLeaderForPartition + break out + } + i, ok, atEnd := pd.searchOffset(rp.FetchOffset) + if atEnd { + continue + } + if !ok { + returnEarly = true // OffsetOutOfRange + break out + } + pbytes := 0 + for _, b := range pd.batches[i:] { + nbytes += b.nbytes + pbytes += b.nbytes + if pbytes >= int(rp.PartitionMaxBytes) { + returnEarly = true + break out + } + } + needp.set(rt.Topic, rp.Partition, int(rp.PartitionMaxBytes)-pbytes) + } + } + } + + wait := time.Duration(req.MaxWaitMillis) * time.Millisecond + deadline := creq.at.Add(wait) + if w == nil && !returnEarly && nbytes < int(req.MinBytes) && time.Now().Before(deadline) { + w := &watchFetch{ + need: int(req.MinBytes) - nbytes, + needp: needp, + deadline: deadline, + creq: creq, + } + w.cb = func() { + select { + case c.watchFetchCh <- w: + case <-c.die: + } + } + for _, rt := range req.Topics { + t, ok := c.data.tps.gett(rt.Topic) + if !ok { + continue + } + for _, rp := range rt.Partitions { + pd, ok := t[rp.Partition] + if !ok || pd.createdAt.After(creq.at) { + continue + } + pd.watch[w] = struct{}{} + w.in = append(w.in, pd) + } + } + w.t = time.AfterFunc(wait, w.cb) + return nil, nil + } + + id2t := make(map[uuid]string) + tidx := make(map[string]int) + + donet := func(t string, id uuid, errCode int16) *kmsg.FetchResponseTopic { + if i, ok := tidx[t]; ok { + return &resp.Topics[i] + } + id2t[id] = t + tidx[t] = len(resp.Topics) + st := kmsg.NewFetchResponseTopic() + st.Topic = t + st.TopicID = id + resp.Topics = append(resp.Topics, st) + return &resp.Topics[len(resp.Topics)-1] + } + donep := func(t string, id uuid, p int32, errCode int16) *kmsg.FetchResponseTopicPartition { + sp := kmsg.NewFetchResponseTopicPartition() + sp.Partition = p + sp.ErrorCode = errCode + st := donet(t, id, 0) + st.Partitions = append(st.Partitions, sp) + return &st.Partitions[len(st.Partitions)-1] + } + + var includeBrokers bool + defer func() { + if includeBrokers { + for _, b := range c.bs { + sb := kmsg.NewFetchResponseBroker() + h, p, _ := net.SplitHostPort(b.ln.Addr().String()) + p32, _ := strconv.Atoi(p) + sb.NodeID = b.node + sb.Host = h + sb.Port = int32(p32) + resp.Brokers = append(resp.Brokers, sb) + } + } + }() + + var batchesAdded int +full: + for _, rt := range req.Topics { + for _, rp := range rt.Partitions { + pd, ok := c.data.tps.getp(rt.Topic, rp.Partition) + if !ok { + if req.Version >= 13 { + donep(rt.Topic, rt.TopicID, rp.Partition, kerr.UnknownTopicID.Code) + } else { + donep(rt.Topic, rt.TopicID, rp.Partition, kerr.UnknownTopicOrPartition.Code) + } + continue + } + if pd.leader != creq.cc.b { + p := donep(rt.Topic, rt.TopicID, rp.Partition, kerr.NotLeaderForPartition.Code) + p.CurrentLeader.LeaderID = pd.leader.node + p.CurrentLeader.LeaderEpoch = pd.epoch + includeBrokers = true + continue + } + sp := donep(rt.Topic, rt.TopicID, rp.Partition, 0) + sp.HighWatermark = pd.highWatermark + sp.LastStableOffset = pd.lastStableOffset + sp.LogStartOffset = pd.logStartOffset + i, ok, atEnd := pd.searchOffset(rp.FetchOffset) + if atEnd { + continue + } + if !ok { + sp.ErrorCode = kerr.OffsetOutOfRange.Code + continue + } + var pbytes int + for _, b := range pd.batches[i:] { + if nbytes = nbytes + b.nbytes; nbytes > int(req.MaxBytes) && batchesAdded > 1 { + break full + } + if pbytes = pbytes + b.nbytes; pbytes > int(rp.PartitionMaxBytes) && batchesAdded > 1 { + break + } + batchesAdded++ + sp.RecordBatches = b.AppendTo(sp.RecordBatches) + } + } + } + + return resp, nil +} + +type watchFetch struct { + need int + needp tps[int] + deadline time.Time + creq *clientReq + + in []*partData + cb func() + t *time.Timer + + once sync.Once + cleaned bool +} + +func (w *watchFetch) push(nbytes int) { + w.need -= nbytes + if w.need <= 0 { + w.once.Do(func() { + go w.cb() + }) + } +} + +func (w *watchFetch) deleted() { + w.once.Do(func() { + go w.cb() + }) +} + +func (w *watchFetch) cleanup(c *Cluster) { + w.cleaned = true + for _, in := range w.in { + delete(in.watch, w) + } + w.t.Stop() +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/02_list_offsets.go b/vendor/github.com/twmb/franz-go/pkg/kfake/02_list_offsets.go new file mode 100644 index 000000000000..587c41a6c777 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/02_list_offsets.go @@ -0,0 +1,99 @@ +package kfake + +import ( + "sort" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(2, 0, 7) } + +func (c *Cluster) handleListOffsets(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.ListOffsetsRequest) + resp := req.ResponseKind().(*kmsg.ListOffsetsResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + tidx := make(map[string]int) + donet := func(t string, errCode int16) *kmsg.ListOffsetsResponseTopic { + if i, ok := tidx[t]; ok { + return &resp.Topics[i] + } + tidx[t] = len(resp.Topics) + st := kmsg.NewListOffsetsResponseTopic() + st.Topic = t + resp.Topics = append(resp.Topics, st) + return &resp.Topics[len(resp.Topics)-1] + } + donep := func(t string, p int32, errCode int16) *kmsg.ListOffsetsResponseTopicPartition { + sp := kmsg.NewListOffsetsResponseTopicPartition() + sp.Partition = p + sp.ErrorCode = errCode + st := donet(t, 0) + st.Partitions = append(st.Partitions, sp) + return &st.Partitions[len(st.Partitions)-1] + } + + for _, rt := range req.Topics { + ps, ok := c.data.tps.gett(rt.Topic) + for _, rp := range rt.Partitions { + if !ok { + donep(rt.Topic, rp.Partition, kerr.UnknownTopicOrPartition.Code) + continue + } + pd, ok := ps[rp.Partition] + if !ok { + donep(rt.Topic, rp.Partition, kerr.UnknownTopicOrPartition.Code) + continue + } + if pd.leader != b { + donep(rt.Topic, rp.Partition, kerr.NotLeaderForPartition.Code) + continue + } + if le := rp.CurrentLeaderEpoch; le != -1 { + if le < pd.epoch { + donep(rt.Topic, rp.Partition, kerr.FencedLeaderEpoch.Code) + continue + } else if le > pd.epoch { + donep(rt.Topic, rp.Partition, kerr.UnknownLeaderEpoch.Code) + continue + } + } + + sp := donep(rt.Topic, rp.Partition, 0) + sp.LeaderEpoch = pd.epoch + switch rp.Timestamp { + case -2: + sp.Offset = pd.logStartOffset + case -1: + if req.IsolationLevel == 1 { + sp.Offset = pd.lastStableOffset + } else { + sp.Offset = pd.highWatermark + } + default: + // returns the index of the first batch _after_ the requested timestamp + idx, _ := sort.Find(len(pd.batches), func(idx int) int { + maxEarlier := pd.batches[idx].maxEarlierTimestamp + switch { + case maxEarlier > rp.Timestamp: + return -1 + case maxEarlier == rp.Timestamp: + return 0 + default: + return 1 + } + }) + if idx == len(pd.batches) { + sp.Offset = -1 + } else { + sp.Offset = pd.batches[idx].FirstOffset + } + } + } + } + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/03_metadata.go b/vendor/github.com/twmb/franz-go/pkg/kfake/03_metadata.go new file mode 100644 index 000000000000..a4f81276725f --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/03_metadata.go @@ -0,0 +1,120 @@ +package kfake + +import ( + "net" + "strconv" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(3, 0, 12) } + +func (c *Cluster) handleMetadata(kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.MetadataRequest) + resp := req.ResponseKind().(*kmsg.MetadataResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + for _, b := range c.bs { + sb := kmsg.NewMetadataResponseBroker() + h, p, _ := net.SplitHostPort(b.ln.Addr().String()) + p32, _ := strconv.Atoi(p) + sb.NodeID = b.node + sb.Host = h + sb.Port = int32(p32) + resp.Brokers = append(resp.Brokers, sb) + } + + resp.ClusterID = &c.cfg.clusterID + resp.ControllerID = c.controller.node + + id2t := make(map[uuid]string) + tidx := make(map[string]int) + + donet := func(t string, id uuid, errCode int16) *kmsg.MetadataResponseTopic { + if i, ok := tidx[t]; ok { + return &resp.Topics[i] + } + id2t[id] = t + tidx[t] = len(resp.Topics) + st := kmsg.NewMetadataResponseTopic() + if t != "" { + st.Topic = kmsg.StringPtr(t) + } + st.TopicID = id + st.ErrorCode = errCode + resp.Topics = append(resp.Topics, st) + return &resp.Topics[len(resp.Topics)-1] + } + donep := func(t string, id uuid, p int32, errCode int16) *kmsg.MetadataResponseTopicPartition { + sp := kmsg.NewMetadataResponseTopicPartition() + sp.Partition = p + sp.ErrorCode = errCode + st := donet(t, id, 0) + st.Partitions = append(st.Partitions, sp) + return &st.Partitions[len(st.Partitions)-1] + } + okp := func(t string, id uuid, p int32, pd *partData) { + nreplicas := c.data.treplicas[t] + if nreplicas > len(c.bs) { + nreplicas = len(c.bs) + } + + sp := donep(t, id, p, 0) + sp.Leader = pd.leader.node + sp.LeaderEpoch = pd.epoch + + for i := 0; i < nreplicas; i++ { + idx := (pd.leader.bsIdx + i) % len(c.bs) + sp.Replicas = append(sp.Replicas, c.bs[idx].node) + } + sp.ISR = sp.Replicas + } + + allowAuto := req.AllowAutoTopicCreation && c.cfg.allowAutoTopic + for _, rt := range req.Topics { + var topic string + var ok bool + // If topic ID is present, we ignore any provided topic. + // Duplicate topics are merged into one response topic. + // Topics with no topic and no ID are ignored. + if rt.TopicID != noID { + if topic, ok = c.data.id2t[rt.TopicID]; !ok { + donet("", rt.TopicID, kerr.UnknownTopicID.Code) + continue + } + } else if rt.Topic == nil { + continue + } else { + topic = *rt.Topic + } + + ps, ok := c.data.tps.gett(topic) + if !ok { + if !allowAuto { + donet(topic, rt.TopicID, kerr.UnknownTopicOrPartition.Code) + continue + } + c.data.mkt(topic, -1, -1, nil) + ps, _ = c.data.tps.gett(topic) + } + + id := c.data.t2id[topic] + for p, pd := range ps { + okp(topic, id, p, pd) + } + } + if req.Topics == nil && c.data.tps != nil { + for topic, ps := range c.data.tps { + id := c.data.t2id[topic] + for p, pd := range ps { + okp(topic, id, p, pd) + } + } + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/08_offset_commit.go b/vendor/github.com/twmb/franz-go/pkg/kfake/08_offset_commit.go new file mode 100644 index 000000000000..b82400e0f4fa --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/08_offset_commit.go @@ -0,0 +1,18 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(8, 0, 8) } + +func (c *Cluster) handleOffsetCommit(creq *clientReq) (kmsg.Response, error) { + req := creq.kreq.(*kmsg.OffsetCommitRequest) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + c.groups.handleOffsetCommit(creq) + return nil, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/09_offset_fetch.go b/vendor/github.com/twmb/franz-go/pkg/kfake/09_offset_fetch.go new file mode 100644 index 000000000000..204339ecff76 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/09_offset_fetch.go @@ -0,0 +1,17 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(9, 0, 8) } + +func (c *Cluster) handleOffsetFetch(creq *clientReq) (kmsg.Response, error) { + req := creq.kreq.(*kmsg.OffsetFetchRequest) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + return c.groups.handleOffsetFetch(creq), nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/10_find_coordinator.go b/vendor/github.com/twmb/franz-go/pkg/kfake/10_find_coordinator.go new file mode 100644 index 000000000000..2873a3320bc1 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/10_find_coordinator.go @@ -0,0 +1,61 @@ +package kfake + +import ( + "net" + "strconv" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(10, 0, 4) } + +func (c *Cluster) handleFindCoordinator(kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.FindCoordinatorRequest) + resp := req.ResponseKind().(*kmsg.FindCoordinatorResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + var unknown bool + if req.CoordinatorType != 0 && req.CoordinatorType != 1 { + unknown = true + } + + if req.Version <= 3 { + req.CoordinatorKeys = append(req.CoordinatorKeys, req.CoordinatorKey) + defer func() { + resp.ErrorCode = resp.Coordinators[0].ErrorCode + resp.ErrorMessage = resp.Coordinators[0].ErrorMessage + resp.NodeID = resp.Coordinators[0].NodeID + resp.Host = resp.Coordinators[0].Host + resp.Port = resp.Coordinators[0].Port + }() + } + + addc := func(key string) *kmsg.FindCoordinatorResponseCoordinator { + sc := kmsg.NewFindCoordinatorResponseCoordinator() + sc.Key = key + resp.Coordinators = append(resp.Coordinators, sc) + return &resp.Coordinators[len(resp.Coordinators)-1] + } + + for _, key := range req.CoordinatorKeys { + sc := addc(key) + if unknown { + sc.ErrorCode = kerr.InvalidRequest.Code + continue + } + + b := c.coordinator(key) + host, port, _ := net.SplitHostPort(b.ln.Addr().String()) + iport, _ := strconv.Atoi(port) + + sc.NodeID = b.node + sc.Host = host + sc.Port = int32(iport) + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/11_join_group.go b/vendor/github.com/twmb/franz-go/pkg/kfake/11_join_group.go new file mode 100644 index 000000000000..70ed1d896e2c --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/11_join_group.go @@ -0,0 +1,18 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(11, 0, 9) } + +func (c *Cluster) handleJoinGroup(creq *clientReq) (kmsg.Response, error) { + req := creq.kreq.(*kmsg.JoinGroupRequest) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + c.groups.handleJoin(creq) + return nil, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/12_heartbeat.go b/vendor/github.com/twmb/franz-go/pkg/kfake/12_heartbeat.go new file mode 100644 index 000000000000..59f12712cc0a --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/12_heartbeat.go @@ -0,0 +1,23 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(12, 0, 4) } + +func (c *Cluster) handleHeartbeat(creq *clientReq) (kmsg.Response, error) { + req := creq.kreq.(*kmsg.HeartbeatRequest) + resp := req.ResponseKind().(*kmsg.HeartbeatResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + if c.groups.handleHeartbeat(creq) { + return nil, nil + } + resp.ErrorCode = kerr.GroupIDNotFound.Code + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/13_leave_group.go b/vendor/github.com/twmb/franz-go/pkg/kfake/13_leave_group.go new file mode 100644 index 000000000000..e941f19ede25 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/13_leave_group.go @@ -0,0 +1,23 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(13, 0, 5) } + +func (c *Cluster) handleLeaveGroup(creq *clientReq) (kmsg.Response, error) { + req := creq.kreq.(*kmsg.LeaveGroupRequest) + resp := req.ResponseKind().(*kmsg.LeaveGroupResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + if c.groups.handleLeave(creq) { + return nil, nil + } + resp.ErrorCode = kerr.GroupIDNotFound.Code + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/14_sync_group.go b/vendor/github.com/twmb/franz-go/pkg/kfake/14_sync_group.go new file mode 100644 index 000000000000..9944b11204ad --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/14_sync_group.go @@ -0,0 +1,23 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(14, 0, 5) } + +func (c *Cluster) handleSyncGroup(creq *clientReq) (kmsg.Response, error) { + req := creq.kreq.(*kmsg.SyncGroupRequest) + resp := req.ResponseKind().(*kmsg.SyncGroupResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + if c.groups.handleSync(creq) { + return nil, nil + } + resp.ErrorCode = kerr.GroupIDNotFound.Code + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/15_describe_groups.go b/vendor/github.com/twmb/franz-go/pkg/kfake/15_describe_groups.go new file mode 100644 index 000000000000..8791759becf7 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/15_describe_groups.go @@ -0,0 +1,17 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(15, 0, 5) } + +func (c *Cluster) handleDescribeGroups(creq *clientReq) (kmsg.Response, error) { + req := creq.kreq.(*kmsg.DescribeGroupsRequest) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + return c.groups.handleDescribe(creq), nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/16_list_groups.go b/vendor/github.com/twmb/franz-go/pkg/kfake/16_list_groups.go new file mode 100644 index 000000000000..6d0189c4b892 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/16_list_groups.go @@ -0,0 +1,17 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(16, 0, 4) } + +func (c *Cluster) handleListGroups(creq *clientReq) (kmsg.Response, error) { + req := creq.kreq.(*kmsg.ListGroupsRequest) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + return c.groups.handleList(creq), nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/17_sasl_handshake.go b/vendor/github.com/twmb/franz-go/pkg/kfake/17_sasl_handshake.go new file mode 100644 index 000000000000..8a80cbf1c824 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/17_sasl_handshake.go @@ -0,0 +1,35 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(17, 1, 1) } + +func (c *Cluster) handleSASLHandshake(creq *clientReq) (kmsg.Response, error) { + req := creq.kreq.(*kmsg.SASLHandshakeRequest) + resp := req.ResponseKind().(*kmsg.SASLHandshakeResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + if creq.cc.saslStage != saslStageBegin { + resp.ErrorCode = kerr.IllegalSaslState.Code + return resp, nil + } + + switch req.Mechanism { + case saslPlain: + creq.cc.saslStage = saslStageAuthPlain + case saslScram256: + creq.cc.saslStage = saslStageAuthScram0_256 + case saslScram512: + creq.cc.saslStage = saslStageAuthScram0_512 + default: + resp.ErrorCode = kerr.UnsupportedSaslMechanism.Code + resp.SupportedMechanisms = []string{saslPlain, saslScram256, saslScram512} + } + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/18_api_versions.go b/vendor/github.com/twmb/franz-go/pkg/kfake/18_api_versions.go new file mode 100644 index 000000000000..c4378da4b8d0 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/18_api_versions.go @@ -0,0 +1,81 @@ +package kfake + +import ( + "fmt" + "sort" + "sync" + + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(18, 0, 3) } + +func (c *Cluster) handleApiVersions(kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.ApiVersionsRequest) + resp := req.ResponseKind().(*kmsg.ApiVersionsResponse) + + if resp.Version > 3 { + resp.Version = 0 // downgrades to 0 if the version is unknown + } + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + // If we are handling ApiVersions, our package is initialized and we + // build our response once. + apiVersionsOnce.Do(func() { + for _, v := range apiVersionsKeys { + apiVersionsSorted = append(apiVersionsSorted, v) + } + sort.Slice(apiVersionsSorted, func(i, j int) bool { + return apiVersionsSorted[i].ApiKey < apiVersionsSorted[j].ApiKey + }) + }) + resp.ApiKeys = apiVersionsSorted + + return resp, nil +} + +// Called at the beginning of every request, this validates that the client +// is sending requests within version ranges we can handle. +func checkReqVersion(key, version int16) error { + v, exists := apiVersionsKeys[key] + if !exists { + return fmt.Errorf("unsupported request key %d", key) + } + if version < v.MinVersion { + return fmt.Errorf("%s version %d below min supported version %d", kmsg.NameForKey(key), version, v.MinVersion) + } + if version > v.MaxVersion { + return fmt.Errorf("%s version %d above max supported version %d", kmsg.NameForKey(key), version, v.MaxVersion) + } + return nil +} + +var ( + apiVersionsMu sync.Mutex + apiVersionsKeys = make(map[int16]kmsg.ApiVersionsResponseApiKey) + + apiVersionsOnce sync.Once + apiVersionsSorted []kmsg.ApiVersionsResponseApiKey +) + +// Every request we implement calls regKey in an init function, allowing us to +// fully correctly build our ApiVersions response. +func regKey(key, min, max int16) { + apiVersionsMu.Lock() + defer apiVersionsMu.Unlock() + + if key < 0 || min < 0 || max < 0 || max < min { + panic(fmt.Sprintf("invalid registration, key: %d, min: %d, max: %d", key, min, max)) + } + if _, exists := apiVersionsKeys[key]; exists { + panic(fmt.Sprintf("doubly registered key %d", key)) + } + apiVersionsKeys[key] = kmsg.ApiVersionsResponseApiKey{ + ApiKey: key, + MinVersion: min, + MaxVersion: max, + } +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/19_create_topics.go b/vendor/github.com/twmb/franz-go/pkg/kfake/19_create_topics.go new file mode 100644 index 000000000000..1f9d00e620bf --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/19_create_topics.go @@ -0,0 +1,85 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// TODO +// +// * Return InvalidTopicException when names collide + +func init() { regKey(19, 0, 7) } + +func (c *Cluster) handleCreateTopics(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.CreateTopicsRequest) + resp := req.ResponseKind().(*kmsg.CreateTopicsResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + donet := func(t string, errCode int16) *kmsg.CreateTopicsResponseTopic { + st := kmsg.NewCreateTopicsResponseTopic() + st.Topic = t + st.ErrorCode = errCode + resp.Topics = append(resp.Topics, st) + return &resp.Topics[len(resp.Topics)-1] + } + donets := func(errCode int16) { + for _, rt := range req.Topics { + donet(rt.Topic, errCode) + } + } + + if b != c.controller { + donets(kerr.NotController.Code) + return resp, nil + } + + uniq := make(map[string]struct{}) + for _, rt := range req.Topics { + if _, ok := uniq[rt.Topic]; ok { + donets(kerr.InvalidRequest.Code) + return resp, nil + } + uniq[rt.Topic] = struct{}{} + } + + for _, rt := range req.Topics { + if _, ok := c.data.tps.gett(rt.Topic); ok { + donet(rt.Topic, kerr.TopicAlreadyExists.Code) + continue + } + if len(rt.ReplicaAssignment) > 0 { + donet(rt.Topic, kerr.InvalidReplicaAssignment.Code) + continue + } + if int(rt.ReplicationFactor) > len(c.bs) { + donet(rt.Topic, kerr.InvalidReplicationFactor.Code) + continue + } + if rt.NumPartitions == 0 { + donet(rt.Topic, kerr.InvalidPartitions.Code) + continue + } + configs := make(map[string]*string) + for _, c := range rt.Configs { + configs[c.Name] = c.Value + } + c.data.mkt(rt.Topic, int(rt.NumPartitions), int(rt.ReplicationFactor), configs) + st := donet(rt.Topic, 0) + st.TopicID = c.data.t2id[rt.Topic] + st.NumPartitions = int32(len(c.data.tps[rt.Topic])) + st.ReplicationFactor = int16(c.data.treplicas[rt.Topic]) + for k, v := range configs { + c := kmsg.NewCreateTopicsResponseTopicConfig() + c.Name = k + c.Value = v + // Source? + st.Configs = append(st.Configs, c) + } + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/20_delete_topics.go b/vendor/github.com/twmb/franz-go/pkg/kfake/20_delete_topics.go new file mode 100644 index 000000000000..eab80235e6db --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/20_delete_topics.go @@ -0,0 +1,94 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(20, 0, 6) } + +func (c *Cluster) handleDeleteTopics(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.DeleteTopicsRequest) + resp := req.ResponseKind().(*kmsg.DeleteTopicsResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + donet := func(t *string, id uuid, errCode int16) *kmsg.DeleteTopicsResponseTopic { + st := kmsg.NewDeleteTopicsResponseTopic() + st.Topic = t + st.TopicID = id + st.ErrorCode = errCode + resp.Topics = append(resp.Topics, st) + return &resp.Topics[len(resp.Topics)-1] + } + donets := func(errCode int16) { + for _, rt := range req.Topics { + donet(rt.Topic, rt.TopicID, errCode) + } + } + + if req.Version <= 5 { + for _, topic := range req.TopicNames { + rt := kmsg.NewDeleteTopicsRequestTopic() + rt.Topic = kmsg.StringPtr(topic) + req.Topics = append(req.Topics, rt) + } + } + + if b != c.controller { + donets(kerr.NotController.Code) + return resp, nil + } + for _, rt := range req.Topics { + if rt.TopicID != noID && rt.Topic != nil { + donets(kerr.InvalidRequest.Code) + return resp, nil + } + } + + type toDelete struct { + topic string + id uuid + } + var toDeletes []toDelete + defer func() { + for _, td := range toDeletes { + delete(c.data.tps, td.topic) + delete(c.data.id2t, td.id) + delete(c.data.t2id, td.topic) + + } + }() + for _, rt := range req.Topics { + var topic string + var id uuid + if rt.Topic != nil { + topic = *rt.Topic + id = c.data.t2id[topic] + } else { + topic = c.data.id2t[rt.TopicID] + id = rt.TopicID + } + t, ok := c.data.tps.gett(topic) + if !ok { + if rt.Topic != nil { + donet(&topic, id, kerr.UnknownTopicOrPartition.Code) + } else { + donet(&topic, id, kerr.UnknownTopicID.Code) + } + continue + } + + donet(&topic, id, 0) + toDeletes = append(toDeletes, toDelete{topic, id}) + for _, pd := range t { + for watch := range pd.watch { + watch.deleted() + } + } + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/21_delete_records.go b/vendor/github.com/twmb/franz-go/pkg/kfake/21_delete_records.go new file mode 100644 index 000000000000..e175045e4cfe --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/21_delete_records.go @@ -0,0 +1,74 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// TODO +// +// * Return InvalidTopicException when names collide + +func init() { regKey(21, 0, 2) } + +func (c *Cluster) handleDeleteRecords(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.DeleteRecordsRequest) + resp := req.ResponseKind().(*kmsg.DeleteRecordsResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + tidx := make(map[string]int) + donet := func(t string, errCode int16) *kmsg.DeleteRecordsResponseTopic { + if i, ok := tidx[t]; ok { + return &resp.Topics[i] + } + tidx[t] = len(resp.Topics) + st := kmsg.NewDeleteRecordsResponseTopic() + st.Topic = t + resp.Topics = append(resp.Topics, st) + return &resp.Topics[len(resp.Topics)-1] + } + donep := func(t string, p int32, errCode int16) *kmsg.DeleteRecordsResponseTopicPartition { + sp := kmsg.NewDeleteRecordsResponseTopicPartition() + sp.Partition = p + sp.ErrorCode = errCode + st := donet(t, 0) + st.Partitions = append(st.Partitions, sp) + return &st.Partitions[len(st.Partitions)-1] + } + + for _, rt := range req.Topics { + ps, ok := c.data.tps.gett(rt.Topic) + for _, rp := range rt.Partitions { + if !ok { + donep(rt.Topic, rp.Partition, kerr.UnknownTopicOrPartition.Code) + continue + } + pd, ok := ps[rp.Partition] + if !ok { + donep(rt.Topic, rp.Partition, kerr.UnknownTopicOrPartition.Code) + continue + } + if pd.leader != b { + donep(rt.Topic, rp.Partition, kerr.NotLeaderForPartition.Code) + continue + } + to := rp.Offset + if to == -1 { + to = pd.highWatermark + } + if to < pd.logStartOffset || to > pd.highWatermark { + donep(rt.Topic, rp.Partition, kerr.OffsetOutOfRange.Code) + continue + } + pd.logStartOffset = to + pd.trimLeft() + sp := donep(rt.Topic, rp.Partition, 0) + sp.LowWatermark = to + } + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/22_init_producer_id.go b/vendor/github.com/twmb/franz-go/pkg/kfake/22_init_producer_id.go new file mode 100644 index 000000000000..a0485905a42e --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/22_init_producer_id.go @@ -0,0 +1,34 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// TODO +// +// * Transactional IDs +// * v3+ + +func init() { regKey(22, 0, 4) } + +func (c *Cluster) handleInitProducerID(kreq kmsg.Request) (kmsg.Response, error) { + var ( + req = kreq.(*kmsg.InitProducerIDRequest) + resp = req.ResponseKind().(*kmsg.InitProducerIDResponse) + ) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + if req.TransactionalID != nil { + resp.ErrorCode = kerr.UnknownServerError.Code + return resp, nil + } + + pid := c.pids.create(nil) + resp.ProducerID = pid.id + resp.ProducerEpoch = pid.epoch + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/23_offset_for_leader_epoch.go b/vendor/github.com/twmb/franz-go/pkg/kfake/23_offset_for_leader_epoch.go new file mode 100644 index 000000000000..f531ecf89cfa --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/23_offset_for_leader_epoch.go @@ -0,0 +1,121 @@ +package kfake + +import ( + "sort" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(23, 3, 4) } + +func (c *Cluster) handleOffsetForLeaderEpoch(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.OffsetForLeaderEpochRequest) + resp := req.ResponseKind().(*kmsg.OffsetForLeaderEpochResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + tidx := make(map[string]int) + donet := func(t string, errCode int16) *kmsg.OffsetForLeaderEpochResponseTopic { + if i, ok := tidx[t]; ok { + return &resp.Topics[i] + } + tidx[t] = len(resp.Topics) + st := kmsg.NewOffsetForLeaderEpochResponseTopic() + st.Topic = t + resp.Topics = append(resp.Topics, st) + return &resp.Topics[len(resp.Topics)-1] + } + donep := func(t string, p int32, errCode int16) *kmsg.OffsetForLeaderEpochResponseTopicPartition { + sp := kmsg.NewOffsetForLeaderEpochResponseTopicPartition() + sp.Partition = p + sp.ErrorCode = errCode + st := donet(t, 0) + st.Partitions = append(st.Partitions, sp) + return &st.Partitions[len(st.Partitions)-1] + } + + for _, rt := range req.Topics { + ps, ok := c.data.tps.gett(rt.Topic) + for _, rp := range rt.Partitions { + if req.ReplicaID != -1 { + donep(rt.Topic, rp.Partition, kerr.UnknownServerError.Code) + continue + } + if !ok { + donep(rt.Topic, rp.Partition, kerr.UnknownTopicOrPartition.Code) + continue + } + pd, ok := ps[rp.Partition] + if !ok { + donep(rt.Topic, rp.Partition, kerr.UnknownTopicOrPartition.Code) + continue + } + if pd.leader != b { + donep(rt.Topic, rp.Partition, kerr.NotLeaderForPartition.Code) + continue + } + if rp.CurrentLeaderEpoch < pd.epoch { + donep(rt.Topic, rp.Partition, kerr.FencedLeaderEpoch.Code) + continue + } else if rp.CurrentLeaderEpoch > pd.epoch { + donep(rt.Topic, rp.Partition, kerr.UnknownLeaderEpoch.Code) + continue + } + + sp := donep(rt.Topic, rp.Partition, 0) + + // If the user is requesting our current epoch, we return the HWM. + if rp.LeaderEpoch == pd.epoch { + sp.LeaderEpoch = pd.epoch + sp.EndOffset = pd.highWatermark + continue + } + + // If our epoch was bumped before anything was + // produced, return the epoch and a start offset of 0. + if len(pd.batches) == 0 { + sp.LeaderEpoch = pd.epoch + sp.EndOffset = 0 + if rp.LeaderEpoch > pd.epoch { + sp.LeaderEpoch = -1 + sp.EndOffset = -1 + } + continue + } + + // What is the largest epoch after the requested epoch? + nextEpoch := rp.LeaderEpoch + 1 + idx, _ := sort.Find(len(pd.batches), func(idx int) int { + batchEpoch := pd.batches[idx].epoch + switch { + case nextEpoch <= batchEpoch: + return -1 + default: + return 1 + } + }) + + // Requested epoch is not yet known: keep -1 returns. + if idx == len(pd.batches) { + sp.LeaderEpoch = -1 + sp.EndOffset = -1 + continue + } + + // Next epoch is actually the first epoch: return the + // requested epoch and the LSO. + if idx == 0 { + sp.LeaderEpoch = rp.LeaderEpoch + sp.EndOffset = pd.logStartOffset + continue + } + + sp.LeaderEpoch = pd.batches[idx-1].epoch + sp.EndOffset = pd.batches[idx].FirstOffset + } + } + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/32_describe_configs.go b/vendor/github.com/twmb/franz-go/pkg/kfake/32_describe_configs.go new file mode 100644 index 000000000000..02662cc5c858 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/32_describe_configs.go @@ -0,0 +1,108 @@ +package kfake + +import ( + "strconv" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(32, 0, 4) } + +func (c *Cluster) handleDescribeConfigs(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.DescribeConfigsRequest) + resp := req.ResponseKind().(*kmsg.DescribeConfigsResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + doner := func(n string, t kmsg.ConfigResourceType, errCode int16) *kmsg.DescribeConfigsResponseResource { + st := kmsg.NewDescribeConfigsResponseResource() + st.ResourceName = n + st.ResourceType = t + st.ErrorCode = errCode + resp.Resources = append(resp.Resources, st) + return &resp.Resources[len(resp.Resources)-1] + } + + rfn := func(r *kmsg.DescribeConfigsResponseResource) func(k string, v *string, src kmsg.ConfigSource, sensitive bool) { + nameIdxs := make(map[string]int) + return func(k string, v *string, src kmsg.ConfigSource, sensitive bool) { + rc := kmsg.NewDescribeConfigsResponseResourceConfig() + rc.Name = k + rc.Value = v + rc.Source = src + rc.ReadOnly = rc.Source == kmsg.ConfigSourceStaticBrokerConfig + rc.IsDefault = rc.Source == kmsg.ConfigSourceDefaultConfig || rc.Source == kmsg.ConfigSourceStaticBrokerConfig + rc.IsSensitive = sensitive + + // We walk configs from static to default to dynamic, + // if this config already exists previously, we move + // the previous config to a synonym and update the + // previous config. + if idx, ok := nameIdxs[k]; ok { + prior := r.Configs[idx] + syn := kmsg.NewDescribeConfigsResponseResourceConfigConfigSynonym() + syn.Name = prior.Name + syn.Value = prior.Value + syn.Source = prior.Source + rc.ConfigSynonyms = append([]kmsg.DescribeConfigsResponseResourceConfigConfigSynonym{syn}, prior.ConfigSynonyms...) + r.Configs[idx] = rc + return + } + nameIdxs[k] = len(r.Configs) + r.Configs = append(r.Configs, rc) + } + } + filter := func(rr *kmsg.DescribeConfigsRequestResource, r *kmsg.DescribeConfigsResponseResource) { + if rr.ConfigNames == nil { + return + } + names := make(map[string]struct{}) + for _, name := range rr.ConfigNames { + names[name] = struct{}{} + } + keep := r.Configs[:0] + for _, rc := range r.Configs { + if _, ok := names[rc.Name]; ok { + keep = append(keep, rc) + } + } + r.Configs = keep + } + +outer: + for i := range req.Resources { + rr := &req.Resources[i] + switch rr.ResourceType { + case kmsg.ConfigResourceTypeBroker: + id := int32(-1) + if rr.ResourceName != "" { + iid, err := strconv.Atoi(rr.ResourceName) + id = int32(iid) + if err != nil || id != b.node { + doner(rr.ResourceName, rr.ResourceType, kerr.InvalidRequest.Code) + continue outer + } + } + r := doner(rr.ResourceName, rr.ResourceType, 0) + c.brokerConfigs(id, rfn(r)) + filter(rr, r) + + case kmsg.ConfigResourceTypeTopic: + if _, ok := c.data.tps.gett(rr.ResourceName); !ok { + doner(rr.ResourceName, rr.ResourceType, kerr.UnknownTopicOrPartition.Code) + continue + } + r := doner(rr.ResourceName, rr.ResourceType, 0) + c.data.configs(rr.ResourceName, rfn(r)) + filter(rr, r) + + default: + doner(rr.ResourceName, rr.ResourceType, kerr.InvalidRequest.Code) + } + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/33_alter_configs.go b/vendor/github.com/twmb/franz-go/pkg/kfake/33_alter_configs.go new file mode 100644 index 000000000000..76e1e5fb576a --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/33_alter_configs.go @@ -0,0 +1,92 @@ +package kfake + +import ( + "strconv" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(33, 0, 2) } + +func (c *Cluster) handleAlterConfigs(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.AlterConfigsRequest) + resp := req.ResponseKind().(*kmsg.AlterConfigsResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + doner := func(n string, t kmsg.ConfigResourceType, errCode int16) *kmsg.AlterConfigsResponseResource { + st := kmsg.NewAlterConfigsResponseResource() + st.ResourceName = n + st.ResourceType = t + st.ErrorCode = errCode + resp.Resources = append(resp.Resources, st) + return &resp.Resources[len(resp.Resources)-1] + } + +outer: + for i := range req.Resources { + rr := &req.Resources[i] + switch rr.ResourceType { + case kmsg.ConfigResourceTypeBroker: + id := int32(-1) + if rr.ResourceName != "" { + iid, err := strconv.Atoi(rr.ResourceName) + id = int32(iid) + if err != nil || id != b.node { + doner(rr.ResourceName, rr.ResourceType, kerr.InvalidRequest.Code) + continue outer + } + } + var invalid bool + for i := range rr.Configs { + rc := &rr.Configs[i] + invalid = invalid || !c.setBrokerConfig(rc.Name, rc.Value, true) + } + if invalid { + doner(rr.ResourceName, rr.ResourceType, kerr.InvalidRequest.Code) + continue + } + doner(rr.ResourceName, rr.ResourceType, 0) + if req.ValidateOnly { + continue + } + c.bcfgs = make(map[string]*string) + for i := range rr.Configs { + rc := &rr.Configs[i] + c.setBrokerConfig(rc.Name, rc.Value, false) + } + + case kmsg.ConfigResourceTypeTopic: + if _, ok := c.data.tps.gett(rr.ResourceName); !ok { + doner(rr.ResourceName, rr.ResourceType, kerr.UnknownTopicOrPartition.Code) + continue + } + var invalid bool + for i := range rr.Configs { + rc := &rr.Configs[i] + invalid = invalid || !c.data.setTopicConfig(rr.ResourceName, rc.Name, rc.Value, true) + } + if invalid { + doner(rr.ResourceName, rr.ResourceType, kerr.InvalidRequest.Code) + continue + } + doner(rr.ResourceName, rr.ResourceType, 0) + if req.ValidateOnly { + continue + } + delete(c.data.tcfgs, rr.ResourceName) + for i := range rr.Configs { + rc := &rr.Configs[i] + c.data.setTopicConfig(rr.ResourceName, rc.Name, rc.Value, false) + } + + default: + doner(rr.ResourceName, rr.ResourceType, kerr.InvalidRequest.Code) + } + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/34_alter_replica_log_dirs.go b/vendor/github.com/twmb/franz-go/pkg/kfake/34_alter_replica_log_dirs.go new file mode 100644 index 000000000000..8031447c3270 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/34_alter_replica_log_dirs.go @@ -0,0 +1,53 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(34, 0, 2) } + +func (c *Cluster) handleAlterReplicaLogDirs(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.AlterReplicaLogDirsRequest) + resp := req.ResponseKind().(*kmsg.AlterReplicaLogDirsResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + tidx := make(map[string]int) + donet := func(t string, errCode int16) *kmsg.AlterReplicaLogDirsResponseTopic { + if i, ok := tidx[t]; ok { + return &resp.Topics[i] + } + tidx[t] = len(resp.Topics) + st := kmsg.NewAlterReplicaLogDirsResponseTopic() + st.Topic = t + resp.Topics = append(resp.Topics, st) + return &resp.Topics[len(resp.Topics)-1] + } + donep := func(t string, p int32, errCode int16) *kmsg.AlterReplicaLogDirsResponseTopicPartition { + sp := kmsg.NewAlterReplicaLogDirsResponseTopicPartition() + sp.Partition = p + sp.ErrorCode = errCode + st := donet(t, 0) + st.Partitions = append(st.Partitions, sp) + return &st.Partitions[len(st.Partitions)-1] + } + + for _, rd := range req.Dirs { + for _, t := range rd.Topics { + for _, p := range t.Partitions { + d, ok := c.data.tps.getp(t.Topic, p) + if !ok { + donep(t.Topic, p, kerr.UnknownTopicOrPartition.Code) + continue + } + d.dir = rd.Dir + donep(t.Topic, p, 0) + } + } + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/35_describe_log_dirs.go b/vendor/github.com/twmb/franz-go/pkg/kfake/35_describe_log_dirs.go new file mode 100644 index 000000000000..e1ea9b8a9f08 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/35_describe_log_dirs.go @@ -0,0 +1,70 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(35, 0, 4) } + +func (c *Cluster) handleDescribeLogDirs(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.DescribeLogDirsRequest) + resp := req.ResponseKind().(*kmsg.DescribeLogDirsResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + totalSpace := make(map[string]int64) + individual := make(map[string]map[string]map[int32]int64) + + add := func(d string, t string, p int32, s int64) { + totalSpace[d] += s + ts, ok := individual[d] + if !ok { + ts = make(map[string]map[int32]int64) + individual[d] = ts + } + ps, ok := ts[t] + if !ok { + ps = make(map[int32]int64) + ts[t] = ps + } + ps[p] += s + } + + if req.Topics == nil { + c.data.tps.each(func(t string, p int32, d *partData) { + add(d.dir, t, p, d.nbytes) + }) + } else { + for _, t := range req.Topics { + for _, p := range t.Partitions { + d, ok := c.data.tps.getp(t.Topic, p) + if ok { + add(d.dir, t.Topic, p, d.nbytes) + } + } + } + } + + for dir, ts := range individual { + rd := kmsg.NewDescribeLogDirsResponseDir() + rd.Dir = dir + rd.TotalBytes = totalSpace[dir] + rd.UsableBytes = 32 << 30 + for t, ps := range ts { + rt := kmsg.NewDescribeLogDirsResponseDirTopic() + rt.Topic = t + for p, s := range ps { + rp := kmsg.NewDescribeLogDirsResponseDirTopicPartition() + rp.Partition = p + rp.Size = s + rt.Partitions = append(rt.Partitions, rp) + } + rd.Topics = append(rd.Topics, rt) + } + resp.Dirs = append(resp.Dirs, rd) + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/36_sasl_authenticate.go b/vendor/github.com/twmb/franz-go/pkg/kfake/36_sasl_authenticate.go new file mode 100644 index 000000000000..b94d2f0118fe --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/36_sasl_authenticate.go @@ -0,0 +1,83 @@ +package kfake + +import ( + "errors" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(36, 0, 2) } + +func (c *Cluster) handleSASLAuthenticate(creq *clientReq) (kmsg.Response, error) { + req := creq.kreq.(*kmsg.SASLAuthenticateRequest) + resp := req.ResponseKind().(*kmsg.SASLAuthenticateResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + switch creq.cc.saslStage { + default: + resp.ErrorCode = kerr.IllegalSaslState.Code + return resp, nil + + case saslStageAuthPlain: + u, p, err := saslSplitPlain(req.SASLAuthBytes) + if err != nil { + return nil, err + } + if c.sasls.plain == nil { + return nil, errors.New("invalid sasl") + } + if p != c.sasls.plain[u] { + return nil, errors.New("invalid sasl") + } + creq.cc.saslStage = saslStageComplete + + case saslStageAuthScram0_256: + c0, err := scramParseClient0(req.SASLAuthBytes) + if err != nil { + return nil, err + } + if c.sasls.scram256 == nil { + return nil, errors.New("invalid sasl") + } + a, ok := c.sasls.scram256[c0.user] + if !ok { + return nil, errors.New("invalid sasl") + } + s0, serverFirst := scramServerFirst(c0, a) + resp.SASLAuthBytes = serverFirst + creq.cc.saslStage = saslStageAuthScram1 + creq.cc.s0 = &s0 + + case saslStageAuthScram0_512: + c0, err := scramParseClient0(req.SASLAuthBytes) + if err != nil { + return nil, err + } + if c.sasls.scram512 == nil { + return nil, errors.New("invalid sasl") + } + a, ok := c.sasls.scram512[c0.user] + if !ok { + return nil, errors.New("invalid sasl") + } + s0, serverFirst := scramServerFirst(c0, a) + resp.SASLAuthBytes = serverFirst + creq.cc.saslStage = saslStageAuthScram1 + creq.cc.s0 = &s0 + + case saslStageAuthScram1: + serverFinal, err := creq.cc.s0.serverFinal(req.SASLAuthBytes) + if err != nil { + return nil, err + } + resp.SASLAuthBytes = serverFinal + creq.cc.saslStage = saslStageComplete + creq.cc.s0 = nil + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/37_create_partitions.go b/vendor/github.com/twmb/franz-go/pkg/kfake/37_create_partitions.go new file mode 100644 index 000000000000..bd4954ef5872 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/37_create_partitions.go @@ -0,0 +1,66 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(37, 0, 3) } + +func (c *Cluster) handleCreatePartitions(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.CreatePartitionsRequest) + resp := req.ResponseKind().(*kmsg.CreatePartitionsResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + donet := func(t string, errCode int16) *kmsg.CreatePartitionsResponseTopic { + st := kmsg.NewCreatePartitionsResponseTopic() + st.Topic = t + st.ErrorCode = errCode + resp.Topics = append(resp.Topics, st) + return &resp.Topics[len(resp.Topics)-1] + } + donets := func(errCode int16) { + for _, rt := range req.Topics { + donet(rt.Topic, errCode) + } + } + + if b != c.controller { + donets(kerr.NotController.Code) + return resp, nil + } + + uniq := make(map[string]struct{}) + for _, rt := range req.Topics { + if _, ok := uniq[rt.Topic]; ok { + donets(kerr.InvalidRequest.Code) + return resp, nil + } + uniq[rt.Topic] = struct{}{} + } + + for _, rt := range req.Topics { + t, ok := c.data.tps.gett(rt.Topic) + if !ok { + donet(rt.Topic, kerr.UnknownTopicOrPartition.Code) + continue + } + if len(rt.Assignment) > 0 { + donet(rt.Topic, kerr.InvalidReplicaAssignment.Code) + continue + } + if rt.Count < int32(len(t)) { + donet(rt.Topic, kerr.InvalidPartitions.Code) + continue + } + for i := int32(len(t)); i < rt.Count; i++ { + c.data.tps.mkp(rt.Topic, i, c.newPartData) + } + donet(rt.Topic, 0) + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/42_delete_groups.go b/vendor/github.com/twmb/franz-go/pkg/kfake/42_delete_groups.go new file mode 100644 index 000000000000..682574157d6d --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/42_delete_groups.go @@ -0,0 +1,17 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(42, 0, 2) } + +func (c *Cluster) handleDeleteGroups(creq *clientReq) (kmsg.Response, error) { + req := creq.kreq.(*kmsg.DeleteGroupsRequest) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + return c.groups.handleDelete(creq), nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/44_incremental_alter_configs.go b/vendor/github.com/twmb/franz-go/pkg/kfake/44_incremental_alter_configs.go new file mode 100644 index 000000000000..a8096bcfc9ff --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/44_incremental_alter_configs.go @@ -0,0 +1,112 @@ +package kfake + +import ( + "strconv" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(44, 0, 1) } + +func (c *Cluster) handleIncrementalAlterConfigs(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + req := kreq.(*kmsg.IncrementalAlterConfigsRequest) + resp := req.ResponseKind().(*kmsg.IncrementalAlterConfigsResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + doner := func(n string, t kmsg.ConfigResourceType, errCode int16) *kmsg.IncrementalAlterConfigsResponseResource { + st := kmsg.NewIncrementalAlterConfigsResponseResource() + st.ResourceName = n + st.ResourceType = t + st.ErrorCode = errCode + resp.Resources = append(resp.Resources, st) + return &resp.Resources[len(resp.Resources)-1] + } + +outer: + for i := range req.Resources { + rr := &req.Resources[i] + switch rr.ResourceType { + case kmsg.ConfigResourceTypeBroker: + id := int32(-1) + if rr.ResourceName != "" { + iid, err := strconv.Atoi(rr.ResourceName) + id = int32(iid) + if err != nil || id != b.node { + doner(rr.ResourceName, rr.ResourceType, kerr.InvalidRequest.Code) + continue outer + } + } + var invalid bool + for i := range rr.Configs { + rc := &rr.Configs[i] + switch rc.Op { + case kmsg.IncrementalAlterConfigOpSet: + invalid = invalid || !c.setBrokerConfig(rr.Configs[i].Name, rr.Configs[i].Value, true) + case kmsg.IncrementalAlterConfigOpDelete: + default: + invalid = true + } + } + if invalid { + doner(rr.ResourceName, rr.ResourceType, kerr.InvalidRequest.Code) + continue + } + doner(rr.ResourceName, rr.ResourceType, 0) + if req.ValidateOnly { + continue + } + for i := range rr.Configs { + rc := &rr.Configs[i] + switch rc.Op { + case kmsg.IncrementalAlterConfigOpSet: + c.setBrokerConfig(rr.Configs[i].Name, rr.Configs[i].Value, false) + case kmsg.IncrementalAlterConfigOpDelete: + delete(c.bcfgs, rc.Name) + } + } + + case kmsg.ConfigResourceTypeTopic: + if _, ok := c.data.tps.gett(rr.ResourceName); !ok { + doner(rr.ResourceName, rr.ResourceType, kerr.UnknownTopicOrPartition.Code) + continue + } + var invalid bool + for i := range rr.Configs { + rc := &rr.Configs[i] + switch rc.Op { + case kmsg.IncrementalAlterConfigOpSet: + invalid = invalid || !c.data.setTopicConfig(rr.ResourceName, rc.Name, rc.Value, true) + case kmsg.IncrementalAlterConfigOpDelete: + default: + invalid = true + } + } + if invalid { + doner(rr.ResourceName, rr.ResourceType, kerr.InvalidRequest.Code) + continue + } + doner(rr.ResourceName, rr.ResourceType, 0) + if req.ValidateOnly { + continue + } + for i := range rr.Configs { + rc := &rr.Configs[i] + switch rc.Op { + case kmsg.IncrementalAlterConfigOpSet: + c.data.setTopicConfig(rr.ResourceName, rc.Name, rc.Value, false) + case kmsg.IncrementalAlterConfigOpDelete: + delete(c.data.tcfgs[rr.ResourceName], rc.Name) + } + } + + default: + doner(rr.ResourceName, rr.ResourceType, kerr.InvalidRequest.Code) + } + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/47_offset_delete.go b/vendor/github.com/twmb/franz-go/pkg/kfake/47_offset_delete.go new file mode 100644 index 000000000000..878e83b45a0e --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/47_offset_delete.go @@ -0,0 +1,23 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(47, 0, 0) } + +func (c *Cluster) handleOffsetDelete(creq *clientReq) (kmsg.Response, error) { + req := creq.kreq.(*kmsg.OffsetDeleteRequest) + resp := req.ResponseKind().(*kmsg.OffsetDeleteResponse) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + if c.groups.handleOffsetDelete(creq) { + return nil, nil + } + resp.ErrorCode = kerr.GroupIDNotFound.Code + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/50_describe_user_scram_credentials.go b/vendor/github.com/twmb/franz-go/pkg/kfake/50_describe_user_scram_credentials.go new file mode 100644 index 000000000000..0cb107d21037 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/50_describe_user_scram_credentials.go @@ -0,0 +1,68 @@ +package kfake + +import ( + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(50, 0, 0) } + +func (c *Cluster) handleDescribeUserSCRAMCredentials(kreq kmsg.Request) (kmsg.Response, error) { + var ( + req = kreq.(*kmsg.DescribeUserSCRAMCredentialsRequest) + resp = req.ResponseKind().(*kmsg.DescribeUserSCRAMCredentialsResponse) + ) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + describe := make(map[string]bool) // if false, user was duplicated + for _, u := range req.Users { + if _, ok := describe[u.Name]; ok { + describe[u.Name] = true + } else { + describe[u.Name] = false + } + } + if req.Users == nil { // null returns all + for u := range c.sasls.scram256 { + describe[u] = false + } + for u := range c.sasls.scram512 { + describe[u] = false + } + } + + addr := func(u string) *kmsg.DescribeUserSCRAMCredentialsResponseResult { + sr := kmsg.NewDescribeUserSCRAMCredentialsResponseResult() + sr.User = u + resp.Results = append(resp.Results, sr) + return &resp.Results[len(resp.Results)-1] + } + + for u, duplicated := range describe { + sr := addr(u) + if duplicated { + sr.ErrorCode = kerr.DuplicateResource.Code + continue + } + if a, ok := c.sasls.scram256[u]; ok { + ci := kmsg.NewDescribeUserSCRAMCredentialsResponseResultCredentialInfo() + ci.Mechanism = 1 + ci.Iterations = int32(a.iterations) + sr.CredentialInfos = append(sr.CredentialInfos, ci) + } + if a, ok := c.sasls.scram512[u]; ok { + ci := kmsg.NewDescribeUserSCRAMCredentialsResponseResultCredentialInfo() + ci.Mechanism = 2 + ci.Iterations = int32(a.iterations) + sr.CredentialInfos = append(sr.CredentialInfos, ci) + } + if len(sr.CredentialInfos) == 0 { + sr.ErrorCode = kerr.ResourceNotFound.Code + } + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/51_alter_user_scram_credentials.go b/vendor/github.com/twmb/franz-go/pkg/kfake/51_alter_user_scram_credentials.go new file mode 100644 index 000000000000..7f853b5618f7 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/51_alter_user_scram_credentials.go @@ -0,0 +1,124 @@ +package kfake + +import ( + "bytes" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +func init() { regKey(51, 0, 0) } + +func (c *Cluster) handleAlterUserSCRAMCredentials(b *broker, kreq kmsg.Request) (kmsg.Response, error) { + var ( + req = kreq.(*kmsg.AlterUserSCRAMCredentialsRequest) + resp = req.ResponseKind().(*kmsg.AlterUserSCRAMCredentialsResponse) + ) + + if err := checkReqVersion(req.Key(), req.Version); err != nil { + return nil, err + } + + addr := func(u string) *kmsg.AlterUserSCRAMCredentialsResponseResult { + sr := kmsg.NewAlterUserSCRAMCredentialsResponseResult() + sr.User = u + resp.Results = append(resp.Results, sr) + return &resp.Results[len(resp.Results)-1] + } + doneu := func(u string, code int16) *kmsg.AlterUserSCRAMCredentialsResponseResult { + sr := addr(u) + sr.ErrorCode = code + return sr + } + + users := make(map[string]int16) + + // Validate everything up front, keeping track of all (and duplicate) + // users. If we are not controller, we fail with our users map. + for _, d := range req.Deletions { + if d.Name == "" { + users[d.Name] = kerr.UnacceptableCredential.Code + continue + } + if d.Mechanism != 1 && d.Mechanism != 2 { + users[d.Name] = kerr.UnsupportedSaslMechanism.Code + continue + } + users[d.Name] = 0 + } + for _, u := range req.Upsertions { + if u.Name == "" || u.Iterations < 4096 || u.Iterations > 16384 { // Kafka min/max + users[u.Name] = kerr.UnacceptableCredential.Code + continue + } + if u.Mechanism != 1 && u.Mechanism != 2 { + users[u.Name] = kerr.UnsupportedSaslMechanism.Code + continue + } + if code, deleting := users[u.Name]; deleting && code == 0 { + users[u.Name] = kerr.DuplicateResource.Code + continue + } + users[u.Name] = 0 + } + + if b != c.controller { + for u := range users { + doneu(u, kerr.NotController.Code) + } + return resp, nil + } + + // Add anything that failed validation. + for u, code := range users { + if code != 0 { + doneu(u, code) + } + } + + // Process all deletions, adding ResourceNotFound as necessary. + for _, d := range req.Deletions { + if users[d.Name] != 0 { + continue + } + m := c.sasls.scram256 + if d.Mechanism == 2 { + m = c.sasls.scram512 + } + if m == nil { + doneu(d.Name, kerr.ResourceNotFound.Code) + continue + } + if _, ok := m[d.Name]; !ok { + doneu(d.Name, kerr.ResourceNotFound.Code) + continue + } + delete(m, d.Name) + doneu(d.Name, 0) + } + + // Process all upsertions. + for _, u := range req.Upsertions { + if users[u.Name] != 0 { + continue + } + m := &c.sasls.scram256 + mech := saslScram256 + if u.Mechanism == 2 { + m = &c.sasls.scram512 + mech = saslScram512 + } + if *m == nil { + *m = make(map[string]scramAuth) + } + (*m)[u.Name] = scramAuth{ + mechanism: mech, + iterations: int(u.Iterations), + saltedPass: bytes.Clone(u.SaltedPassword), + salt: bytes.Clone(u.Salt), + } + doneu(u.Name, 0) + } + + return resp, nil +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/LICENSE b/vendor/github.com/twmb/franz-go/pkg/kfake/LICENSE new file mode 100644 index 000000000000..36e18034325d --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/LICENSE @@ -0,0 +1,24 @@ +Copyright 2020, Travis Bischel. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of the library nor the + names of its contributors may be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY +DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/NOTES b/vendor/github.com/twmb/franz-go/pkg/kfake/NOTES new file mode 100644 index 000000000000..fd7583da67ec --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/NOTES @@ -0,0 +1,62 @@ +ORDER + +BASIC +x Produce +x Metadata +x CreateTopics +x InitProducerID +x ListOffsets +x Fetch +x DeleteTopics +x CreatePartitions + +GROUPS +x OffsetCommit +x OffsetFetch +x FindCoordinator +x JoinGroup +x Heartbeat +x LeaveGroup +x SyncGroup +x DescribeGroups +x ListGroups +x DeleteGroups + +MISC +x OffsetForLeaderEpoch + +SASL +x SaslHandshake +x SaslAuthenticate +x DescribeUserScramCredentials +x AlterUserScramCredentials + +LOW-PRIO +x DeleteRecords +x DescribeConfigs +x AlterConfigs +x IncrementalAlterConfigs +x OffsetDelete +x AlterReplicaLogDirs +x DescribeLogDirs + +TXNS +* AddPartitionsToTxn +* AddOffsetsToTxn +* EndTxn +* TxnOffsetCommit + +ACLS +* DescribeACLs +* CreateACLs +* DeleteACLs + +LOWER-PRIO +* DescribeProducers +* DescribeTransactions +* ListTransactions +* AlterPartitionAssignments +* ListPartitionReassignments +* DescribeClientQuotas +* AlterClientQuotas +DTOKEN: ignore diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/client_conn.go b/vendor/github.com/twmb/franz-go/pkg/kfake/client_conn.go new file mode 100644 index 000000000000..a78b574f3f6f --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/client_conn.go @@ -0,0 +1,182 @@ +package kfake + +import ( + "encoding/binary" + "io" + "net" + "time" + + "github.com/twmb/franz-go/pkg/kbin" + "github.com/twmb/franz-go/pkg/kmsg" +) + +type ( + clientConn struct { + c *Cluster + b *broker + conn net.Conn + respCh chan clientResp + + saslStage saslStage + s0 *scramServer0 + } + + clientReq struct { + cc *clientConn + kreq kmsg.Request + at time.Time + cid string + corr int32 + seq uint32 + } + + clientResp struct { + kresp kmsg.Response + corr int32 + err error + seq uint32 + } +) + +func (creq *clientReq) empty() bool { return creq == nil || creq.cc == nil || creq.kreq == nil } + +func (cc *clientConn) read() { + defer cc.conn.Close() + + type read struct { + body []byte + err error + } + var ( + who = cc.conn.RemoteAddr() + size = make([]byte, 4) + readCh = make(chan read, 1) + seq uint32 + ) + for { + go func() { + if _, err := io.ReadFull(cc.conn, size); err != nil { + readCh <- read{err: err} + return + } + body := make([]byte, binary.BigEndian.Uint32(size)) + _, err := io.ReadFull(cc.conn, body) + readCh <- read{body: body, err: err} + }() + + var read read + select { + case <-cc.c.die: + return + case read = <-readCh: + } + + if err := read.err; err != nil { + cc.c.cfg.logger.Logf(LogLevelDebug, "client %s disconnected from read: %v", who, err) + return + } + + var ( + body = read.body + reader = kbin.Reader{Src: body} + key = reader.Int16() + version = reader.Int16() + corr = reader.Int32() + clientID = reader.NullableString() + kreq = kmsg.RequestForKey(key) + ) + kreq.SetVersion(version) + if kreq.IsFlexible() { + kmsg.SkipTags(&reader) + } + if err := kreq.ReadFrom(reader.Src); err != nil { + cc.c.cfg.logger.Logf(LogLevelDebug, "client %s unable to parse request: %v", who, err) + return + } + + // Within Kafka, a null client ID is treated as an empty string. + var cid string + if clientID != nil { + cid = *clientID + } + + select { + case cc.c.reqCh <- &clientReq{cc, kreq, time.Now(), cid, corr, seq}: + seq++ + case <-cc.c.die: + return + } + } +} + +func (cc *clientConn) write() { + defer cc.conn.Close() + + var ( + who = cc.conn.RemoteAddr() + writeCh = make(chan error, 1) + buf []byte + seq uint32 + + // If a request is by necessity slow (join&sync), and the + // client sends another request down the same conn, we can + // actually handle them out of order because group state is + // managed independently in its own loop. To ensure + // serialization, we capture out of order responses and only + // send them once the prior requests are replied to. + // + // (this is also why there is a seq in the clientReq) + oooresp = make(map[uint32]clientResp) + ) + for { + resp, ok := oooresp[seq] + if !ok { + select { + case resp = <-cc.respCh: + if resp.seq != seq { + oooresp[resp.seq] = resp + continue + } + seq = resp.seq + 1 + case <-cc.c.die: + return + } + } else { + delete(oooresp, seq) + seq++ + } + if err := resp.err; err != nil { + cc.c.cfg.logger.Logf(LogLevelInfo, "client %s request unable to be handled: %v", who, err) + return + } + + // Size, corr, and empty tag section if flexible: 9 bytes max. + buf = append(buf[:0], 0, 0, 0, 0, 0, 0, 0, 0, 0) + buf = resp.kresp.AppendTo(buf) + + start := 0 + l := len(buf) - 4 + if !resp.kresp.IsFlexible() || resp.kresp.Key() == 18 { + l-- + start++ + } + binary.BigEndian.PutUint32(buf[start:], uint32(l)) + binary.BigEndian.PutUint32(buf[start+4:], uint32(resp.corr)) + + go func() { + _, err := cc.conn.Write(buf[start:]) + writeCh <- err + }() + + var err error + select { + case <-cc.c.die: + return + case err = <-writeCh: + } + if err != nil { + cc.c.cfg.logger.Logf(LogLevelDebug, "client %s disconnected from write: %v", who, err) + return + } + } +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/cluster.go b/vendor/github.com/twmb/franz-go/pkg/kfake/cluster.go new file mode 100644 index 000000000000..275b14402add --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/cluster.go @@ -0,0 +1,1084 @@ +package kfake + +import ( + "crypto/tls" + "errors" + "fmt" + "math/rand" + "net" + "strconv" + "sync" + "sync/atomic" + "time" + + "github.com/twmb/franz-go/pkg/kmsg" +) + +// TODO +// +// * Add raft and make the brokers independent +// +// * Support multiple replicas -- we just pass this through + +type ( + + // Cluster is a mock Kafka broker cluster. + Cluster struct { + cfg cfg + + controller *broker + bs []*broker + + coordinatorGen atomic.Uint64 + + adminCh chan func() + reqCh chan *clientReq + wakeCh chan *slept + watchFetchCh chan *watchFetch + + controlMu sync.Mutex + control map[int16]map[*controlCtx]struct{} + currentBroker *broker + currentControl *controlCtx + sleeping map[*clientConn]*bsleep + controlSleep chan sleepChs + + data data + pids pids + groups groups + sasls sasls + bcfgs map[string]*string + + die chan struct{} + dead atomic.Bool + } + + broker struct { + c *Cluster + ln net.Listener + node int32 + bsIdx int + } + + controlFn func(kmsg.Request) (kmsg.Response, error, bool) + + controlCtx struct { + key int16 + fn controlFn + keep bool + drop bool + lastReq map[*clientConn]*clientReq // used to not re-run requests that slept, see doc comments below + } + + controlResp struct { + kresp kmsg.Response + err error + handled bool + } +) + +// MustCluster is like NewCluster, but panics on error. +func MustCluster(opts ...Opt) *Cluster { + c, err := NewCluster(opts...) + if err != nil { + panic(err) + } + return c +} + +// NewCluster returns a new mocked Kafka cluster. +func NewCluster(opts ...Opt) (*Cluster, error) { + cfg := cfg{ + nbrokers: 3, + logger: new(nopLogger), + clusterID: "kfake", + defaultNumParts: 10, + + minSessionTimeout: 6 * time.Second, + maxSessionTimeout: 5 * time.Minute, + + sasls: make(map[struct{ m, u string }]string), + } + for _, opt := range opts { + opt.apply(&cfg) + } + if len(cfg.ports) > 0 { + cfg.nbrokers = len(cfg.ports) + } + + c := &Cluster{ + cfg: cfg, + + adminCh: make(chan func()), + reqCh: make(chan *clientReq, 20), + wakeCh: make(chan *slept, 10), + watchFetchCh: make(chan *watchFetch, 20), + control: make(map[int16]map[*controlCtx]struct{}), + controlSleep: make(chan sleepChs, 1), + + sleeping: make(map[*clientConn]*bsleep), + + data: data{ + id2t: make(map[uuid]string), + t2id: make(map[string]uuid), + treplicas: make(map[string]int), + tcfgs: make(map[string]map[string]*string), + }, + bcfgs: make(map[string]*string), + + die: make(chan struct{}), + } + c.data.c = c + c.groups.c = c + var err error + defer func() { + if err != nil { + c.Close() + } + }() + + for mu, p := range cfg.sasls { + switch mu.m { + case saslPlain: + if c.sasls.plain == nil { + c.sasls.plain = make(map[string]string) + } + c.sasls.plain[mu.u] = p + case saslScram256: + if c.sasls.scram256 == nil { + c.sasls.scram256 = make(map[string]scramAuth) + } + c.sasls.scram256[mu.u] = newScramAuth(saslScram256, p) + case saslScram512: + if c.sasls.scram512 == nil { + c.sasls.scram512 = make(map[string]scramAuth) + } + c.sasls.scram512[mu.u] = newScramAuth(saslScram512, p) + default: + return nil, fmt.Errorf("unknown SASL mechanism %v", mu.m) + } + } + cfg.sasls = nil + + if cfg.enableSASL && c.sasls.empty() { + c.sasls.scram256 = map[string]scramAuth{ + "admin": newScramAuth(saslScram256, "admin"), + } + } + + for i := 0; i < cfg.nbrokers; i++ { + var port int + if len(cfg.ports) > 0 { + port = cfg.ports[i] + } + var ln net.Listener + ln, err = newListener(port, c.cfg.tls) + if err != nil { + return nil, err + } + b := &broker{ + c: c, + ln: ln, + node: int32(i), + bsIdx: len(c.bs), + } + c.bs = append(c.bs, b) + go b.listen() + } + c.controller = c.bs[len(c.bs)-1] + go c.run() + + seedTopics := make(map[string]int32) + for _, sts := range cfg.seedTopics { + p := sts.p + if p < 1 { + p = int32(cfg.defaultNumParts) + } + for _, t := range sts.ts { + seedTopics[t] = p + } + } + for t, p := range seedTopics { + c.data.mkt(t, int(p), -1, nil) + } + return c, nil +} + +// ListenAddrs returns the hostports that the cluster is listening on. +func (c *Cluster) ListenAddrs() []string { + var addrs []string + c.admin(func() { + for _, b := range c.bs { + addrs = append(addrs, b.ln.Addr().String()) + } + }) + return addrs +} + +// Close shuts down the cluster. +func (c *Cluster) Close() { + if c.dead.Swap(true) { + return + } + close(c.die) + for _, b := range c.bs { + b.ln.Close() + } +} + +func newListener(port int, tc *tls.Config) (net.Listener, error) { + l, err := net.Listen("tcp", fmt.Sprintf("127.0.0.1:%d", port)) + if err != nil { + return nil, err + } + if tc != nil { + l = tls.NewListener(l, tc) + } + return l, nil +} + +func (b *broker) listen() { + defer b.ln.Close() + for { + conn, err := b.ln.Accept() + if err != nil { + return + } + + cc := &clientConn{ + c: b.c, + b: b, + conn: conn, + respCh: make(chan clientResp, 2), + } + go cc.read() + go cc.write() + } +} + +func (c *Cluster) run() { +outer: + for { + var ( + creq *clientReq + w *watchFetch + s *slept + kreq kmsg.Request + kresp kmsg.Response + err error + handled bool + ) + + select { + case <-c.die: + return + + case admin := <-c.adminCh: + admin() + continue + + case creq = <-c.reqCh: + if c.cfg.sleepOutOfOrder { + break + } + // If we have any sleeping request on this node, + // we enqueue the new live request to the end and + // wait for the sleeping request to finish. + bs := c.sleeping[creq.cc] + if bs.enqueue(&slept{ + creq: creq, + waiting: true, + }) { + continue + } + + case s = <-c.wakeCh: + // On wakeup, we know we are handling a control + // function that was slept, or a request that was + // waiting for a control function to finish sleeping. + creq = s.creq + if s.waiting { + break + } + + // We continue a previously sleeping request, and + // handle results similar to tryControl. + // + // Control flow is weird here, but is described more + // fully in the finish/resleep/etc methods. + c.continueSleptControl(s) + inner: + for { + select { + case admin := <-c.adminCh: + admin() + continue inner + case res := <-s.res: + c.finishSleptControl(s) + cctx := s.cctx + s = nil + kresp, err, handled = res.kresp, res.err, res.handled + c.maybePopControl(handled, cctx) + if handled { + goto afterControl + } + break inner + case sleepChs := <-c.controlSleep: + c.resleepSleptControl(s, sleepChs) + continue outer + } + } + + case w = <-c.watchFetchCh: + if w.cleaned { + continue // already cleaned up, this is an extraneous timer fire + } + w.cleanup(c) + creq = w.creq + } + + kresp, err, handled = c.tryControl(creq) + if handled { + goto afterControl + } + + if c.cfg.enableSASL { + if allow := c.handleSASL(creq); !allow { + err = errors.New("not allowed given SASL state") + goto afterControl + } + } + + kreq = creq.kreq + switch k := kmsg.Key(kreq.Key()); k { + case kmsg.Produce: + kresp, err = c.handleProduce(creq.cc.b, kreq) + case kmsg.Fetch: + kresp, err = c.handleFetch(creq, w) + case kmsg.ListOffsets: + kresp, err = c.handleListOffsets(creq.cc.b, kreq) + case kmsg.Metadata: + kresp, err = c.handleMetadata(kreq) + case kmsg.OffsetCommit: + kresp, err = c.handleOffsetCommit(creq) + case kmsg.OffsetFetch: + kresp, err = c.handleOffsetFetch(creq) + case kmsg.FindCoordinator: + kresp, err = c.handleFindCoordinator(kreq) + case kmsg.JoinGroup: + kresp, err = c.handleJoinGroup(creq) + case kmsg.Heartbeat: + kresp, err = c.handleHeartbeat(creq) + case kmsg.LeaveGroup: + kresp, err = c.handleLeaveGroup(creq) + case kmsg.SyncGroup: + kresp, err = c.handleSyncGroup(creq) + case kmsg.DescribeGroups: + kresp, err = c.handleDescribeGroups(creq) + case kmsg.ListGroups: + kresp, err = c.handleListGroups(creq) + case kmsg.SASLHandshake: + kresp, err = c.handleSASLHandshake(creq) + case kmsg.ApiVersions: + kresp, err = c.handleApiVersions(kreq) + case kmsg.CreateTopics: + kresp, err = c.handleCreateTopics(creq.cc.b, kreq) + case kmsg.DeleteTopics: + kresp, err = c.handleDeleteTopics(creq.cc.b, kreq) + case kmsg.DeleteRecords: + kresp, err = c.handleDeleteRecords(creq.cc.b, kreq) + case kmsg.InitProducerID: + kresp, err = c.handleInitProducerID(kreq) + case kmsg.OffsetForLeaderEpoch: + kresp, err = c.handleOffsetForLeaderEpoch(creq.cc.b, kreq) + case kmsg.DescribeConfigs: + kresp, err = c.handleDescribeConfigs(creq.cc.b, kreq) + case kmsg.AlterConfigs: + kresp, err = c.handleAlterConfigs(creq.cc.b, kreq) + case kmsg.AlterReplicaLogDirs: + kresp, err = c.handleAlterReplicaLogDirs(creq.cc.b, kreq) + case kmsg.DescribeLogDirs: + kresp, err = c.handleDescribeLogDirs(creq.cc.b, kreq) + case kmsg.SASLAuthenticate: + kresp, err = c.handleSASLAuthenticate(creq) + case kmsg.CreatePartitions: + kresp, err = c.handleCreatePartitions(creq.cc.b, kreq) + case kmsg.DeleteGroups: + kresp, err = c.handleDeleteGroups(creq) + case kmsg.IncrementalAlterConfigs: + kresp, err = c.handleIncrementalAlterConfigs(creq.cc.b, kreq) + case kmsg.OffsetDelete: + kresp, err = c.handleOffsetDelete(creq) + case kmsg.DescribeUserSCRAMCredentials: + kresp, err = c.handleDescribeUserSCRAMCredentials(kreq) + case kmsg.AlterUserSCRAMCredentials: + kresp, err = c.handleAlterUserSCRAMCredentials(creq.cc.b, kreq) + default: + err = fmt.Errorf("unhandled key %v", k) + } + + afterControl: + // If s is non-nil, this is either a previously slept control + // that finished but was not handled, or a previously slept + // waiting request. In either case, we need to signal to the + // sleep dequeue loop to continue. + if s != nil { + s.continueDequeue <- struct{}{} + } + if kresp == nil && err == nil { // produce request with no acks, or otherwise hijacked request (group, sleep) + continue + } + + select { + case creq.cc.respCh <- clientResp{kresp: kresp, corr: creq.corr, err: err, seq: creq.seq}: + case <-c.die: + return + } + } +} + +// Control is a function to call on any client request the cluster handles. +// +// If the control function returns true, then either the response is written +// back to the client or, if there the control function returns an error, the +// client connection is closed. If both returns are nil, then the cluster will +// loop continuing to read from the client and the client will likely have a +// read timeout at some point. +// +// Controlling a request drops the control function from the cluster, meaning +// that a control function can only control *one* request. To keep the control +// function handling more requests, you can call KeepControl within your +// control function. Alternatively, if you want to just run some logic in your +// control function but then have the cluster handle the request as normal, +// you can call DropControl to drop a control function that was not handled. +// +// It is safe to add new control functions within a control function. +// +// Control functions are run serially unless you use SleepControl, multiple +// control functions are "in progress", and you run Cluster.Close. Closing a +// Cluster awakens all sleeping control functions. +func (c *Cluster) Control(fn func(kmsg.Request) (kmsg.Response, error, bool)) { + c.ControlKey(-1, fn) +} + +// Control is a function to call on a specific request key that the cluster +// handles. +// +// If the control function returns true, then either the response is written +// back to the client or, if there the control function returns an error, the +// client connection is closed. If both returns are nil, then the cluster will +// loop continuing to read from the client and the client will likely have a +// read timeout at some point. +// +// Controlling a request drops the control function from the cluster, meaning +// that a control function can only control *one* request. To keep the control +// function handling more requests, you can call KeepControl within your +// control function. Alternatively, if you want to just run some logic in your +// control function but then have the cluster handle the request as normal, +// you can call DropControl to drop a control function that was not handled. +// +// It is safe to add new control functions within a control function. +// +// Control functions are run serially unless you use SleepControl, multiple +// control functions are "in progress", and you run Cluster.Close. Closing a +// Cluster awakens all sleeping control functions. +func (c *Cluster) ControlKey(key int16, fn func(kmsg.Request) (kmsg.Response, error, bool)) { + c.controlMu.Lock() + defer c.controlMu.Unlock() + m := c.control[key] + if m == nil { + m = make(map[*controlCtx]struct{}) + c.control[key] = m + } + m[&controlCtx{ + key: key, + fn: fn, + lastReq: make(map[*clientConn]*clientReq), + }] = struct{}{} +} + +// KeepControl marks the currently running control function to be kept even if +// you handle the request and return true. This can be used to continuously +// control requests without needing to re-add control functions manually. +func (c *Cluster) KeepControl() { + c.controlMu.Lock() + defer c.controlMu.Unlock() + if c.currentControl != nil { + c.currentControl.keep = true + } +} + +// DropControl allows you to drop the current control function. This takes +// precedence over KeepControl. The use of this function is you can run custom +// control logic *once*, drop the control function, and return that the +// function was not handled -- thus allowing other control functions to run, or +// allowing the kfake cluster to process the request as normal. +func (c *Cluster) DropControl() { + c.controlMu.Lock() + defer c.controlMu.Unlock() + if c.currentControl != nil { + c.currentControl.drop = true + } +} + +// SleepControl sleeps the current control function until wakeup returns. This +// yields to run any other connection. +// +// Note that per protocol, requests on the same connection must be replied to +// in order. Many clients write multiple requests to the same connection, so +// if you sleep until a different request runs, you may sleep forever -- you +// must know the semantics of your client to know whether requests run on +// different connections (or, ensure you are writing to different brokers). +// +// For example, franz-go uses a dedicated connection for: +// - produce requests +// - fetch requests +// - join&sync requests +// - requests with a Timeout field +// - all other request +// +// So, for franz-go, there are up to five separate connections depending +// on what you are doing. +// +// You can run SleepControl multiple times in the same control function. If you +// sleep a request you are controlling, and another request of the same key +// comes in, it will run the same control function and may also sleep (i.e., +// you must have logic if you want to avoid sleeping on the same request). +func (c *Cluster) SleepControl(wakeup func()) { + c.controlMu.Lock() + if c.currentControl == nil { + c.controlMu.Unlock() + return + } + c.controlMu.Unlock() + + sleepChs := sleepChs{ + clientWait: make(chan struct{}, 1), + clientCont: make(chan struct{}, 1), + } + go func() { + wakeup() + sleepChs.clientWait <- struct{}{} + }() + + c.controlSleep <- sleepChs + select { + case <-sleepChs.clientCont: + case <-c.die: + } +} + +// CurrentNode is solely valid from within a control function; it returns +// the broker id that the request was received by. +// If there's no request currently inflight, this returns -1. +func (c *Cluster) CurrentNode() int32 { + c.controlMu.Lock() + defer c.controlMu.Unlock() + if b := c.currentBroker; b != nil { + return b.node + } + return -1 +} + +func (c *Cluster) tryControl(creq *clientReq) (kresp kmsg.Response, err error, handled bool) { + c.controlMu.Lock() + defer c.controlMu.Unlock() + if len(c.control) == 0 { + return nil, nil, false + } + kresp, err, handled = c.tryControlKey(creq.kreq.Key(), creq) + if !handled { + kresp, err, handled = c.tryControlKey(-1, creq) + } + return kresp, err, handled +} + +func (c *Cluster) tryControlKey(key int16, creq *clientReq) (kmsg.Response, error, bool) { + for cctx := range c.control[key] { + if cctx.lastReq[creq.cc] == creq { + continue + } + cctx.lastReq[creq.cc] = creq + res := c.runControl(cctx, creq) + for { + select { + case admin := <-c.adminCh: + admin() + continue + case res := <-res: + c.maybePopControl(res.handled, cctx) + return res.kresp, res.err, res.handled + case sleepChs := <-c.controlSleep: + c.beginSleptControl(&slept{ + cctx: cctx, + sleepChs: sleepChs, + res: res, + creq: creq, + }) + return nil, nil, true + } + } + } + return nil, nil, false +} + +func (c *Cluster) runControl(cctx *controlCtx, creq *clientReq) chan controlResp { + res := make(chan controlResp, 1) + c.currentBroker = creq.cc.b + c.currentControl = cctx + // We unlock before entering a control function so that the control + // function can modify / add more control. We re-lock when exiting the + // control function. This does pose some weird control flow issues + // w.r.t. sleeping requests. Here, we have to re-lock before sending + // down res, otherwise we risk unlocking an unlocked mu in + // finishSleepControl. + c.controlMu.Unlock() + go func() { + kresp, err, handled := cctx.fn(creq.kreq) + c.controlMu.Lock() + c.currentControl = nil + c.currentBroker = nil + res <- controlResp{kresp, err, handled} + }() + return res +} + +func (c *Cluster) beginSleptControl(s *slept) { + // Control flow gets really weird here. We unlocked when entering the + // control function, so we have to re-lock now so that tryControl can + // unlock us safely. + bs := c.sleeping[s.creq.cc] + if bs == nil { + bs = &bsleep{ + c: c, + set: make(map[*slept]struct{}), + setWake: make(chan *slept, 1), + } + c.sleeping[s.creq.cc] = bs + } + bs.enqueue(s) + c.controlMu.Lock() + c.currentControl = nil + c.currentBroker = nil +} + +func (c *Cluster) continueSleptControl(s *slept) { + // When continuing a slept control, we are in the main run loop and are + // not currently under the control mu. We need to re-set the current + // broker and current control before resuming. + c.controlMu.Lock() + c.currentBroker = s.creq.cc.b + c.currentControl = s.cctx + c.controlMu.Unlock() + s.sleepChs.clientCont <- struct{}{} +} + +func (c *Cluster) finishSleptControl(s *slept) { + // When finishing a slept control, the control function exited and + // grabbed the control mu. We clear the control, unlock, and allow the + // slept control to be dequeued. + c.currentControl = nil + c.currentBroker = nil + c.controlMu.Unlock() + s.continueDequeue <- struct{}{} +} + +func (c *Cluster) resleepSleptControl(s *slept, sleepChs sleepChs) { + // A control function previously slept and is now again sleeping. We + // need to clear the control broker / etc, update the sleep channels, + // and allow the sleep dequeueing to continue. The control function + // will not be deqeueued in the loop because we updated sleepChs with + // a non-nil clientWait. + c.controlMu.Lock() + c.currentBroker = nil + c.currentControl = nil + c.controlMu.Unlock() + s.sleepChs = sleepChs + s.continueDequeue <- struct{}{} + // For OOO requests, we need to manually trigger a goroutine to + // watch for the sleep to end. + s.bs.maybeWaitOOOWake(s) +} + +func (c *Cluster) maybePopControl(handled bool, cctx *controlCtx) { + if handled && !cctx.keep || cctx.drop { + delete(c.control[cctx.key], cctx) + } +} + +// bsleep manages sleeping requests on a connection to a broker, or +// non-sleeping requests that are waiting for sleeping requests to finish. +type bsleep struct { + c *Cluster + mu sync.Mutex + queue []*slept + set map[*slept]struct{} + setWake chan *slept +} + +type slept struct { + bs *bsleep + cctx *controlCtx + sleepChs sleepChs + res <-chan controlResp + creq *clientReq + waiting bool + + continueDequeue chan struct{} +} + +type sleepChs struct { + clientWait chan struct{} + clientCont chan struct{} +} + +// enqueue has a few potential behaviors. +// +// (1) If s is waiting, this is a new request enqueueing to the back of an +// existing queue, where we are waiting for the head request to finish +// sleeping. Easy case. +// +// (2) If s is not waiting, this is a sleeping request. If the queue is empty, +// this is the first sleeping request on a node. We enqueue and start our wait +// goroutine. Easy. +// +// (3) If s is not waiting, but our queue is non-empty, this must be from a +// convoluted scenario: +// +// (a) the user has SleepOutOfOrder configured, +// (b) or, there was a request in front of us that slept, we were waiting, +// and now we ourselves are sleeping +// (c) or, we are sleeping for the second time in a single control +func (bs *bsleep) enqueue(s *slept) bool { + if bs == nil { + return false // Do not enqueue, nothing sleeping + } + s.continueDequeue = make(chan struct{}, 1) + s.bs = bs + bs.mu.Lock() + defer bs.mu.Unlock() + if s.waiting { + if bs.c.cfg.sleepOutOfOrder { + panic("enqueueing a waiting request even though we are sleeping out of order") + } + if !bs.empty() { + bs.keep(s) // Case (1) + return true + } + return false // We do not enqueue, do not wait: nothing sleeping ahead of us + } + if bs.empty() { + bs.keep(s) + go bs.wait() // Case (2) + return true + } + var q0 *slept + if !bs.c.cfg.sleepOutOfOrder { + q0 = bs.queue[0] // Case (3b) or (3c) -- just update values below + } else { + // Case (3a), out of order sleep: we need to check the entire + // queue to see if this request was already sleeping and, if + // so, update the values. If it was not already sleeping, we + // "keep" the new sleeping item. + bs.keep(s) + return true + } + if q0.creq != s.creq { + panic("internal error: sleeping request not head request") + } + // We do not update continueDequeue because it is actively being read, + // we just reuse the old value. + q0.cctx = s.cctx + q0.sleepChs = s.sleepChs + q0.res = s.res + q0.waiting = s.waiting + return true +} + +// keep stores a sleeping request to be managed. For out of order control, the +// log is a bit more complicated and we need to watch for the control sleep +// finishing here, and forward the "I'm done sleeping" notification to waitSet. +func (bs *bsleep) keep(s *slept) { + if !bs.c.cfg.sleepOutOfOrder { + bs.queue = append(bs.queue, s) + return + } + bs.set[s] = struct{}{} + bs.maybeWaitOOOWake(s) +} + +func (bs *bsleep) maybeWaitOOOWake(s *slept) { + if !bs.c.cfg.sleepOutOfOrder { + return + } + go func() { + select { + case <-bs.c.die: + case <-s.sleepChs.clientWait: + select { + case <-bs.c.die: + case bs.setWake <- s: + } + } + }() +} + +func (bs *bsleep) empty() bool { + return len(bs.queue) == 0 && len(bs.set) == 0 +} + +func (bs *bsleep) wait() { + if bs.c.cfg.sleepOutOfOrder { + bs.waitSet() + } else { + bs.waitQueue() + } +} + +// For out of order control, all control functions run concurrently, serially. +// Whenever they wake up, they send themselves down setWake. waitSet manages +// handling the wake up and interacting with the serial manage goroutine to +// run everything properly. +func (bs *bsleep) waitSet() { + for { + bs.mu.Lock() + if len(bs.set) == 0 { + bs.mu.Unlock() + return + } + bs.mu.Unlock() + + // Wait for a control function to awaken. + var q *slept + select { + case <-bs.c.die: + return + case q = <-bs.setWake: + q.sleepChs.clientWait = nil + } + + // Now, schedule ourselves with the run loop. + select { + case <-bs.c.die: + return + case bs.c.wakeCh <- q: + } + + // Wait for this control function to finish its loop in the run + // function. Once it does, if clientWait is non-nil, the + // control function went back to sleep. If it is nil, the + // control function is done and we remove this from tracking. + select { + case <-bs.c.die: + return + case <-q.continueDequeue: + } + if q.sleepChs.clientWait == nil { + bs.mu.Lock() + delete(bs.set, q) + bs.mu.Unlock() + } + } +} + +// For in-order control functions, the concept is slightly simpler but the +// logic flow is the same. We wait for the head control function to wake up, +// try to run it, and then wait for it to finish. The logic of this function is +// the same as waitSet, minus the middle part where we wait for something to +// wake up. +func (bs *bsleep) waitQueue() { + for { + bs.mu.Lock() + if len(bs.queue) == 0 { + bs.mu.Unlock() + return + } + q0 := bs.queue[0] + bs.mu.Unlock() + + if q0.sleepChs.clientWait != nil { + select { + case <-bs.c.die: + return + case <-q0.sleepChs.clientWait: + q0.sleepChs.clientWait = nil + } + } + + select { + case <-bs.c.die: + return + case bs.c.wakeCh <- q0: + } + + select { + case <-bs.c.die: + return + case <-q0.continueDequeue: + } + if q0.sleepChs.clientWait == nil { + bs.mu.Lock() + bs.queue = bs.queue[1:] + bs.mu.Unlock() + } + } +} + +// Various administrative requests can be passed into the cluster to simulate +// real-world operations. These are performed synchronously in the goroutine +// that handles client requests. + +func (c *Cluster) admin(fn func()) { + ofn := fn + wait := make(chan struct{}) + fn = func() { ofn(); close(wait) } + c.adminCh <- fn + <-wait +} + +// MoveTopicPartition simulates the rebalancing of a partition to an alternative +// broker. This returns an error if the topic, partition, or node does not exit. +func (c *Cluster) MoveTopicPartition(topic string, partition int32, nodeID int32) error { + var err error + c.admin(func() { + var br *broker + for _, b := range c.bs { + if b.node == nodeID { + br = b + break + } + } + if br == nil { + err = fmt.Errorf("node %d not found", nodeID) + return + } + pd, ok := c.data.tps.getp(topic, partition) + if !ok { + err = errors.New("topic/partition not found") + return + } + pd.leader = br + pd.epoch++ + }) + return err +} + +// CoordinatorFor returns the node ID of the group or transaction coordinator +// for the given key. +func (c *Cluster) CoordinatorFor(key string) int32 { + var n int32 + c.admin(func() { + l := len(c.bs) + if l == 0 { + n = -1 + return + } + n = c.coordinator(key).node + }) + return n +} + +// RehashCoordinators simulates group and transacational ID coordinators moving +// around. All group and transactional IDs are rekeyed. This forces clients to +// reload coordinators. +func (c *Cluster) RehashCoordinators() { + c.coordinatorGen.Add(1) +} + +// AddNode adds a node to the cluster. If nodeID is -1, the next node ID is +// used. If port is 0 or negative, a random port is chosen. This returns the +// added node ID and the port used, or an error if the node already exists or +// the port cannot be listened to. +func (c *Cluster) AddNode(nodeID int32, port int) (int32, int, error) { + var err error + c.admin(func() { + if nodeID >= 0 { + for _, b := range c.bs { + if b.node == nodeID { + err = fmt.Errorf("node %d already exists", nodeID) + return + } + } + } else if len(c.bs) > 0 { + // We go one higher than the max current node ID. We + // need to search all nodes because a person may have + // added and removed a bunch, with manual ID overrides. + nodeID = c.bs[0].node + for _, b := range c.bs[1:] { + if b.node > nodeID { + nodeID = b.node + } + } + nodeID++ + } else { + nodeID = 0 + } + if port < 0 { + port = 0 + } + var ln net.Listener + if ln, err = newListener(port, c.cfg.tls); err != nil { + return + } + _, strPort, _ := net.SplitHostPort(ln.Addr().String()) + port, _ = strconv.Atoi(strPort) + b := &broker{ + c: c, + ln: ln, + node: nodeID, + bsIdx: len(c.bs), + } + c.bs = append(c.bs, b) + c.cfg.nbrokers++ + c.shufflePartitionsLocked() + go b.listen() + }) + return nodeID, port, err +} + +// RemoveNode removes a ndoe from the cluster. This returns an error if the +// node does not exist. +func (c *Cluster) RemoveNode(nodeID int32) error { + var err error + c.admin(func() { + for i, b := range c.bs { + if b.node == nodeID { + if len(c.bs) == 1 { + err = errors.New("cannot remove all brokers") + return + } + b.ln.Close() + c.cfg.nbrokers-- + c.bs[i] = c.bs[len(c.bs)-1] + c.bs[i].bsIdx = i + c.bs = c.bs[:len(c.bs)-1] + c.shufflePartitionsLocked() + return + } + } + err = fmt.Errorf("node %d not found", nodeID) + }) + return err +} + +// ShufflePartitionLeaders simulates a leader election for all partitions: all +// partitions have a randomly selected new leader and their internal epochs are +// bumped. +func (c *Cluster) ShufflePartitionLeaders() { + c.admin(func() { + c.shufflePartitionsLocked() + }) +} + +func (c *Cluster) shufflePartitionsLocked() { + c.data.tps.each(func(_ string, _ int32, p *partData) { + var leader *broker + if len(c.bs) == 0 { + leader = c.noLeader() + } else { + leader = c.bs[rand.Intn(len(c.bs))] + } + p.leader = leader + p.epoch++ + }) +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/config.go b/vendor/github.com/twmb/franz-go/pkg/kfake/config.go new file mode 100644 index 000000000000..75b34fb21f66 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/config.go @@ -0,0 +1,126 @@ +package kfake + +import ( + "crypto/tls" + "time" +) + +// Opt is an option to configure a client. +type Opt interface { + apply(*cfg) +} + +type opt struct{ fn func(*cfg) } + +func (opt opt) apply(cfg *cfg) { opt.fn(cfg) } + +type seedTopics struct { + p int32 + ts []string +} + +type cfg struct { + nbrokers int + ports []int + logger Logger + clusterID string + allowAutoTopic bool + defaultNumParts int + seedTopics []seedTopics + + minSessionTimeout time.Duration + maxSessionTimeout time.Duration + + enableSASL bool + sasls map[struct{ m, u string }]string // cleared after client initialization + tls *tls.Config + + sleepOutOfOrder bool +} + +// NumBrokers sets the number of brokers to start in the fake cluster. +func NumBrokers(n int) Opt { + return opt{func(cfg *cfg) { cfg.nbrokers = n }} +} + +// Ports sets the ports to listen on, overriding randomly choosing NumBrokers +// amount of ports. +func Ports(ports ...int) Opt { + return opt{func(cfg *cfg) { cfg.ports = ports }} +} + +// WithLogger sets the logger to use. +func WithLogger(logger Logger) Opt { + return opt{func(cfg *cfg) { cfg.logger = logger }} +} + +// ClusterID sets the cluster ID to return in metadata responses. +func ClusterID(clusterID string) Opt { + return opt{func(cfg *cfg) { cfg.clusterID = clusterID }} +} + +// AllowAutoTopicCreation allows metadata requests to create topics if the +// metadata request has its AllowAutoTopicCreation field set to true. +func AllowAutoTopicCreation() Opt { + return opt{func(cfg *cfg) { cfg.allowAutoTopic = true }} +} + +// DefaultNumPartitions sets the number of partitions to create by default for +// auto created topics / CreateTopics with -1 partitions, overriding the +// default of 10. +func DefaultNumPartitions(n int) Opt { + return opt{func(cfg *cfg) { cfg.defaultNumParts = n }} +} + +// GroupMinSessionTimeout sets the cluster's minimum session timeout allowed +// for groups, overriding the default 6 seconds. +func GroupMinSessionTimeout(d time.Duration) Opt { + return opt{func(cfg *cfg) { cfg.minSessionTimeout = d }} +} + +// GroupMaxSessionTimeout sets the cluster's maximum session timeout allowed +// for groups, overriding the default 5 minutes. +func GroupMaxSessionTimeout(d time.Duration) Opt { + return opt{func(cfg *cfg) { cfg.maxSessionTimeout = d }} +} + +// EnableSASL enables SASL authentication for the cluster. If you do not +// configure a bootstrap user / pass, the default superuser is "admin" / +// "admin" with the SCRAM-SHA-256 SASL mechanisms. +func EnableSASL() Opt { + return opt{func(cfg *cfg) { cfg.enableSASL = true }} +} + +// Superuser seeds the cluster with a superuser. The method must be either +// PLAIN, SCRAM-SHA-256, or SCRAM-SHA-512. +// Note that PLAIN superusers cannot be deleted. +// SCRAM superusers can be modified with AlterUserScramCredentials. +// If you delete all SASL users, the kfake cluster will be unusable. +func Superuser(method, user, pass string) Opt { + return opt{func(cfg *cfg) { cfg.sasls[struct{ m, u string }{method, user}] = pass }} +} + +// TLS enables TLS for the cluster, using the provided TLS config for +// listening. +func TLS(c *tls.Config) Opt { + return opt{func(cfg *cfg) { cfg.tls = c }} +} + +// SeedTopics provides topics to create by default in the cluster. Each topic +// will use the given partitions and use the default internal replication +// factor. If you use a non-positive number for partitions, [DefaultNumPartitions] +// is used. This option can be provided multiple times if you want to seed +// topics with different partition counts. If a topic is provided in multiple +// options, the last specification wins. +func SeedTopics(partitions int32, ts ...string) Opt { + return opt{func(cfg *cfg) { cfg.seedTopics = append(cfg.seedTopics, seedTopics{partitions, ts}) }} +} + +// SleepOutOfOrder allows functions to be handled out of order when control +// functions are sleeping. The functions are be handled internally out of +// order, but responses still wait for the sleeping requests to finish. This +// can be used to set up complicated chains of control where functions only +// advance when you know another request is actively being handled. +func SleepOutOfOrder() Opt { + return opt{func(cfg *cfg) { cfg.sleepOutOfOrder = true }} +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/data.go b/vendor/github.com/twmb/franz-go/pkg/kfake/data.go new file mode 100644 index 000000000000..9f5d46c6b868 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/data.go @@ -0,0 +1,343 @@ +package kfake + +import ( + "crypto/sha256" + "math/rand" + "sort" + "strconv" + "time" + + "github.com/twmb/franz-go/pkg/kmsg" +) + +// TODO +// +// * Write to disk, if configured. +// * When transactional, wait to send out data until txn committed or aborted. + +var noID uuid + +type ( + uuid [16]byte + + data struct { + c *Cluster + tps tps[partData] + + id2t map[uuid]string // topic IDs => topic name + t2id map[string]uuid // topic name => topic IDs + treplicas map[string]int // topic name => # replicas + tcfgs map[string]map[string]*string // topic name => config name => config value + } + + partData struct { + batches []partBatch + dir string + + highWatermark int64 + lastStableOffset int64 + logStartOffset int64 + epoch int32 // current epoch + maxTimestamp int64 // current max timestamp in all batches + nbytes int64 + + // abortedTxns + rf int8 + leader *broker + + watch map[*watchFetch]struct{} + + createdAt time.Time + } + + partBatch struct { + kmsg.RecordBatch + nbytes int + epoch int32 // epoch when appended + + // For list offsets, we may need to return the first offset + // after a given requested timestamp. Client provided + // timestamps gan go forwards and backwards. We answer list + // offsets with a binary search: even if this batch has a small + // timestamp, this is produced _after_ a potentially higher + // timestamp, so it is after it in the list offset response. + // + // When we drop the earlier timestamp, we update all following + // firstMaxTimestamps that match the dropped timestamp. + maxEarlierTimestamp int64 + } +) + +func (d *data) mkt(t string, nparts int, nreplicas int, configs map[string]*string) { + if d.tps != nil { + if _, exists := d.tps[t]; exists { + panic("should have checked existence already") + } + } + var id uuid + for { + sha := sha256.Sum256([]byte(strconv.Itoa(int(time.Now().UnixNano())))) + copy(id[:], sha[:]) + if _, exists := d.id2t[id]; !exists { + break + } + } + + if nparts < 0 { + nparts = d.c.cfg.defaultNumParts + } + if nreplicas < 0 { + nreplicas = 3 // cluster default + } + d.id2t[id] = t + d.t2id[t] = id + d.treplicas[t] = nreplicas + d.tcfgs[t] = configs + for i := 0; i < nparts; i++ { + d.tps.mkp(t, int32(i), d.c.newPartData) + } +} + +func (c *Cluster) noLeader() *broker { + return &broker{ + c: c, + node: -1, + } +} + +func (c *Cluster) newPartData() *partData { + return &partData{ + dir: defLogDir, + leader: c.bs[rand.Intn(len(c.bs))], + watch: make(map[*watchFetch]struct{}), + createdAt: time.Now(), + } +} + +func (pd *partData) pushBatch(nbytes int, b kmsg.RecordBatch) { + maxEarlierTimestamp := b.FirstTimestamp + if maxEarlierTimestamp < pd.maxTimestamp { + maxEarlierTimestamp = pd.maxTimestamp + } else { + pd.maxTimestamp = maxEarlierTimestamp + } + b.FirstOffset = pd.highWatermark + b.PartitionLeaderEpoch = pd.epoch + pd.batches = append(pd.batches, partBatch{b, nbytes, pd.epoch, maxEarlierTimestamp}) + pd.highWatermark += int64(b.NumRecords) + pd.lastStableOffset += int64(b.NumRecords) // TODO + pd.nbytes += int64(nbytes) + for w := range pd.watch { + w.push(nbytes) + } +} + +func (pd *partData) searchOffset(o int64) (index int, found bool, atEnd bool) { + if o < pd.logStartOffset || o > pd.highWatermark { + return 0, false, false + } + if len(pd.batches) == 0 { + if o == 0 { + return 0, false, true + } + } else { + lastBatch := pd.batches[len(pd.batches)-1] + if end := lastBatch.FirstOffset + int64(lastBatch.LastOffsetDelta) + 1; end == o { + return 0, false, true + } + } + + index, found = sort.Find(len(pd.batches), func(idx int) int { + b := &pd.batches[idx] + if o < b.FirstOffset { + return -1 + } + if o >= b.FirstOffset+int64(b.LastOffsetDelta)+1 { + return 1 + } + return 0 + }) + return index, found, false +} + +func (pd *partData) trimLeft() { + for len(pd.batches) > 0 { + b0 := pd.batches[0] + finRec := b0.FirstOffset + int64(b0.LastOffsetDelta) + if finRec >= pd.logStartOffset { + return + } + pd.batches = pd.batches[1:] + pd.nbytes -= int64(b0.nbytes) + } +} + +///////////// +// CONFIGS // +///////////// + +// TODO support modifying config values changing cluster behavior + +// brokerConfigs calls fn for all: +// - static broker configs (read only) +// - default configs +// - dynamic broker configs +func (c *Cluster) brokerConfigs(node int32, fn func(k string, v *string, src kmsg.ConfigSource, sensitive bool)) { + if node >= 0 { + for _, b := range c.bs { + if b.node == node { + id := strconv.Itoa(int(node)) + fn("broker.id", &id, kmsg.ConfigSourceStaticBrokerConfig, false) + break + } + } + } + for _, c := range []struct { + k string + v string + sens bool + }{ + {k: "broker.rack", v: "krack"}, + {k: "sasl.enabled.mechanisms", v: "PLAIN,SCRAM-SHA-256,SCRAM-SHA-512"}, + {k: "super.users", sens: true}, + } { + v := c.v + fn(c.k, &v, kmsg.ConfigSourceStaticBrokerConfig, c.sens) + } + + for k, v := range configDefaults { + if _, ok := validBrokerConfigs[k]; ok { + v := v + fn(k, &v, kmsg.ConfigSourceDefaultConfig, false) + } + } + + for k, v := range c.bcfgs { + fn(k, v, kmsg.ConfigSourceDynamicBrokerConfig, false) + } +} + +// configs calls fn for all +// - static broker configs (read only) +// - default configs +// - dynamic broker configs +// - dynamic topic configs +// +// This differs from brokerConfigs by also including dynamic topic configs. +func (d *data) configs(t string, fn func(k string, v *string, src kmsg.ConfigSource, sensitive bool)) { + for k, v := range configDefaults { + if _, ok := validTopicConfigs[k]; ok { + v := v + fn(k, &v, kmsg.ConfigSourceDefaultConfig, false) + } + } + for k, v := range d.c.bcfgs { + if topicEquiv, ok := validBrokerConfigs[k]; ok && topicEquiv != "" { + fn(k, v, kmsg.ConfigSourceDynamicBrokerConfig, false) + } + } + for k, v := range d.tcfgs[t] { + fn(k, v, kmsg.ConfigSourceDynamicTopicConfig, false) + } +} + +// Unlike Kafka, we validate the value before allowing it to be set. +func (c *Cluster) setBrokerConfig(k string, v *string, dry bool) bool { + if dry { + return true + } + c.bcfgs[k] = v + return true +} + +func (d *data) setTopicConfig(t string, k string, v *string, dry bool) bool { + if dry { + return true + } + if _, ok := d.tcfgs[t]; !ok { + d.tcfgs[t] = make(map[string]*string) + } + d.tcfgs[t][k] = v + return true +} + +// All valid topic configs we support, as well as the equivalent broker +// config if there is one. +var validTopicConfigs = map[string]string{ + "cleanup.policy": "", + "compression.type": "compression.type", + "max.message.bytes": "log.message.max.bytes", + "message.timestamp.type": "log.message.timestamp.type", + "min.insync.replicas": "min.insync.replicas", + "retention.bytes": "log.retention.bytes", + "retention.ms": "log.retention.ms", +} + +// All valid broker configs we support, as well as their equivalent +// topic config if there is one. +var validBrokerConfigs = map[string]string{ + "broker.id": "", + "broker.rack": "", + "compression.type": "compression.type", + "default.replication.factor": "", + "fetch.max.bytes": "", + "log.dir": "", + "log.message.timestamp.type": "message.timestamp.type", + "log.retention.bytes": "retention.bytes", + "log.retention.ms": "retention.ms", + "message.max.bytes": "max.message.bytes", + "min.insync.replicas": "min.insync.replicas", + "sasl.enabled.mechanisms": "", + "super.users": "", +} + +// Default topic and broker configs. +var configDefaults = map[string]string{ + "cleanup.policy": "delete", + "compression.type": "producer", + "max.message.bytes": "1048588", + "message.timestamp.type": "CreateTime", + "min.insync.replicas": "1", + "retention.bytes": "-1", + "retention.ms": "604800000", + + "default.replication.factor": "3", + "fetch.max.bytes": "57671680", + "log.dir": defLogDir, + "log.message.timestamp.type": "CreateTime", + "log.retention.bytes": "-1", + "log.retention.ms": "604800000", + "message.max.bytes": "1048588", +} + +const defLogDir = "/mem/kfake" + +func staticConfig(s ...string) func(*string) bool { + return func(v *string) bool { + if v == nil { + return false + } + for _, ok := range s { + if *v == ok { + return true + } + } + return false + } +} + +func numberConfig(min int, hasMin bool, max int, hasMax bool) func(*string) bool { + return func(v *string) bool { + if v == nil { + return false + } + i, err := strconv.Atoi(*v) + if err != nil { + return false + } + if hasMin && i < min || hasMax && i > max { + return false + } + return true + } +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/groups.go b/vendor/github.com/twmb/franz-go/pkg/kfake/groups.go new file mode 100644 index 000000000000..bc6934d6dcb1 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/groups.go @@ -0,0 +1,1195 @@ +package kfake + +import ( + "bytes" + "fmt" + "sync" + "time" + + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kmsg" +) + +// TODO instance IDs +// TODO persisting groups so commits can happen to client-managed groups +// we need lastCommit, and need to better prune empty groups + +type ( + groups struct { + c *Cluster + gs map[string]*group + } + + group struct { + c *Cluster + gs *groups + name string + + state groupState + + leader string + members map[string]*groupMember + pending map[string]*groupMember + + commits tps[offsetCommit] + + generation int32 + protocolType string + protocols map[string]int + protocol string + + reqCh chan *clientReq + controlCh chan func() + + nJoining int + + tRebalance *time.Timer + + quit sync.Once + quitCh chan struct{} + } + + groupMember struct { + memberID string + clientID string + clientHost string + + join *kmsg.JoinGroupRequest // the latest join request + + // waitingReply is non-nil if a client is waiting for a reply + // from us for a JoinGroupRequest or a SyncGroupRequest. + waitingReply *clientReq + + assignment []byte + + t *time.Timer + last time.Time + } + + offsetCommit struct { + offset int64 + leaderEpoch int32 + metadata *string + } + + groupState int8 +) + +const ( + groupEmpty groupState = iota + groupStable + groupPreparingRebalance + groupCompletingRebalance + groupDead +) + +func (gs groupState) String() string { + switch gs { + case groupEmpty: + return "Empty" + case groupStable: + return "Stable" + case groupPreparingRebalance: + return "PreparingRebalance" + case groupCompletingRebalance: + return "CompletingRebalance" + case groupDead: + return "Dead" + default: + return "Unknown" + } +} + +func (c *Cluster) coordinator(id string) *broker { + gen := c.coordinatorGen.Load() + n := hashString(fmt.Sprintf("%d", gen)+"\x00\x00"+id) % uint64(len(c.bs)) + return c.bs[n] +} + +func (c *Cluster) validateGroup(creq *clientReq, group string) *kerr.Error { + switch key := kmsg.Key(creq.kreq.Key()); key { + case kmsg.OffsetCommit, kmsg.OffsetFetch, kmsg.DescribeGroups, kmsg.DeleteGroups: + default: + if group == "" { + return kerr.InvalidGroupID + } + } + coordinator := c.coordinator(group).node + if coordinator != creq.cc.b.node { + return kerr.NotCoordinator + } + return nil +} + +func generateMemberID(clientID string, instanceID *string) string { + if instanceID == nil { + return clientID + "-" + randStrUUID() + } + return *instanceID + "-" + randStrUUID() +} + +//////////// +// GROUPS // +//////////// + +func (gs *groups) newGroup(name string) *group { + return &group{ + c: gs.c, + gs: gs, + name: name, + members: make(map[string]*groupMember), + pending: make(map[string]*groupMember), + protocols: make(map[string]int), + reqCh: make(chan *clientReq), + controlCh: make(chan func()), + quitCh: make(chan struct{}), + } +} + +// handleJoin completely hijacks the incoming request. +func (gs *groups) handleJoin(creq *clientReq) { + if gs.gs == nil { + gs.gs = make(map[string]*group) + } + req := creq.kreq.(*kmsg.JoinGroupRequest) +start: + g := gs.gs[req.Group] + if g == nil { + g = gs.newGroup(req.Group) + waitJoin := make(chan struct{}) + gs.gs[req.Group] = g + go g.manage(func() { close(waitJoin) }) + defer func() { <-waitJoin }() + } + select { + case g.reqCh <- creq: + case <-g.quitCh: + goto start + } +} + +// Returns true if the request is hijacked and handled, otherwise false if the +// group does not exist. +func (gs *groups) handleHijack(group string, creq *clientReq) bool { + if gs.gs == nil { + return false + } + g := gs.gs[group] + if g == nil { + return false + } + select { + case g.reqCh <- creq: + return true + case <-g.quitCh: + return false + } +} + +func (gs *groups) handleSync(creq *clientReq) bool { + return gs.handleHijack(creq.kreq.(*kmsg.SyncGroupRequest).Group, creq) +} + +func (gs *groups) handleHeartbeat(creq *clientReq) bool { + return gs.handleHijack(creq.kreq.(*kmsg.HeartbeatRequest).Group, creq) +} + +func (gs *groups) handleLeave(creq *clientReq) bool { + return gs.handleHijack(creq.kreq.(*kmsg.LeaveGroupRequest).Group, creq) +} + +func (gs *groups) handleOffsetCommit(creq *clientReq) { + if gs.gs == nil { + gs.gs = make(map[string]*group) + } + req := creq.kreq.(*kmsg.OffsetCommitRequest) +start: + g := gs.gs[req.Group] + if g == nil { + g = gs.newGroup(req.Group) + waitCommit := make(chan struct{}) + gs.gs[req.Group] = g + go g.manage(func() { close(waitCommit) }) + defer func() { <-waitCommit }() + } + select { + case g.reqCh <- creq: + case <-g.quitCh: + goto start + } +} + +func (gs *groups) handleOffsetDelete(creq *clientReq) bool { + return gs.handleHijack(creq.kreq.(*kmsg.OffsetDeleteRequest).Group, creq) +} + +func (gs *groups) handleList(creq *clientReq) *kmsg.ListGroupsResponse { + req := creq.kreq.(*kmsg.ListGroupsRequest) + resp := req.ResponseKind().(*kmsg.ListGroupsResponse) + + var states map[string]struct{} + if len(req.StatesFilter) > 0 { + states = make(map[string]struct{}) + for _, state := range req.StatesFilter { + states[state] = struct{}{} + } + } + + for _, g := range gs.gs { + if g.c.coordinator(g.name).node != creq.cc.b.node { + continue + } + g.waitControl(func() { + if states != nil { + if _, ok := states[g.state.String()]; !ok { + return + } + } + sg := kmsg.NewListGroupsResponseGroup() + sg.Group = g.name + sg.ProtocolType = g.protocolType + sg.GroupState = g.state.String() + resp.Groups = append(resp.Groups, sg) + }) + } + return resp +} + +func (gs *groups) handleDescribe(creq *clientReq) *kmsg.DescribeGroupsResponse { + req := creq.kreq.(*kmsg.DescribeGroupsRequest) + resp := req.ResponseKind().(*kmsg.DescribeGroupsResponse) + + doneg := func(name string) *kmsg.DescribeGroupsResponseGroup { + sg := kmsg.NewDescribeGroupsResponseGroup() + sg.Group = name + resp.Groups = append(resp.Groups, sg) + return &resp.Groups[len(resp.Groups)-1] + } + + for _, rg := range req.Groups { + sg := doneg(rg) + if kerr := gs.c.validateGroup(creq, rg); kerr != nil { + sg.ErrorCode = kerr.Code + continue + } + g, ok := gs.gs[rg] + if !ok { + sg.State = groupDead.String() + continue + } + if !g.waitControl(func() { + sg.State = g.state.String() + sg.ProtocolType = g.protocolType + if g.state == groupStable { + sg.Protocol = g.protocol + } + for _, m := range g.members { + sm := kmsg.NewDescribeGroupsResponseGroupMember() + sm.MemberID = m.memberID + sm.ClientID = m.clientID + sm.ClientHost = m.clientHost + if g.state == groupStable { + for _, p := range m.join.Protocols { + if p.Name == g.protocol { + sm.ProtocolMetadata = p.Metadata + break + } + } + sm.MemberAssignment = m.assignment + } + sg.Members = append(sg.Members, sm) + + } + }) { + sg.State = groupDead.String() + } + } + return resp +} + +func (gs *groups) handleDelete(creq *clientReq) *kmsg.DeleteGroupsResponse { + req := creq.kreq.(*kmsg.DeleteGroupsRequest) + resp := req.ResponseKind().(*kmsg.DeleteGroupsResponse) + + doneg := func(name string) *kmsg.DeleteGroupsResponseGroup { + sg := kmsg.NewDeleteGroupsResponseGroup() + sg.Group = name + resp.Groups = append(resp.Groups, sg) + return &resp.Groups[len(resp.Groups)-1] + } + + for _, rg := range req.Groups { + sg := doneg(rg) + if kerr := gs.c.validateGroup(creq, rg); kerr != nil { + sg.ErrorCode = kerr.Code + continue + } + g, ok := gs.gs[rg] + if !ok { + sg.ErrorCode = kerr.GroupIDNotFound.Code + continue + } + if !g.waitControl(func() { + switch g.state { + case groupDead: + sg.ErrorCode = kerr.GroupIDNotFound.Code + case groupEmpty: + g.quitOnce() + delete(gs.gs, rg) + case groupPreparingRebalance, groupCompletingRebalance, groupStable: + sg.ErrorCode = kerr.NonEmptyGroup.Code + } + }) { + sg.ErrorCode = kerr.GroupIDNotFound.Code + } + } + return resp +} + +func (gs *groups) handleOffsetFetch(creq *clientReq) *kmsg.OffsetFetchResponse { + req := creq.kreq.(*kmsg.OffsetFetchRequest) + resp := req.ResponseKind().(*kmsg.OffsetFetchResponse) + + if req.Version <= 7 { + rg := kmsg.NewOffsetFetchRequestGroup() + rg.Group = req.Group + if req.Topics != nil { + rg.Topics = make([]kmsg.OffsetFetchRequestGroupTopic, len(req.Topics)) + } + for _, t := range req.Topics { + rt := kmsg.NewOffsetFetchRequestGroupTopic() + rt.Topic = t.Topic + rt.Partitions = t.Partitions + rg.Topics = append(rg.Topics, rt) + } + req.Groups = append(req.Groups, rg) + + defer func() { + g0 := resp.Groups[0] + resp.ErrorCode = g0.ErrorCode + for _, t := range g0.Topics { + st := kmsg.NewOffsetFetchResponseTopic() + st.Topic = t.Topic + for _, p := range t.Partitions { + sp := kmsg.NewOffsetFetchResponseTopicPartition() + sp.Partition = p.Partition + sp.Offset = p.Offset + sp.LeaderEpoch = p.LeaderEpoch + sp.Metadata = p.Metadata + sp.ErrorCode = p.ErrorCode + st.Partitions = append(st.Partitions, sp) + } + resp.Topics = append(resp.Topics, st) + } + }() + } + + doneg := func(name string) *kmsg.OffsetFetchResponseGroup { + sg := kmsg.NewOffsetFetchResponseGroup() + sg.Group = name + resp.Groups = append(resp.Groups, sg) + return &resp.Groups[len(resp.Groups)-1] + } + + for _, rg := range req.Groups { + sg := doneg(rg.Group) + if kerr := gs.c.validateGroup(creq, rg.Group); kerr != nil { + sg.ErrorCode = kerr.Code + continue + } + g, ok := gs.gs[rg.Group] + if !ok { + sg.ErrorCode = kerr.GroupIDNotFound.Code + continue + } + if !g.waitControl(func() { + if rg.Topics == nil { + for t, ps := range g.commits { + st := kmsg.NewOffsetFetchResponseGroupTopic() + st.Topic = t + for p, c := range ps { + sp := kmsg.NewOffsetFetchResponseGroupTopicPartition() + sp.Partition = p + sp.Offset = c.offset + sp.LeaderEpoch = c.leaderEpoch + sp.Metadata = c.metadata + st.Partitions = append(st.Partitions, sp) + } + sg.Topics = append(sg.Topics, st) + } + } else { + for _, t := range rg.Topics { + st := kmsg.NewOffsetFetchResponseGroupTopic() + st.Topic = t.Topic + for _, p := range t.Partitions { + sp := kmsg.NewOffsetFetchResponseGroupTopicPartition() + sp.Partition = p + c, ok := g.commits.getp(t.Topic, p) + if !ok { + sp.Offset = -1 + sp.LeaderEpoch = -1 + } else { + sp.Offset = c.offset + sp.LeaderEpoch = c.leaderEpoch + sp.Metadata = c.metadata + } + st.Partitions = append(st.Partitions, sp) + } + sg.Topics = append(sg.Topics, st) + } + } + }) { + sg.ErrorCode = kerr.GroupIDNotFound.Code + } + } + return resp +} + +func (g *group) handleOffsetDelete(creq *clientReq) *kmsg.OffsetDeleteResponse { + req := creq.kreq.(*kmsg.OffsetDeleteRequest) + resp := req.ResponseKind().(*kmsg.OffsetDeleteResponse) + + if kerr := g.c.validateGroup(creq, req.Group); kerr != nil { + resp.ErrorCode = kerr.Code + return resp + } + + tidx := make(map[string]int) + donet := func(t string, errCode int16) *kmsg.OffsetDeleteResponseTopic { + if i, ok := tidx[t]; ok { + return &resp.Topics[i] + } + tidx[t] = len(resp.Topics) + st := kmsg.NewOffsetDeleteResponseTopic() + st.Topic = t + resp.Topics = append(resp.Topics, st) + return &resp.Topics[len(resp.Topics)-1] + } + donep := func(t string, p int32, errCode int16) *kmsg.OffsetDeleteResponseTopicPartition { + sp := kmsg.NewOffsetDeleteResponseTopicPartition() + sp.Partition = p + sp.ErrorCode = errCode + st := donet(t, 0) + st.Partitions = append(st.Partitions, sp) + return &st.Partitions[len(st.Partitions)-1] + } + + // empty: delete everything in request + // preparingRebalance, completingRebalance, stable: + // * if consumer, delete everything not subscribed to + // * if not consumer, delete nothing, error with non_empty_group + subTopics := make(map[string]struct{}) + switch g.state { + default: + resp.ErrorCode = kerr.GroupIDNotFound.Code + return resp + case groupEmpty: + case groupPreparingRebalance, groupCompletingRebalance, groupStable: + if g.protocolType != "consumer" { + resp.ErrorCode = kerr.NonEmptyGroup.Code + return resp + } + for _, m := range []map[string]*groupMember{ + g.members, + g.pending, + } { + for _, m := range m { + if m.join == nil { + continue + } + for _, proto := range m.join.Protocols { + var m kmsg.ConsumerMemberMetadata + if err := m.ReadFrom(proto.Metadata); err == nil { + for _, topic := range m.Topics { + subTopics[topic] = struct{}{} + } + } + } + } + } + } + + for _, t := range req.Topics { + for _, p := range t.Partitions { + if _, ok := subTopics[t.Topic]; ok { + donep(t.Topic, p.Partition, kerr.GroupSubscribedToTopic.Code) + continue + } + g.commits.delp(t.Topic, p.Partition) + donep(t.Topic, p.Partition, 0) + } + } + + return resp +} + +//////////////////// +// GROUP HANDLING // +//////////////////// + +func (g *group) manage(detachNew func()) { + // On the first join only, we want to ensure that if the join is + // invalid, we clean the group up before we detach from the cluster + // serialization loop that is initializing us. + var firstJoin func(bool) + firstJoin = func(ok bool) { + firstJoin = func(bool) {} + if !ok { + delete(g.gs.gs, g.name) + g.quitOnce() + } + detachNew() + } + + defer func() { + for _, m := range g.members { + if m.t != nil { + m.t.Stop() + } + } + for _, m := range g.pending { + if m.t != nil { + m.t.Stop() + } + } + }() + + for { + select { + case <-g.quitCh: + return + case creq := <-g.reqCh: + var kresp kmsg.Response + switch creq.kreq.(type) { + case *kmsg.JoinGroupRequest: + var ok bool + kresp, ok = g.handleJoin(creq) + firstJoin(ok) + case *kmsg.SyncGroupRequest: + kresp = g.handleSync(creq) + case *kmsg.HeartbeatRequest: + kresp = g.handleHeartbeat(creq) + case *kmsg.LeaveGroupRequest: + kresp = g.handleLeave(creq) + case *kmsg.OffsetCommitRequest: + var ok bool + kresp, ok = g.handleOffsetCommit(creq) + firstJoin(ok) + case *kmsg.OffsetDeleteRequest: + kresp = g.handleOffsetDelete(creq) + } + if kresp != nil { + g.reply(creq, kresp, nil) + } + + case fn := <-g.controlCh: + fn() + } + } +} + +func (g *group) waitControl(fn func()) bool { + wait := make(chan struct{}) + wfn := func() { fn(); close(wait) } + select { + case <-g.quitCh: + return false + case g.controlCh <- wfn: + <-wait + return true + } +} + +// Called in the manage loop. +func (g *group) quitOnce() { + g.quit.Do(func() { + g.state = groupDead + close(g.quitCh) + }) +} + +// Handles a join. We do not do the delayed join aspects in Kafka, we just punt +// to the client to immediately rejoin if a new client enters the group. +// +// If this returns nil, the request will be replied to later. +func (g *group) handleJoin(creq *clientReq) (kmsg.Response, bool) { + req := creq.kreq.(*kmsg.JoinGroupRequest) + resp := req.ResponseKind().(*kmsg.JoinGroupResponse) + + if kerr := g.c.validateGroup(creq, req.Group); kerr != nil { + resp.ErrorCode = kerr.Code + return resp, false + } + if req.InstanceID != nil { + resp.ErrorCode = kerr.InvalidGroupID.Code + return resp, false + } + if st := int64(req.SessionTimeoutMillis); st < g.c.cfg.minSessionTimeout.Milliseconds() || st > g.c.cfg.maxSessionTimeout.Milliseconds() { + resp.ErrorCode = kerr.InvalidSessionTimeout.Code + return resp, false + } + if !g.protocolsMatch(req.ProtocolType, req.Protocols) { + resp.ErrorCode = kerr.InconsistentGroupProtocol.Code + return resp, false + } + + // Clients first join with no member ID. For join v4+, we generate + // the member ID and add the member to pending. For v3 and below, + // we immediately enter rebalance. + if req.MemberID == "" { + memberID := generateMemberID(creq.cid, req.InstanceID) + resp.MemberID = memberID + m := &groupMember{ + memberID: memberID, + clientID: creq.cid, + clientHost: creq.cc.conn.RemoteAddr().String(), + join: req, + } + if req.Version >= 4 { + g.addPendingRebalance(m) + resp.ErrorCode = kerr.MemberIDRequired.Code + return resp, true + } + g.addMemberAndRebalance(m, creq, req) + return nil, true + } + + // Pending members rejoining immediately enters rebalance. + if m, ok := g.pending[req.MemberID]; ok { + g.addMemberAndRebalance(m, creq, req) + return nil, true + } + m, ok := g.members[req.MemberID] + if !ok { + resp.ErrorCode = kerr.UnknownMemberID.Code + return resp, false + } + + switch g.state { + default: + resp.ErrorCode = kerr.UnknownMemberID.Code + return resp, false + case groupPreparingRebalance: + g.updateMemberAndRebalance(m, creq, req) + case groupCompletingRebalance: + if m.sameJoin(req) { + g.fillJoinResp(req, resp) + return resp, true + } + g.updateMemberAndRebalance(m, creq, req) + case groupStable: + if g.leader != req.MemberID || m.sameJoin(req) { + g.fillJoinResp(req, resp) + return resp, true + } + g.updateMemberAndRebalance(m, creq, req) + } + return nil, true +} + +// Handles a sync, which can transition us to stable. +func (g *group) handleSync(creq *clientReq) kmsg.Response { + req := creq.kreq.(*kmsg.SyncGroupRequest) + resp := req.ResponseKind().(*kmsg.SyncGroupResponse) + + if kerr := g.c.validateGroup(creq, req.Group); kerr != nil { + resp.ErrorCode = kerr.Code + return resp + } + if req.InstanceID != nil { + resp.ErrorCode = kerr.InvalidGroupID.Code + return resp + } + m, ok := g.members[req.MemberID] + if !ok { + resp.ErrorCode = kerr.UnknownMemberID.Code + return resp + } + if req.Generation != g.generation { + resp.ErrorCode = kerr.IllegalGeneration.Code + return resp + } + if req.ProtocolType != nil && *req.ProtocolType != g.protocolType { + resp.ErrorCode = kerr.InconsistentGroupProtocol.Code + return resp + } + if req.Protocol != nil && *req.Protocol != g.protocol { + resp.ErrorCode = kerr.InconsistentGroupProtocol.Code + return resp + } + + switch g.state { + default: + resp.ErrorCode = kerr.UnknownMemberID.Code + case groupPreparingRebalance: + resp.ErrorCode = kerr.RebalanceInProgress.Code + case groupCompletingRebalance: + m.waitingReply = creq + if req.MemberID == g.leader { + g.completeLeaderSync(req) + } + return nil + case groupStable: // member saw join and is now finally calling sync + resp.ProtocolType = kmsg.StringPtr(g.protocolType) + resp.Protocol = kmsg.StringPtr(g.protocol) + resp.MemberAssignment = m.assignment + } + return resp +} + +// Handles a heartbeat, a relatively simple request that just delays our +// session timeout timer. +func (g *group) handleHeartbeat(creq *clientReq) kmsg.Response { + req := creq.kreq.(*kmsg.HeartbeatRequest) + resp := req.ResponseKind().(*kmsg.HeartbeatResponse) + + if kerr := g.c.validateGroup(creq, req.Group); kerr != nil { + resp.ErrorCode = kerr.Code + return resp + } + if req.InstanceID != nil { + resp.ErrorCode = kerr.InvalidGroupID.Code + return resp + } + m, ok := g.members[req.MemberID] + if !ok { + resp.ErrorCode = kerr.UnknownMemberID.Code + return resp + } + if req.Generation != g.generation { + resp.ErrorCode = kerr.IllegalGeneration.Code + return resp + } + + switch g.state { + default: + resp.ErrorCode = kerr.UnknownMemberID.Code + case groupPreparingRebalance: + resp.ErrorCode = kerr.RebalanceInProgress.Code + g.updateHeartbeat(m) + case groupCompletingRebalance, groupStable: + g.updateHeartbeat(m) + } + return resp +} + +// Handles a leave. We trigger a rebalance for every member leaving in a batch +// request, but that's fine because of our manage serialization. +func (g *group) handleLeave(creq *clientReq) kmsg.Response { + req := creq.kreq.(*kmsg.LeaveGroupRequest) + resp := req.ResponseKind().(*kmsg.LeaveGroupResponse) + + if kerr := g.c.validateGroup(creq, req.Group); kerr != nil { + resp.ErrorCode = kerr.Code + return resp + } + if req.Version < 3 { + req.Members = append(req.Members, kmsg.LeaveGroupRequestMember{ + MemberID: req.MemberID, + }) + defer func() { resp.ErrorCode = resp.Members[0].ErrorCode }() + } + + for _, rm := range req.Members { + mresp := kmsg.NewLeaveGroupResponseMember() + mresp.MemberID = rm.MemberID + mresp.InstanceID = rm.InstanceID + resp.Members = append(resp.Members, mresp) + + r := &resp.Members[len(resp.Members)-1] + if rm.InstanceID != nil { + r.ErrorCode = kerr.UnknownMemberID.Code + continue + } + if m, ok := g.members[rm.MemberID]; !ok { + if p, ok := g.pending[rm.MemberID]; !ok { + r.ErrorCode = kerr.UnknownMemberID.Code + } else { + g.stopPending(p) + } + } else { + g.updateMemberAndRebalance(m, nil, nil) + } + } + + return resp +} + +func fillOffsetCommit(req *kmsg.OffsetCommitRequest, resp *kmsg.OffsetCommitResponse, code int16) { + for _, t := range req.Topics { + st := kmsg.NewOffsetCommitResponseTopic() + st.Topic = t.Topic + for _, p := range t.Partitions { + sp := kmsg.NewOffsetCommitResponseTopicPartition() + sp.Partition = p.Partition + sp.ErrorCode = code + st.Partitions = append(st.Partitions, sp) + } + resp.Topics = append(resp.Topics, st) + } +} + +// Handles a commit. +func (g *group) handleOffsetCommit(creq *clientReq) (*kmsg.OffsetCommitResponse, bool) { + req := creq.kreq.(*kmsg.OffsetCommitRequest) + resp := req.ResponseKind().(*kmsg.OffsetCommitResponse) + + if kerr := g.c.validateGroup(creq, req.Group); kerr != nil { + fillOffsetCommit(req, resp, kerr.Code) + return resp, false + } + if req.InstanceID != nil { + fillOffsetCommit(req, resp, kerr.InvalidGroupID.Code) + return resp, false + } + + var m *groupMember + if len(g.members) > 0 { + var ok bool + m, ok = g.members[req.MemberID] + if !ok { + fillOffsetCommit(req, resp, kerr.UnknownMemberID.Code) + return resp, false + } + if req.Generation != g.generation { + fillOffsetCommit(req, resp, kerr.IllegalGeneration.Code) + return resp, false + } + } else { + if req.MemberID != "" { + fillOffsetCommit(req, resp, kerr.UnknownMemberID.Code) + return resp, false + } + if req.Generation != -1 { + fillOffsetCommit(req, resp, kerr.IllegalGeneration.Code) + return resp, false + } + if g.state != groupEmpty { + panic("invalid state: no members, but group not empty") + } + } + + switch g.state { + default: + fillOffsetCommit(req, resp, kerr.GroupIDNotFound.Code) + return resp, true + case groupEmpty: + for _, t := range req.Topics { + for _, p := range t.Partitions { + g.commits.set(t.Topic, p.Partition, offsetCommit{ + offset: p.Offset, + leaderEpoch: p.LeaderEpoch, + metadata: p.Metadata, + }) + } + } + fillOffsetCommit(req, resp, 0) + case groupPreparingRebalance, groupStable: + for _, t := range req.Topics { + for _, p := range t.Partitions { + g.commits.set(t.Topic, p.Partition, offsetCommit{ + offset: p.Offset, + leaderEpoch: p.LeaderEpoch, + metadata: p.Metadata, + }) + } + } + fillOffsetCommit(req, resp, 0) + g.updateHeartbeat(m) + case groupCompletingRebalance: + fillOffsetCommit(req, resp, kerr.RebalanceInProgress.Code) + g.updateHeartbeat(m) + } + return resp, true +} + +// Transitions the group to the preparing rebalance state. We first need to +// clear any member that is currently sitting in sync. If enough members have +// entered join, we immediately proceed to completeRebalance, otherwise we +// begin a wait timer. +func (g *group) rebalance() { + if g.state == groupCompletingRebalance { + for _, m := range g.members { + m.assignment = nil + if m.waitingReply.empty() { + continue + } + sync, ok := m.waitingReply.kreq.(*kmsg.SyncGroupRequest) + if !ok { + continue + } + resp := sync.ResponseKind().(*kmsg.SyncGroupResponse) + resp.ErrorCode = kerr.RebalanceInProgress.Code + g.reply(m.waitingReply, resp, m) + } + } + + g.state = groupPreparingRebalance + + if g.nJoining >= len(g.members) { + g.completeRebalance() + return + } + + var rebalanceTimeoutMs int32 + for _, m := range g.members { + if m.join.RebalanceTimeoutMillis > rebalanceTimeoutMs { + rebalanceTimeoutMs = m.join.RebalanceTimeoutMillis + } + } + if g.tRebalance == nil { + g.tRebalance = time.AfterFunc(time.Duration(rebalanceTimeoutMs)*time.Millisecond, func() { + select { + case <-g.quitCh: + case g.controlCh <- func() { + g.completeRebalance() + }: + } + }) + } +} + +// Transitions the group to either dead or stable, depending on if any members +// remain by the time we clear those that are not waiting in join. +func (g *group) completeRebalance() { + if g.tRebalance != nil { + g.tRebalance.Stop() + g.tRebalance = nil + } + g.nJoining = 0 + + var foundLeader bool + for _, m := range g.members { + if m.waitingReply.empty() { + for _, p := range m.join.Protocols { + g.protocols[p.Name]-- + } + delete(g.members, m.memberID) + if m.t != nil { + m.t.Stop() + } + continue + } + if m.memberID == g.leader { + foundLeader = true + } + } + + g.generation++ + if g.generation < 0 { + g.generation = 1 + } + if len(g.members) == 0 { + g.state = groupEmpty + return + } + g.state = groupCompletingRebalance + + var foundProto bool + for proto, nsupport := range g.protocols { + if nsupport == len(g.members) { + g.protocol = proto + foundProto = true + break + } + } + if !foundProto { + panic(fmt.Sprint("unable to find commonly supported protocol!", g.protocols, len(g.members))) + } + + for _, m := range g.members { + if !foundLeader { + g.leader = m.memberID + } + req := m.join + resp := req.ResponseKind().(*kmsg.JoinGroupResponse) + g.fillJoinResp(req, resp) + g.reply(m.waitingReply, resp, m) + } +} + +// Transitions the group to stable, the final step of a rebalance. +func (g *group) completeLeaderSync(req *kmsg.SyncGroupRequest) { + for _, m := range g.members { + m.assignment = nil + } + for _, a := range req.GroupAssignment { + m, ok := g.members[a.MemberID] + if !ok { + continue + } + m.assignment = a.MemberAssignment + } + for _, m := range g.members { + if m.waitingReply.empty() { + continue // this member saw join but has not yet called sync + } + resp := m.waitingReply.kreq.ResponseKind().(*kmsg.SyncGroupResponse) + resp.ProtocolType = kmsg.StringPtr(g.protocolType) + resp.Protocol = kmsg.StringPtr(g.protocol) + resp.MemberAssignment = m.assignment + g.reply(m.waitingReply, resp, m) + } + g.state = groupStable +} + +func (g *group) updateHeartbeat(m *groupMember) { + g.atSessionTimeout(m, func() { + g.updateMemberAndRebalance(m, nil, nil) + }) +} + +func (g *group) addPendingRebalance(m *groupMember) { + g.pending[m.memberID] = m + g.atSessionTimeout(m, func() { + delete(g.pending, m.memberID) + }) +} + +func (g *group) stopPending(m *groupMember) { + delete(g.pending, m.memberID) + if m.t != nil { + m.t.Stop() + } +} + +func (g *group) atSessionTimeout(m *groupMember, fn func()) { + if m.t != nil { + m.t.Stop() + } + timeout := time.Millisecond * time.Duration(m.join.SessionTimeoutMillis) + m.last = time.Now() + tfn := func() { + select { + case <-g.quitCh: + case g.controlCh <- func() { + if time.Since(m.last) >= timeout { + fn() + } + }: + } + } + m.t = time.AfterFunc(timeout, tfn) +} + +// This is used to update a member from a new join request, or to clear a +// member from failed heartbeats. +func (g *group) updateMemberAndRebalance(m *groupMember, waitingReply *clientReq, newJoin *kmsg.JoinGroupRequest) { + for _, p := range m.join.Protocols { + g.protocols[p.Name]-- + } + m.join = newJoin + if m.join != nil { + for _, p := range m.join.Protocols { + g.protocols[p.Name]++ + } + if m.waitingReply.empty() && !waitingReply.empty() { + g.nJoining++ + } + m.waitingReply = waitingReply + } else { + delete(g.members, m.memberID) + if m.t != nil { + m.t.Stop() + } + if !m.waitingReply.empty() { + g.nJoining-- + } + } + g.rebalance() +} + +// Adds a new member to the group and rebalances. +func (g *group) addMemberAndRebalance(m *groupMember, waitingReply *clientReq, join *kmsg.JoinGroupRequest) { + g.stopPending(m) + m.join = join + for _, p := range m.join.Protocols { + g.protocols[p.Name]++ + } + g.members[m.memberID] = m + g.nJoining++ + m.waitingReply = waitingReply + g.rebalance() +} + +// Returns if a new join can even join the group based on the join's supported +// protocols. +func (g *group) protocolsMatch(protocolType string, protocols []kmsg.JoinGroupRequestProtocol) bool { + if g.protocolType == "" { + if protocolType == "" || len(protocols) == 0 { + return false + } + g.protocolType = protocolType + return true + } + if protocolType != g.protocolType { + return false + } + if len(g.protocols) == 0 { + return true + } + for _, p := range protocols { + if _, ok := g.protocols[p.Name]; ok { + return true + } + } + return false +} + +// Returns if a new join request is the same as an old request; if so, for +// non-leaders, we just return the old join response. +func (m *groupMember) sameJoin(req *kmsg.JoinGroupRequest) bool { + if len(m.join.Protocols) != len(req.Protocols) { + return false + } + for i := range m.join.Protocols { + if m.join.Protocols[i].Name != req.Protocols[i].Name { + return false + } + if !bytes.Equal(m.join.Protocols[i].Metadata, req.Protocols[i].Metadata) { + return false + } + } + return true +} + +func (g *group) fillJoinResp(req *kmsg.JoinGroupRequest, resp *kmsg.JoinGroupResponse) { + resp.Generation = g.generation + resp.ProtocolType = kmsg.StringPtr(g.protocolType) + resp.Protocol = kmsg.StringPtr(g.protocol) + resp.LeaderID = g.leader + resp.MemberID = req.MemberID + if g.leader == req.MemberID { + resp.Members = g.joinResponseMetadata() + } +} + +func (g *group) joinResponseMetadata() []kmsg.JoinGroupResponseMember { + metadata := make([]kmsg.JoinGroupResponseMember, 0, len(g.members)) +members: + for _, m := range g.members { + for _, p := range m.join.Protocols { + if p.Name == g.protocol { + metadata = append(metadata, kmsg.JoinGroupResponseMember{ + MemberID: m.memberID, + ProtocolMetadata: p.Metadata, + }) + continue members + } + } + panic("inconsistent group protocol within saved members") + } + return metadata +} + +func (g *group) reply(creq *clientReq, kresp kmsg.Response, m *groupMember) { + select { + case creq.cc.respCh <- clientResp{kresp: kresp, corr: creq.corr, seq: creq.seq}: + case <-g.c.die: + return + } + if m != nil { + m.waitingReply = nil + g.updateHeartbeat(m) + } +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/logger.go b/vendor/github.com/twmb/franz-go/pkg/kfake/logger.go new file mode 100644 index 000000000000..cc674bc8b7c3 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/logger.go @@ -0,0 +1,65 @@ +package kfake + +import ( + "fmt" + "io" +) + +// LogLevel designates which level the logger should log at. +type LogLevel int8 + +const ( + // LogLevelNone disables logging. + LogLevelNone LogLevel = iota + // LogLevelError logs all errors. Generally, these should not happen. + LogLevelError + // LogLevelWarn logs all warnings, such as request failures. + LogLevelWarn + // LogLevelInfo logs informational messages, such as requests. This is + // usually the default log level. + LogLevelInfo + // LogLevelDebug logs verbose information, and is usually not used in + // production. + LogLevelDebug +) + +func (l LogLevel) String() string { + switch l { + case LogLevelError: + return "ERR" + case LogLevelWarn: + return "WRN" + case LogLevelInfo: + return "INF" + case LogLevelDebug: + return "DBG" + default: + return "NON" + } +} + +// Logger can be provided to hook into the fake cluster's logs. +type Logger interface { + Logf(LogLevel, string, ...any) +} + +type nopLogger struct{} + +func (*nopLogger) Logf(LogLevel, string, ...any) {} + +// BasicLogger returns a logger that writes newline delimited messages to dst. +func BasicLogger(dst io.Writer, level LogLevel) Logger { + return &basicLogger{dst, level} +} + +type basicLogger struct { + dst io.Writer + level LogLevel +} + +func (b *basicLogger) Logf(level LogLevel, msg string, args ...any) { + if b.level < level { + return + } + fmt.Fprintf(b.dst, "[%s] "+msg+"\n", append([]any{level}, args...)...) +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/main.go b/vendor/github.com/twmb/franz-go/pkg/kfake/main.go new file mode 100644 index 000000000000..6d6596c9d551 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/main.go @@ -0,0 +1,31 @@ +//go:build none + +package main + +import ( + "fmt" + "os" + "os/signal" + + "github.com/twmb/franz-go/pkg/kfake" +) + +func main() { + c, err := kfake.NewCluster( + kfake.Ports(9092, 9093, 9094), + kfake.SeedTopics(-1, "foo"), + ) + if err != nil { + panic(err) + } + defer c.Close() + + addrs := c.ListenAddrs() + for _, addr := range addrs { + fmt.Println(addr) + } + + sigs := make(chan os.Signal, 2) + signal.Notify(sigs, os.Interrupt) + <-sigs +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/misc.go b/vendor/github.com/twmb/franz-go/pkg/kfake/misc.go new file mode 100644 index 000000000000..75f2f2449947 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/misc.go @@ -0,0 +1,63 @@ +package kfake + +import ( + "crypto/rand" + "crypto/sha256" + "encoding/binary" + "fmt" + "io" + "sync" +) + +func randFill(slice []byte) { + randPoolFill(slice) +} + +func randBytes(n int) []byte { + r := make([]byte, n) + randPoolFill(r) + return r +} + +func randUUID() [16]byte { + var uuid [16]byte + randPoolFill(uuid[:]) + return uuid +} + +func randStrUUID() string { + uuid := randUUID() + return fmt.Sprintf("%x", uuid[:]) +} + +func hashString(s string) uint64 { + sum := sha256.Sum256([]byte(s)) + var n uint64 + for i := 0; i < 4; i++ { + v := binary.BigEndian.Uint64(sum[i*8:]) + n ^= v + } + return n +} + +var ( + mu sync.Mutex + randPool = make([]byte, 4<<10) + randPoolAt = len(randPool) +) + +func randPoolFill(into []byte) { + mu.Lock() + defer mu.Unlock() + for len(into) != 0 { + n := copy(into, randPool[randPoolAt:]) + into = into[n:] + randPoolAt += n + if randPoolAt == cap(randPool) { + if _, err := io.ReadFull(rand.Reader, randPool); err != nil { + panic(fmt.Sprintf("unable to read %d bytes from crypto/rand: %v", len(randPool), err)) + } + randPoolAt = 0 + } + } +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/pid.go b/vendor/github.com/twmb/franz-go/pkg/kfake/pid.go new file mode 100644 index 000000000000..eacf2cf87bd1 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/pid.go @@ -0,0 +1,96 @@ +package kfake + +import ( + "hash/fnv" + "math" + "math/rand" +) + +// TODO +// +// * Convert pids to struct, add heap of last use, add index to pidseqs, and +// remove pidseqs as they exhaust max # of pids configured. +// +// * Wrap epochs + +type ( + pids map[int64]*pidMap + + pidMap struct { + id int64 + epoch int16 + tps tps[pidseqs] + } + + pid struct { + id int64 + epoch int16 + } + + pidseqs struct { + seqs [5]int32 + at uint8 + } +) + +func (pids *pids) get(id int64, epoch int16, t string, p int32) (*pidseqs, int16) { + if *pids == nil { + return nil, 0 + } + pm := (*pids)[id] + if pm == nil { + return nil, 0 + } + return pm.tps.mkpDefault(t, p), pm.epoch +} + +func (pids *pids) create(txnalID *string) pid { + if *pids == nil { + *pids = make(map[int64]*pidMap) + } + var id int64 + if txnalID != nil { + hasher := fnv.New64() + hasher.Write([]byte(*txnalID)) + id = int64(hasher.Sum64()) & math.MaxInt64 + } else { + for { + id = int64(rand.Uint64()) & math.MaxInt64 + if _, exists := (*pids)[id]; !exists { + break + } + } + } + pm, exists := (*pids)[id] + if exists { + pm.epoch++ + return pid{id, pm.epoch} + } + pm = &pidMap{id: id} + (*pids)[id] = pm + return pid{id, 0} +} + +func (seqs *pidseqs) pushAndValidate(firstSeq, numRecs int32) (ok, dup bool) { + // If there is no pid, we do not do duplicate detection. + if seqs == nil { + return true, false + } + var ( + seq = firstSeq + seq64 = int64(seq) + next64 = (seq64 + int64(numRecs)) % math.MaxInt32 + next = int32(next64) + ) + for i := 0; i < 5; i++ { + if seqs.seqs[i] == seq && seqs.seqs[(i+1)%5] == next { + return true, true + } + } + if seqs.seqs[seqs.at] != seq { + return false, false + } + seqs.at = (seqs.at + 1) % 5 + seqs.seqs[seqs.at] = next + return true, false +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/sasl.go b/vendor/github.com/twmb/franz-go/pkg/kfake/sasl.go new file mode 100644 index 000000000000..413bb3bd1c77 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/sasl.go @@ -0,0 +1,296 @@ +package kfake + +import ( + "bytes" + "crypto/hmac" + "crypto/sha256" + "crypto/sha512" + "encoding/base64" + "errors" + "fmt" + "regexp" + "strings" + + "github.com/twmb/franz-go/pkg/kmsg" + "golang.org/x/crypto/pbkdf2" +) + +// TODO server-error-value in serverFinal + +const ( + saslPlain = "PLAIN" + saslScram256 = "SCRAM-SHA-256" + saslScram512 = "SCRAM-SHA-512" + scramIterations = 4096 +) + +type ( + sasls struct { + plain map[string]string // user => pass + scram256 map[string]scramAuth // user => scram auth + scram512 map[string]scramAuth // user => scram auth + } + + saslStage uint8 +) + +func (s sasls) empty() bool { + return len(s.plain) == 0 && len(s.scram256) == 0 && len(s.scram512) == 0 +} + +const ( + saslStageBegin saslStage = iota + saslStageAuthPlain + saslStageAuthScram0_256 + saslStageAuthScram0_512 + saslStageAuthScram1 + saslStageComplete +) + +func (c *Cluster) handleSASL(creq *clientReq) (allow bool) { + switch creq.cc.saslStage { + case saslStageBegin: + switch creq.kreq.(type) { + case *kmsg.ApiVersionsRequest, + *kmsg.SASLHandshakeRequest: + return true + default: + return false + } + case saslStageAuthPlain, + saslStageAuthScram0_256, + saslStageAuthScram0_512, + saslStageAuthScram1: + switch creq.kreq.(type) { + case *kmsg.ApiVersionsRequest, + *kmsg.SASLAuthenticateRequest: + return true + default: + return false + } + case saslStageComplete: + return true + default: + panic("unreachable") + } +} + +/////////// +// PLAIN // +/////////// + +func saslSplitPlain(auth []byte) (user, pass string, err error) { + parts := strings.SplitN(string(auth), "\x00", 3) + if len(parts) != 3 { + return "", "", errors.New("invalid plain auth") + } + if len(parts[0]) != 0 && parts[0] != parts[1] { + return "", "", errors.New("authzid is not equal to username") // see below + } + return parts[1], parts[2], nil +} + +/////////// +// SCRAM // +/////////// + +func newScramAuth(mechanism, pass string) scramAuth { + var saltedPass []byte + salt := randBytes(10) + switch mechanism { + case saslScram256: + saltedPass = pbkdf2.Key([]byte(pass), salt, scramIterations, sha256.Size, sha256.New) + case saslScram512: + saltedPass = pbkdf2.Key([]byte(pass), salt, scramIterations, sha512.Size, sha512.New) + default: + panic("unreachable") + } + return scramAuth{ + mechanism: mechanism, + iterations: scramIterations, + saltedPass: saltedPass, + salt: salt, + } +} + +type scramAuth struct { + mechanism string // scram 256 or 512 + iterations int + saltedPass []byte + salt []byte +} + +// client-first-message +type scramClient0 struct { + user string + bare []byte // client-first-message-bare + nonce []byte // nonce in client0 +} + +var scramUnescaper = strings.NewReplacer("=3D", "=", "=2C", ",") + +func scramParseClient0(client0 []byte) (scramClient0, error) { + m := reClient0.FindSubmatch(client0) + if len(m) == 0 { + return scramClient0{}, errors.New("invalid client0") + } + var ( + zid = string(m[1]) + bare = bytes.Clone(m[2]) + user = string(m[3]) + nonce = bytes.Clone(m[4]) + ext = string(m[5]) + ) + if len(ext) != 0 { + return scramClient0{}, errors.New("invalid extensions") + } + if zid != "" && zid != user { + return scramClient0{}, errors.New("authzid is not equal to username") // Kafka & Redpanda enforce that a present zid == username + } + return scramClient0{ + user: scramUnescaper.Replace(user), + bare: bare, + nonce: nonce, + }, nil +} + +func scramServerFirst(client0 scramClient0, auth scramAuth) (scramServer0, []byte) { + nonce := append(client0.nonce, base64.RawStdEncoding.EncodeToString(randBytes(16))...) + serverFirst := []byte(fmt.Sprintf("r=%s,s=%s,i=%d", + nonce, + base64.StdEncoding.EncodeToString(auth.salt), + scramIterations, + )) + return scramServer0{ + a: auth, + c0bare: client0.bare, + s0: serverFirst, + }, serverFirst +} + +// server-first-message +type scramServer0 struct { + a scramAuth + c0bare []byte + s0 []byte +} + +// validates client-final-message and replies with server-final-message +func (s *scramServer0) serverFinal(clientFinal []byte) ([]byte, error) { + m := reClientFinal.FindSubmatch(clientFinal) + if len(m) == 0 { + return nil, errors.New("invalid client-final-message") + } + var ( + finalWithoutProof = m[1] + channel = m[2] + clientProof64 = m[3] + h = sha256.New + ) + if s.a.mechanism == saslScram512 { + h = sha512.New + } + if !bytes.Equal(channel, []byte("biws")) { // "biws" == base64("n,,") + return nil, errors.New("invalid channel binding") + } + clientProof, err := base64.StdEncoding.DecodeString(string(clientProof64)) + if err != nil { + return nil, errors.New("client proof is not std-base64") + } + if len(clientProof) != h().Size() { + return nil, fmt.Errorf("len(client proof) %d != expected %d", len(clientProof), h().Size()) + } + + var clientKey []byte // := HMAC(SaltedPass, "Client Key") + { + mac := hmac.New(h, s.a.saltedPass) + mac.Write([]byte("Client Key")) + clientKey = mac.Sum(nil) + } + + var storedKey []byte // := H(ClientKey) + { + h := h() + h.Write(clientKey) + storedKey = h.Sum(nil) + } + + var authMessage []byte // := client-first-bare-message + "," + server-first-message + "," + client-final-message-without-proof + { + authMessage = append(s.c0bare, ',') + authMessage = append(authMessage, s.s0...) + authMessage = append(authMessage, ',') + authMessage = append(authMessage, finalWithoutProof...) + } + + var clientSignature []byte // := HMAC(StoredKey, AuthMessage) + { + mac := hmac.New(h, storedKey) + mac.Write(authMessage) + clientSignature = mac.Sum(nil) + } + + usedKey := clientProof // := ClientKey XOR ClientSignature + { + for i, b := range clientSignature { + usedKey[i] ^= b + } + h := h() + h.Write(usedKey) + usedKey = h.Sum(nil) + } + if !bytes.Equal(usedKey, storedKey) { + return nil, errors.New("invalid password") + } + + var serverKey []byte // := HMAC(SaltedPass, "Server Key") + { + mac := hmac.New(h, s.a.saltedPass) + mac.Write([]byte("Server Key")) + serverKey = mac.Sum(nil) + } + var serverSignature []byte // := HMAC(ServerKey, AuthMessage) + { + mac := hmac.New(h, serverKey) + mac.Write(authMessage) + serverSignature = mac.Sum(nil) + } + + serverFinal := []byte(fmt.Sprintf("v=%s", base64.StdEncoding.EncodeToString(serverSignature))) + return serverFinal, nil +} + +var reClient0, reClientFinal *regexp.Regexp + +func init() { + // https://datatracker.ietf.org/doc/html/rfc5802#section-7 + const ( + valueSafe = "[\x01-\x2b\x2d-\x3c\x3e-\x7f]+" // all except \0 - , + value = "[\x01-\x2b\x2d-\x7f]+" // all except \0 , + printable = "[\x21-\x2b\x2d-\x7e]+" // all except , (and DEL, unnoted) + saslName = "(?:[\x01-\x2b\x2d-\x3c\x3e-\x7f]|=2C|=3D)+" // valueSafe | others; kafka is lazy here + b64 = `[a-zA-Z0-9/+]+={0,3}` // we are lazy here matching up to 3 = + ext = "(?:,[a-zA-Z]+=[\x01-\x2b\x2d-\x7f]+)*" + ) + + // 0: entire match + // 1: authzid + // 2: client-first-message-bare + // 3: username + // 4: nonce + // 5: ext + client0 := fmt.Sprintf("^n,(?:a=(%s))?,((?:m=%s,)?n=(%s),r=(%s)(%s))$", saslName, value, saslName, printable, ext) + + // We reject extensions in client0. Kafka does not validate the nonce + // and some clients may generate it incorrectly (i.e. old franz-go), so + // we do not validate it. + // + // 0: entire match + // 1: channel-final-message-without-proof + // 2: channel binding + // 3: proof + clientFinal := fmt.Sprintf("^(c=(%s),r=%s),p=(%s)$", b64, printable, b64) + + reClient0 = regexp.MustCompile(client0) + reClientFinal = regexp.MustCompile(clientFinal) +} diff --git a/vendor/github.com/twmb/franz-go/pkg/kfake/topic_partition.go b/vendor/github.com/twmb/franz-go/pkg/kfake/topic_partition.go new file mode 100644 index 000000000000..ac409c53e7d6 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/pkg/kfake/topic_partition.go @@ -0,0 +1,75 @@ +package kfake + +type tps[V any] map[string]map[int32]*V + +func (tps *tps[V]) getp(t string, p int32) (*V, bool) { + if *tps == nil { + return nil, false + } + ps := (*tps)[t] + if ps == nil { + return nil, false + } + v, ok := ps[p] + return v, ok +} + +func (tps *tps[V]) gett(t string) (map[int32]*V, bool) { + if tps == nil { + return nil, false + } + ps, ok := (*tps)[t] + return ps, ok +} + +func (tps *tps[V]) mkt(t string) map[int32]*V { + if *tps == nil { + *tps = make(map[string]map[int32]*V) + } + ps := (*tps)[t] + if ps == nil { + ps = make(map[int32]*V) + (*tps)[t] = ps + } + return ps +} + +func (tps *tps[V]) mkp(t string, p int32, newFn func() *V) *V { + ps := tps.mkt(t) + v, ok := ps[p] + if !ok { + v = newFn() + ps[p] = v + } + return v +} + +func (tps *tps[V]) mkpDefault(t string, p int32) *V { + return tps.mkp(t, p, func() *V { return new(V) }) +} + +func (tps *tps[V]) set(t string, p int32, v V) { + *tps.mkpDefault(t, p) = v +} + +func (tps *tps[V]) each(fn func(t string, p int32, v *V)) { + for t, ps := range *tps { + for p, v := range ps { + fn(t, p, v) + } + } +} + +func (tps *tps[V]) delp(t string, p int32) { + if *tps == nil { + return + } + ps := (*tps)[t] + if ps == nil { + return + } + delete(ps, p) + if len(ps) == 0 { + delete(*tps, t) + } +} diff --git a/vendor/github.com/twmb/franz-go/plugin/kotel/LICENSE b/vendor/github.com/twmb/franz-go/plugin/kotel/LICENSE new file mode 100644 index 000000000000..36e18034325d --- /dev/null +++ b/vendor/github.com/twmb/franz-go/plugin/kotel/LICENSE @@ -0,0 +1,24 @@ +Copyright 2020, Travis Bischel. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of the library nor the + names of its contributors may be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY +DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/twmb/franz-go/plugin/kotel/README.md b/vendor/github.com/twmb/franz-go/plugin/kotel/README.md new file mode 100644 index 000000000000..4ed1b5ae781c --- /dev/null +++ b/vendor/github.com/twmb/franz-go/plugin/kotel/README.md @@ -0,0 +1,194 @@ +kotel +=== + +Kotel is an OpenTelemetry instrumentation plug-in package for franz-go. It +provides [tracing](https://pkg.go.dev/go.opentelemetry.io/otel/trace) +and [metrics](https://pkg.go.dev/go.opentelemetry.io/otel/metric) options +through +a [`kgo.Hook`](https://pkg.go.dev/github.com/twmb/franz-go/pkg/kgo#Hook). With +kotel, you can trace records produced or consumed with franz-go. You can pass +parent traces into records and extract parent traces from records. It also +tracks metrics related to connections, errors, and bytes transferred. + +To learn more about how to use kotel, see the usage sections in the README and +refer to the [OpenTelemetry documentation](https://opentelemetry.io/docs) for +additional information about OpenTelemetry and how it can be used in your +franz-go projects. + +## Tracing + +kotel provides tracing capabilities for Kafka using OpenTelemetry +specifications. It allows for the creation of three different span +operations: "publish", "receive", and "process". Additionally, it also provides +a set of attributes to use with these spans. + +### How it works + +The kotel tracer module uses hooks to automatically create and close "publish" +and "receive" spans as a `kgo.Record` flows through the application. However, +for the "process" span, it uses a convenience method that must be manually +invoked and closed in the consumer code to capture processing. + +The following table provides a visual representation of the lineage of the +span operations: + +| Order | Hook/Method | Operation | State | +|-------|---------------------------------|-----------|-------| +| 1 | kgo.HookProduceRecordBuffered | Publish | Start | +| 2 | kgo.HookProduceRecordUnbuffered | Publish | End | +| 3 | kgo.HookFetchRecordBuffered | Receive | Start | +| 4 | kgo.HookFetchRecordUnbuffered | Receive | End | +| 5 | kotel.Tracer.WithProcessSpan | Process | Start | + +### Getting started + +To start using kotel for tracing, you will need to: + +1. Set up a tracer provider +2. Configure any desired tracer options +3. Create a new kotel tracer +4. Create a new kotel service hook +5. Create a new Kafka client and pass in the kotel hook + +Here's an example of how you might do this: + +```go +// Initialize tracer provider. +tracerProvider, err := initTracerProvider() + +// Create a new kotel tracer. +tracerOpts := []kotel.TracerOpt{ + kotel.TracerProvider(tracerProvider), + kotel.TracerPropagator(propagation.NewCompositeTextMapPropagator(propagation.TraceContext{})), +} +tracer := kotel.NewTracer(tracerOpts...) + +// Create a new kotel service. +kotelOps := []kotel.Opt{ + kotel.WithTracer(tracer), +} +kotelService := kotel.NewKotel(kotelOps...) + +// Create a new Kafka client. +cl, err := kgo.NewClient( + // Pass in the kotel hook. + kgo.WithHooks(kotelService.Hooks()...), + // ...other opts. +) +``` + +### Sending records + +When producing a record with franz-go, it will traced by kotel. To include +parent traces, pass in an instrumented context. + +Here's an example of how to do this: + +```go +func httpHandler(w http.ResponseWriter, r *http.Request) { + // Start a new span with options. + opts := []trace.SpanStartOption{ + trace.WithSpanKind(trace.SpanKindServer), + trace.WithAttributes([]attribute.KeyValue{attribute.String("some-key", "foo")}...), + } + ctx, span := tracer.Start(r.Context(), "request", opts...) + // End the span when function exits. + defer span.End() + + var wg sync.WaitGroup + wg.Add(1) + record := &kgo.Record{Topic: "topic", Value: []byte("foo")} + // Pass in the context from the tracer.Start() call to ensure that the span + // created is linked to the parent span. + cl.Produce(ctx, record, func(_ *kgo.Record, err error) { + defer wg.Done() + if err != nil { + fmt.Printf("record had a produce error: %v\n", err) + span.SetStatus(codes.Error, err.Error()) + span.RecordError(err) + } + }) + wg.Wait() +} +``` + +### Processing Records + +Use the `kotel.Tracer.WithProcessSpan` method to start a "process" span. Make +sure to end the span after you finish processing the record. The trace can be +continued to the next processing step if desired. + +Here is an example of how you might do this: + +```go +func processRecord(record *kgo.Record, tracer *kotel.Tracer) { + ctx, span := tracer.WithProcessSpan(record) + // Process the record here. + // End the span when function exits. + defer span.End() + // optionally pass the context to the next processing step. + fmt.Printf( + "processed offset '%s' with key '%s' and value '%s'\n", + strconv.FormatInt(record.Offset, 10), + string(record.Key), + string(record.Value), + ) +} +``` + +## Metrics + +The kotel meter module tracks various metrics related to the processing of +records, such as the number of successful and unsuccessful connections, bytes +written and read, and the number of buffered records. These metrics are all +counters and are tracked under the following names: + +``` +messaging.kafka.connects.count{node_id = "#{node}"} +messaging.kafka.connect_errors.count{node_id = "#{node}"} +messaging.kafka.disconnects.count{node_id = "#{node}"} +messaging.kafka.write_errors.count{node_id = "#{node}"} +messaging.kafka.write_bytes{node_id = "#{node}"} +messaging.kafka.read_errors.count{node_id = "#{node}"} +messaging.kafka.read_bytes.count{node_id = "#{node}"} +messaging.kafka.produce_bytes.count{node_id = "#{node}", topic = "#{topic}"} +messaging.kafka.produce_records.count{node_id = "#{node}", topic = "#{topic}"} +messaging.kafka.fetch_bytes.count{node_id = "#{node}", topic = "#{topic}"} +messaging.kafka.fetch_records.count{node_id = "#{node}", topic = "#{topic}"} +``` + +### Getting started + +To start using kotel for metrics, you will need to: + +1. Set up a meter provider +2. Configure any desired meter options +3. Create a new kotel meter +4. Create a new kotel service hook +5. Create a new Kafka client and pass in the kotel hook + +Here's an example of how you might do this: + +```go +// Initialize meter provider. +meterProvider, err := initMeterProvider() + +// Create a new kotel meter. +meterOpts := []kotel.MeterOpt{kotel.MeterProvider(meterProvider)} +meter := kotel.NewMeter(meterOpts...) + +// Pass the meter to NewKotel hook. +kotelOps := []kotel.Opt{ + kotel.WithMeter(meter), +} + +// Create a new kotel service. +kotelService := kotel.NewKotel(kotelOps...) + +// Create a new Kafka client. +cl, err := kgo.NewClient( + // Pass in the kotel hook. + kgo.WithHooks(kotelService.Hooks()...), + // ...other opts. +) +``` diff --git a/vendor/github.com/twmb/franz-go/plugin/kotel/carrier.go b/vendor/github.com/twmb/franz-go/plugin/kotel/carrier.go new file mode 100644 index 000000000000..e851ab6f9fa7 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/plugin/kotel/carrier.go @@ -0,0 +1,53 @@ +package kotel + +import ( + "github.com/twmb/franz-go/pkg/kgo" +) + +// RecordCarrier injects and extracts traces from a kgo.Record. +// +// This type exists to satisfy the otel/propagation.TextMapCarrier interface. +type RecordCarrier struct { + record *kgo.Record +} + +// NewRecordCarrier creates a new RecordCarrier. +func NewRecordCarrier(record *kgo.Record) RecordCarrier { + return RecordCarrier{record: record} +} + +// Get retrieves a single value for a given key if it exists. +func (c RecordCarrier) Get(key string) string { + for _, h := range c.record.Headers { + if h.Key == key { + return string(h.Value) + } + } + return "" +} + +// Set sets a header. +func (c RecordCarrier) Set(key, val string) { + // Check if key already exists. + for i, h := range c.record.Headers { + if h.Key == key { + // Key exist, update the value. + c.record.Headers[i].Value = []byte(val) + return + } + } + // Key does not exist, append new header. + c.record.Headers = append(c.record.Headers, kgo.RecordHeader{ + Key: key, + Value: []byte(val), + }) +} + +// Keys returns a slice of all key identifiers in the carrier. +func (c RecordCarrier) Keys() []string { + out := make([]string, len(c.record.Headers)) + for i, h := range c.record.Headers { + out[i] = h.Key + } + return out +} diff --git a/vendor/github.com/twmb/franz-go/plugin/kotel/kotel.go b/vendor/github.com/twmb/franz-go/plugin/kotel/kotel.go new file mode 100644 index 000000000000..47e443b8ea42 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/plugin/kotel/kotel.go @@ -0,0 +1,61 @@ +package kotel + +import ( + "github.com/twmb/franz-go/pkg/kgo" +) + +const ( + instrumentationName = "github.com/twmb/franz-go/plugin/kotel" +) + +// Kotel represents the configuration options available for the kotel plugin. +type Kotel struct { + meter *Meter + tracer *Tracer +} + +// Opt interface used for setting optional kotel properties. +type Opt interface{ apply(*Kotel) } + +type optFunc func(*Kotel) + +func (o optFunc) apply(c *Kotel) { o(c) } + +// WithTracer configures Kotel with a Tracer. +func WithTracer(t *Tracer) Opt { + return optFunc(func(k *Kotel) { + if t != nil { + k.tracer = t + } + }) +} + +// WithMeter configures Kotel with a Meter. +func WithMeter(m *Meter) Opt { + return optFunc(func(k *Kotel) { + if m != nil { + k.meter = m + } + }) +} + +// Hooks return a list of kgo.hooks compatible with its interface. +func (k *Kotel) Hooks() []kgo.Hook { + var hooks []kgo.Hook + if k.tracer != nil { + hooks = append(hooks, k.tracer) + } + if k.meter != nil { + hooks = append(hooks, k.meter) + } + return hooks +} + +// NewKotel creates a new Kotel struct and applies opts to it. +func NewKotel(opts ...Opt) *Kotel { + k := &Kotel{} + for _, opt := range opts { + opt.apply(k) + } + return k +} diff --git a/vendor/github.com/twmb/franz-go/plugin/kotel/meter.go b/vendor/github.com/twmb/franz-go/plugin/kotel/meter.go new file mode 100644 index 000000000000..8c56a8c7002d --- /dev/null +++ b/vendor/github.com/twmb/franz-go/plugin/kotel/meter.go @@ -0,0 +1,375 @@ +package kotel + +import ( + "context" + "log" + "math" + "net" + "strconv" + "time" + + "github.com/twmb/franz-go/pkg/kgo" + "go.opentelemetry.io/otel" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/metric" + semconv "go.opentelemetry.io/otel/semconv/v1.18.0" +) + +var ( // interface checks to ensure we implement the hooks properly + _ kgo.HookBrokerConnect = new(Meter) + _ kgo.HookBrokerDisconnect = new(Meter) + _ kgo.HookBrokerWrite = new(Meter) + _ kgo.HookBrokerRead = new(Meter) + _ kgo.HookProduceBatchWritten = new(Meter) + _ kgo.HookFetchBatchRead = new(Meter) +) + +const ( + dimensionless = "1" + bytes = "by" +) + +type Meter struct { + provider metric.MeterProvider + meter metric.Meter + instruments instruments + + mergeConnectsMeter bool +} + +// MeterOpt interface used for setting optional config properties. +type MeterOpt interface { + apply(*Meter) +} + +type meterOptFunc func(*Meter) + +// MeterProvider takes a metric.MeterProvider and applies it to the Meter +// If none is specified, the global provider is used. +func MeterProvider(provider metric.MeterProvider) MeterOpt { + return meterOptFunc(func(m *Meter) { + if provider != nil { + m.provider = provider + } + }) +} + +// WithMergedConnectsMeter merges the `messaging.kafka.connect_errors.count` +// counter into the `messaging.kafka.connects.count` counter, adding an +// attribute "outcome" with the values "success" or "failure". This option +// shall be used when a single metric with different dimensions is preferred +// over two separate metrics that produce data at alternating intervals. +// For example, it becomes possible to alert on the metric no longer +// producing data. +func WithMergedConnectsMeter() MeterOpt { + return meterOptFunc(func(m *Meter) { + m.mergeConnectsMeter = true + }) + +} + +func (o meterOptFunc) apply(m *Meter) { + o(m) +} + +// NewMeter returns a Meter, used as option for kotel to instrument franz-go +// with instruments. +func NewMeter(opts ...MeterOpt) *Meter { + m := &Meter{} + for _, opt := range opts { + opt.apply(m) + } + if m.provider == nil { + m.provider = otel.GetMeterProvider() + } + m.meter = m.provider.Meter( + instrumentationName, + metric.WithInstrumentationVersion(semVersion()), + metric.WithSchemaURL(semconv.SchemaURL), + ) + m.instruments = m.newInstruments() + return m +} + +// instruments --------------------------------------------------------------- + +type instruments struct { + connects metric.Int64Counter + connectErrs metric.Int64Counter + disconnects metric.Int64Counter + + writeErrs metric.Int64Counter + writeBytes metric.Int64Counter + + readErrs metric.Int64Counter + readBytes metric.Int64Counter + + produceBytes metric.Int64Counter + produceRecords metric.Int64Counter + fetchBytes metric.Int64Counter + fetchRecords metric.Int64Counter +} + +func (m *Meter) newInstruments() instruments { + // connects and disconnects + connects, err := m.meter.Int64Counter( + "messaging.kafka.connects.count", + metric.WithUnit(dimensionless), + metric.WithDescription("Total number of connections opened, by broker"), + ) + if err != nil { + log.Printf("failed to create connects instrument, %v", err) + } + + var connectErrs metric.Int64Counter + if !m.mergeConnectsMeter { + var err error + connectErrs, err = m.meter.Int64Counter( + "messaging.kafka.connect_errors.count", + metric.WithUnit(dimensionless), + metric.WithDescription("Total number of connection errors, by broker"), + ) + if err != nil { + log.Printf("failed to create connectErrs instrument, %v", err) + } + } + + disconnects, err := m.meter.Int64Counter( + "messaging.kafka.disconnects.count", + metric.WithUnit(dimensionless), + metric.WithDescription("Total number of connections closed, by broker"), + ) + if err != nil { + log.Printf("failed to create disconnects instrument, %v", err) + } + + // write + + writeErrs, err := m.meter.Int64Counter( + "messaging.kafka.write_errors.count", + metric.WithUnit(dimensionless), + metric.WithDescription("Total number of write errors, by broker"), + ) + if err != nil { + log.Printf("failed to create writeErrs instrument, %v", err) + } + + writeBytes, err := m.meter.Int64Counter( + "messaging.kafka.write_bytes", + metric.WithUnit(bytes), + metric.WithDescription("Total number of bytes written, by broker"), + ) + if err != nil { + log.Printf("failed to create writeBytes instrument, %v", err) + } + + // read + + readErrs, err := m.meter.Int64Counter( + "messaging.kafka.read_errors.count", + metric.WithUnit(dimensionless), + metric.WithDescription("Total number of read errors, by broker"), + ) + if err != nil { + log.Printf("failed to create readErrs instrument, %v", err) + } + + readBytes, err := m.meter.Int64Counter( + "messaging.kafka.read_bytes.count", + metric.WithUnit(bytes), + metric.WithDescription("Total number of bytes read, by broker"), + ) + if err != nil { + log.Printf("failed to create readBytes instrument, %v", err) + } + + // produce & consume + + produceBytes, err := m.meter.Int64Counter( + "messaging.kafka.produce_bytes.count", + metric.WithUnit(bytes), + metric.WithDescription("Total number of uncompressed bytes produced, by broker and topic"), + ) + if err != nil { + log.Printf("failed to create produceBytes instrument, %v", err) + } + + produceRecords, err := m.meter.Int64Counter( + "messaging.kafka.produce_records.count", + metric.WithUnit(dimensionless), + metric.WithDescription("Total number of produced records, by broker and topic"), + ) + if err != nil { + log.Printf("failed to create produceRecords instrument, %v", err) + } + + fetchBytes, err := m.meter.Int64Counter( + "messaging.kafka.fetch_bytes.count", + metric.WithUnit(bytes), + metric.WithDescription("Total number of uncompressed bytes fetched, by broker and topic"), + ) + if err != nil { + log.Printf("failed to create fetchBytes instrument, %v", err) + } + + fetchRecords, err := m.meter.Int64Counter( + "messaging.kafka.fetch_records.count", + metric.WithUnit(dimensionless), + metric.WithDescription("Total number of fetched records, by broker and topic"), + ) + if err != nil { + log.Printf("failed to create fetchRecords instrument, %v", err) + } + + return instruments{ + connects: connects, + connectErrs: connectErrs, + disconnects: disconnects, + + writeErrs: writeErrs, + writeBytes: writeBytes, + + readErrs: readErrs, + readBytes: readBytes, + + produceBytes: produceBytes, + produceRecords: produceRecords, + fetchBytes: fetchBytes, + fetchRecords: fetchRecords, + } +} + +// Helpers ------------------------------------------------------------------- + +func strnode(node int32) string { + if node < 0 { + return "seed_" + strconv.Itoa(int(node)-math.MinInt32) + } + return strconv.Itoa(int(node)) +} + +// Hooks --------------------------------------------------------------------- + +func (m *Meter) OnBrokerConnect(meta kgo.BrokerMetadata, _ time.Duration, _ net.Conn, err error) { + node := strnode(meta.NodeID) + + if m.mergeConnectsMeter { + if err != nil { + m.instruments.connects.Add( + context.Background(), + 1, + metric.WithAttributeSet(attribute.NewSet( + attribute.String("node_id", node), + attribute.String("outcome", "failure"), + )), + ) + return + } + m.instruments.connects.Add( + context.Background(), + 1, + metric.WithAttributeSet(attribute.NewSet( + attribute.String("node_id", node), + attribute.String("outcome", "success"), + )), + ) + return + } + + attributes := attribute.NewSet(attribute.String("node_id", node)) + if err != nil { + m.instruments.connectErrs.Add( + context.Background(), + 1, + metric.WithAttributeSet(attributes), + ) + return + } + m.instruments.connects.Add( + context.Background(), + 1, + metric.WithAttributeSet(attributes), + ) +} + +func (m *Meter) OnBrokerDisconnect(meta kgo.BrokerMetadata, _ net.Conn) { + node := strnode(meta.NodeID) + attributes := attribute.NewSet(attribute.String("node_id", node)) + m.instruments.disconnects.Add( + context.Background(), + 1, + metric.WithAttributeSet(attributes), + ) +} + +func (m *Meter) OnBrokerWrite(meta kgo.BrokerMetadata, _ int16, bytesWritten int, _, _ time.Duration, err error) { + node := strnode(meta.NodeID) + attributes := attribute.NewSet(attribute.String("node_id", node)) + if err != nil { + m.instruments.writeErrs.Add( + context.Background(), + 1, + metric.WithAttributeSet(attributes), + ) + return + } + m.instruments.writeBytes.Add( + context.Background(), + int64(bytesWritten), + metric.WithAttributeSet(attributes), + ) +} + +func (m *Meter) OnBrokerRead(meta kgo.BrokerMetadata, _ int16, bytesRead int, _, _ time.Duration, err error) { + node := strnode(meta.NodeID) + attributes := attribute.NewSet(attribute.String("node_id", node)) + if err != nil { + m.instruments.readErrs.Add( + context.Background(), + 1, + metric.WithAttributeSet(attributes), + ) + return + } + m.instruments.readBytes.Add( + context.Background(), + int64(bytesRead), + metric.WithAttributeSet(attributes), + ) +} + +func (m *Meter) OnProduceBatchWritten(meta kgo.BrokerMetadata, topic string, _ int32, pbm kgo.ProduceBatchMetrics) { + node := strnode(meta.NodeID) + attributes := attribute.NewSet( + attribute.String("node_id", node), + attribute.String("topic", topic), + ) + m.instruments.produceBytes.Add( + context.Background(), + int64(pbm.UncompressedBytes), + metric.WithAttributeSet(attributes), + ) + m.instruments.produceRecords.Add( + context.Background(), + int64(pbm.NumRecords), + metric.WithAttributeSet(attributes), + ) +} + +func (m *Meter) OnFetchBatchRead(meta kgo.BrokerMetadata, topic string, _ int32, fbm kgo.FetchBatchMetrics) { + node := strnode(meta.NodeID) + attributes := attribute.NewSet( + attribute.String("node_id", node), + attribute.String("topic", topic), + ) + m.instruments.fetchBytes.Add( + context.Background(), + int64(fbm.UncompressedBytes), + metric.WithAttributeSet(attributes), + ) + m.instruments.fetchRecords.Add( + context.Background(), + int64(fbm.NumRecords), + metric.WithAttributeSet(attributes), + ) +} diff --git a/vendor/github.com/twmb/franz-go/plugin/kotel/tracer.go b/vendor/github.com/twmb/franz-go/plugin/kotel/tracer.go new file mode 100644 index 000000000000..0e3f6cf518b0 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/plugin/kotel/tracer.go @@ -0,0 +1,254 @@ +package kotel + +import ( + "context" + "unicode/utf8" + + "github.com/twmb/franz-go/pkg/kgo" + "go.opentelemetry.io/otel" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/codes" + "go.opentelemetry.io/otel/propagation" + semconv "go.opentelemetry.io/otel/semconv/v1.18.0" + "go.opentelemetry.io/otel/trace" +) + +var ( // interface checks to ensure we implement the hooks properly. + _ kgo.HookProduceRecordBuffered = new(Tracer) + _ kgo.HookProduceRecordUnbuffered = new(Tracer) + _ kgo.HookFetchRecordBuffered = new(Tracer) + _ kgo.HookFetchRecordUnbuffered = new(Tracer) +) + +type Tracer struct { + tracerProvider trace.TracerProvider + propagators propagation.TextMapPropagator + tracer trace.Tracer + clientID string + consumerGroup string + keyFormatter func(*kgo.Record) (string, error) +} + +// TracerOpt interface used for setting optional config properties. +type TracerOpt interface{ apply(*Tracer) } + +type tracerOptFunc func(*Tracer) + +func (o tracerOptFunc) apply(t *Tracer) { o(t) } + +// TracerProvider takes a trace.TracerProvider and applies it to the Tracer. +// If none is specified, the global provider is used. +func TracerProvider(provider trace.TracerProvider) TracerOpt { + return tracerOptFunc(func(t *Tracer) { t.tracerProvider = provider }) +} + +// TracerPropagator takes a propagation.TextMapPropagator and applies it to the +// Tracer. +// +// If none is specified, the global Propagator is used. +func TracerPropagator(propagator propagation.TextMapPropagator) TracerOpt { + return tracerOptFunc(func(t *Tracer) { t.propagators = propagator }) +} + +// ClientID sets the optional client_id attribute value. +func ClientID(id string) TracerOpt { + return tracerOptFunc(func(t *Tracer) { t.clientID = id }) +} + +// ConsumerGroup sets the optional group attribute value. +func ConsumerGroup(group string) TracerOpt { + return tracerOptFunc(func(t *Tracer) { t.consumerGroup = group }) +} + +// KeyFormatter formats a Record's key for use in a span's attributes, +// overriding the default of string(Record.Key). +// +// This option can be used to parse binary data and return a canonical string +// representation. If the returned string is not valid UTF-8 or if the +// formatter returns an error, the key is not attached to the span. +func KeyFormatter(fn func(*kgo.Record) (string, error)) TracerOpt { + return tracerOptFunc(func(t *Tracer) { t.keyFormatter = fn }) +} + +// NewTracer returns a Tracer, used as option for kotel to instrument franz-go +// with tracing. +func NewTracer(opts ...TracerOpt) *Tracer { + t := &Tracer{} + for _, opt := range opts { + opt.apply(t) + } + if t.tracerProvider == nil { + t.tracerProvider = otel.GetTracerProvider() + } + if t.propagators == nil { + t.propagators = otel.GetTextMapPropagator() + } + t.tracer = t.tracerProvider.Tracer( + instrumentationName, + trace.WithInstrumentationVersion(semVersion()), + trace.WithSchemaURL(semconv.SchemaURL), + ) + return t +} + +func (t *Tracer) maybeKeyAttr(attrs []attribute.KeyValue, r *kgo.Record) []attribute.KeyValue { + if r.Key == nil { + return attrs + } + var keykey string + if t.keyFormatter != nil { + k, err := t.keyFormatter(r) + if err != nil || !utf8.ValidString(k) { + return attrs + } + keykey = k + } else { + if !utf8.Valid(r.Key) { + return attrs + } + keykey = string(r.Key) + } + return append(attrs, semconv.MessagingKafkaMessageKeyKey.String(keykey)) +} + +// WithProcessSpan starts a new span for the "process" operation on a consumer +// record. +// +// It sets up the span options. The user's application code is responsible for +// ending the span. +// +// This should only ever be called within a polling loop of a consumed record and +// not a record which has been created for producing, so call this at the start of each +// iteration of your processing for the record. +func (t *Tracer) WithProcessSpan(r *kgo.Record) (context.Context, trace.Span) { + // Set up the span options. + attrs := []attribute.KeyValue{ + semconv.MessagingSystemKey.String("kafka"), + semconv.MessagingSourceKindTopic, + semconv.MessagingSourceName(r.Topic), + semconv.MessagingOperationProcess, + semconv.MessagingKafkaSourcePartition(int(r.Partition)), + semconv.MessagingKafkaMessageOffset(int(r.Offset)), + } + attrs = t.maybeKeyAttr(attrs, r) + if t.clientID != "" { + attrs = append(attrs, semconv.MessagingKafkaClientIDKey.String(t.clientID)) + } + if t.consumerGroup != "" { + attrs = append(attrs, semconv.MessagingKafkaConsumerGroupKey.String(t.consumerGroup)) + } + if r.Key != nil && r.Value == nil { + attrs = append(attrs, semconv.MessagingKafkaMessageTombstoneKey.Bool(true)) + } + opts := []trace.SpanStartOption{ + trace.WithAttributes(attrs...), + trace.WithSpanKind(trace.SpanKindConsumer), + } + + if r.Context == nil { + r.Context = context.Background() + } + // Start a new span using the provided context and options. + return t.tracer.Start(r.Context, r.Topic+" process", opts...) +} + +// Hooks ---------------------------------------------------------------------- + +// OnProduceRecordBuffered starts a new span for the "publish" operation on a +// buffered record. +// +// It sets span options and injects the span context into record and updates +// the record's context, so it can be ended in the OnProduceRecordUnbuffered +// hook. +func (t *Tracer) OnProduceRecordBuffered(r *kgo.Record) { + // Set up span options. + attrs := []attribute.KeyValue{ + semconv.MessagingSystemKey.String("kafka"), + semconv.MessagingDestinationKindTopic, + semconv.MessagingDestinationName(r.Topic), + semconv.MessagingOperationPublish, + } + attrs = t.maybeKeyAttr(attrs, r) + if t.clientID != "" { + attrs = append(attrs, semconv.MessagingKafkaClientIDKey.String(t.clientID)) + } + if r.Key != nil && r.Value == nil { + attrs = append(attrs, semconv.MessagingKafkaMessageTombstoneKey.Bool(true)) + } + opts := []trace.SpanStartOption{ + trace.WithAttributes(attrs...), + trace.WithSpanKind(trace.SpanKindProducer), + } + // Start the "publish" span. + ctx, _ := t.tracer.Start(r.Context, r.Topic+" publish", opts...) + // Inject the span context into the record. + t.propagators.Inject(ctx, NewRecordCarrier(r)) + // Update the record context. + r.Context = ctx +} + +// OnProduceRecordUnbuffered continues and ends the "publish" span for an +// unbuffered record. +// +// It sets attributes with values unset when producing and records any error +// that occurred during the publish operation. +func (t *Tracer) OnProduceRecordUnbuffered(r *kgo.Record, err error) { + span := trace.SpanFromContext(r.Context) + defer span.End() + span.SetAttributes( + semconv.MessagingKafkaDestinationPartition(int(r.Partition)), + ) + if err != nil { + span.SetStatus(codes.Error, err.Error()) + span.RecordError(err) + } +} + +// OnFetchRecordBuffered starts a new span for the "receive" operation on a +// buffered record. +// +// It sets the span options and extracts the span context from the record, +// updates the record's context to ensure it can be ended in the +// OnFetchRecordUnbuffered hook and can be used in downstream consumer +// processing. +func (t *Tracer) OnFetchRecordBuffered(r *kgo.Record) { + // Set up the span options. + attrs := []attribute.KeyValue{ + semconv.MessagingSystemKey.String("kafka"), + semconv.MessagingSourceKindTopic, + semconv.MessagingSourceName(r.Topic), + semconv.MessagingOperationReceive, + semconv.MessagingKafkaSourcePartition(int(r.Partition)), + } + attrs = t.maybeKeyAttr(attrs, r) + if t.clientID != "" { + attrs = append(attrs, semconv.MessagingKafkaClientIDKey.String(t.clientID)) + } + if t.consumerGroup != "" { + attrs = append(attrs, semconv.MessagingKafkaConsumerGroupKey.String(t.consumerGroup)) + } + if r.Key != nil && r.Value == nil { + attrs = append(attrs, semconv.MessagingKafkaMessageTombstoneKey.Bool(true)) + } + opts := []trace.SpanStartOption{ + trace.WithAttributes(attrs...), + trace.WithSpanKind(trace.SpanKindConsumer), + } + + if r.Context == nil { + r.Context = context.Background() + } + // Extract the span context from the record. + ctx := t.propagators.Extract(r.Context, NewRecordCarrier(r)) + // Start the "receive" span. + newCtx, _ := t.tracer.Start(ctx, r.Topic+" receive", opts...) + // Update the record context. + r.Context = newCtx +} + +// OnFetchRecordUnbuffered continues and ends the "receive" span for an +// unbuffered record. +func (t *Tracer) OnFetchRecordUnbuffered(r *kgo.Record, _ bool) { + span := trace.SpanFromContext(r.Context) + defer span.End() +} diff --git a/vendor/github.com/twmb/franz-go/plugin/kotel/version.go b/vendor/github.com/twmb/franz-go/plugin/kotel/version.go new file mode 100644 index 000000000000..0152aa7012c6 --- /dev/null +++ b/vendor/github.com/twmb/franz-go/plugin/kotel/version.go @@ -0,0 +1,21 @@ +package kotel + +import "runtime/debug" + +// version is the current release version of the kotel instrumentation. +func version() string { + info, ok := debug.ReadBuildInfo() + if ok { + for _, dep := range info.Deps { + if dep.Path == instrumentationName { + return dep.Version + } + } + } + return "unknown" +} + +// semVersion is the semantic version to be supplied to tracer/meter creation. +func semVersion() string { + return "semver:" + version() +} diff --git a/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/README.md b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/README.md new file mode 100644 index 000000000000..5f03e01386f2 --- /dev/null +++ b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/README.md @@ -0,0 +1,3 @@ +# Semconv v1.18.0 + +[![PkgGoDev](https://pkg.go.dev/badge/go.opentelemetry.io/otel/semconv/v1.18.0)](https://pkg.go.dev/go.opentelemetry.io/otel/semconv/v1.18.0) diff --git a/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/doc.go b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/doc.go new file mode 100644 index 000000000000..ff55fe79b588 --- /dev/null +++ b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/doc.go @@ -0,0 +1,9 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +// Package semconv implements OpenTelemetry semantic conventions. +// +// OpenTelemetry semantic conventions are agreed standardized naming +// patterns for OpenTelemetry things. This package represents the conventions +// as of the v1.18.0 version of the OpenTelemetry specification. +package semconv // import "go.opentelemetry.io/otel/semconv/v1.18.0" diff --git a/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/event.go b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/event.go new file mode 100644 index 000000000000..60ef182ffcd0 --- /dev/null +++ b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/event.go @@ -0,0 +1,188 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +// Code generated from semantic convention specification. DO NOT EDIT. + +package semconv // import "go.opentelemetry.io/otel/semconv/v1.18.0" + +import "go.opentelemetry.io/otel/attribute" + +// This semantic convention defines the attributes used to represent a feature +// flag evaluation as an event. +const ( + // FeatureFlagKeyKey is the attribute Key conforming to the + // "feature_flag.key" semantic conventions. It represents the unique + // identifier of the feature flag. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'logo-color' + FeatureFlagKeyKey = attribute.Key("feature_flag.key") + + // FeatureFlagProviderNameKey is the attribute Key conforming to the + // "feature_flag.provider_name" semantic conventions. It represents the + // name of the service provider that performs the flag evaluation. + // + // Type: string + // RequirementLevel: Recommended + // Stability: stable + // Examples: 'Flag Manager' + FeatureFlagProviderNameKey = attribute.Key("feature_flag.provider_name") + + // FeatureFlagVariantKey is the attribute Key conforming to the + // "feature_flag.variant" semantic conventions. It represents the sHOULD be + // a semantic identifier for a value. If one is unavailable, a stringified + // version of the value can be used. + // + // Type: string + // RequirementLevel: Recommended + // Stability: stable + // Examples: 'red', 'true', 'on' + // Note: A semantic identifier, commonly referred to as a variant, provides + // a means + // for referring to a value without including the value itself. This can + // provide additional context for understanding the meaning behind a value. + // For example, the variant `red` maybe be used for the value `#c05543`. + // + // A stringified version of the value can be used in situations where a + // semantic identifier is unavailable. String representation of the value + // should be determined by the implementer. + FeatureFlagVariantKey = attribute.Key("feature_flag.variant") +) + +// FeatureFlagKey returns an attribute KeyValue conforming to the +// "feature_flag.key" semantic conventions. It represents the unique identifier +// of the feature flag. +func FeatureFlagKey(val string) attribute.KeyValue { + return FeatureFlagKeyKey.String(val) +} + +// FeatureFlagProviderName returns an attribute KeyValue conforming to the +// "feature_flag.provider_name" semantic conventions. It represents the name of +// the service provider that performs the flag evaluation. +func FeatureFlagProviderName(val string) attribute.KeyValue { + return FeatureFlagProviderNameKey.String(val) +} + +// FeatureFlagVariant returns an attribute KeyValue conforming to the +// "feature_flag.variant" semantic conventions. It represents the sHOULD be a +// semantic identifier for a value. If one is unavailable, a stringified +// version of the value can be used. +func FeatureFlagVariant(val string) attribute.KeyValue { + return FeatureFlagVariantKey.String(val) +} + +// RPC received/sent message. +const ( + // MessageTypeKey is the attribute Key conforming to the "message.type" + // semantic conventions. It represents the whether this is a received or + // sent message. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + MessageTypeKey = attribute.Key("message.type") + + // MessageIDKey is the attribute Key conforming to the "message.id" + // semantic conventions. It represents the mUST be calculated as two + // different counters starting from `1` one for sent messages and one for + // received message. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Note: This way we guarantee that the values will be consistent between + // different implementations. + MessageIDKey = attribute.Key("message.id") + + // MessageCompressedSizeKey is the attribute Key conforming to the + // "message.compressed_size" semantic conventions. It represents the + // compressed size of the message in bytes. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + MessageCompressedSizeKey = attribute.Key("message.compressed_size") + + // MessageUncompressedSizeKey is the attribute Key conforming to the + // "message.uncompressed_size" semantic conventions. It represents the + // uncompressed size of the message in bytes. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + MessageUncompressedSizeKey = attribute.Key("message.uncompressed_size") +) + +var ( + // sent + MessageTypeSent = MessageTypeKey.String("SENT") + // received + MessageTypeReceived = MessageTypeKey.String("RECEIVED") +) + +// MessageID returns an attribute KeyValue conforming to the "message.id" +// semantic conventions. It represents the mUST be calculated as two different +// counters starting from `1` one for sent messages and one for received +// message. +func MessageID(val int) attribute.KeyValue { + return MessageIDKey.Int(val) +} + +// MessageCompressedSize returns an attribute KeyValue conforming to the +// "message.compressed_size" semantic conventions. It represents the compressed +// size of the message in bytes. +func MessageCompressedSize(val int) attribute.KeyValue { + return MessageCompressedSizeKey.Int(val) +} + +// MessageUncompressedSize returns an attribute KeyValue conforming to the +// "message.uncompressed_size" semantic conventions. It represents the +// uncompressed size of the message in bytes. +func MessageUncompressedSize(val int) attribute.KeyValue { + return MessageUncompressedSizeKey.Int(val) +} + +// The attributes used to report a single exception associated with a span. +const ( + // ExceptionEscapedKey is the attribute Key conforming to the + // "exception.escaped" semantic conventions. It represents the sHOULD be + // set to true if the exception event is recorded at a point where it is + // known that the exception is escaping the scope of the span. + // + // Type: boolean + // RequirementLevel: Optional + // Stability: stable + // Note: An exception is considered to have escaped (or left) the scope of + // a span, + // if that span is ended while the exception is still logically "in + // flight". + // This may be actually "in flight" in some languages (e.g. if the + // exception + // is passed to a Context manager's `__exit__` method in Python) but will + // usually be caught at the point of recording the exception in most + // languages. + // + // It is usually not possible to determine at the point where an exception + // is thrown + // whether it will escape the scope of a span. + // However, it is trivial to know that an exception + // will escape, if one checks for an active exception just before ending + // the span, + // as done in the [example above](#recording-an-exception). + // + // It follows that an exception may still escape the scope of the span + // even if the `exception.escaped` attribute was not set or set to false, + // since the event might have been recorded at a time where it was not + // clear whether the exception will escape. + ExceptionEscapedKey = attribute.Key("exception.escaped") +) + +// ExceptionEscaped returns an attribute KeyValue conforming to the +// "exception.escaped" semantic conventions. It represents the sHOULD be set to +// true if the exception event is recorded at a point where it is known that +// the exception is escaping the scope of the span. +func ExceptionEscaped(val bool) attribute.KeyValue { + return ExceptionEscapedKey.Bool(val) +} diff --git a/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/exception.go b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/exception.go new file mode 100644 index 000000000000..d7b2de12475d --- /dev/null +++ b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/exception.go @@ -0,0 +1,9 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package semconv // import "go.opentelemetry.io/otel/semconv/v1.18.0" + +const ( + // ExceptionEventName is the name of the Span event representing an exception. + ExceptionEventName = "exception" +) diff --git a/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/http.go b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/http.go new file mode 100644 index 000000000000..9c5d10fe5562 --- /dev/null +++ b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/http.go @@ -0,0 +1,10 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package semconv // import "go.opentelemetry.io/otel/semconv/v1.18.0" + +// HTTP scheme attributes. +var ( + HTTPSchemeHTTP = HTTPSchemeKey.String("http") + HTTPSchemeHTTPS = HTTPSchemeKey.String("https") +) diff --git a/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/resource.go b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/resource.go new file mode 100644 index 000000000000..5f8c8fd4c5de --- /dev/null +++ b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/resource.go @@ -0,0 +1,1999 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +// Code generated from semantic convention specification. DO NOT EDIT. + +package semconv // import "go.opentelemetry.io/otel/semconv/v1.18.0" + +import "go.opentelemetry.io/otel/attribute" + +// The web browser in which the application represented by the resource is +// running. The `browser.*` attributes MUST be used only for resources that +// represent applications running in a web browser (regardless of whether +// running on a mobile or desktop device). +const ( + // BrowserBrandsKey is the attribute Key conforming to the "browser.brands" + // semantic conventions. It represents the array of brand name and version + // separated by a space + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: ' Not A;Brand 99', 'Chromium 99', 'Chrome 99' + // Note: This value is intended to be taken from the [UA client hints + // API](https://wicg.github.io/ua-client-hints/#interface) + // (`navigator.userAgentData.brands`). + BrowserBrandsKey = attribute.Key("browser.brands") + + // BrowserPlatformKey is the attribute Key conforming to the + // "browser.platform" semantic conventions. It represents the platform on + // which the browser is running + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Windows', 'macOS', 'Android' + // Note: This value is intended to be taken from the [UA client hints + // API](https://wicg.github.io/ua-client-hints/#interface) + // (`navigator.userAgentData.platform`). If unavailable, the legacy + // `navigator.platform` API SHOULD NOT be used instead and this attribute + // SHOULD be left unset in order for the values to be consistent. + // The list of possible values is defined in the [W3C User-Agent Client + // Hints + // specification](https://wicg.github.io/ua-client-hints/#sec-ch-ua-platform). + // Note that some (but not all) of these values can overlap with values in + // the [`os.type` and `os.name` attributes](./os.md). However, for + // consistency, the values in the `browser.platform` attribute should + // capture the exact value that the user agent provides. + BrowserPlatformKey = attribute.Key("browser.platform") + + // BrowserMobileKey is the attribute Key conforming to the "browser.mobile" + // semantic conventions. It represents a boolean that is true if the + // browser is running on a mobile device + // + // Type: boolean + // RequirementLevel: Optional + // Stability: stable + // Note: This value is intended to be taken from the [UA client hints + // API](https://wicg.github.io/ua-client-hints/#interface) + // (`navigator.userAgentData.mobile`). If unavailable, this attribute + // SHOULD be left unset. + BrowserMobileKey = attribute.Key("browser.mobile") + + // BrowserUserAgentKey is the attribute Key conforming to the + // "browser.user_agent" semantic conventions. It represents the full + // user-agent string provided by the browser + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Mozilla/5.0 (Macintosh; Intel Mac OS X 10_15_7) + // AppleWebKit/537.36 (KHTML, ' + // 'like Gecko) Chrome/95.0.4638.54 Safari/537.36' + // Note: The user-agent value SHOULD be provided only from browsers that do + // not have a mechanism to retrieve brands and platform individually from + // the User-Agent Client Hints API. To retrieve the value, the legacy + // `navigator.userAgent` API can be used. + BrowserUserAgentKey = attribute.Key("browser.user_agent") + + // BrowserLanguageKey is the attribute Key conforming to the + // "browser.language" semantic conventions. It represents the preferred + // language of the user using the browser + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'en', 'en-US', 'fr', 'fr-FR' + // Note: This value is intended to be taken from the Navigator API + // `navigator.language`. + BrowserLanguageKey = attribute.Key("browser.language") +) + +// BrowserBrands returns an attribute KeyValue conforming to the +// "browser.brands" semantic conventions. It represents the array of brand name +// and version separated by a space +func BrowserBrands(val ...string) attribute.KeyValue { + return BrowserBrandsKey.StringSlice(val) +} + +// BrowserPlatform returns an attribute KeyValue conforming to the +// "browser.platform" semantic conventions. It represents the platform on which +// the browser is running +func BrowserPlatform(val string) attribute.KeyValue { + return BrowserPlatformKey.String(val) +} + +// BrowserMobile returns an attribute KeyValue conforming to the +// "browser.mobile" semantic conventions. It represents a boolean that is true +// if the browser is running on a mobile device +func BrowserMobile(val bool) attribute.KeyValue { + return BrowserMobileKey.Bool(val) +} + +// BrowserUserAgent returns an attribute KeyValue conforming to the +// "browser.user_agent" semantic conventions. It represents the full user-agent +// string provided by the browser +func BrowserUserAgent(val string) attribute.KeyValue { + return BrowserUserAgentKey.String(val) +} + +// BrowserLanguage returns an attribute KeyValue conforming to the +// "browser.language" semantic conventions. It represents the preferred +// language of the user using the browser +func BrowserLanguage(val string) attribute.KeyValue { + return BrowserLanguageKey.String(val) +} + +// A cloud environment (e.g. GCP, Azure, AWS) +const ( + // CloudProviderKey is the attribute Key conforming to the "cloud.provider" + // semantic conventions. It represents the name of the cloud provider. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + CloudProviderKey = attribute.Key("cloud.provider") + + // CloudAccountIDKey is the attribute Key conforming to the + // "cloud.account.id" semantic conventions. It represents the cloud account + // ID the resource is assigned to. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '111111111111', 'opentelemetry' + CloudAccountIDKey = attribute.Key("cloud.account.id") + + // CloudRegionKey is the attribute Key conforming to the "cloud.region" + // semantic conventions. It represents the geographical region the resource + // is running. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'us-central1', 'us-east-1' + // Note: Refer to your provider's docs to see the available regions, for + // example [Alibaba Cloud + // regions](https://www.alibabacloud.com/help/doc-detail/40654.htm), [AWS + // regions](https://aws.amazon.com/about-aws/global-infrastructure/regions_az/), + // [Azure + // regions](https://azure.microsoft.com/en-us/global-infrastructure/geographies/), + // [Google Cloud regions](https://cloud.google.com/about/locations), or + // [Tencent Cloud + // regions](https://intl.cloud.tencent.com/document/product/213/6091). + CloudRegionKey = attribute.Key("cloud.region") + + // CloudAvailabilityZoneKey is the attribute Key conforming to the + // "cloud.availability_zone" semantic conventions. It represents the cloud + // regions often have multiple, isolated locations known as zones to + // increase availability. Availability zone represents the zone where the + // resource is running. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'us-east-1c' + // Note: Availability zones are called "zones" on Alibaba Cloud and Google + // Cloud. + CloudAvailabilityZoneKey = attribute.Key("cloud.availability_zone") + + // CloudPlatformKey is the attribute Key conforming to the "cloud.platform" + // semantic conventions. It represents the cloud platform in use. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + // Note: The prefix of the service SHOULD match the one specified in + // `cloud.provider`. + CloudPlatformKey = attribute.Key("cloud.platform") +) + +var ( + // Alibaba Cloud + CloudProviderAlibabaCloud = CloudProviderKey.String("alibaba_cloud") + // Amazon Web Services + CloudProviderAWS = CloudProviderKey.String("aws") + // Microsoft Azure + CloudProviderAzure = CloudProviderKey.String("azure") + // Google Cloud Platform + CloudProviderGCP = CloudProviderKey.String("gcp") + // IBM Cloud + CloudProviderIbmCloud = CloudProviderKey.String("ibm_cloud") + // Tencent Cloud + CloudProviderTencentCloud = CloudProviderKey.String("tencent_cloud") +) + +var ( + // Alibaba Cloud Elastic Compute Service + CloudPlatformAlibabaCloudECS = CloudPlatformKey.String("alibaba_cloud_ecs") + // Alibaba Cloud Function Compute + CloudPlatformAlibabaCloudFc = CloudPlatformKey.String("alibaba_cloud_fc") + // Red Hat OpenShift on Alibaba Cloud + CloudPlatformAlibabaCloudOpenshift = CloudPlatformKey.String("alibaba_cloud_openshift") + // AWS Elastic Compute Cloud + CloudPlatformAWSEC2 = CloudPlatformKey.String("aws_ec2") + // AWS Elastic Container Service + CloudPlatformAWSECS = CloudPlatformKey.String("aws_ecs") + // AWS Elastic Kubernetes Service + CloudPlatformAWSEKS = CloudPlatformKey.String("aws_eks") + // AWS Lambda + CloudPlatformAWSLambda = CloudPlatformKey.String("aws_lambda") + // AWS Elastic Beanstalk + CloudPlatformAWSElasticBeanstalk = CloudPlatformKey.String("aws_elastic_beanstalk") + // AWS App Runner + CloudPlatformAWSAppRunner = CloudPlatformKey.String("aws_app_runner") + // Red Hat OpenShift on AWS (ROSA) + CloudPlatformAWSOpenshift = CloudPlatformKey.String("aws_openshift") + // Azure Virtual Machines + CloudPlatformAzureVM = CloudPlatformKey.String("azure_vm") + // Azure Container Instances + CloudPlatformAzureContainerInstances = CloudPlatformKey.String("azure_container_instances") + // Azure Kubernetes Service + CloudPlatformAzureAKS = CloudPlatformKey.String("azure_aks") + // Azure Functions + CloudPlatformAzureFunctions = CloudPlatformKey.String("azure_functions") + // Azure App Service + CloudPlatformAzureAppService = CloudPlatformKey.String("azure_app_service") + // Azure Red Hat OpenShift + CloudPlatformAzureOpenshift = CloudPlatformKey.String("azure_openshift") + // Google Cloud Compute Engine (GCE) + CloudPlatformGCPComputeEngine = CloudPlatformKey.String("gcp_compute_engine") + // Google Cloud Run + CloudPlatformGCPCloudRun = CloudPlatformKey.String("gcp_cloud_run") + // Google Cloud Kubernetes Engine (GKE) + CloudPlatformGCPKubernetesEngine = CloudPlatformKey.String("gcp_kubernetes_engine") + // Google Cloud Functions (GCF) + CloudPlatformGCPCloudFunctions = CloudPlatformKey.String("gcp_cloud_functions") + // Google Cloud App Engine (GAE) + CloudPlatformGCPAppEngine = CloudPlatformKey.String("gcp_app_engine") + // Red Hat OpenShift on Google Cloud + CloudPlatformGCPOpenshift = CloudPlatformKey.String("gcp_openshift") + // Red Hat OpenShift on IBM Cloud + CloudPlatformIbmCloudOpenshift = CloudPlatformKey.String("ibm_cloud_openshift") + // Tencent Cloud Cloud Virtual Machine (CVM) + CloudPlatformTencentCloudCvm = CloudPlatformKey.String("tencent_cloud_cvm") + // Tencent Cloud Elastic Kubernetes Service (EKS) + CloudPlatformTencentCloudEKS = CloudPlatformKey.String("tencent_cloud_eks") + // Tencent Cloud Serverless Cloud Function (SCF) + CloudPlatformTencentCloudScf = CloudPlatformKey.String("tencent_cloud_scf") +) + +// CloudAccountID returns an attribute KeyValue conforming to the +// "cloud.account.id" semantic conventions. It represents the cloud account ID +// the resource is assigned to. +func CloudAccountID(val string) attribute.KeyValue { + return CloudAccountIDKey.String(val) +} + +// CloudRegion returns an attribute KeyValue conforming to the +// "cloud.region" semantic conventions. It represents the geographical region +// the resource is running. +func CloudRegion(val string) attribute.KeyValue { + return CloudRegionKey.String(val) +} + +// CloudAvailabilityZone returns an attribute KeyValue conforming to the +// "cloud.availability_zone" semantic conventions. It represents the cloud +// regions often have multiple, isolated locations known as zones to increase +// availability. Availability zone represents the zone where the resource is +// running. +func CloudAvailabilityZone(val string) attribute.KeyValue { + return CloudAvailabilityZoneKey.String(val) +} + +// Resources used by AWS Elastic Container Service (ECS). +const ( + // AWSECSContainerARNKey is the attribute Key conforming to the + // "aws.ecs.container.arn" semantic conventions. It represents the Amazon + // Resource Name (ARN) of an [ECS container + // instance](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/ECS_instances.html). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: + // 'arn:aws:ecs:us-west-1:123456789123:container/32624152-9086-4f0e-acae-1a75b14fe4d9' + AWSECSContainerARNKey = attribute.Key("aws.ecs.container.arn") + + // AWSECSClusterARNKey is the attribute Key conforming to the + // "aws.ecs.cluster.arn" semantic conventions. It represents the ARN of an + // [ECS + // cluster](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/clusters.html). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'arn:aws:ecs:us-west-2:123456789123:cluster/my-cluster' + AWSECSClusterARNKey = attribute.Key("aws.ecs.cluster.arn") + + // AWSECSLaunchtypeKey is the attribute Key conforming to the + // "aws.ecs.launchtype" semantic conventions. It represents the [launch + // type](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/launch_types.html) + // for an ECS task. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + AWSECSLaunchtypeKey = attribute.Key("aws.ecs.launchtype") + + // AWSECSTaskARNKey is the attribute Key conforming to the + // "aws.ecs.task.arn" semantic conventions. It represents the ARN of an + // [ECS task + // definition](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task_definitions.html). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: + // 'arn:aws:ecs:us-west-1:123456789123:task/10838bed-421f-43ef-870a-f43feacbbb5b' + AWSECSTaskARNKey = attribute.Key("aws.ecs.task.arn") + + // AWSECSTaskFamilyKey is the attribute Key conforming to the + // "aws.ecs.task.family" semantic conventions. It represents the task + // definition family this task definition is a member of. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'opentelemetry-family' + AWSECSTaskFamilyKey = attribute.Key("aws.ecs.task.family") + + // AWSECSTaskRevisionKey is the attribute Key conforming to the + // "aws.ecs.task.revision" semantic conventions. It represents the revision + // for this task definition. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '8', '26' + AWSECSTaskRevisionKey = attribute.Key("aws.ecs.task.revision") +) + +var ( + // ec2 + AWSECSLaunchtypeEC2 = AWSECSLaunchtypeKey.String("ec2") + // fargate + AWSECSLaunchtypeFargate = AWSECSLaunchtypeKey.String("fargate") +) + +// AWSECSContainerARN returns an attribute KeyValue conforming to the +// "aws.ecs.container.arn" semantic conventions. It represents the Amazon +// Resource Name (ARN) of an [ECS container +// instance](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/ECS_instances.html). +func AWSECSContainerARN(val string) attribute.KeyValue { + return AWSECSContainerARNKey.String(val) +} + +// AWSECSClusterARN returns an attribute KeyValue conforming to the +// "aws.ecs.cluster.arn" semantic conventions. It represents the ARN of an [ECS +// cluster](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/clusters.html). +func AWSECSClusterARN(val string) attribute.KeyValue { + return AWSECSClusterARNKey.String(val) +} + +// AWSECSTaskARN returns an attribute KeyValue conforming to the +// "aws.ecs.task.arn" semantic conventions. It represents the ARN of an [ECS +// task +// definition](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task_definitions.html). +func AWSECSTaskARN(val string) attribute.KeyValue { + return AWSECSTaskARNKey.String(val) +} + +// AWSECSTaskFamily returns an attribute KeyValue conforming to the +// "aws.ecs.task.family" semantic conventions. It represents the task +// definition family this task definition is a member of. +func AWSECSTaskFamily(val string) attribute.KeyValue { + return AWSECSTaskFamilyKey.String(val) +} + +// AWSECSTaskRevision returns an attribute KeyValue conforming to the +// "aws.ecs.task.revision" semantic conventions. It represents the revision for +// this task definition. +func AWSECSTaskRevision(val string) attribute.KeyValue { + return AWSECSTaskRevisionKey.String(val) +} + +// Resources used by AWS Elastic Kubernetes Service (EKS). +const ( + // AWSEKSClusterARNKey is the attribute Key conforming to the + // "aws.eks.cluster.arn" semantic conventions. It represents the ARN of an + // EKS cluster. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'arn:aws:ecs:us-west-2:123456789123:cluster/my-cluster' + AWSEKSClusterARNKey = attribute.Key("aws.eks.cluster.arn") +) + +// AWSEKSClusterARN returns an attribute KeyValue conforming to the +// "aws.eks.cluster.arn" semantic conventions. It represents the ARN of an EKS +// cluster. +func AWSEKSClusterARN(val string) attribute.KeyValue { + return AWSEKSClusterARNKey.String(val) +} + +// Resources specific to Amazon Web Services. +const ( + // AWSLogGroupNamesKey is the attribute Key conforming to the + // "aws.log.group.names" semantic conventions. It represents the name(s) of + // the AWS log group(s) an application is writing to. + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: '/aws/lambda/my-function', 'opentelemetry-service' + // Note: Multiple log groups must be supported for cases like + // multi-container applications, where a single application has sidecar + // containers, and each write to their own log group. + AWSLogGroupNamesKey = attribute.Key("aws.log.group.names") + + // AWSLogGroupARNsKey is the attribute Key conforming to the + // "aws.log.group.arns" semantic conventions. It represents the Amazon + // Resource Name(s) (ARN) of the AWS log group(s). + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: + // 'arn:aws:logs:us-west-1:123456789012:log-group:/aws/my/group:*' + // Note: See the [log group ARN format + // documentation](https://docs.aws.amazon.com/AmazonCloudWatch/latest/logs/iam-access-control-overview-cwl.html#CWL_ARN_Format). + AWSLogGroupARNsKey = attribute.Key("aws.log.group.arns") + + // AWSLogStreamNamesKey is the attribute Key conforming to the + // "aws.log.stream.names" semantic conventions. It represents the name(s) + // of the AWS log stream(s) an application is writing to. + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: 'logs/main/10838bed-421f-43ef-870a-f43feacbbb5b' + AWSLogStreamNamesKey = attribute.Key("aws.log.stream.names") + + // AWSLogStreamARNsKey is the attribute Key conforming to the + // "aws.log.stream.arns" semantic conventions. It represents the ARN(s) of + // the AWS log stream(s). + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: + // 'arn:aws:logs:us-west-1:123456789012:log-group:/aws/my/group:log-stream:logs/main/10838bed-421f-43ef-870a-f43feacbbb5b' + // Note: See the [log stream ARN format + // documentation](https://docs.aws.amazon.com/AmazonCloudWatch/latest/logs/iam-access-control-overview-cwl.html#CWL_ARN_Format). + // One log group can contain several log streams, so these ARNs necessarily + // identify both a log group and a log stream. + AWSLogStreamARNsKey = attribute.Key("aws.log.stream.arns") +) + +// AWSLogGroupNames returns an attribute KeyValue conforming to the +// "aws.log.group.names" semantic conventions. It represents the name(s) of the +// AWS log group(s) an application is writing to. +func AWSLogGroupNames(val ...string) attribute.KeyValue { + return AWSLogGroupNamesKey.StringSlice(val) +} + +// AWSLogGroupARNs returns an attribute KeyValue conforming to the +// "aws.log.group.arns" semantic conventions. It represents the Amazon Resource +// Name(s) (ARN) of the AWS log group(s). +func AWSLogGroupARNs(val ...string) attribute.KeyValue { + return AWSLogGroupARNsKey.StringSlice(val) +} + +// AWSLogStreamNames returns an attribute KeyValue conforming to the +// "aws.log.stream.names" semantic conventions. It represents the name(s) of +// the AWS log stream(s) an application is writing to. +func AWSLogStreamNames(val ...string) attribute.KeyValue { + return AWSLogStreamNamesKey.StringSlice(val) +} + +// AWSLogStreamARNs returns an attribute KeyValue conforming to the +// "aws.log.stream.arns" semantic conventions. It represents the ARN(s) of the +// AWS log stream(s). +func AWSLogStreamARNs(val ...string) attribute.KeyValue { + return AWSLogStreamARNsKey.StringSlice(val) +} + +// A container instance. +const ( + // ContainerNameKey is the attribute Key conforming to the "container.name" + // semantic conventions. It represents the container name used by container + // runtime. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'opentelemetry-autoconf' + ContainerNameKey = attribute.Key("container.name") + + // ContainerIDKey is the attribute Key conforming to the "container.id" + // semantic conventions. It represents the container ID. Usually a UUID, as + // for example used to [identify Docker + // containers](https://docs.docker.com/engine/reference/run/#container-identification). + // The UUID might be abbreviated. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'a3bf90e006b2' + ContainerIDKey = attribute.Key("container.id") + + // ContainerRuntimeKey is the attribute Key conforming to the + // "container.runtime" semantic conventions. It represents the container + // runtime managing this container. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'docker', 'containerd', 'rkt' + ContainerRuntimeKey = attribute.Key("container.runtime") + + // ContainerImageNameKey is the attribute Key conforming to the + // "container.image.name" semantic conventions. It represents the name of + // the image the container was built on. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'gcr.io/opentelemetry/operator' + ContainerImageNameKey = attribute.Key("container.image.name") + + // ContainerImageTagKey is the attribute Key conforming to the + // "container.image.tag" semantic conventions. It represents the container + // image tag. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '0.1' + ContainerImageTagKey = attribute.Key("container.image.tag") +) + +// ContainerName returns an attribute KeyValue conforming to the +// "container.name" semantic conventions. It represents the container name used +// by container runtime. +func ContainerName(val string) attribute.KeyValue { + return ContainerNameKey.String(val) +} + +// ContainerID returns an attribute KeyValue conforming to the +// "container.id" semantic conventions. It represents the container ID. Usually +// a UUID, as for example used to [identify Docker +// containers](https://docs.docker.com/engine/reference/run/#container-identification). +// The UUID might be abbreviated. +func ContainerID(val string) attribute.KeyValue { + return ContainerIDKey.String(val) +} + +// ContainerRuntime returns an attribute KeyValue conforming to the +// "container.runtime" semantic conventions. It represents the container +// runtime managing this container. +func ContainerRuntime(val string) attribute.KeyValue { + return ContainerRuntimeKey.String(val) +} + +// ContainerImageName returns an attribute KeyValue conforming to the +// "container.image.name" semantic conventions. It represents the name of the +// image the container was built on. +func ContainerImageName(val string) attribute.KeyValue { + return ContainerImageNameKey.String(val) +} + +// ContainerImageTag returns an attribute KeyValue conforming to the +// "container.image.tag" semantic conventions. It represents the container +// image tag. +func ContainerImageTag(val string) attribute.KeyValue { + return ContainerImageTagKey.String(val) +} + +// The software deployment. +const ( + // DeploymentEnvironmentKey is the attribute Key conforming to the + // "deployment.environment" semantic conventions. It represents the name of + // the [deployment + // environment](https://en.wikipedia.org/wiki/Deployment_environment) (aka + // deployment tier). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'staging', 'production' + DeploymentEnvironmentKey = attribute.Key("deployment.environment") +) + +// DeploymentEnvironment returns an attribute KeyValue conforming to the +// "deployment.environment" semantic conventions. It represents the name of the +// [deployment +// environment](https://en.wikipedia.org/wiki/Deployment_environment) (aka +// deployment tier). +func DeploymentEnvironment(val string) attribute.KeyValue { + return DeploymentEnvironmentKey.String(val) +} + +// The device on which the process represented by this resource is running. +const ( + // DeviceIDKey is the attribute Key conforming to the "device.id" semantic + // conventions. It represents a unique identifier representing the device + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '2ab2916d-a51f-4ac8-80ee-45ac31a28092' + // Note: The device identifier MUST only be defined using the values + // outlined below. This value is not an advertising identifier and MUST NOT + // be used as such. On iOS (Swift or Objective-C), this value MUST be equal + // to the [vendor + // identifier](https://developer.apple.com/documentation/uikit/uidevice/1620059-identifierforvendor). + // On Android (Java or Kotlin), this value MUST be equal to the Firebase + // Installation ID or a globally unique UUID which is persisted across + // sessions in your application. More information can be found + // [here](https://developer.android.com/training/articles/user-data-ids) on + // best practices and exact implementation details. Caution should be taken + // when storing personal data or anything which can identify a user. GDPR + // and data protection laws may apply, ensure you do your own due + // diligence. + DeviceIDKey = attribute.Key("device.id") + + // DeviceModelIdentifierKey is the attribute Key conforming to the + // "device.model.identifier" semantic conventions. It represents the model + // identifier for the device + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'iPhone3,4', 'SM-G920F' + // Note: It's recommended this value represents a machine readable version + // of the model identifier rather than the market or consumer-friendly name + // of the device. + DeviceModelIdentifierKey = attribute.Key("device.model.identifier") + + // DeviceModelNameKey is the attribute Key conforming to the + // "device.model.name" semantic conventions. It represents the marketing + // name for the device model + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'iPhone 6s Plus', 'Samsung Galaxy S6' + // Note: It's recommended this value represents a human readable version of + // the device model rather than a machine readable alternative. + DeviceModelNameKey = attribute.Key("device.model.name") + + // DeviceManufacturerKey is the attribute Key conforming to the + // "device.manufacturer" semantic conventions. It represents the name of + // the device manufacturer + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Apple', 'Samsung' + // Note: The Android OS provides this field via + // [Build](https://developer.android.com/reference/android/os/Build#MANUFACTURER). + // iOS apps SHOULD hardcode the value `Apple`. + DeviceManufacturerKey = attribute.Key("device.manufacturer") +) + +// DeviceID returns an attribute KeyValue conforming to the "device.id" +// semantic conventions. It represents a unique identifier representing the +// device +func DeviceID(val string) attribute.KeyValue { + return DeviceIDKey.String(val) +} + +// DeviceModelIdentifier returns an attribute KeyValue conforming to the +// "device.model.identifier" semantic conventions. It represents the model +// identifier for the device +func DeviceModelIdentifier(val string) attribute.KeyValue { + return DeviceModelIdentifierKey.String(val) +} + +// DeviceModelName returns an attribute KeyValue conforming to the +// "device.model.name" semantic conventions. It represents the marketing name +// for the device model +func DeviceModelName(val string) attribute.KeyValue { + return DeviceModelNameKey.String(val) +} + +// DeviceManufacturer returns an attribute KeyValue conforming to the +// "device.manufacturer" semantic conventions. It represents the name of the +// device manufacturer +func DeviceManufacturer(val string) attribute.KeyValue { + return DeviceManufacturerKey.String(val) +} + +// A serverless instance. +const ( + // FaaSNameKey is the attribute Key conforming to the "faas.name" semantic + // conventions. It represents the name of the single function that this + // runtime instance executes. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'my-function', 'myazurefunctionapp/some-function-name' + // Note: This is the name of the function as configured/deployed on the + // FaaS + // platform and is usually different from the name of the callback + // function (which may be stored in the + // [`code.namespace`/`code.function`](../../trace/semantic_conventions/span-general.md#source-code-attributes) + // span attributes). + // + // For some cloud providers, the above definition is ambiguous. The + // following + // definition of function name MUST be used for this attribute + // (and consequently the span name) for the listed cloud + // providers/products: + // + // * **Azure:** The full name `/`, i.e., function app name + // followed by a forward slash followed by the function name (this form + // can also be seen in the resource JSON for the function). + // This means that a span attribute MUST be used, as an Azure function + // app can host multiple functions that would usually share + // a TracerProvider (see also the `faas.id` attribute). + FaaSNameKey = attribute.Key("faas.name") + + // FaaSIDKey is the attribute Key conforming to the "faas.id" semantic + // conventions. It represents the unique ID of the single function that + // this runtime instance executes. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'arn:aws:lambda:us-west-2:123456789012:function:my-function' + // Note: On some cloud providers, it may not be possible to determine the + // full ID at startup, + // so consider setting `faas.id` as a span attribute instead. + // + // The exact value to use for `faas.id` depends on the cloud provider: + // + // * **AWS Lambda:** The function + // [ARN](https://docs.aws.amazon.com/general/latest/gr/aws-arns-and-namespaces.html). + // Take care not to use the "invoked ARN" directly but replace any + // [alias + // suffix](https://docs.aws.amazon.com/lambda/latest/dg/configuration-aliases.html) + // with the resolved function version, as the same runtime instance may + // be invokable with + // multiple different aliases. + // * **GCP:** The [URI of the + // resource](https://cloud.google.com/iam/docs/full-resource-names) + // * **Azure:** The [Fully Qualified Resource + // ID](https://docs.microsoft.com/en-us/rest/api/resources/resources/get-by-id) + // of the invoked function, + // *not* the function app, having the form + // `/subscriptions//resourceGroups//providers/Microsoft.Web/sites//functions/`. + // This means that a span attribute MUST be used, as an Azure function + // app can host multiple functions that would usually share + // a TracerProvider. + FaaSIDKey = attribute.Key("faas.id") + + // FaaSVersionKey is the attribute Key conforming to the "faas.version" + // semantic conventions. It represents the immutable version of the + // function being executed. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '26', 'pinkfroid-00002' + // Note: Depending on the cloud provider and platform, use: + // + // * **AWS Lambda:** The [function + // version](https://docs.aws.amazon.com/lambda/latest/dg/configuration-versions.html) + // (an integer represented as a decimal string). + // * **Google Cloud Run:** The + // [revision](https://cloud.google.com/run/docs/managing/revisions) + // (i.e., the function name plus the revision suffix). + // * **Google Cloud Functions:** The value of the + // [`K_REVISION` environment + // variable](https://cloud.google.com/functions/docs/env-var#runtime_environment_variables_set_automatically). + // * **Azure Functions:** Not applicable. Do not set this attribute. + FaaSVersionKey = attribute.Key("faas.version") + + // FaaSInstanceKey is the attribute Key conforming to the "faas.instance" + // semantic conventions. It represents the execution environment ID as a + // string, that will be potentially reused for other invocations to the + // same function/function version. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '2021/06/28/[$LATEST]2f399eb14537447da05ab2a2e39309de' + // Note: * **AWS Lambda:** Use the (full) log stream name. + FaaSInstanceKey = attribute.Key("faas.instance") + + // FaaSMaxMemoryKey is the attribute Key conforming to the + // "faas.max_memory" semantic conventions. It represents the amount of + // memory available to the serverless function in MiB. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 128 + // Note: It's recommended to set this attribute since e.g. too little + // memory can easily stop a Java AWS Lambda function from working + // correctly. On AWS Lambda, the environment variable + // `AWS_LAMBDA_FUNCTION_MEMORY_SIZE` provides this information. + FaaSMaxMemoryKey = attribute.Key("faas.max_memory") +) + +// FaaSName returns an attribute KeyValue conforming to the "faas.name" +// semantic conventions. It represents the name of the single function that +// this runtime instance executes. +func FaaSName(val string) attribute.KeyValue { + return FaaSNameKey.String(val) +} + +// FaaSID returns an attribute KeyValue conforming to the "faas.id" semantic +// conventions. It represents the unique ID of the single function that this +// runtime instance executes. +func FaaSID(val string) attribute.KeyValue { + return FaaSIDKey.String(val) +} + +// FaaSVersion returns an attribute KeyValue conforming to the +// "faas.version" semantic conventions. It represents the immutable version of +// the function being executed. +func FaaSVersion(val string) attribute.KeyValue { + return FaaSVersionKey.String(val) +} + +// FaaSInstance returns an attribute KeyValue conforming to the +// "faas.instance" semantic conventions. It represents the execution +// environment ID as a string, that will be potentially reused for other +// invocations to the same function/function version. +func FaaSInstance(val string) attribute.KeyValue { + return FaaSInstanceKey.String(val) +} + +// FaaSMaxMemory returns an attribute KeyValue conforming to the +// "faas.max_memory" semantic conventions. It represents the amount of memory +// available to the serverless function in MiB. +func FaaSMaxMemory(val int) attribute.KeyValue { + return FaaSMaxMemoryKey.Int(val) +} + +// A host is defined as a general computing instance. +const ( + // HostIDKey is the attribute Key conforming to the "host.id" semantic + // conventions. It represents the unique host ID. For Cloud, this must be + // the instance_id assigned by the cloud provider. For non-containerized + // Linux systems, the `machine-id` located in `/etc/machine-id` or + // `/var/lib/dbus/machine-id` may be used. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'fdbf79e8af94cb7f9e8df36789187052' + HostIDKey = attribute.Key("host.id") + + // HostNameKey is the attribute Key conforming to the "host.name" semantic + // conventions. It represents the name of the host. On Unix systems, it may + // contain what the hostname command returns, or the fully qualified + // hostname, or another name specified by the user. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'opentelemetry-test' + HostNameKey = attribute.Key("host.name") + + // HostTypeKey is the attribute Key conforming to the "host.type" semantic + // conventions. It represents the type of host. For Cloud, this must be the + // machine type. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'n1-standard-1' + HostTypeKey = attribute.Key("host.type") + + // HostArchKey is the attribute Key conforming to the "host.arch" semantic + // conventions. It represents the CPU architecture the host system is + // running on. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + HostArchKey = attribute.Key("host.arch") + + // HostImageNameKey is the attribute Key conforming to the + // "host.image.name" semantic conventions. It represents the name of the VM + // image or OS install the host was instantiated from. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'infra-ami-eks-worker-node-7d4ec78312', 'CentOS-8-x86_64-1905' + HostImageNameKey = attribute.Key("host.image.name") + + // HostImageIDKey is the attribute Key conforming to the "host.image.id" + // semantic conventions. It represents the vM image ID. For Cloud, this + // value is from the provider. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'ami-07b06b442921831e5' + HostImageIDKey = attribute.Key("host.image.id") + + // HostImageVersionKey is the attribute Key conforming to the + // "host.image.version" semantic conventions. It represents the version + // string of the VM image as defined in [Version + // Attributes](README.md#version-attributes). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '0.1' + HostImageVersionKey = attribute.Key("host.image.version") +) + +var ( + // AMD64 + HostArchAMD64 = HostArchKey.String("amd64") + // ARM32 + HostArchARM32 = HostArchKey.String("arm32") + // ARM64 + HostArchARM64 = HostArchKey.String("arm64") + // Itanium + HostArchIA64 = HostArchKey.String("ia64") + // 32-bit PowerPC + HostArchPPC32 = HostArchKey.String("ppc32") + // 64-bit PowerPC + HostArchPPC64 = HostArchKey.String("ppc64") + // IBM z/Architecture + HostArchS390x = HostArchKey.String("s390x") + // 32-bit x86 + HostArchX86 = HostArchKey.String("x86") +) + +// HostID returns an attribute KeyValue conforming to the "host.id" semantic +// conventions. It represents the unique host ID. For Cloud, this must be the +// instance_id assigned by the cloud provider. For non-containerized Linux +// systems, the `machine-id` located in `/etc/machine-id` or +// `/var/lib/dbus/machine-id` may be used. +func HostID(val string) attribute.KeyValue { + return HostIDKey.String(val) +} + +// HostName returns an attribute KeyValue conforming to the "host.name" +// semantic conventions. It represents the name of the host. On Unix systems, +// it may contain what the hostname command returns, or the fully qualified +// hostname, or another name specified by the user. +func HostName(val string) attribute.KeyValue { + return HostNameKey.String(val) +} + +// HostType returns an attribute KeyValue conforming to the "host.type" +// semantic conventions. It represents the type of host. For Cloud, this must +// be the machine type. +func HostType(val string) attribute.KeyValue { + return HostTypeKey.String(val) +} + +// HostImageName returns an attribute KeyValue conforming to the +// "host.image.name" semantic conventions. It represents the name of the VM +// image or OS install the host was instantiated from. +func HostImageName(val string) attribute.KeyValue { + return HostImageNameKey.String(val) +} + +// HostImageID returns an attribute KeyValue conforming to the +// "host.image.id" semantic conventions. It represents the vM image ID. For +// Cloud, this value is from the provider. +func HostImageID(val string) attribute.KeyValue { + return HostImageIDKey.String(val) +} + +// HostImageVersion returns an attribute KeyValue conforming to the +// "host.image.version" semantic conventions. It represents the version string +// of the VM image as defined in [Version +// Attributes](README.md#version-attributes). +func HostImageVersion(val string) attribute.KeyValue { + return HostImageVersionKey.String(val) +} + +// A Kubernetes Cluster. +const ( + // K8SClusterNameKey is the attribute Key conforming to the + // "k8s.cluster.name" semantic conventions. It represents the name of the + // cluster. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'opentelemetry-cluster' + K8SClusterNameKey = attribute.Key("k8s.cluster.name") +) + +// K8SClusterName returns an attribute KeyValue conforming to the +// "k8s.cluster.name" semantic conventions. It represents the name of the +// cluster. +func K8SClusterName(val string) attribute.KeyValue { + return K8SClusterNameKey.String(val) +} + +// A Kubernetes Node object. +const ( + // K8SNodeNameKey is the attribute Key conforming to the "k8s.node.name" + // semantic conventions. It represents the name of the Node. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'node-1' + K8SNodeNameKey = attribute.Key("k8s.node.name") + + // K8SNodeUIDKey is the attribute Key conforming to the "k8s.node.uid" + // semantic conventions. It represents the UID of the Node. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '1eb3a0c6-0477-4080-a9cb-0cb7db65c6a2' + K8SNodeUIDKey = attribute.Key("k8s.node.uid") +) + +// K8SNodeName returns an attribute KeyValue conforming to the +// "k8s.node.name" semantic conventions. It represents the name of the Node. +func K8SNodeName(val string) attribute.KeyValue { + return K8SNodeNameKey.String(val) +} + +// K8SNodeUID returns an attribute KeyValue conforming to the "k8s.node.uid" +// semantic conventions. It represents the UID of the Node. +func K8SNodeUID(val string) attribute.KeyValue { + return K8SNodeUIDKey.String(val) +} + +// A Kubernetes Namespace. +const ( + // K8SNamespaceNameKey is the attribute Key conforming to the + // "k8s.namespace.name" semantic conventions. It represents the name of the + // namespace that the pod is running in. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'default' + K8SNamespaceNameKey = attribute.Key("k8s.namespace.name") +) + +// K8SNamespaceName returns an attribute KeyValue conforming to the +// "k8s.namespace.name" semantic conventions. It represents the name of the +// namespace that the pod is running in. +func K8SNamespaceName(val string) attribute.KeyValue { + return K8SNamespaceNameKey.String(val) +} + +// A Kubernetes Pod object. +const ( + // K8SPodUIDKey is the attribute Key conforming to the "k8s.pod.uid" + // semantic conventions. It represents the UID of the Pod. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '275ecb36-5aa8-4c2a-9c47-d8bb681b9aff' + K8SPodUIDKey = attribute.Key("k8s.pod.uid") + + // K8SPodNameKey is the attribute Key conforming to the "k8s.pod.name" + // semantic conventions. It represents the name of the Pod. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'opentelemetry-pod-autoconf' + K8SPodNameKey = attribute.Key("k8s.pod.name") +) + +// K8SPodUID returns an attribute KeyValue conforming to the "k8s.pod.uid" +// semantic conventions. It represents the UID of the Pod. +func K8SPodUID(val string) attribute.KeyValue { + return K8SPodUIDKey.String(val) +} + +// K8SPodName returns an attribute KeyValue conforming to the "k8s.pod.name" +// semantic conventions. It represents the name of the Pod. +func K8SPodName(val string) attribute.KeyValue { + return K8SPodNameKey.String(val) +} + +// A container in a +// [PodTemplate](https://kubernetes.io/docs/concepts/workloads/pods/#pod-templates). +const ( + // K8SContainerNameKey is the attribute Key conforming to the + // "k8s.container.name" semantic conventions. It represents the name of the + // Container from Pod specification, must be unique within a Pod. Container + // runtime usually uses different globally unique name (`container.name`). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'redis' + K8SContainerNameKey = attribute.Key("k8s.container.name") + + // K8SContainerRestartCountKey is the attribute Key conforming to the + // "k8s.container.restart_count" semantic conventions. It represents the + // number of times the container was restarted. This attribute can be used + // to identify a particular container (running or stopped) within a + // container spec. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 0, 2 + K8SContainerRestartCountKey = attribute.Key("k8s.container.restart_count") +) + +// K8SContainerName returns an attribute KeyValue conforming to the +// "k8s.container.name" semantic conventions. It represents the name of the +// Container from Pod specification, must be unique within a Pod. Container +// runtime usually uses different globally unique name (`container.name`). +func K8SContainerName(val string) attribute.KeyValue { + return K8SContainerNameKey.String(val) +} + +// K8SContainerRestartCount returns an attribute KeyValue conforming to the +// "k8s.container.restart_count" semantic conventions. It represents the number +// of times the container was restarted. This attribute can be used to identify +// a particular container (running or stopped) within a container spec. +func K8SContainerRestartCount(val int) attribute.KeyValue { + return K8SContainerRestartCountKey.Int(val) +} + +// A Kubernetes ReplicaSet object. +const ( + // K8SReplicaSetUIDKey is the attribute Key conforming to the + // "k8s.replicaset.uid" semantic conventions. It represents the UID of the + // ReplicaSet. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '275ecb36-5aa8-4c2a-9c47-d8bb681b9aff' + K8SReplicaSetUIDKey = attribute.Key("k8s.replicaset.uid") + + // K8SReplicaSetNameKey is the attribute Key conforming to the + // "k8s.replicaset.name" semantic conventions. It represents the name of + // the ReplicaSet. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'opentelemetry' + K8SReplicaSetNameKey = attribute.Key("k8s.replicaset.name") +) + +// K8SReplicaSetUID returns an attribute KeyValue conforming to the +// "k8s.replicaset.uid" semantic conventions. It represents the UID of the +// ReplicaSet. +func K8SReplicaSetUID(val string) attribute.KeyValue { + return K8SReplicaSetUIDKey.String(val) +} + +// K8SReplicaSetName returns an attribute KeyValue conforming to the +// "k8s.replicaset.name" semantic conventions. It represents the name of the +// ReplicaSet. +func K8SReplicaSetName(val string) attribute.KeyValue { + return K8SReplicaSetNameKey.String(val) +} + +// A Kubernetes Deployment object. +const ( + // K8SDeploymentUIDKey is the attribute Key conforming to the + // "k8s.deployment.uid" semantic conventions. It represents the UID of the + // Deployment. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '275ecb36-5aa8-4c2a-9c47-d8bb681b9aff' + K8SDeploymentUIDKey = attribute.Key("k8s.deployment.uid") + + // K8SDeploymentNameKey is the attribute Key conforming to the + // "k8s.deployment.name" semantic conventions. It represents the name of + // the Deployment. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'opentelemetry' + K8SDeploymentNameKey = attribute.Key("k8s.deployment.name") +) + +// K8SDeploymentUID returns an attribute KeyValue conforming to the +// "k8s.deployment.uid" semantic conventions. It represents the UID of the +// Deployment. +func K8SDeploymentUID(val string) attribute.KeyValue { + return K8SDeploymentUIDKey.String(val) +} + +// K8SDeploymentName returns an attribute KeyValue conforming to the +// "k8s.deployment.name" semantic conventions. It represents the name of the +// Deployment. +func K8SDeploymentName(val string) attribute.KeyValue { + return K8SDeploymentNameKey.String(val) +} + +// A Kubernetes StatefulSet object. +const ( + // K8SStatefulSetUIDKey is the attribute Key conforming to the + // "k8s.statefulset.uid" semantic conventions. It represents the UID of the + // StatefulSet. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '275ecb36-5aa8-4c2a-9c47-d8bb681b9aff' + K8SStatefulSetUIDKey = attribute.Key("k8s.statefulset.uid") + + // K8SStatefulSetNameKey is the attribute Key conforming to the + // "k8s.statefulset.name" semantic conventions. It represents the name of + // the StatefulSet. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'opentelemetry' + K8SStatefulSetNameKey = attribute.Key("k8s.statefulset.name") +) + +// K8SStatefulSetUID returns an attribute KeyValue conforming to the +// "k8s.statefulset.uid" semantic conventions. It represents the UID of the +// StatefulSet. +func K8SStatefulSetUID(val string) attribute.KeyValue { + return K8SStatefulSetUIDKey.String(val) +} + +// K8SStatefulSetName returns an attribute KeyValue conforming to the +// "k8s.statefulset.name" semantic conventions. It represents the name of the +// StatefulSet. +func K8SStatefulSetName(val string) attribute.KeyValue { + return K8SStatefulSetNameKey.String(val) +} + +// A Kubernetes DaemonSet object. +const ( + // K8SDaemonSetUIDKey is the attribute Key conforming to the + // "k8s.daemonset.uid" semantic conventions. It represents the UID of the + // DaemonSet. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '275ecb36-5aa8-4c2a-9c47-d8bb681b9aff' + K8SDaemonSetUIDKey = attribute.Key("k8s.daemonset.uid") + + // K8SDaemonSetNameKey is the attribute Key conforming to the + // "k8s.daemonset.name" semantic conventions. It represents the name of the + // DaemonSet. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'opentelemetry' + K8SDaemonSetNameKey = attribute.Key("k8s.daemonset.name") +) + +// K8SDaemonSetUID returns an attribute KeyValue conforming to the +// "k8s.daemonset.uid" semantic conventions. It represents the UID of the +// DaemonSet. +func K8SDaemonSetUID(val string) attribute.KeyValue { + return K8SDaemonSetUIDKey.String(val) +} + +// K8SDaemonSetName returns an attribute KeyValue conforming to the +// "k8s.daemonset.name" semantic conventions. It represents the name of the +// DaemonSet. +func K8SDaemonSetName(val string) attribute.KeyValue { + return K8SDaemonSetNameKey.String(val) +} + +// A Kubernetes Job object. +const ( + // K8SJobUIDKey is the attribute Key conforming to the "k8s.job.uid" + // semantic conventions. It represents the UID of the Job. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '275ecb36-5aa8-4c2a-9c47-d8bb681b9aff' + K8SJobUIDKey = attribute.Key("k8s.job.uid") + + // K8SJobNameKey is the attribute Key conforming to the "k8s.job.name" + // semantic conventions. It represents the name of the Job. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'opentelemetry' + K8SJobNameKey = attribute.Key("k8s.job.name") +) + +// K8SJobUID returns an attribute KeyValue conforming to the "k8s.job.uid" +// semantic conventions. It represents the UID of the Job. +func K8SJobUID(val string) attribute.KeyValue { + return K8SJobUIDKey.String(val) +} + +// K8SJobName returns an attribute KeyValue conforming to the "k8s.job.name" +// semantic conventions. It represents the name of the Job. +func K8SJobName(val string) attribute.KeyValue { + return K8SJobNameKey.String(val) +} + +// A Kubernetes CronJob object. +const ( + // K8SCronJobUIDKey is the attribute Key conforming to the + // "k8s.cronjob.uid" semantic conventions. It represents the UID of the + // CronJob. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '275ecb36-5aa8-4c2a-9c47-d8bb681b9aff' + K8SCronJobUIDKey = attribute.Key("k8s.cronjob.uid") + + // K8SCronJobNameKey is the attribute Key conforming to the + // "k8s.cronjob.name" semantic conventions. It represents the name of the + // CronJob. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'opentelemetry' + K8SCronJobNameKey = attribute.Key("k8s.cronjob.name") +) + +// K8SCronJobUID returns an attribute KeyValue conforming to the +// "k8s.cronjob.uid" semantic conventions. It represents the UID of the +// CronJob. +func K8SCronJobUID(val string) attribute.KeyValue { + return K8SCronJobUIDKey.String(val) +} + +// K8SCronJobName returns an attribute KeyValue conforming to the +// "k8s.cronjob.name" semantic conventions. It represents the name of the +// CronJob. +func K8SCronJobName(val string) attribute.KeyValue { + return K8SCronJobNameKey.String(val) +} + +// The operating system (OS) on which the process represented by this resource +// is running. +const ( + // OSTypeKey is the attribute Key conforming to the "os.type" semantic + // conventions. It represents the operating system type. + // + // Type: Enum + // RequirementLevel: Required + // Stability: stable + OSTypeKey = attribute.Key("os.type") + + // OSDescriptionKey is the attribute Key conforming to the "os.description" + // semantic conventions. It represents the human readable (not intended to + // be parsed) OS version information, like e.g. reported by `ver` or + // `lsb_release -a` commands. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Microsoft Windows [Version 10.0.18363.778]', 'Ubuntu 18.04.1 + // LTS' + OSDescriptionKey = attribute.Key("os.description") + + // OSNameKey is the attribute Key conforming to the "os.name" semantic + // conventions. It represents the human readable operating system name. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'iOS', 'Android', 'Ubuntu' + OSNameKey = attribute.Key("os.name") + + // OSVersionKey is the attribute Key conforming to the "os.version" + // semantic conventions. It represents the version string of the operating + // system as defined in [Version + // Attributes](../../resource/semantic_conventions/README.md#version-attributes). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '14.2.1', '18.04.1' + OSVersionKey = attribute.Key("os.version") +) + +var ( + // Microsoft Windows + OSTypeWindows = OSTypeKey.String("windows") + // Linux + OSTypeLinux = OSTypeKey.String("linux") + // Apple Darwin + OSTypeDarwin = OSTypeKey.String("darwin") + // FreeBSD + OSTypeFreeBSD = OSTypeKey.String("freebsd") + // NetBSD + OSTypeNetBSD = OSTypeKey.String("netbsd") + // OpenBSD + OSTypeOpenBSD = OSTypeKey.String("openbsd") + // DragonFly BSD + OSTypeDragonflyBSD = OSTypeKey.String("dragonflybsd") + // HP-UX (Hewlett Packard Unix) + OSTypeHPUX = OSTypeKey.String("hpux") + // AIX (Advanced Interactive eXecutive) + OSTypeAIX = OSTypeKey.String("aix") + // SunOS, Oracle Solaris + OSTypeSolaris = OSTypeKey.String("solaris") + // IBM z/OS + OSTypeZOS = OSTypeKey.String("z_os") +) + +// OSDescription returns an attribute KeyValue conforming to the +// "os.description" semantic conventions. It represents the human readable (not +// intended to be parsed) OS version information, like e.g. reported by `ver` +// or `lsb_release -a` commands. +func OSDescription(val string) attribute.KeyValue { + return OSDescriptionKey.String(val) +} + +// OSName returns an attribute KeyValue conforming to the "os.name" semantic +// conventions. It represents the human readable operating system name. +func OSName(val string) attribute.KeyValue { + return OSNameKey.String(val) +} + +// OSVersion returns an attribute KeyValue conforming to the "os.version" +// semantic conventions. It represents the version string of the operating +// system as defined in [Version +// Attributes](../../resource/semantic_conventions/README.md#version-attributes). +func OSVersion(val string) attribute.KeyValue { + return OSVersionKey.String(val) +} + +// An operating system process. +const ( + // ProcessPIDKey is the attribute Key conforming to the "process.pid" + // semantic conventions. It represents the process identifier (PID). + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 1234 + ProcessPIDKey = attribute.Key("process.pid") + + // ProcessParentPIDKey is the attribute Key conforming to the + // "process.parent_pid" semantic conventions. It represents the parent + // Process identifier (PID). + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 111 + ProcessParentPIDKey = attribute.Key("process.parent_pid") + + // ProcessExecutableNameKey is the attribute Key conforming to the + // "process.executable.name" semantic conventions. It represents the name + // of the process executable. On Linux based systems, can be set to the + // `Name` in `proc/[pid]/status`. On Windows, can be set to the base name + // of `GetProcessImageFileNameW`. + // + // Type: string + // RequirementLevel: ConditionallyRequired (See alternative attributes + // below.) + // Stability: stable + // Examples: 'otelcol' + ProcessExecutableNameKey = attribute.Key("process.executable.name") + + // ProcessExecutablePathKey is the attribute Key conforming to the + // "process.executable.path" semantic conventions. It represents the full + // path to the process executable. On Linux based systems, can be set to + // the target of `proc/[pid]/exe`. On Windows, can be set to the result of + // `GetProcessImageFileNameW`. + // + // Type: string + // RequirementLevel: ConditionallyRequired (See alternative attributes + // below.) + // Stability: stable + // Examples: '/usr/bin/cmd/otelcol' + ProcessExecutablePathKey = attribute.Key("process.executable.path") + + // ProcessCommandKey is the attribute Key conforming to the + // "process.command" semantic conventions. It represents the command used + // to launch the process (i.e. the command name). On Linux based systems, + // can be set to the zeroth string in `proc/[pid]/cmdline`. On Windows, can + // be set to the first parameter extracted from `GetCommandLineW`. + // + // Type: string + // RequirementLevel: ConditionallyRequired (See alternative attributes + // below.) + // Stability: stable + // Examples: 'cmd/otelcol' + ProcessCommandKey = attribute.Key("process.command") + + // ProcessCommandLineKey is the attribute Key conforming to the + // "process.command_line" semantic conventions. It represents the full + // command used to launch the process as a single string representing the + // full command. On Windows, can be set to the result of `GetCommandLineW`. + // Do not set this if you have to assemble it just for monitoring; use + // `process.command_args` instead. + // + // Type: string + // RequirementLevel: ConditionallyRequired (See alternative attributes + // below.) + // Stability: stable + // Examples: 'C:\\cmd\\otecol --config="my directory\\config.yaml"' + ProcessCommandLineKey = attribute.Key("process.command_line") + + // ProcessCommandArgsKey is the attribute Key conforming to the + // "process.command_args" semantic conventions. It represents the all the + // command arguments (including the command/executable itself) as received + // by the process. On Linux-based systems (and some other Unixoid systems + // supporting procfs), can be set according to the list of null-delimited + // strings extracted from `proc/[pid]/cmdline`. For libc-based executables, + // this would be the full argv vector passed to `main`. + // + // Type: string[] + // RequirementLevel: ConditionallyRequired (See alternative attributes + // below.) + // Stability: stable + // Examples: 'cmd/otecol', '--config=config.yaml' + ProcessCommandArgsKey = attribute.Key("process.command_args") + + // ProcessOwnerKey is the attribute Key conforming to the "process.owner" + // semantic conventions. It represents the username of the user that owns + // the process. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'root' + ProcessOwnerKey = attribute.Key("process.owner") +) + +// ProcessPID returns an attribute KeyValue conforming to the "process.pid" +// semantic conventions. It represents the process identifier (PID). +func ProcessPID(val int) attribute.KeyValue { + return ProcessPIDKey.Int(val) +} + +// ProcessParentPID returns an attribute KeyValue conforming to the +// "process.parent_pid" semantic conventions. It represents the parent Process +// identifier (PID). +func ProcessParentPID(val int) attribute.KeyValue { + return ProcessParentPIDKey.Int(val) +} + +// ProcessExecutableName returns an attribute KeyValue conforming to the +// "process.executable.name" semantic conventions. It represents the name of +// the process executable. On Linux based systems, can be set to the `Name` in +// `proc/[pid]/status`. On Windows, can be set to the base name of +// `GetProcessImageFileNameW`. +func ProcessExecutableName(val string) attribute.KeyValue { + return ProcessExecutableNameKey.String(val) +} + +// ProcessExecutablePath returns an attribute KeyValue conforming to the +// "process.executable.path" semantic conventions. It represents the full path +// to the process executable. On Linux based systems, can be set to the target +// of `proc/[pid]/exe`. On Windows, can be set to the result of +// `GetProcessImageFileNameW`. +func ProcessExecutablePath(val string) attribute.KeyValue { + return ProcessExecutablePathKey.String(val) +} + +// ProcessCommand returns an attribute KeyValue conforming to the +// "process.command" semantic conventions. It represents the command used to +// launch the process (i.e. the command name). On Linux based systems, can be +// set to the zeroth string in `proc/[pid]/cmdline`. On Windows, can be set to +// the first parameter extracted from `GetCommandLineW`. +func ProcessCommand(val string) attribute.KeyValue { + return ProcessCommandKey.String(val) +} + +// ProcessCommandLine returns an attribute KeyValue conforming to the +// "process.command_line" semantic conventions. It represents the full command +// used to launch the process as a single string representing the full command. +// On Windows, can be set to the result of `GetCommandLineW`. Do not set this +// if you have to assemble it just for monitoring; use `process.command_args` +// instead. +func ProcessCommandLine(val string) attribute.KeyValue { + return ProcessCommandLineKey.String(val) +} + +// ProcessCommandArgs returns an attribute KeyValue conforming to the +// "process.command_args" semantic conventions. It represents the all the +// command arguments (including the command/executable itself) as received by +// the process. On Linux-based systems (and some other Unixoid systems +// supporting procfs), can be set according to the list of null-delimited +// strings extracted from `proc/[pid]/cmdline`. For libc-based executables, +// this would be the full argv vector passed to `main`. +func ProcessCommandArgs(val ...string) attribute.KeyValue { + return ProcessCommandArgsKey.StringSlice(val) +} + +// ProcessOwner returns an attribute KeyValue conforming to the +// "process.owner" semantic conventions. It represents the username of the user +// that owns the process. +func ProcessOwner(val string) attribute.KeyValue { + return ProcessOwnerKey.String(val) +} + +// The single (language) runtime instance which is monitored. +const ( + // ProcessRuntimeNameKey is the attribute Key conforming to the + // "process.runtime.name" semantic conventions. It represents the name of + // the runtime of this process. For compiled native binaries, this SHOULD + // be the name of the compiler. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'OpenJDK Runtime Environment' + ProcessRuntimeNameKey = attribute.Key("process.runtime.name") + + // ProcessRuntimeVersionKey is the attribute Key conforming to the + // "process.runtime.version" semantic conventions. It represents the + // version of the runtime of this process, as returned by the runtime + // without modification. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '14.0.2' + ProcessRuntimeVersionKey = attribute.Key("process.runtime.version") + + // ProcessRuntimeDescriptionKey is the attribute Key conforming to the + // "process.runtime.description" semantic conventions. It represents an + // additional description about the runtime of the process, for example a + // specific vendor customization of the runtime environment. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Eclipse OpenJ9 Eclipse OpenJ9 VM openj9-0.21.0' + ProcessRuntimeDescriptionKey = attribute.Key("process.runtime.description") +) + +// ProcessRuntimeName returns an attribute KeyValue conforming to the +// "process.runtime.name" semantic conventions. It represents the name of the +// runtime of this process. For compiled native binaries, this SHOULD be the +// name of the compiler. +func ProcessRuntimeName(val string) attribute.KeyValue { + return ProcessRuntimeNameKey.String(val) +} + +// ProcessRuntimeVersion returns an attribute KeyValue conforming to the +// "process.runtime.version" semantic conventions. It represents the version of +// the runtime of this process, as returned by the runtime without +// modification. +func ProcessRuntimeVersion(val string) attribute.KeyValue { + return ProcessRuntimeVersionKey.String(val) +} + +// ProcessRuntimeDescription returns an attribute KeyValue conforming to the +// "process.runtime.description" semantic conventions. It represents an +// additional description about the runtime of the process, for example a +// specific vendor customization of the runtime environment. +func ProcessRuntimeDescription(val string) attribute.KeyValue { + return ProcessRuntimeDescriptionKey.String(val) +} + +// A service instance. +const ( + // ServiceNameKey is the attribute Key conforming to the "service.name" + // semantic conventions. It represents the logical name of the service. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'shoppingcart' + // Note: MUST be the same for all instances of horizontally scaled + // services. If the value was not specified, SDKs MUST fallback to + // `unknown_service:` concatenated with + // [`process.executable.name`](process.md#process), e.g. + // `unknown_service:bash`. If `process.executable.name` is not available, + // the value MUST be set to `unknown_service`. + ServiceNameKey = attribute.Key("service.name") + + // ServiceNamespaceKey is the attribute Key conforming to the + // "service.namespace" semantic conventions. It represents a namespace for + // `service.name`. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Shop' + // Note: A string value having a meaning that helps to distinguish a group + // of services, for example the team name that owns a group of services. + // `service.name` is expected to be unique within the same namespace. If + // `service.namespace` is not specified in the Resource then `service.name` + // is expected to be unique for all services that have no explicit + // namespace defined (so the empty/unspecified namespace is simply one more + // valid namespace). Zero-length namespace string is assumed equal to + // unspecified namespace. + ServiceNamespaceKey = attribute.Key("service.namespace") + + // ServiceInstanceIDKey is the attribute Key conforming to the + // "service.instance.id" semantic conventions. It represents the string ID + // of the service instance. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '627cc493-f310-47de-96bd-71410b7dec09' + // Note: MUST be unique for each instance of the same + // `service.namespace,service.name` pair (in other words + // `service.namespace,service.name,service.instance.id` triplet MUST be + // globally unique). The ID helps to distinguish instances of the same + // service that exist at the same time (e.g. instances of a horizontally + // scaled service). It is preferable for the ID to be persistent and stay + // the same for the lifetime of the service instance, however it is + // acceptable that the ID is ephemeral and changes during important + // lifetime events for the service (e.g. service restarts). If the service + // has no inherent unique ID that can be used as the value of this + // attribute it is recommended to generate a random Version 1 or Version 4 + // RFC 4122 UUID (services aiming for reproducible UUIDs may also use + // Version 5, see RFC 4122 for more recommendations). + ServiceInstanceIDKey = attribute.Key("service.instance.id") + + // ServiceVersionKey is the attribute Key conforming to the + // "service.version" semantic conventions. It represents the version string + // of the service API or implementation. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '2.0.0' + ServiceVersionKey = attribute.Key("service.version") +) + +// ServiceName returns an attribute KeyValue conforming to the +// "service.name" semantic conventions. It represents the logical name of the +// service. +func ServiceName(val string) attribute.KeyValue { + return ServiceNameKey.String(val) +} + +// ServiceNamespace returns an attribute KeyValue conforming to the +// "service.namespace" semantic conventions. It represents a namespace for +// `service.name`. +func ServiceNamespace(val string) attribute.KeyValue { + return ServiceNamespaceKey.String(val) +} + +// ServiceInstanceID returns an attribute KeyValue conforming to the +// "service.instance.id" semantic conventions. It represents the string ID of +// the service instance. +func ServiceInstanceID(val string) attribute.KeyValue { + return ServiceInstanceIDKey.String(val) +} + +// ServiceVersion returns an attribute KeyValue conforming to the +// "service.version" semantic conventions. It represents the version string of +// the service API or implementation. +func ServiceVersion(val string) attribute.KeyValue { + return ServiceVersionKey.String(val) +} + +// The telemetry SDK used to capture data recorded by the instrumentation +// libraries. +const ( + // TelemetrySDKNameKey is the attribute Key conforming to the + // "telemetry.sdk.name" semantic conventions. It represents the name of the + // telemetry SDK as defined above. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'opentelemetry' + TelemetrySDKNameKey = attribute.Key("telemetry.sdk.name") + + // TelemetrySDKLanguageKey is the attribute Key conforming to the + // "telemetry.sdk.language" semantic conventions. It represents the + // language of the telemetry SDK. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + TelemetrySDKLanguageKey = attribute.Key("telemetry.sdk.language") + + // TelemetrySDKVersionKey is the attribute Key conforming to the + // "telemetry.sdk.version" semantic conventions. It represents the version + // string of the telemetry SDK. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '1.2.3' + TelemetrySDKVersionKey = attribute.Key("telemetry.sdk.version") + + // TelemetryAutoVersionKey is the attribute Key conforming to the + // "telemetry.auto.version" semantic conventions. It represents the version + // string of the auto instrumentation agent, if used. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '1.2.3' + TelemetryAutoVersionKey = attribute.Key("telemetry.auto.version") +) + +var ( + // cpp + TelemetrySDKLanguageCPP = TelemetrySDKLanguageKey.String("cpp") + // dotnet + TelemetrySDKLanguageDotnet = TelemetrySDKLanguageKey.String("dotnet") + // erlang + TelemetrySDKLanguageErlang = TelemetrySDKLanguageKey.String("erlang") + // go + TelemetrySDKLanguageGo = TelemetrySDKLanguageKey.String("go") + // java + TelemetrySDKLanguageJava = TelemetrySDKLanguageKey.String("java") + // nodejs + TelemetrySDKLanguageNodejs = TelemetrySDKLanguageKey.String("nodejs") + // php + TelemetrySDKLanguagePHP = TelemetrySDKLanguageKey.String("php") + // python + TelemetrySDKLanguagePython = TelemetrySDKLanguageKey.String("python") + // ruby + TelemetrySDKLanguageRuby = TelemetrySDKLanguageKey.String("ruby") + // webjs + TelemetrySDKLanguageWebjs = TelemetrySDKLanguageKey.String("webjs") + // swift + TelemetrySDKLanguageSwift = TelemetrySDKLanguageKey.String("swift") +) + +// TelemetrySDKName returns an attribute KeyValue conforming to the +// "telemetry.sdk.name" semantic conventions. It represents the name of the +// telemetry SDK as defined above. +func TelemetrySDKName(val string) attribute.KeyValue { + return TelemetrySDKNameKey.String(val) +} + +// TelemetrySDKVersion returns an attribute KeyValue conforming to the +// "telemetry.sdk.version" semantic conventions. It represents the version +// string of the telemetry SDK. +func TelemetrySDKVersion(val string) attribute.KeyValue { + return TelemetrySDKVersionKey.String(val) +} + +// TelemetryAutoVersion returns an attribute KeyValue conforming to the +// "telemetry.auto.version" semantic conventions. It represents the version +// string of the auto instrumentation agent, if used. +func TelemetryAutoVersion(val string) attribute.KeyValue { + return TelemetryAutoVersionKey.String(val) +} + +// Resource describing the packaged software running the application code. Web +// engines are typically executed using process.runtime. +const ( + // WebEngineNameKey is the attribute Key conforming to the "webengine.name" + // semantic conventions. It represents the name of the web engine. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'WildFly' + WebEngineNameKey = attribute.Key("webengine.name") + + // WebEngineVersionKey is the attribute Key conforming to the + // "webengine.version" semantic conventions. It represents the version of + // the web engine. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '21.0.0' + WebEngineVersionKey = attribute.Key("webengine.version") + + // WebEngineDescriptionKey is the attribute Key conforming to the + // "webengine.description" semantic conventions. It represents the + // additional description of the web engine (e.g. detailed version and + // edition information). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'WildFly Full 21.0.0.Final (WildFly Core 13.0.1.Final) - + // 2.2.2.Final' + WebEngineDescriptionKey = attribute.Key("webengine.description") +) + +// WebEngineName returns an attribute KeyValue conforming to the +// "webengine.name" semantic conventions. It represents the name of the web +// engine. +func WebEngineName(val string) attribute.KeyValue { + return WebEngineNameKey.String(val) +} + +// WebEngineVersion returns an attribute KeyValue conforming to the +// "webengine.version" semantic conventions. It represents the version of the +// web engine. +func WebEngineVersion(val string) attribute.KeyValue { + return WebEngineVersionKey.String(val) +} + +// WebEngineDescription returns an attribute KeyValue conforming to the +// "webengine.description" semantic conventions. It represents the additional +// description of the web engine (e.g. detailed version and edition +// information). +func WebEngineDescription(val string) attribute.KeyValue { + return WebEngineDescriptionKey.String(val) +} + +// Attributes used by non-OTLP exporters to represent OpenTelemetry Scope's +// concepts. +const ( + // OTelScopeNameKey is the attribute Key conforming to the + // "otel.scope.name" semantic conventions. It represents the name of the + // instrumentation scope - (`InstrumentationScope.Name` in OTLP). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'io.opentelemetry.contrib.mongodb' + OTelScopeNameKey = attribute.Key("otel.scope.name") + + // OTelScopeVersionKey is the attribute Key conforming to the + // "otel.scope.version" semantic conventions. It represents the version of + // the instrumentation scope - (`InstrumentationScope.Version` in OTLP). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '1.0.0' + OTelScopeVersionKey = attribute.Key("otel.scope.version") +) + +// OTelScopeName returns an attribute KeyValue conforming to the +// "otel.scope.name" semantic conventions. It represents the name of the +// instrumentation scope - (`InstrumentationScope.Name` in OTLP). +func OTelScopeName(val string) attribute.KeyValue { + return OTelScopeNameKey.String(val) +} + +// OTelScopeVersion returns an attribute KeyValue conforming to the +// "otel.scope.version" semantic conventions. It represents the version of the +// instrumentation scope - (`InstrumentationScope.Version` in OTLP). +func OTelScopeVersion(val string) attribute.KeyValue { + return OTelScopeVersionKey.String(val) +} + +// Span attributes used by non-OTLP exporters to represent OpenTelemetry +// Scope's concepts. +const ( + // OTelLibraryNameKey is the attribute Key conforming to the + // "otel.library.name" semantic conventions. It represents the deprecated, + // use the `otel.scope.name` attribute. + // + // Type: string + // RequirementLevel: Optional + // Stability: deprecated + // Examples: 'io.opentelemetry.contrib.mongodb' + OTelLibraryNameKey = attribute.Key("otel.library.name") + + // OTelLibraryVersionKey is the attribute Key conforming to the + // "otel.library.version" semantic conventions. It represents the + // deprecated, use the `otel.scope.version` attribute. + // + // Type: string + // RequirementLevel: Optional + // Stability: deprecated + // Examples: '1.0.0' + OTelLibraryVersionKey = attribute.Key("otel.library.version") +) + +// OTelLibraryName returns an attribute KeyValue conforming to the +// "otel.library.name" semantic conventions. It represents the deprecated, use +// the `otel.scope.name` attribute. +func OTelLibraryName(val string) attribute.KeyValue { + return OTelLibraryNameKey.String(val) +} + +// OTelLibraryVersion returns an attribute KeyValue conforming to the +// "otel.library.version" semantic conventions. It represents the deprecated, +// use the `otel.scope.version` attribute. +func OTelLibraryVersion(val string) attribute.KeyValue { + return OTelLibraryVersionKey.String(val) +} diff --git a/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/schema.go b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/schema.go new file mode 100644 index 000000000000..70ad4d1b6cce --- /dev/null +++ b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/schema.go @@ -0,0 +1,9 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package semconv // import "go.opentelemetry.io/otel/semconv/v1.18.0" + +// SchemaURL is the schema URL that matches the version of the semantic conventions +// that this package defines. Semconv packages starting from v1.4.0 must declare +// non-empty schema URL in the form https://opentelemetry.io/schemas/ +const SchemaURL = "https://opentelemetry.io/schemas/1.18.0" diff --git a/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/trace.go b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/trace.go new file mode 100644 index 000000000000..03a64d90e420 --- /dev/null +++ b/vendor/go.opentelemetry.io/otel/semconv/v1.18.0/trace.go @@ -0,0 +1,3370 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +// Code generated from semantic convention specification. DO NOT EDIT. + +package semconv // import "go.opentelemetry.io/otel/semconv/v1.18.0" + +import "go.opentelemetry.io/otel/attribute" + +// The shared attributes used to report a single exception associated with a +// span or log. +const ( + // ExceptionTypeKey is the attribute Key conforming to the "exception.type" + // semantic conventions. It represents the type of the exception (its + // fully-qualified class name, if applicable). The dynamic type of the + // exception should be preferred over the static type in languages that + // support it. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'java.net.ConnectException', 'OSError' + ExceptionTypeKey = attribute.Key("exception.type") + + // ExceptionMessageKey is the attribute Key conforming to the + // "exception.message" semantic conventions. It represents the exception + // message. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Division by zero', "Can't convert 'int' object to str + // implicitly" + ExceptionMessageKey = attribute.Key("exception.message") + + // ExceptionStacktraceKey is the attribute Key conforming to the + // "exception.stacktrace" semantic conventions. It represents a stacktrace + // as a string in the natural representation for the language runtime. The + // representation is to be determined and documented by each language SIG. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Exception in thread "main" java.lang.RuntimeException: Test + // exception\\n at ' + // 'com.example.GenerateTrace.methodB(GenerateTrace.java:13)\\n at ' + // 'com.example.GenerateTrace.methodA(GenerateTrace.java:9)\\n at ' + // 'com.example.GenerateTrace.main(GenerateTrace.java:5)' + ExceptionStacktraceKey = attribute.Key("exception.stacktrace") +) + +// ExceptionType returns an attribute KeyValue conforming to the +// "exception.type" semantic conventions. It represents the type of the +// exception (its fully-qualified class name, if applicable). The dynamic type +// of the exception should be preferred over the static type in languages that +// support it. +func ExceptionType(val string) attribute.KeyValue { + return ExceptionTypeKey.String(val) +} + +// ExceptionMessage returns an attribute KeyValue conforming to the +// "exception.message" semantic conventions. It represents the exception +// message. +func ExceptionMessage(val string) attribute.KeyValue { + return ExceptionMessageKey.String(val) +} + +// ExceptionStacktrace returns an attribute KeyValue conforming to the +// "exception.stacktrace" semantic conventions. It represents a stacktrace as a +// string in the natural representation for the language runtime. The +// representation is to be determined and documented by each language SIG. +func ExceptionStacktrace(val string) attribute.KeyValue { + return ExceptionStacktraceKey.String(val) +} + +// Attributes for Events represented using Log Records. +const ( + // EventNameKey is the attribute Key conforming to the "event.name" + // semantic conventions. It represents the name identifies the event. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'click', 'exception' + EventNameKey = attribute.Key("event.name") + + // EventDomainKey is the attribute Key conforming to the "event.domain" + // semantic conventions. It represents the domain identifies the business + // context for the events. + // + // Type: Enum + // RequirementLevel: Required + // Stability: stable + // Note: Events across different domains may have same `event.name`, yet be + // unrelated events. + EventDomainKey = attribute.Key("event.domain") +) + +var ( + // Events from browser apps + EventDomainBrowser = EventDomainKey.String("browser") + // Events from mobile apps + EventDomainDevice = EventDomainKey.String("device") + // Events from Kubernetes + EventDomainK8S = EventDomainKey.String("k8s") +) + +// EventName returns an attribute KeyValue conforming to the "event.name" +// semantic conventions. It represents the name identifies the event. +func EventName(val string) attribute.KeyValue { + return EventNameKey.String(val) +} + +// Span attributes used by AWS Lambda (in addition to general `faas` +// attributes). +const ( + // AWSLambdaInvokedARNKey is the attribute Key conforming to the + // "aws.lambda.invoked_arn" semantic conventions. It represents the full + // invoked ARN as provided on the `Context` passed to the function + // (`Lambda-Runtime-Invoked-Function-ARN` header on the + // `/runtime/invocation/next` applicable). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'arn:aws:lambda:us-east-1:123456:function:myfunction:myalias' + // Note: This may be different from `faas.id` if an alias is involved. + AWSLambdaInvokedARNKey = attribute.Key("aws.lambda.invoked_arn") +) + +// AWSLambdaInvokedARN returns an attribute KeyValue conforming to the +// "aws.lambda.invoked_arn" semantic conventions. It represents the full +// invoked ARN as provided on the `Context` passed to the function +// (`Lambda-Runtime-Invoked-Function-ARN` header on the +// `/runtime/invocation/next` applicable). +func AWSLambdaInvokedARN(val string) attribute.KeyValue { + return AWSLambdaInvokedARNKey.String(val) +} + +// Attributes for CloudEvents. CloudEvents is a specification on how to define +// event data in a standard way. These attributes can be attached to spans when +// performing operations with CloudEvents, regardless of the protocol being +// used. +const ( + // CloudeventsEventIDKey is the attribute Key conforming to the + // "cloudevents.event_id" semantic conventions. It represents the + // [event_id](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#id) + // uniquely identifies the event. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: '123e4567-e89b-12d3-a456-426614174000', '0001' + CloudeventsEventIDKey = attribute.Key("cloudevents.event_id") + + // CloudeventsEventSourceKey is the attribute Key conforming to the + // "cloudevents.event_source" semantic conventions. It represents the + // [source](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#source-1) + // identifies the context in which an event happened. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'https://github.com/cloudevents', + // '/cloudevents/spec/pull/123', 'my-service' + CloudeventsEventSourceKey = attribute.Key("cloudevents.event_source") + + // CloudeventsEventSpecVersionKey is the attribute Key conforming to the + // "cloudevents.event_spec_version" semantic conventions. It represents the + // [version of the CloudEvents + // specification](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#specversion) + // which the event uses. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '1.0' + CloudeventsEventSpecVersionKey = attribute.Key("cloudevents.event_spec_version") + + // CloudeventsEventTypeKey is the attribute Key conforming to the + // "cloudevents.event_type" semantic conventions. It represents the + // [event_type](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#type) + // contains a value describing the type of event related to the originating + // occurrence. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'com.github.pull_request.opened', + // 'com.example.object.deleted.v2' + CloudeventsEventTypeKey = attribute.Key("cloudevents.event_type") + + // CloudeventsEventSubjectKey is the attribute Key conforming to the + // "cloudevents.event_subject" semantic conventions. It represents the + // [subject](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#subject) + // of the event in the context of the event producer (identified by + // source). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'mynewfile.jpg' + CloudeventsEventSubjectKey = attribute.Key("cloudevents.event_subject") +) + +// CloudeventsEventID returns an attribute KeyValue conforming to the +// "cloudevents.event_id" semantic conventions. It represents the +// [event_id](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#id) +// uniquely identifies the event. +func CloudeventsEventID(val string) attribute.KeyValue { + return CloudeventsEventIDKey.String(val) +} + +// CloudeventsEventSource returns an attribute KeyValue conforming to the +// "cloudevents.event_source" semantic conventions. It represents the +// [source](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#source-1) +// identifies the context in which an event happened. +func CloudeventsEventSource(val string) attribute.KeyValue { + return CloudeventsEventSourceKey.String(val) +} + +// CloudeventsEventSpecVersion returns an attribute KeyValue conforming to +// the "cloudevents.event_spec_version" semantic conventions. It represents the +// [version of the CloudEvents +// specification](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#specversion) +// which the event uses. +func CloudeventsEventSpecVersion(val string) attribute.KeyValue { + return CloudeventsEventSpecVersionKey.String(val) +} + +// CloudeventsEventType returns an attribute KeyValue conforming to the +// "cloudevents.event_type" semantic conventions. It represents the +// [event_type](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#type) +// contains a value describing the type of event related to the originating +// occurrence. +func CloudeventsEventType(val string) attribute.KeyValue { + return CloudeventsEventTypeKey.String(val) +} + +// CloudeventsEventSubject returns an attribute KeyValue conforming to the +// "cloudevents.event_subject" semantic conventions. It represents the +// [subject](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#subject) +// of the event in the context of the event producer (identified by source). +func CloudeventsEventSubject(val string) attribute.KeyValue { + return CloudeventsEventSubjectKey.String(val) +} + +// Semantic conventions for the OpenTracing Shim +const ( + // OpentracingRefTypeKey is the attribute Key conforming to the + // "opentracing.ref_type" semantic conventions. It represents the + // parent-child Reference type + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + // Note: The causal relationship between a child Span and a parent Span. + OpentracingRefTypeKey = attribute.Key("opentracing.ref_type") +) + +var ( + // The parent Span depends on the child Span in some capacity + OpentracingRefTypeChildOf = OpentracingRefTypeKey.String("child_of") + // The parent Span does not depend in any way on the result of the child Span + OpentracingRefTypeFollowsFrom = OpentracingRefTypeKey.String("follows_from") +) + +// The attributes used to perform database client calls. +const ( + // DBSystemKey is the attribute Key conforming to the "db.system" semantic + // conventions. It represents an identifier for the database management + // system (DBMS) product being used. See below for a list of well-known + // identifiers. + // + // Type: Enum + // RequirementLevel: Required + // Stability: stable + DBSystemKey = attribute.Key("db.system") + + // DBConnectionStringKey is the attribute Key conforming to the + // "db.connection_string" semantic conventions. It represents the + // connection string used to connect to the database. It is recommended to + // remove embedded credentials. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Server=(localdb)\\v11.0;Integrated Security=true;' + DBConnectionStringKey = attribute.Key("db.connection_string") + + // DBUserKey is the attribute Key conforming to the "db.user" semantic + // conventions. It represents the username for accessing the database. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'readonly_user', 'reporting_user' + DBUserKey = attribute.Key("db.user") + + // DBJDBCDriverClassnameKey is the attribute Key conforming to the + // "db.jdbc.driver_classname" semantic conventions. It represents the + // fully-qualified class name of the [Java Database Connectivity + // (JDBC)](https://docs.oracle.com/javase/8/docs/technotes/guides/jdbc/) + // driver used to connect. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'org.postgresql.Driver', + // 'com.microsoft.sqlserver.jdbc.SQLServerDriver' + DBJDBCDriverClassnameKey = attribute.Key("db.jdbc.driver_classname") + + // DBNameKey is the attribute Key conforming to the "db.name" semantic + // conventions. It represents the this attribute is used to report the name + // of the database being accessed. For commands that switch the database, + // this should be set to the target database (even if the command fails). + // + // Type: string + // RequirementLevel: ConditionallyRequired (If applicable.) + // Stability: stable + // Examples: 'customers', 'main' + // Note: In some SQL databases, the database name to be used is called + // "schema name". In case there are multiple layers that could be + // considered for database name (e.g. Oracle instance name and schema + // name), the database name to be used is the more specific layer (e.g. + // Oracle schema name). + DBNameKey = attribute.Key("db.name") + + // DBStatementKey is the attribute Key conforming to the "db.statement" + // semantic conventions. It represents the database statement being + // executed. + // + // Type: string + // RequirementLevel: ConditionallyRequired (If applicable and not + // explicitly disabled via instrumentation configuration.) + // Stability: stable + // Examples: 'SELECT * FROM wuser_table', 'SET mykey "WuValue"' + // Note: The value may be sanitized to exclude sensitive information. + DBStatementKey = attribute.Key("db.statement") + + // DBOperationKey is the attribute Key conforming to the "db.operation" + // semantic conventions. It represents the name of the operation being + // executed, e.g. the [MongoDB command + // name](https://docs.mongodb.com/manual/reference/command/#database-operations) + // such as `findAndModify`, or the SQL keyword. + // + // Type: string + // RequirementLevel: ConditionallyRequired (If `db.statement` is not + // applicable.) + // Stability: stable + // Examples: 'findAndModify', 'HMSET', 'SELECT' + // Note: When setting this to an SQL keyword, it is not recommended to + // attempt any client-side parsing of `db.statement` just to get this + // property, but it should be set if the operation name is provided by the + // library being instrumented. If the SQL statement has an ambiguous + // operation, or performs more than one operation, this value may be + // omitted. + DBOperationKey = attribute.Key("db.operation") +) + +var ( + // Some other SQL database. Fallback only. See notes + DBSystemOtherSQL = DBSystemKey.String("other_sql") + // Microsoft SQL Server + DBSystemMSSQL = DBSystemKey.String("mssql") + // Microsoft SQL Server Compact + DBSystemMssqlcompact = DBSystemKey.String("mssqlcompact") + // MySQL + DBSystemMySQL = DBSystemKey.String("mysql") + // Oracle Database + DBSystemOracle = DBSystemKey.String("oracle") + // IBM DB2 + DBSystemDB2 = DBSystemKey.String("db2") + // PostgreSQL + DBSystemPostgreSQL = DBSystemKey.String("postgresql") + // Amazon Redshift + DBSystemRedshift = DBSystemKey.String("redshift") + // Apache Hive + DBSystemHive = DBSystemKey.String("hive") + // Cloudscape + DBSystemCloudscape = DBSystemKey.String("cloudscape") + // HyperSQL DataBase + DBSystemHSQLDB = DBSystemKey.String("hsqldb") + // Progress Database + DBSystemProgress = DBSystemKey.String("progress") + // SAP MaxDB + DBSystemMaxDB = DBSystemKey.String("maxdb") + // SAP HANA + DBSystemHanaDB = DBSystemKey.String("hanadb") + // Ingres + DBSystemIngres = DBSystemKey.String("ingres") + // FirstSQL + DBSystemFirstSQL = DBSystemKey.String("firstsql") + // EnterpriseDB + DBSystemEDB = DBSystemKey.String("edb") + // InterSystems Caché + DBSystemCache = DBSystemKey.String("cache") + // Adabas (Adaptable Database System) + DBSystemAdabas = DBSystemKey.String("adabas") + // Firebird + DBSystemFirebird = DBSystemKey.String("firebird") + // Apache Derby + DBSystemDerby = DBSystemKey.String("derby") + // FileMaker + DBSystemFilemaker = DBSystemKey.String("filemaker") + // Informix + DBSystemInformix = DBSystemKey.String("informix") + // InstantDB + DBSystemInstantDB = DBSystemKey.String("instantdb") + // InterBase + DBSystemInterbase = DBSystemKey.String("interbase") + // MariaDB + DBSystemMariaDB = DBSystemKey.String("mariadb") + // Netezza + DBSystemNetezza = DBSystemKey.String("netezza") + // Pervasive PSQL + DBSystemPervasive = DBSystemKey.String("pervasive") + // PointBase + DBSystemPointbase = DBSystemKey.String("pointbase") + // SQLite + DBSystemSqlite = DBSystemKey.String("sqlite") + // Sybase + DBSystemSybase = DBSystemKey.String("sybase") + // Teradata + DBSystemTeradata = DBSystemKey.String("teradata") + // Vertica + DBSystemVertica = DBSystemKey.String("vertica") + // H2 + DBSystemH2 = DBSystemKey.String("h2") + // ColdFusion IMQ + DBSystemColdfusion = DBSystemKey.String("coldfusion") + // Apache Cassandra + DBSystemCassandra = DBSystemKey.String("cassandra") + // Apache HBase + DBSystemHBase = DBSystemKey.String("hbase") + // MongoDB + DBSystemMongoDB = DBSystemKey.String("mongodb") + // Redis + DBSystemRedis = DBSystemKey.String("redis") + // Couchbase + DBSystemCouchbase = DBSystemKey.String("couchbase") + // CouchDB + DBSystemCouchDB = DBSystemKey.String("couchdb") + // Microsoft Azure Cosmos DB + DBSystemCosmosDB = DBSystemKey.String("cosmosdb") + // Amazon DynamoDB + DBSystemDynamoDB = DBSystemKey.String("dynamodb") + // Neo4j + DBSystemNeo4j = DBSystemKey.String("neo4j") + // Apache Geode + DBSystemGeode = DBSystemKey.String("geode") + // Elasticsearch + DBSystemElasticsearch = DBSystemKey.String("elasticsearch") + // Memcached + DBSystemMemcached = DBSystemKey.String("memcached") + // CockroachDB + DBSystemCockroachdb = DBSystemKey.String("cockroachdb") + // OpenSearch + DBSystemOpensearch = DBSystemKey.String("opensearch") + // ClickHouse + DBSystemClickhouse = DBSystemKey.String("clickhouse") + // Cloud Spanner + DBSystemSpanner = DBSystemKey.String("spanner") +) + +// DBConnectionString returns an attribute KeyValue conforming to the +// "db.connection_string" semantic conventions. It represents the connection +// string used to connect to the database. It is recommended to remove embedded +// credentials. +func DBConnectionString(val string) attribute.KeyValue { + return DBConnectionStringKey.String(val) +} + +// DBUser returns an attribute KeyValue conforming to the "db.user" semantic +// conventions. It represents the username for accessing the database. +func DBUser(val string) attribute.KeyValue { + return DBUserKey.String(val) +} + +// DBJDBCDriverClassname returns an attribute KeyValue conforming to the +// "db.jdbc.driver_classname" semantic conventions. It represents the +// fully-qualified class name of the [Java Database Connectivity +// (JDBC)](https://docs.oracle.com/javase/8/docs/technotes/guides/jdbc/) driver +// used to connect. +func DBJDBCDriverClassname(val string) attribute.KeyValue { + return DBJDBCDriverClassnameKey.String(val) +} + +// DBName returns an attribute KeyValue conforming to the "db.name" semantic +// conventions. It represents the this attribute is used to report the name of +// the database being accessed. For commands that switch the database, this +// should be set to the target database (even if the command fails). +func DBName(val string) attribute.KeyValue { + return DBNameKey.String(val) +} + +// DBStatement returns an attribute KeyValue conforming to the +// "db.statement" semantic conventions. It represents the database statement +// being executed. +func DBStatement(val string) attribute.KeyValue { + return DBStatementKey.String(val) +} + +// DBOperation returns an attribute KeyValue conforming to the +// "db.operation" semantic conventions. It represents the name of the operation +// being executed, e.g. the [MongoDB command +// name](https://docs.mongodb.com/manual/reference/command/#database-operations) +// such as `findAndModify`, or the SQL keyword. +func DBOperation(val string) attribute.KeyValue { + return DBOperationKey.String(val) +} + +// Connection-level attributes for Microsoft SQL Server +const ( + // DBMSSQLInstanceNameKey is the attribute Key conforming to the + // "db.mssql.instance_name" semantic conventions. It represents the + // Microsoft SQL Server [instance + // name](https://docs.microsoft.com/en-us/sql/connect/jdbc/building-the-connection-url?view=sql-server-ver15) + // connecting to. This name is used to determine the port of a named + // instance. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'MSSQLSERVER' + // Note: If setting a `db.mssql.instance_name`, `net.peer.port` is no + // longer required (but still recommended if non-standard). + DBMSSQLInstanceNameKey = attribute.Key("db.mssql.instance_name") +) + +// DBMSSQLInstanceName returns an attribute KeyValue conforming to the +// "db.mssql.instance_name" semantic conventions. It represents the Microsoft +// SQL Server [instance +// name](https://docs.microsoft.com/en-us/sql/connect/jdbc/building-the-connection-url?view=sql-server-ver15) +// connecting to. This name is used to determine the port of a named instance. +func DBMSSQLInstanceName(val string) attribute.KeyValue { + return DBMSSQLInstanceNameKey.String(val) +} + +// Call-level attributes for Cassandra +const ( + // DBCassandraPageSizeKey is the attribute Key conforming to the + // "db.cassandra.page_size" semantic conventions. It represents the fetch + // size used for paging, i.e. how many rows will be returned at once. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 5000 + DBCassandraPageSizeKey = attribute.Key("db.cassandra.page_size") + + // DBCassandraConsistencyLevelKey is the attribute Key conforming to the + // "db.cassandra.consistency_level" semantic conventions. It represents the + // consistency level of the query. Based on consistency values from + // [CQL](https://docs.datastax.com/en/cassandra-oss/3.0/cassandra/dml/dmlConfigConsistency.html). + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + DBCassandraConsistencyLevelKey = attribute.Key("db.cassandra.consistency_level") + + // DBCassandraTableKey is the attribute Key conforming to the + // "db.cassandra.table" semantic conventions. It represents the name of the + // primary table that the operation is acting upon, including the keyspace + // name (if applicable). + // + // Type: string + // RequirementLevel: Recommended + // Stability: stable + // Examples: 'mytable' + // Note: This mirrors the db.sql.table attribute but references cassandra + // rather than sql. It is not recommended to attempt any client-side + // parsing of `db.statement` just to get this property, but it should be + // set if it is provided by the library being instrumented. If the + // operation is acting upon an anonymous table, or more than one table, + // this value MUST NOT be set. + DBCassandraTableKey = attribute.Key("db.cassandra.table") + + // DBCassandraIdempotenceKey is the attribute Key conforming to the + // "db.cassandra.idempotence" semantic conventions. It represents the + // whether or not the query is idempotent. + // + // Type: boolean + // RequirementLevel: Optional + // Stability: stable + DBCassandraIdempotenceKey = attribute.Key("db.cassandra.idempotence") + + // DBCassandraSpeculativeExecutionCountKey is the attribute Key conforming + // to the "db.cassandra.speculative_execution_count" semantic conventions. + // It represents the number of times a query was speculatively executed. + // Not set or `0` if the query was not executed speculatively. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 0, 2 + DBCassandraSpeculativeExecutionCountKey = attribute.Key("db.cassandra.speculative_execution_count") + + // DBCassandraCoordinatorIDKey is the attribute Key conforming to the + // "db.cassandra.coordinator.id" semantic conventions. It represents the ID + // of the coordinating node for a query. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'be13faa2-8574-4d71-926d-27f16cf8a7af' + DBCassandraCoordinatorIDKey = attribute.Key("db.cassandra.coordinator.id") + + // DBCassandraCoordinatorDCKey is the attribute Key conforming to the + // "db.cassandra.coordinator.dc" semantic conventions. It represents the + // data center of the coordinating node for a query. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'us-west-2' + DBCassandraCoordinatorDCKey = attribute.Key("db.cassandra.coordinator.dc") +) + +var ( + // all + DBCassandraConsistencyLevelAll = DBCassandraConsistencyLevelKey.String("all") + // each_quorum + DBCassandraConsistencyLevelEachQuorum = DBCassandraConsistencyLevelKey.String("each_quorum") + // quorum + DBCassandraConsistencyLevelQuorum = DBCassandraConsistencyLevelKey.String("quorum") + // local_quorum + DBCassandraConsistencyLevelLocalQuorum = DBCassandraConsistencyLevelKey.String("local_quorum") + // one + DBCassandraConsistencyLevelOne = DBCassandraConsistencyLevelKey.String("one") + // two + DBCassandraConsistencyLevelTwo = DBCassandraConsistencyLevelKey.String("two") + // three + DBCassandraConsistencyLevelThree = DBCassandraConsistencyLevelKey.String("three") + // local_one + DBCassandraConsistencyLevelLocalOne = DBCassandraConsistencyLevelKey.String("local_one") + // any + DBCassandraConsistencyLevelAny = DBCassandraConsistencyLevelKey.String("any") + // serial + DBCassandraConsistencyLevelSerial = DBCassandraConsistencyLevelKey.String("serial") + // local_serial + DBCassandraConsistencyLevelLocalSerial = DBCassandraConsistencyLevelKey.String("local_serial") +) + +// DBCassandraPageSize returns an attribute KeyValue conforming to the +// "db.cassandra.page_size" semantic conventions. It represents the fetch size +// used for paging, i.e. how many rows will be returned at once. +func DBCassandraPageSize(val int) attribute.KeyValue { + return DBCassandraPageSizeKey.Int(val) +} + +// DBCassandraTable returns an attribute KeyValue conforming to the +// "db.cassandra.table" semantic conventions. It represents the name of the +// primary table that the operation is acting upon, including the keyspace name +// (if applicable). +func DBCassandraTable(val string) attribute.KeyValue { + return DBCassandraTableKey.String(val) +} + +// DBCassandraIdempotence returns an attribute KeyValue conforming to the +// "db.cassandra.idempotence" semantic conventions. It represents the whether +// or not the query is idempotent. +func DBCassandraIdempotence(val bool) attribute.KeyValue { + return DBCassandraIdempotenceKey.Bool(val) +} + +// DBCassandraSpeculativeExecutionCount returns an attribute KeyValue +// conforming to the "db.cassandra.speculative_execution_count" semantic +// conventions. It represents the number of times a query was speculatively +// executed. Not set or `0` if the query was not executed speculatively. +func DBCassandraSpeculativeExecutionCount(val int) attribute.KeyValue { + return DBCassandraSpeculativeExecutionCountKey.Int(val) +} + +// DBCassandraCoordinatorID returns an attribute KeyValue conforming to the +// "db.cassandra.coordinator.id" semantic conventions. It represents the ID of +// the coordinating node for a query. +func DBCassandraCoordinatorID(val string) attribute.KeyValue { + return DBCassandraCoordinatorIDKey.String(val) +} + +// DBCassandraCoordinatorDC returns an attribute KeyValue conforming to the +// "db.cassandra.coordinator.dc" semantic conventions. It represents the data +// center of the coordinating node for a query. +func DBCassandraCoordinatorDC(val string) attribute.KeyValue { + return DBCassandraCoordinatorDCKey.String(val) +} + +// Call-level attributes for Redis +const ( + // DBRedisDBIndexKey is the attribute Key conforming to the + // "db.redis.database_index" semantic conventions. It represents the index + // of the database being accessed as used in the [`SELECT` + // command](https://redis.io/commands/select), provided as an integer. To + // be used instead of the generic `db.name` attribute. + // + // Type: int + // RequirementLevel: ConditionallyRequired (If other than the default + // database (`0`).) + // Stability: stable + // Examples: 0, 1, 15 + DBRedisDBIndexKey = attribute.Key("db.redis.database_index") +) + +// DBRedisDBIndex returns an attribute KeyValue conforming to the +// "db.redis.database_index" semantic conventions. It represents the index of +// the database being accessed as used in the [`SELECT` +// command](https://redis.io/commands/select), provided as an integer. To be +// used instead of the generic `db.name` attribute. +func DBRedisDBIndex(val int) attribute.KeyValue { + return DBRedisDBIndexKey.Int(val) +} + +// Call-level attributes for MongoDB +const ( + // DBMongoDBCollectionKey is the attribute Key conforming to the + // "db.mongodb.collection" semantic conventions. It represents the + // collection being accessed within the database stated in `db.name`. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'customers', 'products' + DBMongoDBCollectionKey = attribute.Key("db.mongodb.collection") +) + +// DBMongoDBCollection returns an attribute KeyValue conforming to the +// "db.mongodb.collection" semantic conventions. It represents the collection +// being accessed within the database stated in `db.name`. +func DBMongoDBCollection(val string) attribute.KeyValue { + return DBMongoDBCollectionKey.String(val) +} + +// Call-level attributes for SQL databases +const ( + // DBSQLTableKey is the attribute Key conforming to the "db.sql.table" + // semantic conventions. It represents the name of the primary table that + // the operation is acting upon, including the database name (if + // applicable). + // + // Type: string + // RequirementLevel: Recommended + // Stability: stable + // Examples: 'public.users', 'customers' + // Note: It is not recommended to attempt any client-side parsing of + // `db.statement` just to get this property, but it should be set if it is + // provided by the library being instrumented. If the operation is acting + // upon an anonymous table, or more than one table, this value MUST NOT be + // set. + DBSQLTableKey = attribute.Key("db.sql.table") +) + +// DBSQLTable returns an attribute KeyValue conforming to the "db.sql.table" +// semantic conventions. It represents the name of the primary table that the +// operation is acting upon, including the database name (if applicable). +func DBSQLTable(val string) attribute.KeyValue { + return DBSQLTableKey.String(val) +} + +// Span attributes used by non-OTLP exporters to represent OpenTelemetry Span's +// concepts. +const ( + // OTelStatusCodeKey is the attribute Key conforming to the + // "otel.status_code" semantic conventions. It represents the name of the + // code, either "OK" or "ERROR". MUST NOT be set if the status code is + // UNSET. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + OTelStatusCodeKey = attribute.Key("otel.status_code") + + // OTelStatusDescriptionKey is the attribute Key conforming to the + // "otel.status_description" semantic conventions. It represents the + // description of the Status if it has a value, otherwise not set. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'resource not found' + OTelStatusDescriptionKey = attribute.Key("otel.status_description") +) + +var ( + // The operation has been validated by an Application developer or Operator to have completed successfully + OTelStatusCodeOk = OTelStatusCodeKey.String("OK") + // The operation contains an error + OTelStatusCodeError = OTelStatusCodeKey.String("ERROR") +) + +// OTelStatusDescription returns an attribute KeyValue conforming to the +// "otel.status_description" semantic conventions. It represents the +// description of the Status if it has a value, otherwise not set. +func OTelStatusDescription(val string) attribute.KeyValue { + return OTelStatusDescriptionKey.String(val) +} + +// This semantic convention describes an instance of a function that runs +// without provisioning or managing of servers (also known as serverless +// functions or Function as a Service (FaaS)) with spans. +const ( + // FaaSTriggerKey is the attribute Key conforming to the "faas.trigger" + // semantic conventions. It represents the type of the trigger which caused + // this function execution. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + // Note: For the server/consumer span on the incoming side, + // `faas.trigger` MUST be set. + // + // Clients invoking FaaS instances usually cannot set `faas.trigger`, + // since they would typically need to look in the payload to determine + // the event type. If clients set it, it should be the same as the + // trigger that corresponding incoming would have (i.e., this has + // nothing to do with the underlying transport used to make the API + // call to invoke the lambda, which is often HTTP). + FaaSTriggerKey = attribute.Key("faas.trigger") + + // FaaSExecutionKey is the attribute Key conforming to the "faas.execution" + // semantic conventions. It represents the execution ID of the current + // function execution. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'af9d5aa4-a685-4c5f-a22b-444f80b3cc28' + FaaSExecutionKey = attribute.Key("faas.execution") +) + +var ( + // A response to some data source operation such as a database or filesystem read/write + FaaSTriggerDatasource = FaaSTriggerKey.String("datasource") + // To provide an answer to an inbound HTTP request + FaaSTriggerHTTP = FaaSTriggerKey.String("http") + // A function is set to be executed when messages are sent to a messaging system + FaaSTriggerPubsub = FaaSTriggerKey.String("pubsub") + // A function is scheduled to be executed regularly + FaaSTriggerTimer = FaaSTriggerKey.String("timer") + // If none of the others apply + FaaSTriggerOther = FaaSTriggerKey.String("other") +) + +// FaaSExecution returns an attribute KeyValue conforming to the +// "faas.execution" semantic conventions. It represents the execution ID of the +// current function execution. +func FaaSExecution(val string) attribute.KeyValue { + return FaaSExecutionKey.String(val) +} + +// Semantic Convention for FaaS triggered as a response to some data source +// operation such as a database or filesystem read/write. +const ( + // FaaSDocumentCollectionKey is the attribute Key conforming to the + // "faas.document.collection" semantic conventions. It represents the name + // of the source on which the triggering operation was performed. For + // example, in Cloud Storage or S3 corresponds to the bucket name, and in + // Cosmos DB to the database name. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'myBucketName', 'myDBName' + FaaSDocumentCollectionKey = attribute.Key("faas.document.collection") + + // FaaSDocumentOperationKey is the attribute Key conforming to the + // "faas.document.operation" semantic conventions. It represents the + // describes the type of the operation that was performed on the data. + // + // Type: Enum + // RequirementLevel: Required + // Stability: stable + FaaSDocumentOperationKey = attribute.Key("faas.document.operation") + + // FaaSDocumentTimeKey is the attribute Key conforming to the + // "faas.document.time" semantic conventions. It represents a string + // containing the time when the data was accessed in the [ISO + // 8601](https://www.iso.org/iso-8601-date-and-time-format.html) format + // expressed in [UTC](https://www.w3.org/TR/NOTE-datetime). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '2020-01-23T13:47:06Z' + FaaSDocumentTimeKey = attribute.Key("faas.document.time") + + // FaaSDocumentNameKey is the attribute Key conforming to the + // "faas.document.name" semantic conventions. It represents the document + // name/table subjected to the operation. For example, in Cloud Storage or + // S3 is the name of the file, and in Cosmos DB the table name. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'myFile.txt', 'myTableName' + FaaSDocumentNameKey = attribute.Key("faas.document.name") +) + +var ( + // When a new object is created + FaaSDocumentOperationInsert = FaaSDocumentOperationKey.String("insert") + // When an object is modified + FaaSDocumentOperationEdit = FaaSDocumentOperationKey.String("edit") + // When an object is deleted + FaaSDocumentOperationDelete = FaaSDocumentOperationKey.String("delete") +) + +// FaaSDocumentCollection returns an attribute KeyValue conforming to the +// "faas.document.collection" semantic conventions. It represents the name of +// the source on which the triggering operation was performed. For example, in +// Cloud Storage or S3 corresponds to the bucket name, and in Cosmos DB to the +// database name. +func FaaSDocumentCollection(val string) attribute.KeyValue { + return FaaSDocumentCollectionKey.String(val) +} + +// FaaSDocumentTime returns an attribute KeyValue conforming to the +// "faas.document.time" semantic conventions. It represents a string containing +// the time when the data was accessed in the [ISO +// 8601](https://www.iso.org/iso-8601-date-and-time-format.html) format +// expressed in [UTC](https://www.w3.org/TR/NOTE-datetime). +func FaaSDocumentTime(val string) attribute.KeyValue { + return FaaSDocumentTimeKey.String(val) +} + +// FaaSDocumentName returns an attribute KeyValue conforming to the +// "faas.document.name" semantic conventions. It represents the document +// name/table subjected to the operation. For example, in Cloud Storage or S3 +// is the name of the file, and in Cosmos DB the table name. +func FaaSDocumentName(val string) attribute.KeyValue { + return FaaSDocumentNameKey.String(val) +} + +// Semantic Convention for FaaS scheduled to be executed regularly. +const ( + // FaaSTimeKey is the attribute Key conforming to the "faas.time" semantic + // conventions. It represents a string containing the function invocation + // time in the [ISO + // 8601](https://www.iso.org/iso-8601-date-and-time-format.html) format + // expressed in [UTC](https://www.w3.org/TR/NOTE-datetime). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '2020-01-23T13:47:06Z' + FaaSTimeKey = attribute.Key("faas.time") + + // FaaSCronKey is the attribute Key conforming to the "faas.cron" semantic + // conventions. It represents a string containing the schedule period as + // [Cron + // Expression](https://docs.oracle.com/cd/E12058_01/doc/doc.1014/e12030/cron_expressions.htm). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '0/5 * * * ? *' + FaaSCronKey = attribute.Key("faas.cron") +) + +// FaaSTime returns an attribute KeyValue conforming to the "faas.time" +// semantic conventions. It represents a string containing the function +// invocation time in the [ISO +// 8601](https://www.iso.org/iso-8601-date-and-time-format.html) format +// expressed in [UTC](https://www.w3.org/TR/NOTE-datetime). +func FaaSTime(val string) attribute.KeyValue { + return FaaSTimeKey.String(val) +} + +// FaaSCron returns an attribute KeyValue conforming to the "faas.cron" +// semantic conventions. It represents a string containing the schedule period +// as [Cron +// Expression](https://docs.oracle.com/cd/E12058_01/doc/doc.1014/e12030/cron_expressions.htm). +func FaaSCron(val string) attribute.KeyValue { + return FaaSCronKey.String(val) +} + +// Contains additional attributes for incoming FaaS spans. +const ( + // FaaSColdstartKey is the attribute Key conforming to the "faas.coldstart" + // semantic conventions. It represents a boolean that is true if the + // serverless function is executed for the first time (aka cold-start). + // + // Type: boolean + // RequirementLevel: Optional + // Stability: stable + FaaSColdstartKey = attribute.Key("faas.coldstart") +) + +// FaaSColdstart returns an attribute KeyValue conforming to the +// "faas.coldstart" semantic conventions. It represents a boolean that is true +// if the serverless function is executed for the first time (aka cold-start). +func FaaSColdstart(val bool) attribute.KeyValue { + return FaaSColdstartKey.Bool(val) +} + +// Contains additional attributes for outgoing FaaS spans. +const ( + // FaaSInvokedNameKey is the attribute Key conforming to the + // "faas.invoked_name" semantic conventions. It represents the name of the + // invoked function. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'my-function' + // Note: SHOULD be equal to the `faas.name` resource attribute of the + // invoked function. + FaaSInvokedNameKey = attribute.Key("faas.invoked_name") + + // FaaSInvokedProviderKey is the attribute Key conforming to the + // "faas.invoked_provider" semantic conventions. It represents the cloud + // provider of the invoked function. + // + // Type: Enum + // RequirementLevel: Required + // Stability: stable + // Note: SHOULD be equal to the `cloud.provider` resource attribute of the + // invoked function. + FaaSInvokedProviderKey = attribute.Key("faas.invoked_provider") + + // FaaSInvokedRegionKey is the attribute Key conforming to the + // "faas.invoked_region" semantic conventions. It represents the cloud + // region of the invoked function. + // + // Type: string + // RequirementLevel: ConditionallyRequired (For some cloud providers, like + // AWS or GCP, the region in which a function is hosted is essential to + // uniquely identify the function and also part of its endpoint. Since it's + // part of the endpoint being called, the region is always known to + // clients. In these cases, `faas.invoked_region` MUST be set accordingly. + // If the region is unknown to the client or not required for identifying + // the invoked function, setting `faas.invoked_region` is optional.) + // Stability: stable + // Examples: 'eu-central-1' + // Note: SHOULD be equal to the `cloud.region` resource attribute of the + // invoked function. + FaaSInvokedRegionKey = attribute.Key("faas.invoked_region") +) + +var ( + // Alibaba Cloud + FaaSInvokedProviderAlibabaCloud = FaaSInvokedProviderKey.String("alibaba_cloud") + // Amazon Web Services + FaaSInvokedProviderAWS = FaaSInvokedProviderKey.String("aws") + // Microsoft Azure + FaaSInvokedProviderAzure = FaaSInvokedProviderKey.String("azure") + // Google Cloud Platform + FaaSInvokedProviderGCP = FaaSInvokedProviderKey.String("gcp") + // Tencent Cloud + FaaSInvokedProviderTencentCloud = FaaSInvokedProviderKey.String("tencent_cloud") +) + +// FaaSInvokedName returns an attribute KeyValue conforming to the +// "faas.invoked_name" semantic conventions. It represents the name of the +// invoked function. +func FaaSInvokedName(val string) attribute.KeyValue { + return FaaSInvokedNameKey.String(val) +} + +// FaaSInvokedRegion returns an attribute KeyValue conforming to the +// "faas.invoked_region" semantic conventions. It represents the cloud region +// of the invoked function. +func FaaSInvokedRegion(val string) attribute.KeyValue { + return FaaSInvokedRegionKey.String(val) +} + +// These attributes may be used for any network related operation. +const ( + // NetTransportKey is the attribute Key conforming to the "net.transport" + // semantic conventions. It represents the transport protocol used. See + // note below. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + NetTransportKey = attribute.Key("net.transport") + + // NetAppProtocolNameKey is the attribute Key conforming to the + // "net.app.protocol.name" semantic conventions. It represents the + // application layer protocol used. The value SHOULD be normalized to + // lowercase. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'amqp', 'http', 'mqtt' + NetAppProtocolNameKey = attribute.Key("net.app.protocol.name") + + // NetAppProtocolVersionKey is the attribute Key conforming to the + // "net.app.protocol.version" semantic conventions. It represents the + // version of the application layer protocol used. See note below. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '3.1.1' + // Note: `net.app.protocol.version` refers to the version of the protocol + // used and might be different from the protocol client's version. If the + // HTTP client used has a version of `0.27.2`, but sends HTTP version + // `1.1`, this attribute should be set to `1.1`. + NetAppProtocolVersionKey = attribute.Key("net.app.protocol.version") + + // NetSockPeerNameKey is the attribute Key conforming to the + // "net.sock.peer.name" semantic conventions. It represents the remote + // socket peer name. + // + // Type: string + // RequirementLevel: Recommended (If available and different from + // `net.peer.name` and if `net.sock.peer.addr` is set.) + // Stability: stable + // Examples: 'proxy.example.com' + NetSockPeerNameKey = attribute.Key("net.sock.peer.name") + + // NetSockPeerAddrKey is the attribute Key conforming to the + // "net.sock.peer.addr" semantic conventions. It represents the remote + // socket peer address: IPv4 or IPv6 for internet protocols, path for local + // communication, + // [etc](https://man7.org/linux/man-pages/man7/address_families.7.html). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '127.0.0.1', '/tmp/mysql.sock' + NetSockPeerAddrKey = attribute.Key("net.sock.peer.addr") + + // NetSockPeerPortKey is the attribute Key conforming to the + // "net.sock.peer.port" semantic conventions. It represents the remote + // socket peer port. + // + // Type: int + // RequirementLevel: Recommended (If defined for the address family and if + // different than `net.peer.port` and if `net.sock.peer.addr` is set.) + // Stability: stable + // Examples: 16456 + NetSockPeerPortKey = attribute.Key("net.sock.peer.port") + + // NetSockFamilyKey is the attribute Key conforming to the + // "net.sock.family" semantic conventions. It represents the protocol + // [address + // family](https://man7.org/linux/man-pages/man7/address_families.7.html) + // which is used for communication. + // + // Type: Enum + // RequirementLevel: ConditionallyRequired (If different than `inet` and if + // any of `net.sock.peer.addr` or `net.sock.host.addr` are set. Consumers + // of telemetry SHOULD accept both IPv4 and IPv6 formats for the address in + // `net.sock.peer.addr` if `net.sock.family` is not set. This is to support + // instrumentations that follow previous versions of this document.) + // Stability: stable + // Examples: 'inet6', 'bluetooth' + NetSockFamilyKey = attribute.Key("net.sock.family") + + // NetPeerNameKey is the attribute Key conforming to the "net.peer.name" + // semantic conventions. It represents the logical remote hostname, see + // note below. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'example.com' + // Note: `net.peer.name` SHOULD NOT be set if capturing it would require an + // extra DNS lookup. + NetPeerNameKey = attribute.Key("net.peer.name") + + // NetPeerPortKey is the attribute Key conforming to the "net.peer.port" + // semantic conventions. It represents the logical remote port number + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 80, 8080, 443 + NetPeerPortKey = attribute.Key("net.peer.port") + + // NetHostNameKey is the attribute Key conforming to the "net.host.name" + // semantic conventions. It represents the logical local hostname or + // similar, see note below. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'localhost' + NetHostNameKey = attribute.Key("net.host.name") + + // NetHostPortKey is the attribute Key conforming to the "net.host.port" + // semantic conventions. It represents the logical local port number, + // preferably the one that the peer used to connect + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 8080 + NetHostPortKey = attribute.Key("net.host.port") + + // NetSockHostAddrKey is the attribute Key conforming to the + // "net.sock.host.addr" semantic conventions. It represents the local + // socket address. Useful in case of a multi-IP host. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '192.168.0.1' + NetSockHostAddrKey = attribute.Key("net.sock.host.addr") + + // NetSockHostPortKey is the attribute Key conforming to the + // "net.sock.host.port" semantic conventions. It represents the local + // socket port number. + // + // Type: int + // RequirementLevel: Recommended (If defined for the address family and if + // different than `net.host.port` and if `net.sock.host.addr` is set.) + // Stability: stable + // Examples: 35555 + NetSockHostPortKey = attribute.Key("net.sock.host.port") + + // NetHostConnectionTypeKey is the attribute Key conforming to the + // "net.host.connection.type" semantic conventions. It represents the + // internet connection type currently being used by the host. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + // Examples: 'wifi' + NetHostConnectionTypeKey = attribute.Key("net.host.connection.type") + + // NetHostConnectionSubtypeKey is the attribute Key conforming to the + // "net.host.connection.subtype" semantic conventions. It represents the + // this describes more details regarding the connection.type. It may be the + // type of cell technology connection, but it could be used for describing + // details about a wifi connection. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + // Examples: 'LTE' + NetHostConnectionSubtypeKey = attribute.Key("net.host.connection.subtype") + + // NetHostCarrierNameKey is the attribute Key conforming to the + // "net.host.carrier.name" semantic conventions. It represents the name of + // the mobile carrier. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'sprint' + NetHostCarrierNameKey = attribute.Key("net.host.carrier.name") + + // NetHostCarrierMccKey is the attribute Key conforming to the + // "net.host.carrier.mcc" semantic conventions. It represents the mobile + // carrier country code. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '310' + NetHostCarrierMccKey = attribute.Key("net.host.carrier.mcc") + + // NetHostCarrierMncKey is the attribute Key conforming to the + // "net.host.carrier.mnc" semantic conventions. It represents the mobile + // carrier network code. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '001' + NetHostCarrierMncKey = attribute.Key("net.host.carrier.mnc") + + // NetHostCarrierIccKey is the attribute Key conforming to the + // "net.host.carrier.icc" semantic conventions. It represents the ISO + // 3166-1 alpha-2 2-character country code associated with the mobile + // carrier network. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'DE' + NetHostCarrierIccKey = attribute.Key("net.host.carrier.icc") +) + +var ( + // ip_tcp + NetTransportTCP = NetTransportKey.String("ip_tcp") + // ip_udp + NetTransportUDP = NetTransportKey.String("ip_udp") + // Named or anonymous pipe. See note below + NetTransportPipe = NetTransportKey.String("pipe") + // In-process communication + NetTransportInProc = NetTransportKey.String("inproc") + // Something else (non IP-based) + NetTransportOther = NetTransportKey.String("other") +) + +var ( + // IPv4 address + NetSockFamilyInet = NetSockFamilyKey.String("inet") + // IPv6 address + NetSockFamilyInet6 = NetSockFamilyKey.String("inet6") + // Unix domain socket path + NetSockFamilyUnix = NetSockFamilyKey.String("unix") +) + +var ( + // wifi + NetHostConnectionTypeWifi = NetHostConnectionTypeKey.String("wifi") + // wired + NetHostConnectionTypeWired = NetHostConnectionTypeKey.String("wired") + // cell + NetHostConnectionTypeCell = NetHostConnectionTypeKey.String("cell") + // unavailable + NetHostConnectionTypeUnavailable = NetHostConnectionTypeKey.String("unavailable") + // unknown + NetHostConnectionTypeUnknown = NetHostConnectionTypeKey.String("unknown") +) + +var ( + // GPRS + NetHostConnectionSubtypeGprs = NetHostConnectionSubtypeKey.String("gprs") + // EDGE + NetHostConnectionSubtypeEdge = NetHostConnectionSubtypeKey.String("edge") + // UMTS + NetHostConnectionSubtypeUmts = NetHostConnectionSubtypeKey.String("umts") + // CDMA + NetHostConnectionSubtypeCdma = NetHostConnectionSubtypeKey.String("cdma") + // EVDO Rel. 0 + NetHostConnectionSubtypeEvdo0 = NetHostConnectionSubtypeKey.String("evdo_0") + // EVDO Rev. A + NetHostConnectionSubtypeEvdoA = NetHostConnectionSubtypeKey.String("evdo_a") + // CDMA2000 1XRTT + NetHostConnectionSubtypeCdma20001xrtt = NetHostConnectionSubtypeKey.String("cdma2000_1xrtt") + // HSDPA + NetHostConnectionSubtypeHsdpa = NetHostConnectionSubtypeKey.String("hsdpa") + // HSUPA + NetHostConnectionSubtypeHsupa = NetHostConnectionSubtypeKey.String("hsupa") + // HSPA + NetHostConnectionSubtypeHspa = NetHostConnectionSubtypeKey.String("hspa") + // IDEN + NetHostConnectionSubtypeIden = NetHostConnectionSubtypeKey.String("iden") + // EVDO Rev. B + NetHostConnectionSubtypeEvdoB = NetHostConnectionSubtypeKey.String("evdo_b") + // LTE + NetHostConnectionSubtypeLte = NetHostConnectionSubtypeKey.String("lte") + // EHRPD + NetHostConnectionSubtypeEhrpd = NetHostConnectionSubtypeKey.String("ehrpd") + // HSPAP + NetHostConnectionSubtypeHspap = NetHostConnectionSubtypeKey.String("hspap") + // GSM + NetHostConnectionSubtypeGsm = NetHostConnectionSubtypeKey.String("gsm") + // TD-SCDMA + NetHostConnectionSubtypeTdScdma = NetHostConnectionSubtypeKey.String("td_scdma") + // IWLAN + NetHostConnectionSubtypeIwlan = NetHostConnectionSubtypeKey.String("iwlan") + // 5G NR (New Radio) + NetHostConnectionSubtypeNr = NetHostConnectionSubtypeKey.String("nr") + // 5G NRNSA (New Radio Non-Standalone) + NetHostConnectionSubtypeNrnsa = NetHostConnectionSubtypeKey.String("nrnsa") + // LTE CA + NetHostConnectionSubtypeLteCa = NetHostConnectionSubtypeKey.String("lte_ca") +) + +// NetAppProtocolName returns an attribute KeyValue conforming to the +// "net.app.protocol.name" semantic conventions. It represents the application +// layer protocol used. The value SHOULD be normalized to lowercase. +func NetAppProtocolName(val string) attribute.KeyValue { + return NetAppProtocolNameKey.String(val) +} + +// NetAppProtocolVersion returns an attribute KeyValue conforming to the +// "net.app.protocol.version" semantic conventions. It represents the version +// of the application layer protocol used. See note below. +func NetAppProtocolVersion(val string) attribute.KeyValue { + return NetAppProtocolVersionKey.String(val) +} + +// NetSockPeerName returns an attribute KeyValue conforming to the +// "net.sock.peer.name" semantic conventions. It represents the remote socket +// peer name. +func NetSockPeerName(val string) attribute.KeyValue { + return NetSockPeerNameKey.String(val) +} + +// NetSockPeerAddr returns an attribute KeyValue conforming to the +// "net.sock.peer.addr" semantic conventions. It represents the remote socket +// peer address: IPv4 or IPv6 for internet protocols, path for local +// communication, +// [etc](https://man7.org/linux/man-pages/man7/address_families.7.html). +func NetSockPeerAddr(val string) attribute.KeyValue { + return NetSockPeerAddrKey.String(val) +} + +// NetSockPeerPort returns an attribute KeyValue conforming to the +// "net.sock.peer.port" semantic conventions. It represents the remote socket +// peer port. +func NetSockPeerPort(val int) attribute.KeyValue { + return NetSockPeerPortKey.Int(val) +} + +// NetPeerName returns an attribute KeyValue conforming to the +// "net.peer.name" semantic conventions. It represents the logical remote +// hostname, see note below. +func NetPeerName(val string) attribute.KeyValue { + return NetPeerNameKey.String(val) +} + +// NetPeerPort returns an attribute KeyValue conforming to the +// "net.peer.port" semantic conventions. It represents the logical remote port +// number +func NetPeerPort(val int) attribute.KeyValue { + return NetPeerPortKey.Int(val) +} + +// NetHostName returns an attribute KeyValue conforming to the +// "net.host.name" semantic conventions. It represents the logical local +// hostname or similar, see note below. +func NetHostName(val string) attribute.KeyValue { + return NetHostNameKey.String(val) +} + +// NetHostPort returns an attribute KeyValue conforming to the +// "net.host.port" semantic conventions. It represents the logical local port +// number, preferably the one that the peer used to connect +func NetHostPort(val int) attribute.KeyValue { + return NetHostPortKey.Int(val) +} + +// NetSockHostAddr returns an attribute KeyValue conforming to the +// "net.sock.host.addr" semantic conventions. It represents the local socket +// address. Useful in case of a multi-IP host. +func NetSockHostAddr(val string) attribute.KeyValue { + return NetSockHostAddrKey.String(val) +} + +// NetSockHostPort returns an attribute KeyValue conforming to the +// "net.sock.host.port" semantic conventions. It represents the local socket +// port number. +func NetSockHostPort(val int) attribute.KeyValue { + return NetSockHostPortKey.Int(val) +} + +// NetHostCarrierName returns an attribute KeyValue conforming to the +// "net.host.carrier.name" semantic conventions. It represents the name of the +// mobile carrier. +func NetHostCarrierName(val string) attribute.KeyValue { + return NetHostCarrierNameKey.String(val) +} + +// NetHostCarrierMcc returns an attribute KeyValue conforming to the +// "net.host.carrier.mcc" semantic conventions. It represents the mobile +// carrier country code. +func NetHostCarrierMcc(val string) attribute.KeyValue { + return NetHostCarrierMccKey.String(val) +} + +// NetHostCarrierMnc returns an attribute KeyValue conforming to the +// "net.host.carrier.mnc" semantic conventions. It represents the mobile +// carrier network code. +func NetHostCarrierMnc(val string) attribute.KeyValue { + return NetHostCarrierMncKey.String(val) +} + +// NetHostCarrierIcc returns an attribute KeyValue conforming to the +// "net.host.carrier.icc" semantic conventions. It represents the ISO 3166-1 +// alpha-2 2-character country code associated with the mobile carrier network. +func NetHostCarrierIcc(val string) attribute.KeyValue { + return NetHostCarrierIccKey.String(val) +} + +// Operations that access some remote service. +const ( + // PeerServiceKey is the attribute Key conforming to the "peer.service" + // semantic conventions. It represents the + // [`service.name`](../../resource/semantic_conventions/README.md#service) + // of the remote service. SHOULD be equal to the actual `service.name` + // resource attribute of the remote service if any. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'AuthTokenCache' + PeerServiceKey = attribute.Key("peer.service") +) + +// PeerService returns an attribute KeyValue conforming to the +// "peer.service" semantic conventions. It represents the +// [`service.name`](../../resource/semantic_conventions/README.md#service) of +// the remote service. SHOULD be equal to the actual `service.name` resource +// attribute of the remote service if any. +func PeerService(val string) attribute.KeyValue { + return PeerServiceKey.String(val) +} + +// These attributes may be used for any operation with an authenticated and/or +// authorized enduser. +const ( + // EnduserIDKey is the attribute Key conforming to the "enduser.id" + // semantic conventions. It represents the username or client_id extracted + // from the access token or + // [Authorization](https://tools.ietf.org/html/rfc7235#section-4.2) header + // in the inbound request from outside the system. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'username' + EnduserIDKey = attribute.Key("enduser.id") + + // EnduserRoleKey is the attribute Key conforming to the "enduser.role" + // semantic conventions. It represents the actual/assumed role the client + // is making the request under extracted from token or application security + // context. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'admin' + EnduserRoleKey = attribute.Key("enduser.role") + + // EnduserScopeKey is the attribute Key conforming to the "enduser.scope" + // semantic conventions. It represents the scopes or granted authorities + // the client currently possesses extracted from token or application + // security context. The value would come from the scope associated with an + // [OAuth 2.0 Access + // Token](https://tools.ietf.org/html/rfc6749#section-3.3) or an attribute + // value in a [SAML 2.0 + // Assertion](http://docs.oasis-open.org/security/saml/Post2.0/sstc-saml-tech-overview-2.0.html). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'read:message, write:files' + EnduserScopeKey = attribute.Key("enduser.scope") +) + +// EnduserID returns an attribute KeyValue conforming to the "enduser.id" +// semantic conventions. It represents the username or client_id extracted from +// the access token or +// [Authorization](https://tools.ietf.org/html/rfc7235#section-4.2) header in +// the inbound request from outside the system. +func EnduserID(val string) attribute.KeyValue { + return EnduserIDKey.String(val) +} + +// EnduserRole returns an attribute KeyValue conforming to the +// "enduser.role" semantic conventions. It represents the actual/assumed role +// the client is making the request under extracted from token or application +// security context. +func EnduserRole(val string) attribute.KeyValue { + return EnduserRoleKey.String(val) +} + +// EnduserScope returns an attribute KeyValue conforming to the +// "enduser.scope" semantic conventions. It represents the scopes or granted +// authorities the client currently possesses extracted from token or +// application security context. The value would come from the scope associated +// with an [OAuth 2.0 Access +// Token](https://tools.ietf.org/html/rfc6749#section-3.3) or an attribute +// value in a [SAML 2.0 +// Assertion](http://docs.oasis-open.org/security/saml/Post2.0/sstc-saml-tech-overview-2.0.html). +func EnduserScope(val string) attribute.KeyValue { + return EnduserScopeKey.String(val) +} + +// These attributes may be used for any operation to store information about a +// thread that started a span. +const ( + // ThreadIDKey is the attribute Key conforming to the "thread.id" semantic + // conventions. It represents the current "managed" thread ID (as opposed + // to OS thread ID). + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 42 + ThreadIDKey = attribute.Key("thread.id") + + // ThreadNameKey is the attribute Key conforming to the "thread.name" + // semantic conventions. It represents the current thread name. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'main' + ThreadNameKey = attribute.Key("thread.name") +) + +// ThreadID returns an attribute KeyValue conforming to the "thread.id" +// semantic conventions. It represents the current "managed" thread ID (as +// opposed to OS thread ID). +func ThreadID(val int) attribute.KeyValue { + return ThreadIDKey.Int(val) +} + +// ThreadName returns an attribute KeyValue conforming to the "thread.name" +// semantic conventions. It represents the current thread name. +func ThreadName(val string) attribute.KeyValue { + return ThreadNameKey.String(val) +} + +// These attributes allow to report this unit of code and therefore to provide +// more context about the span. +const ( + // CodeFunctionKey is the attribute Key conforming to the "code.function" + // semantic conventions. It represents the method or function name, or + // equivalent (usually rightmost part of the code unit's name). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'serveRequest' + CodeFunctionKey = attribute.Key("code.function") + + // CodeNamespaceKey is the attribute Key conforming to the "code.namespace" + // semantic conventions. It represents the "namespace" within which + // `code.function` is defined. Usually the qualified class or module name, + // such that `code.namespace` + some separator + `code.function` form a + // unique identifier for the code unit. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'com.example.MyHTTPService' + CodeNamespaceKey = attribute.Key("code.namespace") + + // CodeFilepathKey is the attribute Key conforming to the "code.filepath" + // semantic conventions. It represents the source code file name that + // identifies the code unit as uniquely as possible (preferably an absolute + // file path). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '/usr/local/MyApplication/content_root/app/index.php' + CodeFilepathKey = attribute.Key("code.filepath") + + // CodeLineNumberKey is the attribute Key conforming to the "code.lineno" + // semantic conventions. It represents the line number in `code.filepath` + // best representing the operation. It SHOULD point within the code unit + // named in `code.function`. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 42 + CodeLineNumberKey = attribute.Key("code.lineno") + + // CodeColumnKey is the attribute Key conforming to the "code.column" + // semantic conventions. It represents the column number in `code.filepath` + // best representing the operation. It SHOULD point within the code unit + // named in `code.function`. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 16 + CodeColumnKey = attribute.Key("code.column") +) + +// CodeFunction returns an attribute KeyValue conforming to the +// "code.function" semantic conventions. It represents the method or function +// name, or equivalent (usually rightmost part of the code unit's name). +func CodeFunction(val string) attribute.KeyValue { + return CodeFunctionKey.String(val) +} + +// CodeNamespace returns an attribute KeyValue conforming to the +// "code.namespace" semantic conventions. It represents the "namespace" within +// which `code.function` is defined. Usually the qualified class or module +// name, such that `code.namespace` + some separator + `code.function` form a +// unique identifier for the code unit. +func CodeNamespace(val string) attribute.KeyValue { + return CodeNamespaceKey.String(val) +} + +// CodeFilepath returns an attribute KeyValue conforming to the +// "code.filepath" semantic conventions. It represents the source code file +// name that identifies the code unit as uniquely as possible (preferably an +// absolute file path). +func CodeFilepath(val string) attribute.KeyValue { + return CodeFilepathKey.String(val) +} + +// CodeLineNumber returns an attribute KeyValue conforming to the "code.lineno" +// semantic conventions. It represents the line number in `code.filepath` best +// representing the operation. It SHOULD point within the code unit named in +// `code.function`. +func CodeLineNumber(val int) attribute.KeyValue { + return CodeLineNumberKey.Int(val) +} + +// CodeColumn returns an attribute KeyValue conforming to the "code.column" +// semantic conventions. It represents the column number in `code.filepath` +// best representing the operation. It SHOULD point within the code unit named +// in `code.function`. +func CodeColumn(val int) attribute.KeyValue { + return CodeColumnKey.Int(val) +} + +// Semantic conventions for HTTP client and server Spans. +const ( + // HTTPMethodKey is the attribute Key conforming to the "http.method" + // semantic conventions. It represents the hTTP request method. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'GET', 'POST', 'HEAD' + HTTPMethodKey = attribute.Key("http.method") + + // HTTPStatusCodeKey is the attribute Key conforming to the + // "http.status_code" semantic conventions. It represents the [HTTP + // response status code](https://tools.ietf.org/html/rfc7231#section-6). + // + // Type: int + // RequirementLevel: ConditionallyRequired (If and only if one was + // received/sent.) + // Stability: stable + // Examples: 200 + HTTPStatusCodeKey = attribute.Key("http.status_code") + + // HTTPFlavorKey is the attribute Key conforming to the "http.flavor" + // semantic conventions. It represents the kind of HTTP protocol used. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + // Note: If `net.transport` is not specified, it can be assumed to be + // `IP.TCP` except if `http.flavor` is `QUIC`, in which case `IP.UDP` is + // assumed. + HTTPFlavorKey = attribute.Key("http.flavor") + + // HTTPUserAgentKey is the attribute Key conforming to the + // "http.user_agent" semantic conventions. It represents the value of the + // [HTTP + // User-Agent](https://www.rfc-editor.org/rfc/rfc9110.html#field.user-agent) + // header sent by the client. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'CERN-LineMode/2.15 libwww/2.17b3' + HTTPUserAgentKey = attribute.Key("http.user_agent") + + // HTTPRequestContentLengthKey is the attribute Key conforming to the + // "http.request_content_length" semantic conventions. It represents the + // size of the request payload body in bytes. This is the number of bytes + // transferred excluding headers and is often, but not always, present as + // the + // [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) + // header. For requests using transport encoding, this should be the + // compressed size. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 3495 + HTTPRequestContentLengthKey = attribute.Key("http.request_content_length") + + // HTTPResponseContentLengthKey is the attribute Key conforming to the + // "http.response_content_length" semantic conventions. It represents the + // size of the response payload body in bytes. This is the number of bytes + // transferred excluding headers and is often, but not always, present as + // the + // [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) + // header. For requests using transport encoding, this should be the + // compressed size. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 3495 + HTTPResponseContentLengthKey = attribute.Key("http.response_content_length") +) + +var ( + // HTTP/1.0 + HTTPFlavorHTTP10 = HTTPFlavorKey.String("1.0") + // HTTP/1.1 + HTTPFlavorHTTP11 = HTTPFlavorKey.String("1.1") + // HTTP/2 + HTTPFlavorHTTP20 = HTTPFlavorKey.String("2.0") + // HTTP/3 + HTTPFlavorHTTP30 = HTTPFlavorKey.String("3.0") + // SPDY protocol + HTTPFlavorSPDY = HTTPFlavorKey.String("SPDY") + // QUIC protocol + HTTPFlavorQUIC = HTTPFlavorKey.String("QUIC") +) + +// HTTPMethod returns an attribute KeyValue conforming to the "http.method" +// semantic conventions. It represents the hTTP request method. +func HTTPMethod(val string) attribute.KeyValue { + return HTTPMethodKey.String(val) +} + +// HTTPStatusCode returns an attribute KeyValue conforming to the +// "http.status_code" semantic conventions. It represents the [HTTP response +// status code](https://tools.ietf.org/html/rfc7231#section-6). +func HTTPStatusCode(val int) attribute.KeyValue { + return HTTPStatusCodeKey.Int(val) +} + +// HTTPUserAgent returns an attribute KeyValue conforming to the +// "http.user_agent" semantic conventions. It represents the value of the [HTTP +// User-Agent](https://www.rfc-editor.org/rfc/rfc9110.html#field.user-agent) +// header sent by the client. +func HTTPUserAgent(val string) attribute.KeyValue { + return HTTPUserAgentKey.String(val) +} + +// HTTPRequestContentLength returns an attribute KeyValue conforming to the +// "http.request_content_length" semantic conventions. It represents the size +// of the request payload body in bytes. This is the number of bytes +// transferred excluding headers and is often, but not always, present as the +// [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) +// header. For requests using transport encoding, this should be the compressed +// size. +func HTTPRequestContentLength(val int) attribute.KeyValue { + return HTTPRequestContentLengthKey.Int(val) +} + +// HTTPResponseContentLength returns an attribute KeyValue conforming to the +// "http.response_content_length" semantic conventions. It represents the size +// of the response payload body in bytes. This is the number of bytes +// transferred excluding headers and is often, but not always, present as the +// [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) +// header. For requests using transport encoding, this should be the compressed +// size. +func HTTPResponseContentLength(val int) attribute.KeyValue { + return HTTPResponseContentLengthKey.Int(val) +} + +// Semantic Convention for HTTP Client +const ( + // HTTPURLKey is the attribute Key conforming to the "http.url" semantic + // conventions. It represents the full HTTP request URL in the form + // `scheme://host[:port]/path?query[#fragment]`. Usually the fragment is + // not transmitted over HTTP, but if it is known, it should be included + // nevertheless. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'https://www.foo.bar/search?q=OpenTelemetry#SemConv' + // Note: `http.url` MUST NOT contain credentials passed via URL in form of + // `https://username:password@www.example.com/`. In such case the + // attribute's value should be `https://www.example.com/`. + HTTPURLKey = attribute.Key("http.url") + + // HTTPResendCountKey is the attribute Key conforming to the + // "http.resend_count" semantic conventions. It represents the ordinal + // number of request resending attempt (for any reason, including + // redirects). + // + // Type: int + // RequirementLevel: Recommended (if and only if request was retried.) + // Stability: stable + // Examples: 3 + // Note: The resend count SHOULD be updated each time an HTTP request gets + // resent by the client, regardless of what was the cause of the resending + // (e.g. redirection, authorization failure, 503 Server Unavailable, + // network issues, or any other). + HTTPResendCountKey = attribute.Key("http.resend_count") +) + +// HTTPURL returns an attribute KeyValue conforming to the "http.url" +// semantic conventions. It represents the full HTTP request URL in the form +// `scheme://host[:port]/path?query[#fragment]`. Usually the fragment is not +// transmitted over HTTP, but if it is known, it should be included +// nevertheless. +func HTTPURL(val string) attribute.KeyValue { + return HTTPURLKey.String(val) +} + +// HTTPResendCount returns an attribute KeyValue conforming to the +// "http.resend_count" semantic conventions. It represents the ordinal number +// of request resending attempt (for any reason, including redirects). +func HTTPResendCount(val int) attribute.KeyValue { + return HTTPResendCountKey.Int(val) +} + +// Semantic Convention for HTTP Server +const ( + // HTTPSchemeKey is the attribute Key conforming to the "http.scheme" + // semantic conventions. It represents the URI scheme identifying the used + // protocol. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'http', 'https' + HTTPSchemeKey = attribute.Key("http.scheme") + + // HTTPTargetKey is the attribute Key conforming to the "http.target" + // semantic conventions. It represents the full request target as passed in + // a HTTP request line or equivalent. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: '/path/12314/?q=ddds' + HTTPTargetKey = attribute.Key("http.target") + + // HTTPRouteKey is the attribute Key conforming to the "http.route" + // semantic conventions. It represents the matched route (path template in + // the format used by the respective server framework). See note below + // + // Type: string + // RequirementLevel: ConditionallyRequired (If and only if it's available) + // Stability: stable + // Examples: '/users/:userID?', '{controller}/{action}/{id?}' + // Note: MUST NOT be populated when this is not supported by the HTTP + // server framework as the route attribute should have low-cardinality and + // the URI path can NOT substitute it. + // SHOULD include the [application root](#http-server-definitions) if there + // is one. + HTTPRouteKey = attribute.Key("http.route") + + // HTTPClientIPKey is the attribute Key conforming to the "http.client_ip" + // semantic conventions. It represents the IP address of the original + // client behind all proxies, if known (e.g. from + // [X-Forwarded-For](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/X-Forwarded-For)). + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '83.164.160.102' + // Note: This is not necessarily the same as `net.sock.peer.addr`, which + // would + // identify the network-level peer, which may be a proxy. + // + // This attribute should be set when a source of information different + // from the one used for `net.sock.peer.addr`, is available even if that + // other + // source just confirms the same value as `net.sock.peer.addr`. + // Rationale: For `net.sock.peer.addr`, one typically does not know if it + // comes from a proxy, reverse proxy, or the actual client. Setting + // `http.client_ip` when it's the same as `net.sock.peer.addr` means that + // one is at least somewhat confident that the address is not that of + // the closest proxy. + HTTPClientIPKey = attribute.Key("http.client_ip") +) + +// HTTPScheme returns an attribute KeyValue conforming to the "http.scheme" +// semantic conventions. It represents the URI scheme identifying the used +// protocol. +func HTTPScheme(val string) attribute.KeyValue { + return HTTPSchemeKey.String(val) +} + +// HTTPTarget returns an attribute KeyValue conforming to the "http.target" +// semantic conventions. It represents the full request target as passed in a +// HTTP request line or equivalent. +func HTTPTarget(val string) attribute.KeyValue { + return HTTPTargetKey.String(val) +} + +// HTTPRoute returns an attribute KeyValue conforming to the "http.route" +// semantic conventions. It represents the matched route (path template in the +// format used by the respective server framework). See note below +func HTTPRoute(val string) attribute.KeyValue { + return HTTPRouteKey.String(val) +} + +// HTTPClientIP returns an attribute KeyValue conforming to the +// "http.client_ip" semantic conventions. It represents the IP address of the +// original client behind all proxies, if known (e.g. from +// [X-Forwarded-For](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/X-Forwarded-For)). +func HTTPClientIP(val string) attribute.KeyValue { + return HTTPClientIPKey.String(val) +} + +// Attributes that exist for multiple DynamoDB request types. +const ( + // AWSDynamoDBTableNamesKey is the attribute Key conforming to the + // "aws.dynamodb.table_names" semantic conventions. It represents the keys + // in the `RequestItems` object field. + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Users', 'Cats' + AWSDynamoDBTableNamesKey = attribute.Key("aws.dynamodb.table_names") + + // AWSDynamoDBConsumedCapacityKey is the attribute Key conforming to the + // "aws.dynamodb.consumed_capacity" semantic conventions. It represents the + // JSON-serialized value of each item in the `ConsumedCapacity` response + // field. + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: '{ "CapacityUnits": number, "GlobalSecondaryIndexes": { + // "string" : { "CapacityUnits": number, "ReadCapacityUnits": number, + // "WriteCapacityUnits": number } }, "LocalSecondaryIndexes": { "string" : + // { "CapacityUnits": number, "ReadCapacityUnits": number, + // "WriteCapacityUnits": number } }, "ReadCapacityUnits": number, "Table": + // { "CapacityUnits": number, "ReadCapacityUnits": number, + // "WriteCapacityUnits": number }, "TableName": "string", + // "WriteCapacityUnits": number }' + AWSDynamoDBConsumedCapacityKey = attribute.Key("aws.dynamodb.consumed_capacity") + + // AWSDynamoDBItemCollectionMetricsKey is the attribute Key conforming to + // the "aws.dynamodb.item_collection_metrics" semantic conventions. It + // represents the JSON-serialized value of the `ItemCollectionMetrics` + // response field. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '{ "string" : [ { "ItemCollectionKey": { "string" : { "B": + // blob, "BOOL": boolean, "BS": [ blob ], "L": [ "AttributeValue" ], "M": { + // "string" : "AttributeValue" }, "N": "string", "NS": [ "string" ], + // "NULL": boolean, "S": "string", "SS": [ "string" ] } }, + // "SizeEstimateRangeGB": [ number ] } ] }' + AWSDynamoDBItemCollectionMetricsKey = attribute.Key("aws.dynamodb.item_collection_metrics") + + // AWSDynamoDBProvisionedReadCapacityKey is the attribute Key conforming to + // the "aws.dynamodb.provisioned_read_capacity" semantic conventions. It + // represents the value of the `ProvisionedThroughput.ReadCapacityUnits` + // request parameter. + // + // Type: double + // RequirementLevel: Optional + // Stability: stable + // Examples: 1.0, 2.0 + AWSDynamoDBProvisionedReadCapacityKey = attribute.Key("aws.dynamodb.provisioned_read_capacity") + + // AWSDynamoDBProvisionedWriteCapacityKey is the attribute Key conforming + // to the "aws.dynamodb.provisioned_write_capacity" semantic conventions. + // It represents the value of the + // `ProvisionedThroughput.WriteCapacityUnits` request parameter. + // + // Type: double + // RequirementLevel: Optional + // Stability: stable + // Examples: 1.0, 2.0 + AWSDynamoDBProvisionedWriteCapacityKey = attribute.Key("aws.dynamodb.provisioned_write_capacity") + + // AWSDynamoDBConsistentReadKey is the attribute Key conforming to the + // "aws.dynamodb.consistent_read" semantic conventions. It represents the + // value of the `ConsistentRead` request parameter. + // + // Type: boolean + // RequirementLevel: Optional + // Stability: stable + AWSDynamoDBConsistentReadKey = attribute.Key("aws.dynamodb.consistent_read") + + // AWSDynamoDBProjectionKey is the attribute Key conforming to the + // "aws.dynamodb.projection" semantic conventions. It represents the value + // of the `ProjectionExpression` request parameter. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Title', 'Title, Price, Color', 'Title, Description, + // RelatedItems, ProductReviews' + AWSDynamoDBProjectionKey = attribute.Key("aws.dynamodb.projection") + + // AWSDynamoDBLimitKey is the attribute Key conforming to the + // "aws.dynamodb.limit" semantic conventions. It represents the value of + // the `Limit` request parameter. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 10 + AWSDynamoDBLimitKey = attribute.Key("aws.dynamodb.limit") + + // AWSDynamoDBAttributesToGetKey is the attribute Key conforming to the + // "aws.dynamodb.attributes_to_get" semantic conventions. It represents the + // value of the `AttributesToGet` request parameter. + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: 'lives', 'id' + AWSDynamoDBAttributesToGetKey = attribute.Key("aws.dynamodb.attributes_to_get") + + // AWSDynamoDBIndexNameKey is the attribute Key conforming to the + // "aws.dynamodb.index_name" semantic conventions. It represents the value + // of the `IndexName` request parameter. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'name_to_group' + AWSDynamoDBIndexNameKey = attribute.Key("aws.dynamodb.index_name") + + // AWSDynamoDBSelectKey is the attribute Key conforming to the + // "aws.dynamodb.select" semantic conventions. It represents the value of + // the `Select` request parameter. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'ALL_ATTRIBUTES', 'COUNT' + AWSDynamoDBSelectKey = attribute.Key("aws.dynamodb.select") +) + +// AWSDynamoDBTableNames returns an attribute KeyValue conforming to the +// "aws.dynamodb.table_names" semantic conventions. It represents the keys in +// the `RequestItems` object field. +func AWSDynamoDBTableNames(val ...string) attribute.KeyValue { + return AWSDynamoDBTableNamesKey.StringSlice(val) +} + +// AWSDynamoDBConsumedCapacity returns an attribute KeyValue conforming to +// the "aws.dynamodb.consumed_capacity" semantic conventions. It represents the +// JSON-serialized value of each item in the `ConsumedCapacity` response field. +func AWSDynamoDBConsumedCapacity(val ...string) attribute.KeyValue { + return AWSDynamoDBConsumedCapacityKey.StringSlice(val) +} + +// AWSDynamoDBItemCollectionMetrics returns an attribute KeyValue conforming +// to the "aws.dynamodb.item_collection_metrics" semantic conventions. It +// represents the JSON-serialized value of the `ItemCollectionMetrics` response +// field. +func AWSDynamoDBItemCollectionMetrics(val string) attribute.KeyValue { + return AWSDynamoDBItemCollectionMetricsKey.String(val) +} + +// AWSDynamoDBProvisionedReadCapacity returns an attribute KeyValue +// conforming to the "aws.dynamodb.provisioned_read_capacity" semantic +// conventions. It represents the value of the +// `ProvisionedThroughput.ReadCapacityUnits` request parameter. +func AWSDynamoDBProvisionedReadCapacity(val float64) attribute.KeyValue { + return AWSDynamoDBProvisionedReadCapacityKey.Float64(val) +} + +// AWSDynamoDBProvisionedWriteCapacity returns an attribute KeyValue +// conforming to the "aws.dynamodb.provisioned_write_capacity" semantic +// conventions. It represents the value of the +// `ProvisionedThroughput.WriteCapacityUnits` request parameter. +func AWSDynamoDBProvisionedWriteCapacity(val float64) attribute.KeyValue { + return AWSDynamoDBProvisionedWriteCapacityKey.Float64(val) +} + +// AWSDynamoDBConsistentRead returns an attribute KeyValue conforming to the +// "aws.dynamodb.consistent_read" semantic conventions. It represents the value +// of the `ConsistentRead` request parameter. +func AWSDynamoDBConsistentRead(val bool) attribute.KeyValue { + return AWSDynamoDBConsistentReadKey.Bool(val) +} + +// AWSDynamoDBProjection returns an attribute KeyValue conforming to the +// "aws.dynamodb.projection" semantic conventions. It represents the value of +// the `ProjectionExpression` request parameter. +func AWSDynamoDBProjection(val string) attribute.KeyValue { + return AWSDynamoDBProjectionKey.String(val) +} + +// AWSDynamoDBLimit returns an attribute KeyValue conforming to the +// "aws.dynamodb.limit" semantic conventions. It represents the value of the +// `Limit` request parameter. +func AWSDynamoDBLimit(val int) attribute.KeyValue { + return AWSDynamoDBLimitKey.Int(val) +} + +// AWSDynamoDBAttributesToGet returns an attribute KeyValue conforming to +// the "aws.dynamodb.attributes_to_get" semantic conventions. It represents the +// value of the `AttributesToGet` request parameter. +func AWSDynamoDBAttributesToGet(val ...string) attribute.KeyValue { + return AWSDynamoDBAttributesToGetKey.StringSlice(val) +} + +// AWSDynamoDBIndexName returns an attribute KeyValue conforming to the +// "aws.dynamodb.index_name" semantic conventions. It represents the value of +// the `IndexName` request parameter. +func AWSDynamoDBIndexName(val string) attribute.KeyValue { + return AWSDynamoDBIndexNameKey.String(val) +} + +// AWSDynamoDBSelect returns an attribute KeyValue conforming to the +// "aws.dynamodb.select" semantic conventions. It represents the value of the +// `Select` request parameter. +func AWSDynamoDBSelect(val string) attribute.KeyValue { + return AWSDynamoDBSelectKey.String(val) +} + +// DynamoDB.CreateTable +const ( + // AWSDynamoDBGlobalSecondaryIndexesKey is the attribute Key conforming to + // the "aws.dynamodb.global_secondary_indexes" semantic conventions. It + // represents the JSON-serialized value of each item of the + // `GlobalSecondaryIndexes` request field + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: '{ "IndexName": "string", "KeySchema": [ { "AttributeName": + // "string", "KeyType": "string" } ], "Projection": { "NonKeyAttributes": [ + // "string" ], "ProjectionType": "string" }, "ProvisionedThroughput": { + // "ReadCapacityUnits": number, "WriteCapacityUnits": number } }' + AWSDynamoDBGlobalSecondaryIndexesKey = attribute.Key("aws.dynamodb.global_secondary_indexes") + + // AWSDynamoDBLocalSecondaryIndexesKey is the attribute Key conforming to + // the "aws.dynamodb.local_secondary_indexes" semantic conventions. It + // represents the JSON-serialized value of each item of the + // `LocalSecondaryIndexes` request field. + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: '{ "IndexARN": "string", "IndexName": "string", + // "IndexSizeBytes": number, "ItemCount": number, "KeySchema": [ { + // "AttributeName": "string", "KeyType": "string" } ], "Projection": { + // "NonKeyAttributes": [ "string" ], "ProjectionType": "string" } }' + AWSDynamoDBLocalSecondaryIndexesKey = attribute.Key("aws.dynamodb.local_secondary_indexes") +) + +// AWSDynamoDBGlobalSecondaryIndexes returns an attribute KeyValue +// conforming to the "aws.dynamodb.global_secondary_indexes" semantic +// conventions. It represents the JSON-serialized value of each item of the +// `GlobalSecondaryIndexes` request field +func AWSDynamoDBGlobalSecondaryIndexes(val ...string) attribute.KeyValue { + return AWSDynamoDBGlobalSecondaryIndexesKey.StringSlice(val) +} + +// AWSDynamoDBLocalSecondaryIndexes returns an attribute KeyValue conforming +// to the "aws.dynamodb.local_secondary_indexes" semantic conventions. It +// represents the JSON-serialized value of each item of the +// `LocalSecondaryIndexes` request field. +func AWSDynamoDBLocalSecondaryIndexes(val ...string) attribute.KeyValue { + return AWSDynamoDBLocalSecondaryIndexesKey.StringSlice(val) +} + +// DynamoDB.ListTables +const ( + // AWSDynamoDBExclusiveStartTableKey is the attribute Key conforming to the + // "aws.dynamodb.exclusive_start_table" semantic conventions. It represents + // the value of the `ExclusiveStartTableName` request parameter. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Users', 'CatsTable' + AWSDynamoDBExclusiveStartTableKey = attribute.Key("aws.dynamodb.exclusive_start_table") + + // AWSDynamoDBTableCountKey is the attribute Key conforming to the + // "aws.dynamodb.table_count" semantic conventions. It represents the the + // number of items in the `TableNames` response parameter. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 20 + AWSDynamoDBTableCountKey = attribute.Key("aws.dynamodb.table_count") +) + +// AWSDynamoDBExclusiveStartTable returns an attribute KeyValue conforming +// to the "aws.dynamodb.exclusive_start_table" semantic conventions. It +// represents the value of the `ExclusiveStartTableName` request parameter. +func AWSDynamoDBExclusiveStartTable(val string) attribute.KeyValue { + return AWSDynamoDBExclusiveStartTableKey.String(val) +} + +// AWSDynamoDBTableCount returns an attribute KeyValue conforming to the +// "aws.dynamodb.table_count" semantic conventions. It represents the the +// number of items in the `TableNames` response parameter. +func AWSDynamoDBTableCount(val int) attribute.KeyValue { + return AWSDynamoDBTableCountKey.Int(val) +} + +// DynamoDB.Query +const ( + // AWSDynamoDBScanForwardKey is the attribute Key conforming to the + // "aws.dynamodb.scan_forward" semantic conventions. It represents the + // value of the `ScanIndexForward` request parameter. + // + // Type: boolean + // RequirementLevel: Optional + // Stability: stable + AWSDynamoDBScanForwardKey = attribute.Key("aws.dynamodb.scan_forward") +) + +// AWSDynamoDBScanForward returns an attribute KeyValue conforming to the +// "aws.dynamodb.scan_forward" semantic conventions. It represents the value of +// the `ScanIndexForward` request parameter. +func AWSDynamoDBScanForward(val bool) attribute.KeyValue { + return AWSDynamoDBScanForwardKey.Bool(val) +} + +// DynamoDB.Scan +const ( + // AWSDynamoDBSegmentKey is the attribute Key conforming to the + // "aws.dynamodb.segment" semantic conventions. It represents the value of + // the `Segment` request parameter. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 10 + AWSDynamoDBSegmentKey = attribute.Key("aws.dynamodb.segment") + + // AWSDynamoDBTotalSegmentsKey is the attribute Key conforming to the + // "aws.dynamodb.total_segments" semantic conventions. It represents the + // value of the `TotalSegments` request parameter. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 100 + AWSDynamoDBTotalSegmentsKey = attribute.Key("aws.dynamodb.total_segments") + + // AWSDynamoDBCountKey is the attribute Key conforming to the + // "aws.dynamodb.count" semantic conventions. It represents the value of + // the `Count` response parameter. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 10 + AWSDynamoDBCountKey = attribute.Key("aws.dynamodb.count") + + // AWSDynamoDBScannedCountKey is the attribute Key conforming to the + // "aws.dynamodb.scanned_count" semantic conventions. It represents the + // value of the `ScannedCount` response parameter. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 50 + AWSDynamoDBScannedCountKey = attribute.Key("aws.dynamodb.scanned_count") +) + +// AWSDynamoDBSegment returns an attribute KeyValue conforming to the +// "aws.dynamodb.segment" semantic conventions. It represents the value of the +// `Segment` request parameter. +func AWSDynamoDBSegment(val int) attribute.KeyValue { + return AWSDynamoDBSegmentKey.Int(val) +} + +// AWSDynamoDBTotalSegments returns an attribute KeyValue conforming to the +// "aws.dynamodb.total_segments" semantic conventions. It represents the value +// of the `TotalSegments` request parameter. +func AWSDynamoDBTotalSegments(val int) attribute.KeyValue { + return AWSDynamoDBTotalSegmentsKey.Int(val) +} + +// AWSDynamoDBCount returns an attribute KeyValue conforming to the +// "aws.dynamodb.count" semantic conventions. It represents the value of the +// `Count` response parameter. +func AWSDynamoDBCount(val int) attribute.KeyValue { + return AWSDynamoDBCountKey.Int(val) +} + +// AWSDynamoDBScannedCount returns an attribute KeyValue conforming to the +// "aws.dynamodb.scanned_count" semantic conventions. It represents the value +// of the `ScannedCount` response parameter. +func AWSDynamoDBScannedCount(val int) attribute.KeyValue { + return AWSDynamoDBScannedCountKey.Int(val) +} + +// DynamoDB.UpdateTable +const ( + // AWSDynamoDBAttributeDefinitionsKey is the attribute Key conforming to + // the "aws.dynamodb.attribute_definitions" semantic conventions. It + // represents the JSON-serialized value of each item in the + // `AttributeDefinitions` request field. + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: '{ "AttributeName": "string", "AttributeType": "string" }' + AWSDynamoDBAttributeDefinitionsKey = attribute.Key("aws.dynamodb.attribute_definitions") + + // AWSDynamoDBGlobalSecondaryIndexUpdatesKey is the attribute Key + // conforming to the "aws.dynamodb.global_secondary_index_updates" semantic + // conventions. It represents the JSON-serialized value of each item in the + // the `GlobalSecondaryIndexUpdates` request field. + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: '{ "Create": { "IndexName": "string", "KeySchema": [ { + // "AttributeName": "string", "KeyType": "string" } ], "Projection": { + // "NonKeyAttributes": [ "string" ], "ProjectionType": "string" }, + // "ProvisionedThroughput": { "ReadCapacityUnits": number, + // "WriteCapacityUnits": number } }' + AWSDynamoDBGlobalSecondaryIndexUpdatesKey = attribute.Key("aws.dynamodb.global_secondary_index_updates") +) + +// AWSDynamoDBAttributeDefinitions returns an attribute KeyValue conforming +// to the "aws.dynamodb.attribute_definitions" semantic conventions. It +// represents the JSON-serialized value of each item in the +// `AttributeDefinitions` request field. +func AWSDynamoDBAttributeDefinitions(val ...string) attribute.KeyValue { + return AWSDynamoDBAttributeDefinitionsKey.StringSlice(val) +} + +// AWSDynamoDBGlobalSecondaryIndexUpdates returns an attribute KeyValue +// conforming to the "aws.dynamodb.global_secondary_index_updates" semantic +// conventions. It represents the JSON-serialized value of each item in the the +// `GlobalSecondaryIndexUpdates` request field. +func AWSDynamoDBGlobalSecondaryIndexUpdates(val ...string) attribute.KeyValue { + return AWSDynamoDBGlobalSecondaryIndexUpdatesKey.StringSlice(val) +} + +// Semantic conventions to apply when instrumenting the GraphQL implementation. +// They map GraphQL operations to attributes on a Span. +const ( + // GraphqlOperationNameKey is the attribute Key conforming to the + // "graphql.operation.name" semantic conventions. It represents the name of + // the operation being executed. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'findBookByID' + GraphqlOperationNameKey = attribute.Key("graphql.operation.name") + + // GraphqlOperationTypeKey is the attribute Key conforming to the + // "graphql.operation.type" semantic conventions. It represents the type of + // the operation being executed. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + // Examples: 'query', 'mutation', 'subscription' + GraphqlOperationTypeKey = attribute.Key("graphql.operation.type") + + // GraphqlDocumentKey is the attribute Key conforming to the + // "graphql.document" semantic conventions. It represents the GraphQL + // document being executed. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'query findBookByID { bookByID(id: ?) { name } }' + // Note: The value may be sanitized to exclude sensitive information. + GraphqlDocumentKey = attribute.Key("graphql.document") +) + +var ( + // GraphQL query + GraphqlOperationTypeQuery = GraphqlOperationTypeKey.String("query") + // GraphQL mutation + GraphqlOperationTypeMutation = GraphqlOperationTypeKey.String("mutation") + // GraphQL subscription + GraphqlOperationTypeSubscription = GraphqlOperationTypeKey.String("subscription") +) + +// GraphqlOperationName returns an attribute KeyValue conforming to the +// "graphql.operation.name" semantic conventions. It represents the name of the +// operation being executed. +func GraphqlOperationName(val string) attribute.KeyValue { + return GraphqlOperationNameKey.String(val) +} + +// GraphqlDocument returns an attribute KeyValue conforming to the +// "graphql.document" semantic conventions. It represents the GraphQL document +// being executed. +func GraphqlDocument(val string) attribute.KeyValue { + return GraphqlDocumentKey.String(val) +} + +// Semantic convention describing per-message attributes populated on messaging +// spans or links. +const ( + // MessagingMessageIDKey is the attribute Key conforming to the + // "messaging.message.id" semantic conventions. It represents a value used + // by the messaging system as an identifier for the message, represented as + // a string. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '452a7c7c7c7048c2f887f61572b18fc2' + MessagingMessageIDKey = attribute.Key("messaging.message.id") + + // MessagingMessageConversationIDKey is the attribute Key conforming to the + // "messaging.message.conversation_id" semantic conventions. It represents + // the [conversation ID](#conversations) identifying the conversation to + // which the message belongs, represented as a string. Sometimes called + // "Correlation ID". + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'MyConversationID' + MessagingMessageConversationIDKey = attribute.Key("messaging.message.conversation_id") + + // MessagingMessagePayloadSizeBytesKey is the attribute Key conforming to + // the "messaging.message.payload_size_bytes" semantic conventions. It + // represents the (uncompressed) size of the message payload in bytes. Also + // use this attribute if it is unknown whether the compressed or + // uncompressed payload size is reported. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 2738 + MessagingMessagePayloadSizeBytesKey = attribute.Key("messaging.message.payload_size_bytes") + + // MessagingMessagePayloadCompressedSizeBytesKey is the attribute Key + // conforming to the "messaging.message.payload_compressed_size_bytes" + // semantic conventions. It represents the compressed size of the message + // payload in bytes. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 2048 + MessagingMessagePayloadCompressedSizeBytesKey = attribute.Key("messaging.message.payload_compressed_size_bytes") +) + +// MessagingMessageID returns an attribute KeyValue conforming to the +// "messaging.message.id" semantic conventions. It represents a value used by +// the messaging system as an identifier for the message, represented as a +// string. +func MessagingMessageID(val string) attribute.KeyValue { + return MessagingMessageIDKey.String(val) +} + +// MessagingMessageConversationID returns an attribute KeyValue conforming +// to the "messaging.message.conversation_id" semantic conventions. It +// represents the [conversation ID](#conversations) identifying the +// conversation to which the message belongs, represented as a string. +// Sometimes called "Correlation ID". +func MessagingMessageConversationID(val string) attribute.KeyValue { + return MessagingMessageConversationIDKey.String(val) +} + +// MessagingMessagePayloadSizeBytes returns an attribute KeyValue conforming +// to the "messaging.message.payload_size_bytes" semantic conventions. It +// represents the (uncompressed) size of the message payload in bytes. Also use +// this attribute if it is unknown whether the compressed or uncompressed +// payload size is reported. +func MessagingMessagePayloadSizeBytes(val int) attribute.KeyValue { + return MessagingMessagePayloadSizeBytesKey.Int(val) +} + +// MessagingMessagePayloadCompressedSizeBytes returns an attribute KeyValue +// conforming to the "messaging.message.payload_compressed_size_bytes" semantic +// conventions. It represents the compressed size of the message payload in +// bytes. +func MessagingMessagePayloadCompressedSizeBytes(val int) attribute.KeyValue { + return MessagingMessagePayloadCompressedSizeBytesKey.Int(val) +} + +// Semantic convention for attributes that describe messaging destination on +// broker +const ( + // MessagingDestinationNameKey is the attribute Key conforming to the + // "messaging.destination.name" semantic conventions. It represents the + // message destination name + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'MyQueue', 'MyTopic' + // Note: Destination name SHOULD uniquely identify a specific queue, topic + // or other entity within the broker. If + // the broker does not have such notion, the destination name SHOULD + // uniquely identify the broker. + MessagingDestinationNameKey = attribute.Key("messaging.destination.name") + + // MessagingDestinationKindKey is the attribute Key conforming to the + // "messaging.destination.kind" semantic conventions. It represents the + // kind of message destination + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + MessagingDestinationKindKey = attribute.Key("messaging.destination.kind") + + // MessagingDestinationTemplateKey is the attribute Key conforming to the + // "messaging.destination.template" semantic conventions. It represents the + // low cardinality representation of the messaging destination name + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '/customers/{customerID}' + // Note: Destination names could be constructed from templates. An example + // would be a destination name involving a user name or product id. + // Although the destination name in this case is of high cardinality, the + // underlying template is of low cardinality and can be effectively used + // for grouping and aggregation. + MessagingDestinationTemplateKey = attribute.Key("messaging.destination.template") + + // MessagingDestinationTemporaryKey is the attribute Key conforming to the + // "messaging.destination.temporary" semantic conventions. It represents a + // boolean that is true if the message destination is temporary and might + // not exist anymore after messages are processed. + // + // Type: boolean + // RequirementLevel: Optional + // Stability: stable + MessagingDestinationTemporaryKey = attribute.Key("messaging.destination.temporary") + + // MessagingDestinationAnonymousKey is the attribute Key conforming to the + // "messaging.destination.anonymous" semantic conventions. It represents a + // boolean that is true if the message destination is anonymous (could be + // unnamed or have auto-generated name). + // + // Type: boolean + // RequirementLevel: Optional + // Stability: stable + MessagingDestinationAnonymousKey = attribute.Key("messaging.destination.anonymous") +) + +var ( + // A message sent to a queue + MessagingDestinationKindQueue = MessagingDestinationKindKey.String("queue") + // A message sent to a topic + MessagingDestinationKindTopic = MessagingDestinationKindKey.String("topic") +) + +// MessagingDestinationName returns an attribute KeyValue conforming to the +// "messaging.destination.name" semantic conventions. It represents the message +// destination name +func MessagingDestinationName(val string) attribute.KeyValue { + return MessagingDestinationNameKey.String(val) +} + +// MessagingDestinationTemplate returns an attribute KeyValue conforming to +// the "messaging.destination.template" semantic conventions. It represents the +// low cardinality representation of the messaging destination name +func MessagingDestinationTemplate(val string) attribute.KeyValue { + return MessagingDestinationTemplateKey.String(val) +} + +// MessagingDestinationTemporary returns an attribute KeyValue conforming to +// the "messaging.destination.temporary" semantic conventions. It represents a +// boolean that is true if the message destination is temporary and might not +// exist anymore after messages are processed. +func MessagingDestinationTemporary(val bool) attribute.KeyValue { + return MessagingDestinationTemporaryKey.Bool(val) +} + +// MessagingDestinationAnonymous returns an attribute KeyValue conforming to +// the "messaging.destination.anonymous" semantic conventions. It represents a +// boolean that is true if the message destination is anonymous (could be +// unnamed or have auto-generated name). +func MessagingDestinationAnonymous(val bool) attribute.KeyValue { + return MessagingDestinationAnonymousKey.Bool(val) +} + +// Semantic convention for attributes that describe messaging source on broker +const ( + // MessagingSourceNameKey is the attribute Key conforming to the + // "messaging.source.name" semantic conventions. It represents the message + // source name + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'MyQueue', 'MyTopic' + // Note: Source name SHOULD uniquely identify a specific queue, topic, or + // other entity within the broker. If + // the broker does not have such notion, the source name SHOULD uniquely + // identify the broker. + MessagingSourceNameKey = attribute.Key("messaging.source.name") + + // MessagingSourceKindKey is the attribute Key conforming to the + // "messaging.source.kind" semantic conventions. It represents the kind of + // message source + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + MessagingSourceKindKey = attribute.Key("messaging.source.kind") + + // MessagingSourceTemplateKey is the attribute Key conforming to the + // "messaging.source.template" semantic conventions. It represents the low + // cardinality representation of the messaging source name + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '/customers/{customerID}' + // Note: Source names could be constructed from templates. An example would + // be a source name involving a user name or product id. Although the + // source name in this case is of high cardinality, the underlying template + // is of low cardinality and can be effectively used for grouping and + // aggregation. + MessagingSourceTemplateKey = attribute.Key("messaging.source.template") + + // MessagingSourceTemporaryKey is the attribute Key conforming to the + // "messaging.source.temporary" semantic conventions. It represents a + // boolean that is true if the message source is temporary and might not + // exist anymore after messages are processed. + // + // Type: boolean + // RequirementLevel: Optional + // Stability: stable + MessagingSourceTemporaryKey = attribute.Key("messaging.source.temporary") + + // MessagingSourceAnonymousKey is the attribute Key conforming to the + // "messaging.source.anonymous" semantic conventions. It represents a + // boolean that is true if the message source is anonymous (could be + // unnamed or have auto-generated name). + // + // Type: boolean + // RequirementLevel: Optional + // Stability: stable + MessagingSourceAnonymousKey = attribute.Key("messaging.source.anonymous") +) + +var ( + // A message received from a queue + MessagingSourceKindQueue = MessagingSourceKindKey.String("queue") + // A message received from a topic + MessagingSourceKindTopic = MessagingSourceKindKey.String("topic") +) + +// MessagingSourceName returns an attribute KeyValue conforming to the +// "messaging.source.name" semantic conventions. It represents the message +// source name +func MessagingSourceName(val string) attribute.KeyValue { + return MessagingSourceNameKey.String(val) +} + +// MessagingSourceTemplate returns an attribute KeyValue conforming to the +// "messaging.source.template" semantic conventions. It represents the low +// cardinality representation of the messaging source name +func MessagingSourceTemplate(val string) attribute.KeyValue { + return MessagingSourceTemplateKey.String(val) +} + +// MessagingSourceTemporary returns an attribute KeyValue conforming to the +// "messaging.source.temporary" semantic conventions. It represents a boolean +// that is true if the message source is temporary and might not exist anymore +// after messages are processed. +func MessagingSourceTemporary(val bool) attribute.KeyValue { + return MessagingSourceTemporaryKey.Bool(val) +} + +// MessagingSourceAnonymous returns an attribute KeyValue conforming to the +// "messaging.source.anonymous" semantic conventions. It represents a boolean +// that is true if the message source is anonymous (could be unnamed or have +// auto-generated name). +func MessagingSourceAnonymous(val bool) attribute.KeyValue { + return MessagingSourceAnonymousKey.Bool(val) +} + +// General attributes used in messaging systems. +const ( + // MessagingSystemKey is the attribute Key conforming to the + // "messaging.system" semantic conventions. It represents a string + // identifying the messaging system. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'kafka', 'rabbitmq', 'rocketmq', 'activemq', 'AmazonSQS' + MessagingSystemKey = attribute.Key("messaging.system") + + // MessagingOperationKey is the attribute Key conforming to the + // "messaging.operation" semantic conventions. It represents a string + // identifying the kind of messaging operation as defined in the [Operation + // names](#operation-names) section above. + // + // Type: Enum + // RequirementLevel: Required + // Stability: stable + // Note: If a custom value is used, it MUST be of low cardinality. + MessagingOperationKey = attribute.Key("messaging.operation") + + // MessagingBatchMessageCountKey is the attribute Key conforming to the + // "messaging.batch.message_count" semantic conventions. It represents the + // number of messages sent, received, or processed in the scope of the + // batching operation. + // + // Type: int + // RequirementLevel: ConditionallyRequired (If the span describes an + // operation on a batch of messages.) + // Stability: stable + // Examples: 0, 1, 2 + // Note: Instrumentations SHOULD NOT set `messaging.batch.message_count` on + // spans that operate with a single message. When a messaging client + // library supports both batch and single-message API for the same + // operation, instrumentations SHOULD use `messaging.batch.message_count` + // for batching APIs and SHOULD NOT use it for single-message APIs. + MessagingBatchMessageCountKey = attribute.Key("messaging.batch.message_count") +) + +var ( + // publish + MessagingOperationPublish = MessagingOperationKey.String("publish") + // receive + MessagingOperationReceive = MessagingOperationKey.String("receive") + // process + MessagingOperationProcess = MessagingOperationKey.String("process") +) + +// MessagingSystem returns an attribute KeyValue conforming to the +// "messaging.system" semantic conventions. It represents a string identifying +// the messaging system. +func MessagingSystem(val string) attribute.KeyValue { + return MessagingSystemKey.String(val) +} + +// MessagingBatchMessageCount returns an attribute KeyValue conforming to +// the "messaging.batch.message_count" semantic conventions. It represents the +// number of messages sent, received, or processed in the scope of the batching +// operation. +func MessagingBatchMessageCount(val int) attribute.KeyValue { + return MessagingBatchMessageCountKey.Int(val) +} + +// Semantic convention for a consumer of messages received from a messaging +// system +const ( + // MessagingConsumerIDKey is the attribute Key conforming to the + // "messaging.consumer.id" semantic conventions. It represents the + // identifier for the consumer receiving a message. For Kafka, set it to + // `{messaging.kafka.consumer.group} - {messaging.kafka.client_id}`, if + // both are present, or only `messaging.kafka.consumer.group`. For brokers, + // such as RabbitMQ and Artemis, set it to the `client_id` of the client + // consuming the message. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'mygroup - client-6' + MessagingConsumerIDKey = attribute.Key("messaging.consumer.id") +) + +// MessagingConsumerID returns an attribute KeyValue conforming to the +// "messaging.consumer.id" semantic conventions. It represents the identifier +// for the consumer receiving a message. For Kafka, set it to +// `{messaging.kafka.consumer.group} - {messaging.kafka.client_id}`, if both +// are present, or only `messaging.kafka.consumer.group`. For brokers, such as +// RabbitMQ and Artemis, set it to the `client_id` of the client consuming the +// message. +func MessagingConsumerID(val string) attribute.KeyValue { + return MessagingConsumerIDKey.String(val) +} + +// Attributes for RabbitMQ +const ( + // MessagingRabbitmqDestinationRoutingKeyKey is the attribute Key + // conforming to the "messaging.rabbitmq.destination.routing_key" semantic + // conventions. It represents the rabbitMQ message routing key. + // + // Type: string + // RequirementLevel: ConditionallyRequired (If not empty.) + // Stability: stable + // Examples: 'myKey' + MessagingRabbitmqDestinationRoutingKeyKey = attribute.Key("messaging.rabbitmq.destination.routing_key") +) + +// MessagingRabbitmqDestinationRoutingKey returns an attribute KeyValue +// conforming to the "messaging.rabbitmq.destination.routing_key" semantic +// conventions. It represents the rabbitMQ message routing key. +func MessagingRabbitmqDestinationRoutingKey(val string) attribute.KeyValue { + return MessagingRabbitmqDestinationRoutingKeyKey.String(val) +} + +// Attributes for Apache Kafka +const ( + // MessagingKafkaMessageKeyKey is the attribute Key conforming to the + // "messaging.kafka.message.key" semantic conventions. It represents the + // message keys in Kafka are used for grouping alike messages to ensure + // they're processed on the same partition. They differ from + // `messaging.message.id` in that they're not unique. If the key is `null`, + // the attribute MUST NOT be set. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'myKey' + // Note: If the key type is not string, it's string representation has to + // be supplied for the attribute. If the key has no unambiguous, canonical + // string form, don't include its value. + MessagingKafkaMessageKeyKey = attribute.Key("messaging.kafka.message.key") + + // MessagingKafkaConsumerGroupKey is the attribute Key conforming to the + // "messaging.kafka.consumer.group" semantic conventions. It represents the + // name of the Kafka Consumer Group that is handling the message. Only + // applies to consumers, not producers. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'my-group' + MessagingKafkaConsumerGroupKey = attribute.Key("messaging.kafka.consumer.group") + + // MessagingKafkaClientIDKey is the attribute Key conforming to the + // "messaging.kafka.client_id" semantic conventions. It represents the + // client ID for the Consumer or Producer that is handling the message. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'client-5' + MessagingKafkaClientIDKey = attribute.Key("messaging.kafka.client_id") + + // MessagingKafkaDestinationPartitionKey is the attribute Key conforming to + // the "messaging.kafka.destination.partition" semantic conventions. It + // represents the partition the message is sent to. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 2 + MessagingKafkaDestinationPartitionKey = attribute.Key("messaging.kafka.destination.partition") + + // MessagingKafkaSourcePartitionKey is the attribute Key conforming to the + // "messaging.kafka.source.partition" semantic conventions. It represents + // the partition the message is received from. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 2 + MessagingKafkaSourcePartitionKey = attribute.Key("messaging.kafka.source.partition") + + // MessagingKafkaMessageOffsetKey is the attribute Key conforming to the + // "messaging.kafka.message.offset" semantic conventions. It represents the + // offset of a record in the corresponding Kafka partition. + // + // Type: int + // RequirementLevel: Optional + // Stability: stable + // Examples: 42 + MessagingKafkaMessageOffsetKey = attribute.Key("messaging.kafka.message.offset") + + // MessagingKafkaMessageTombstoneKey is the attribute Key conforming to the + // "messaging.kafka.message.tombstone" semantic conventions. It represents + // a boolean that is true if the message is a tombstone. + // + // Type: boolean + // RequirementLevel: ConditionallyRequired (If value is `true`. When + // missing, the value is assumed to be `false`.) + // Stability: stable + MessagingKafkaMessageTombstoneKey = attribute.Key("messaging.kafka.message.tombstone") +) + +// MessagingKafkaMessageKey returns an attribute KeyValue conforming to the +// "messaging.kafka.message.key" semantic conventions. It represents the +// message keys in Kafka are used for grouping alike messages to ensure they're +// processed on the same partition. They differ from `messaging.message.id` in +// that they're not unique. If the key is `null`, the attribute MUST NOT be +// set. +func MessagingKafkaMessageKey(val string) attribute.KeyValue { + return MessagingKafkaMessageKeyKey.String(val) +} + +// MessagingKafkaConsumerGroup returns an attribute KeyValue conforming to +// the "messaging.kafka.consumer.group" semantic conventions. It represents the +// name of the Kafka Consumer Group that is handling the message. Only applies +// to consumers, not producers. +func MessagingKafkaConsumerGroup(val string) attribute.KeyValue { + return MessagingKafkaConsumerGroupKey.String(val) +} + +// MessagingKafkaClientID returns an attribute KeyValue conforming to the +// "messaging.kafka.client_id" semantic conventions. It represents the client +// ID for the Consumer or Producer that is handling the message. +func MessagingKafkaClientID(val string) attribute.KeyValue { + return MessagingKafkaClientIDKey.String(val) +} + +// MessagingKafkaDestinationPartition returns an attribute KeyValue +// conforming to the "messaging.kafka.destination.partition" semantic +// conventions. It represents the partition the message is sent to. +func MessagingKafkaDestinationPartition(val int) attribute.KeyValue { + return MessagingKafkaDestinationPartitionKey.Int(val) +} + +// MessagingKafkaSourcePartition returns an attribute KeyValue conforming to +// the "messaging.kafka.source.partition" semantic conventions. It represents +// the partition the message is received from. +func MessagingKafkaSourcePartition(val int) attribute.KeyValue { + return MessagingKafkaSourcePartitionKey.Int(val) +} + +// MessagingKafkaMessageOffset returns an attribute KeyValue conforming to +// the "messaging.kafka.message.offset" semantic conventions. It represents the +// offset of a record in the corresponding Kafka partition. +func MessagingKafkaMessageOffset(val int) attribute.KeyValue { + return MessagingKafkaMessageOffsetKey.Int(val) +} + +// MessagingKafkaMessageTombstone returns an attribute KeyValue conforming +// to the "messaging.kafka.message.tombstone" semantic conventions. It +// represents a boolean that is true if the message is a tombstone. +func MessagingKafkaMessageTombstone(val bool) attribute.KeyValue { + return MessagingKafkaMessageTombstoneKey.Bool(val) +} + +// Attributes for Apache RocketMQ +const ( + // MessagingRocketmqNamespaceKey is the attribute Key conforming to the + // "messaging.rocketmq.namespace" semantic conventions. It represents the + // namespace of RocketMQ resources, resources in different namespaces are + // individual. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'myNamespace' + MessagingRocketmqNamespaceKey = attribute.Key("messaging.rocketmq.namespace") + + // MessagingRocketmqClientGroupKey is the attribute Key conforming to the + // "messaging.rocketmq.client_group" semantic conventions. It represents + // the name of the RocketMQ producer/consumer group that is handling the + // message. The client type is identified by the SpanKind. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'myConsumerGroup' + MessagingRocketmqClientGroupKey = attribute.Key("messaging.rocketmq.client_group") + + // MessagingRocketmqClientIDKey is the attribute Key conforming to the + // "messaging.rocketmq.client_id" semantic conventions. It represents the + // unique identifier for each client. + // + // Type: string + // RequirementLevel: Required + // Stability: stable + // Examples: 'myhost@8742@s8083jm' + MessagingRocketmqClientIDKey = attribute.Key("messaging.rocketmq.client_id") + + // MessagingRocketmqMessageDeliveryTimestampKey is the attribute Key + // conforming to the "messaging.rocketmq.message.delivery_timestamp" + // semantic conventions. It represents the timestamp in milliseconds that + // the delay message is expected to be delivered to consumer. + // + // Type: int + // RequirementLevel: ConditionallyRequired (If the message type is delay + // and delay time level is not specified.) + // Stability: stable + // Examples: 1665987217045 + MessagingRocketmqMessageDeliveryTimestampKey = attribute.Key("messaging.rocketmq.message.delivery_timestamp") + + // MessagingRocketmqMessageDelayTimeLevelKey is the attribute Key + // conforming to the "messaging.rocketmq.message.delay_time_level" semantic + // conventions. It represents the delay time level for delay message, which + // determines the message delay time. + // + // Type: int + // RequirementLevel: ConditionallyRequired (If the message type is delay + // and delivery timestamp is not specified.) + // Stability: stable + // Examples: 3 + MessagingRocketmqMessageDelayTimeLevelKey = attribute.Key("messaging.rocketmq.message.delay_time_level") + + // MessagingRocketmqMessageGroupKey is the attribute Key conforming to the + // "messaging.rocketmq.message.group" semantic conventions. It represents + // the it is essential for FIFO message. Messages that belong to the same + // message group are always processed one by one within the same consumer + // group. + // + // Type: string + // RequirementLevel: ConditionallyRequired (If the message type is FIFO.) + // Stability: stable + // Examples: 'myMessageGroup' + MessagingRocketmqMessageGroupKey = attribute.Key("messaging.rocketmq.message.group") + + // MessagingRocketmqMessageTypeKey is the attribute Key conforming to the + // "messaging.rocketmq.message.type" semantic conventions. It represents + // the type of message. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + MessagingRocketmqMessageTypeKey = attribute.Key("messaging.rocketmq.message.type") + + // MessagingRocketmqMessageTagKey is the attribute Key conforming to the + // "messaging.rocketmq.message.tag" semantic conventions. It represents the + // secondary classifier of message besides topic. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'tagA' + MessagingRocketmqMessageTagKey = attribute.Key("messaging.rocketmq.message.tag") + + // MessagingRocketmqMessageKeysKey is the attribute Key conforming to the + // "messaging.rocketmq.message.keys" semantic conventions. It represents + // the key(s) of message, another way to mark message besides message id. + // + // Type: string[] + // RequirementLevel: Optional + // Stability: stable + // Examples: 'keyA', 'keyB' + MessagingRocketmqMessageKeysKey = attribute.Key("messaging.rocketmq.message.keys") + + // MessagingRocketmqConsumptionModelKey is the attribute Key conforming to + // the "messaging.rocketmq.consumption_model" semantic conventions. It + // represents the model of message consumption. This only applies to + // consumer spans. + // + // Type: Enum + // RequirementLevel: Optional + // Stability: stable + MessagingRocketmqConsumptionModelKey = attribute.Key("messaging.rocketmq.consumption_model") +) + +var ( + // Normal message + MessagingRocketmqMessageTypeNormal = MessagingRocketmqMessageTypeKey.String("normal") + // FIFO message + MessagingRocketmqMessageTypeFifo = MessagingRocketmqMessageTypeKey.String("fifo") + // Delay message + MessagingRocketmqMessageTypeDelay = MessagingRocketmqMessageTypeKey.String("delay") + // Transaction message + MessagingRocketmqMessageTypeTransaction = MessagingRocketmqMessageTypeKey.String("transaction") +) + +var ( + // Clustering consumption model + MessagingRocketmqConsumptionModelClustering = MessagingRocketmqConsumptionModelKey.String("clustering") + // Broadcasting consumption model + MessagingRocketmqConsumptionModelBroadcasting = MessagingRocketmqConsumptionModelKey.String("broadcasting") +) + +// MessagingRocketmqNamespace returns an attribute KeyValue conforming to +// the "messaging.rocketmq.namespace" semantic conventions. It represents the +// namespace of RocketMQ resources, resources in different namespaces are +// individual. +func MessagingRocketmqNamespace(val string) attribute.KeyValue { + return MessagingRocketmqNamespaceKey.String(val) +} + +// MessagingRocketmqClientGroup returns an attribute KeyValue conforming to +// the "messaging.rocketmq.client_group" semantic conventions. It represents +// the name of the RocketMQ producer/consumer group that is handling the +// message. The client type is identified by the SpanKind. +func MessagingRocketmqClientGroup(val string) attribute.KeyValue { + return MessagingRocketmqClientGroupKey.String(val) +} + +// MessagingRocketmqClientID returns an attribute KeyValue conforming to the +// "messaging.rocketmq.client_id" semantic conventions. It represents the +// unique identifier for each client. +func MessagingRocketmqClientID(val string) attribute.KeyValue { + return MessagingRocketmqClientIDKey.String(val) +} + +// MessagingRocketmqMessageDeliveryTimestamp returns an attribute KeyValue +// conforming to the "messaging.rocketmq.message.delivery_timestamp" semantic +// conventions. It represents the timestamp in milliseconds that the delay +// message is expected to be delivered to consumer. +func MessagingRocketmqMessageDeliveryTimestamp(val int) attribute.KeyValue { + return MessagingRocketmqMessageDeliveryTimestampKey.Int(val) +} + +// MessagingRocketmqMessageDelayTimeLevel returns an attribute KeyValue +// conforming to the "messaging.rocketmq.message.delay_time_level" semantic +// conventions. It represents the delay time level for delay message, which +// determines the message delay time. +func MessagingRocketmqMessageDelayTimeLevel(val int) attribute.KeyValue { + return MessagingRocketmqMessageDelayTimeLevelKey.Int(val) +} + +// MessagingRocketmqMessageGroup returns an attribute KeyValue conforming to +// the "messaging.rocketmq.message.group" semantic conventions. It represents +// the it is essential for FIFO message. Messages that belong to the same +// message group are always processed one by one within the same consumer +// group. +func MessagingRocketmqMessageGroup(val string) attribute.KeyValue { + return MessagingRocketmqMessageGroupKey.String(val) +} + +// MessagingRocketmqMessageTag returns an attribute KeyValue conforming to +// the "messaging.rocketmq.message.tag" semantic conventions. It represents the +// secondary classifier of message besides topic. +func MessagingRocketmqMessageTag(val string) attribute.KeyValue { + return MessagingRocketmqMessageTagKey.String(val) +} + +// MessagingRocketmqMessageKeys returns an attribute KeyValue conforming to +// the "messaging.rocketmq.message.keys" semantic conventions. It represents +// the key(s) of message, another way to mark message besides message id. +func MessagingRocketmqMessageKeys(val ...string) attribute.KeyValue { + return MessagingRocketmqMessageKeysKey.StringSlice(val) +} + +// Semantic conventions for remote procedure calls. +const ( + // RPCSystemKey is the attribute Key conforming to the "rpc.system" + // semantic conventions. It represents a string identifying the remoting + // system. See below for a list of well-known identifiers. + // + // Type: Enum + // RequirementLevel: Required + // Stability: stable + RPCSystemKey = attribute.Key("rpc.system") + + // RPCServiceKey is the attribute Key conforming to the "rpc.service" + // semantic conventions. It represents the full (logical) name of the + // service being called, including its package name, if applicable. + // + // Type: string + // RequirementLevel: Recommended + // Stability: stable + // Examples: 'myservice.EchoService' + // Note: This is the logical name of the service from the RPC interface + // perspective, which can be different from the name of any implementing + // class. The `code.namespace` attribute may be used to store the latter + // (despite the attribute name, it may include a class name; e.g., class + // with method actually executing the call on the server side, RPC client + // stub class on the client side). + RPCServiceKey = attribute.Key("rpc.service") + + // RPCMethodKey is the attribute Key conforming to the "rpc.method" + // semantic conventions. It represents the name of the (logical) method + // being called, must be equal to the $method part in the span name. + // + // Type: string + // RequirementLevel: Recommended + // Stability: stable + // Examples: 'exampleMethod' + // Note: This is the logical name of the method from the RPC interface + // perspective, which can be different from the name of any implementing + // method/function. The `code.function` attribute may be used to store the + // latter (e.g., method actually executing the call on the server side, RPC + // client stub method on the client side). + RPCMethodKey = attribute.Key("rpc.method") +) + +var ( + // gRPC + RPCSystemGRPC = RPCSystemKey.String("grpc") + // Java RMI + RPCSystemJavaRmi = RPCSystemKey.String("java_rmi") + // .NET WCF + RPCSystemDotnetWcf = RPCSystemKey.String("dotnet_wcf") + // Apache Dubbo + RPCSystemApacheDubbo = RPCSystemKey.String("apache_dubbo") +) + +// RPCService returns an attribute KeyValue conforming to the "rpc.service" +// semantic conventions. It represents the full (logical) name of the service +// being called, including its package name, if applicable. +func RPCService(val string) attribute.KeyValue { + return RPCServiceKey.String(val) +} + +// RPCMethod returns an attribute KeyValue conforming to the "rpc.method" +// semantic conventions. It represents the name of the (logical) method being +// called, must be equal to the $method part in the span name. +func RPCMethod(val string) attribute.KeyValue { + return RPCMethodKey.String(val) +} + +// Tech-specific attributes for gRPC. +const ( + // RPCGRPCStatusCodeKey is the attribute Key conforming to the + // "rpc.grpc.status_code" semantic conventions. It represents the [numeric + // status + // code](https://github.com/grpc/grpc/blob/v1.33.2/doc/statuscodes.md) of + // the gRPC request. + // + // Type: Enum + // RequirementLevel: Required + // Stability: stable + RPCGRPCStatusCodeKey = attribute.Key("rpc.grpc.status_code") +) + +var ( + // OK + RPCGRPCStatusCodeOk = RPCGRPCStatusCodeKey.Int(0) + // CANCELLED + RPCGRPCStatusCodeCancelled = RPCGRPCStatusCodeKey.Int(1) + // UNKNOWN + RPCGRPCStatusCodeUnknown = RPCGRPCStatusCodeKey.Int(2) + // INVALID_ARGUMENT + RPCGRPCStatusCodeInvalidArgument = RPCGRPCStatusCodeKey.Int(3) + // DEADLINE_EXCEEDED + RPCGRPCStatusCodeDeadlineExceeded = RPCGRPCStatusCodeKey.Int(4) + // NOT_FOUND + RPCGRPCStatusCodeNotFound = RPCGRPCStatusCodeKey.Int(5) + // ALREADY_EXISTS + RPCGRPCStatusCodeAlreadyExists = RPCGRPCStatusCodeKey.Int(6) + // PERMISSION_DENIED + RPCGRPCStatusCodePermissionDenied = RPCGRPCStatusCodeKey.Int(7) + // RESOURCE_EXHAUSTED + RPCGRPCStatusCodeResourceExhausted = RPCGRPCStatusCodeKey.Int(8) + // FAILED_PRECONDITION + RPCGRPCStatusCodeFailedPrecondition = RPCGRPCStatusCodeKey.Int(9) + // ABORTED + RPCGRPCStatusCodeAborted = RPCGRPCStatusCodeKey.Int(10) + // OUT_OF_RANGE + RPCGRPCStatusCodeOutOfRange = RPCGRPCStatusCodeKey.Int(11) + // UNIMPLEMENTED + RPCGRPCStatusCodeUnimplemented = RPCGRPCStatusCodeKey.Int(12) + // INTERNAL + RPCGRPCStatusCodeInternal = RPCGRPCStatusCodeKey.Int(13) + // UNAVAILABLE + RPCGRPCStatusCodeUnavailable = RPCGRPCStatusCodeKey.Int(14) + // DATA_LOSS + RPCGRPCStatusCodeDataLoss = RPCGRPCStatusCodeKey.Int(15) + // UNAUTHENTICATED + RPCGRPCStatusCodeUnauthenticated = RPCGRPCStatusCodeKey.Int(16) +) + +// Tech-specific attributes for [JSON RPC](https://www.jsonrpc.org/). +const ( + // RPCJsonrpcVersionKey is the attribute Key conforming to the + // "rpc.jsonrpc.version" semantic conventions. It represents the protocol + // version as in `jsonrpc` property of request/response. Since JSON-RPC 1.0 + // does not specify this, the value can be omitted. + // + // Type: string + // RequirementLevel: ConditionallyRequired (If other than the default + // version (`1.0`)) + // Stability: stable + // Examples: '2.0', '1.0' + RPCJsonrpcVersionKey = attribute.Key("rpc.jsonrpc.version") + + // RPCJsonrpcRequestIDKey is the attribute Key conforming to the + // "rpc.jsonrpc.request_id" semantic conventions. It represents the `id` + // property of request or response. Since protocol allows id to be int, + // string, `null` or missing (for notifications), value is expected to be + // cast to string for simplicity. Use empty string in case of `null` value. + // Omit entirely if this is a notification. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: '10', 'request-7', '' + RPCJsonrpcRequestIDKey = attribute.Key("rpc.jsonrpc.request_id") + + // RPCJsonrpcErrorCodeKey is the attribute Key conforming to the + // "rpc.jsonrpc.error_code" semantic conventions. It represents the + // `error.code` property of response if it is an error response. + // + // Type: int + // RequirementLevel: ConditionallyRequired (If response is not successful.) + // Stability: stable + // Examples: -32700, 100 + RPCJsonrpcErrorCodeKey = attribute.Key("rpc.jsonrpc.error_code") + + // RPCJsonrpcErrorMessageKey is the attribute Key conforming to the + // "rpc.jsonrpc.error_message" semantic conventions. It represents the + // `error.message` property of response if it is an error response. + // + // Type: string + // RequirementLevel: Optional + // Stability: stable + // Examples: 'Parse error', 'User already exists' + RPCJsonrpcErrorMessageKey = attribute.Key("rpc.jsonrpc.error_message") +) + +// RPCJsonrpcVersion returns an attribute KeyValue conforming to the +// "rpc.jsonrpc.version" semantic conventions. It represents the protocol +// version as in `jsonrpc` property of request/response. Since JSON-RPC 1.0 +// does not specify this, the value can be omitted. +func RPCJsonrpcVersion(val string) attribute.KeyValue { + return RPCJsonrpcVersionKey.String(val) +} + +// RPCJsonrpcRequestID returns an attribute KeyValue conforming to the +// "rpc.jsonrpc.request_id" semantic conventions. It represents the `id` +// property of request or response. Since protocol allows id to be int, string, +// `null` or missing (for notifications), value is expected to be cast to +// string for simplicity. Use empty string in case of `null` value. Omit +// entirely if this is a notification. +func RPCJsonrpcRequestID(val string) attribute.KeyValue { + return RPCJsonrpcRequestIDKey.String(val) +} + +// RPCJsonrpcErrorCode returns an attribute KeyValue conforming to the +// "rpc.jsonrpc.error_code" semantic conventions. It represents the +// `error.code` property of response if it is an error response. +func RPCJsonrpcErrorCode(val int) attribute.KeyValue { + return RPCJsonrpcErrorCodeKey.Int(val) +} + +// RPCJsonrpcErrorMessage returns an attribute KeyValue conforming to the +// "rpc.jsonrpc.error_message" semantic conventions. It represents the +// `error.message` property of response if it is an error response. +func RPCJsonrpcErrorMessage(val string) attribute.KeyValue { + return RPCJsonrpcErrorMessageKey.String(val) +} diff --git a/vendor/modules.txt b/vendor/modules.txt index af829edd451b..ee311110825f 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1602,10 +1602,19 @@ github.com/twmb/franz-go/pkg/kgo github.com/twmb/franz-go/pkg/kgo/internal/sticky github.com/twmb/franz-go/pkg/kversion github.com/twmb/franz-go/pkg/sasl +# github.com/twmb/franz-go/pkg/kadm v1.13.0 +## explicit; go 1.21 +github.com/twmb/franz-go/pkg/kadm +# github.com/twmb/franz-go/pkg/kfake v0.0.0-20240821035758-b77dd13e2bfa +## explicit; go 1.21 +github.com/twmb/franz-go/pkg/kfake # github.com/twmb/franz-go/pkg/kmsg v1.8.0 ## explicit; go 1.19 github.com/twmb/franz-go/pkg/kmsg github.com/twmb/franz-go/pkg/kmsg/internal/kbin +# github.com/twmb/franz-go/plugin/kotel v1.5.0 +## explicit; go 1.21 +github.com/twmb/franz-go/plugin/kotel # github.com/twmb/franz-go/plugin/kprom v1.1.0 ## explicit; go 1.18 github.com/twmb/franz-go/plugin/kprom @@ -1758,6 +1767,7 @@ go.opentelemetry.io/otel/internal/baggage go.opentelemetry.io/otel/internal/global go.opentelemetry.io/otel/propagation go.opentelemetry.io/otel/semconv/v1.17.0 +go.opentelemetry.io/otel/semconv/v1.18.0 go.opentelemetry.io/otel/semconv/v1.20.0 go.opentelemetry.io/otel/semconv/v1.21.0 go.opentelemetry.io/otel/semconv/v1.24.0