From e625d7f1caa4c9373215a46cb018ecdfc6563c65 Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Wed, 24 Jan 2024 17:06:01 +0530 Subject: [PATCH 01/23] otel: add support for per tenant configuration for mapping otlp data to loki format (#11143) **What this PR does / why we need it**: In OTEL, we pick select Resource Attributes to identify the streams and store all the other attributes as Structured Metadata, as explained here. The problem however is that the list of Resource Attributes that are picked as Stream labels are hardcoded, and there is no way to drop unwanted data. This PR adds support for configuring how data is mapped from OTEL to Loki format per tenant. It also adds support for dropping unwanted data. We decided to make the config look similar to Prometheus's relabling config to make it familiar. **Special notes for your reviewer**: Opening a draft PR to get some initial feedback. I will add documentation in a separate PR. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated --- CHANGELOG.md | 1 + .../promtail/targets/lokipush/pushtarget.go | 2 +- docs/sources/configure/_index.md | 11 + pkg/distributor/http.go | 2 +- pkg/distributor/limits.go | 2 + pkg/loghttp/push/otlp.go | 142 ++++--- pkg/loghttp/push/otlp_config.go | 166 ++++++++ pkg/loghttp/push/otlp_config_test.go | 355 ++++++++++++++++++ pkg/loghttp/push/otlp_test.go | 160 +++++++- pkg/loghttp/push/push.go | 12 +- pkg/loghttp/push/push_test.go | 2 +- pkg/validation/limits.go | 12 +- pkg/validation/limits_test.go | 19 + 13 files changed, 813 insertions(+), 73 deletions(-) create mode 100644 pkg/loghttp/push/otlp_config.go create mode 100644 pkg/loghttp/push/otlp_config_test.go diff --git a/CHANGELOG.md b/CHANGELOG.md index 04b692efcb00e..8c22d9126f9c6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -51,6 +51,7 @@ * [11654](https://github.com/grafana/loki/pull/11654) **dannykopping** Cache: atomically check background cache size limit correctly. * [11682](https://github.com/grafana/loki/pull/11682) **ashwanthgoli** Metadata cache: Adds `frontend.max-metadata-cache-freshness` to configure the time window for which metadata results are not cached. This helps avoid returning inaccurate results by not caching recent results. * [11679](https://github.com/grafana/loki/pull/11679) **dannykopping** Cache: extending #11535 to align custom ingester query split with cache keys for correct caching of results. +* [11143](https://github.com/grafana/loki/pull/11143) **sandeepsukhani** otel: Add support for per tenant configuration for mapping otlp data to loki format * [11499](https://github.com/grafana/loki/pull/11284) **jmichalek132** Config: Adds `frontend.log-query-request-headers` to enable logging of request headers in query logs. ##### Fixes diff --git a/clients/pkg/promtail/targets/lokipush/pushtarget.go b/clients/pkg/promtail/targets/lokipush/pushtarget.go index 1e8affb389a17..c981de0de3dda 100644 --- a/clients/pkg/promtail/targets/lokipush/pushtarget.go +++ b/clients/pkg/promtail/targets/lokipush/pushtarget.go @@ -111,7 +111,7 @@ func (t *PushTarget) run() error { func (t *PushTarget) handleLoki(w http.ResponseWriter, r *http.Request) { logger := util_log.WithContext(r.Context(), util_log.Logger) userID, _ := tenant.TenantID(r.Context()) - req, err := push.ParseRequest(logger, userID, r, nil, push.ParseLokiRequest) + req, err := push.ParseRequest(logger, userID, r, nil, nil, push.ParseLokiRequest) if err != nil { level.Warn(t.logger).Log("msg", "failed to parse incoming push request", "err", err.Error()) http.Error(w, err.Error(), http.StatusBadRequest) diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index edb394733a3c5..2a2c51544f013 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -3140,6 +3140,17 @@ shard_streams: # Maximum number of structured metadata entries per log line. # CLI flag: -limits.max-structured-metadata-entries-count [max_structured_metadata_entries_count: | default = 128] + +# OTLP log ingestion configurations +otlp_config: + resource_attributes: + [ignore_defaults: ] + + [attributes: ] + + [scope_attributes: ] + + [log_attributes: ] ``` ### frontend_worker diff --git a/pkg/distributor/http.go b/pkg/distributor/http.go index 67db8e5c5ba7b..ce242355e077b 100644 --- a/pkg/distributor/http.go +++ b/pkg/distributor/http.go @@ -34,7 +34,7 @@ func (d *Distributor) pushHandler(w http.ResponseWriter, r *http.Request, pushRe http.Error(w, err.Error(), http.StatusBadRequest) return } - req, err := push.ParseRequest(logger, tenantID, r, d.tenantsRetention, pushRequestParser) + req, err := push.ParseRequest(logger, tenantID, r, d.tenantsRetention, d.validator.Limits, pushRequestParser) if err != nil { if d.tenantConfigs.LogPushRequest(tenantID) { level.Debug(logger).Log( diff --git a/pkg/distributor/limits.go b/pkg/distributor/limits.go index add9d17708dbd..6db6995662dd2 100644 --- a/pkg/distributor/limits.go +++ b/pkg/distributor/limits.go @@ -5,6 +5,7 @@ import ( "github.com/grafana/loki/pkg/compactor/retention" "github.com/grafana/loki/pkg/distributor/shardstreams" + "github.com/grafana/loki/pkg/loghttp/push" ) // Limits is an interface for distributor limits/related configs @@ -29,4 +30,5 @@ type Limits interface { AllowStructuredMetadata(userID string) bool MaxStructuredMetadataSize(userID string) int MaxStructuredMetadataCount(userID string) int + OTLPConfig(userID string) push.OTLPConfig } diff --git a/pkg/loghttp/push/otlp.go b/pkg/loghttp/push/otlp.go index 737e9b78ae72a..f4f937b93dc33 100644 --- a/pkg/loghttp/push/otlp.go +++ b/pkg/loghttp/push/otlp.go @@ -25,28 +25,9 @@ import ( const ( pbContentType = "application/x-protobuf" gzipContentEncoding = "gzip" + attrServiceName = "service.name" ) -var blessedAttributes = []string{ - "service.name", - "service.namespace", - "service.instance.id", - "deployment.environment", - "cloud.region", - "cloud.availability_zone", - "k8s.cluster.name", - "k8s.namespace.name", - "k8s.pod.name", - "k8s.container.name", - "container.name", - "k8s.replicaset.name", - "k8s.deployment.name", - "k8s.statefulset.name", - "k8s.daemonset.name", - "k8s.cronjob.name", - "k8s.job.name", -} - var blessedAttributesNormalized = make([]string, len(blessedAttributes)) func init() { @@ -62,14 +43,14 @@ func newPushStats() *Stats { } } -func ParseOTLPRequest(userID string, r *http.Request, tenantsRetention TenantsRetention) (*logproto.PushRequest, *Stats, error) { +func ParseOTLPRequest(userID string, r *http.Request, tenantsRetention TenantsRetention, limits Limits) (*logproto.PushRequest, *Stats, error) { stats := newPushStats() otlpLogs, err := extractLogs(r, stats) if err != nil { return nil, nil, err } - req := otlpToLokiPushRequest(otlpLogs, userID, tenantsRetention, stats) + req := otlpToLokiPushRequest(otlpLogs, userID, tenantsRetention, limits.OTLPConfig(userID), stats) return req, stats, nil } @@ -120,7 +101,7 @@ func extractLogs(r *http.Request, pushStats *Stats) (plog.Logs, error) { return req.Logs(), nil } -func otlpToLokiPushRequest(ld plog.Logs, userID string, tenantsRetention TenantsRetention, stats *Stats) *logproto.PushRequest { +func otlpToLokiPushRequest(ld plog.Logs, userID string, tenantsRetention TenantsRetention, otlpConfig OTLPConfig, stats *Stats) *logproto.PushRequest { if ld.LogRecordCount() == 0 { return &logproto.PushRequest{} } @@ -131,29 +112,31 @@ func otlpToLokiPushRequest(ld plog.Logs, userID string, tenantsRetention Tenants for i := 0; i < rls.Len(); i++ { sls := rls.At(i).ScopeLogs() res := rls.At(i).Resource() + resAttrs := res.Attributes() - flattenedResourceAttributes := labels.NewBuilder(logproto.FromLabelAdaptersToLabels(attributesToLabels(res.Attributes(), ""))) - // service.name is a required Resource Attribute. If it is not present, we will set it to "unknown_service". - if flattenedResourceAttributes.Get("service_name") == "" { - flattenedResourceAttributes = flattenedResourceAttributes.Set("service_name", "unknown_service") + if v, _ := resAttrs.Get(attrServiceName); v.AsString() == "" { + resAttrs.PutStr(attrServiceName, "unknown_service") } + resourceAttributesAsStructuredMetadata := make(push.LabelsAdapter, 0, resAttrs.Len()) + streamLabels := make(model.LabelSet, len(blessedAttributesNormalized)) - if dac := res.DroppedAttributesCount(); dac != 0 { - flattenedResourceAttributes = flattenedResourceAttributes.Set("resource_dropped_attributes_count", fmt.Sprintf("%d", dac)) - } + resAttrs.Range(func(k string, v pcommon.Value) bool { + action := otlpConfig.ActionForResourceAttribute(k) + if action == Drop { + return true + } - // copy blessed attributes to stream labels - streamLabels := make(model.LabelSet, len(blessedAttributesNormalized)) - for _, ba := range blessedAttributesNormalized { - v := flattenedResourceAttributes.Get(ba) - if v == "" { - continue + attributeAsLabels := attributeToLabels(k, v, "") + if action == IndexLabel { + for _, lbl := range attributeAsLabels { + streamLabels[model.LabelName(lbl.Name)] = model.LabelValue(lbl.Value) + } + } else if action == StructuredMetadata { + resourceAttributesAsStructuredMetadata = append(resourceAttributesAsStructuredMetadata, attributeAsLabels...) } - streamLabels[model.LabelName(ba)] = model.LabelValue(v) - // remove the blessed attributes copied to stream labels - flattenedResourceAttributes.Del(ba) - } + return true + }) if err := streamLabels.Validate(); err != nil { stats.errs = append(stats.errs, fmt.Errorf("invalid labels: %w", err)) @@ -161,9 +144,6 @@ func otlpToLokiPushRequest(ld plog.Logs, userID string, tenantsRetention Tenants } labelsStr := streamLabels.String() - // convert the remaining resource attributes to structured metadata - resourceAttributesAsStructuredMetadata := logproto.FromLabelsToLabelAdapters(flattenedResourceAttributes.Labels()) - lbs := modelLabelsSetToLabelsList(streamLabels) if _, ok := pushRequestsByStream[labelsStr]; !ok { pushRequestsByStream[labelsStr] = logproto.Stream{ @@ -178,6 +158,7 @@ func otlpToLokiPushRequest(ld plog.Logs, userID string, tenantsRetention Tenants for j := 0; j < sls.Len(); j++ { scope := sls.At(j).Scope() logs := sls.At(j).LogRecords() + scopeAttrs := scope.Attributes() // it would be rare to have multiple scopes so if the entries slice is empty, pre-allocate it for the number of log entries if cap(pushRequestsByStream[labelsStr].Entries) == 0 { @@ -187,7 +168,20 @@ func otlpToLokiPushRequest(ld plog.Logs, userID string, tenantsRetention Tenants } // use fields and attributes from scope as structured metadata - scopeAttributesAsStructuredMetadata := attributesToLabels(scope.Attributes(), "") + scopeAttributesAsStructuredMetadata := make(push.LabelsAdapter, 0, scopeAttrs.Len()+3) + scopeAttrs.Range(func(k string, v pcommon.Value) bool { + action := otlpConfig.ActionForScopeAttribute(k) + if action == Drop { + return true + } + + attributeAsLabels := attributeToLabels(k, v, "") + if action == StructuredMetadata { + scopeAttributesAsStructuredMetadata = append(scopeAttributesAsStructuredMetadata, attributeAsLabels...) + } + + return true + }) if scopeName := scope.Name(); scopeName != "" { scopeAttributesAsStructuredMetadata = append(scopeAttributesAsStructuredMetadata, push.LabelAdapter{ @@ -213,7 +207,7 @@ func otlpToLokiPushRequest(ld plog.Logs, userID string, tenantsRetention Tenants for k := 0; k < logs.Len(); k++ { log := logs.At(k) - entry := otlpLogToPushEntry(log) + entry := otlpLogToPushEntry(log, otlpConfig) // if entry.StructuredMetadata doesn't have capacity to add resource and scope attributes, make a new slice with enough capacity attributesAsStructuredMetadataLen := len(resourceAttributesAsStructuredMetadata) + len(scopeAttributesAsStructuredMetadata) @@ -251,9 +245,23 @@ func otlpToLokiPushRequest(ld plog.Logs, userID string, tenantsRetention Tenants } // otlpLogToPushEntry converts an OTLP log record to a Loki push.Entry. -func otlpLogToPushEntry(log plog.LogRecord) push.Entry { +func otlpLogToPushEntry(log plog.LogRecord, otlpConfig OTLPConfig) push.Entry { // copy log attributes and all the fields from log(except log.Body) to structured metadata - structuredMetadata := attributesToLabels(log.Attributes(), "") + logAttrs := log.Attributes() + structuredMetadata := make(push.LabelsAdapter, 0, logAttrs.Len()+7) + logAttrs.Range(func(k string, v pcommon.Value) bool { + action := otlpConfig.ActionForLogAttribute(k) + if action == Drop { + return true + } + + attributeAsLabels := attributeToLabels(k, v, "") + if action == StructuredMetadata { + structuredMetadata = append(structuredMetadata, attributeAsLabels...) + } + + return true + }) // if log.Timestamp() is 0, we would have already stored log.ObservedTimestamp as log timestamp so no need to store again in structured metadata if log.Timestamp() != 0 && log.ObservedTimestamp() != 0 { @@ -316,25 +324,39 @@ func attributesToLabels(attrs pcommon.Map, prefix string) push.LabelsAdapter { } attrs.Range(func(k string, v pcommon.Value) bool { - keyWithPrefix := k - if prefix != "" { - keyWithPrefix = prefix + "_" + k - } - keyWithPrefix = prometheustranslator.NormalizeLabel(keyWithPrefix) - - typ := v.Type() - if typ == pcommon.ValueTypeMap { - labelsAdapter = append(labelsAdapter, attributesToLabels(v.Map(), keyWithPrefix)...) - } else { - labelsAdapter = append(labelsAdapter, push.LabelAdapter{Name: keyWithPrefix, Value: v.AsString()}) - } - + labelsAdapter = append(labelsAdapter, attributeToLabels(k, v, prefix)...) return true }) return labelsAdapter } +func attributeToLabels(k string, v pcommon.Value, prefix string) push.LabelsAdapter { + var labelsAdapter push.LabelsAdapter + + keyWithPrefix := k + if prefix != "" { + keyWithPrefix = prefix + "_" + k + } + keyWithPrefix = prometheustranslator.NormalizeLabel(keyWithPrefix) + + typ := v.Type() + if typ == pcommon.ValueTypeMap { + mv := v.Map() + labelsAdapter = make(push.LabelsAdapter, 0, mv.Len()) + mv.Range(func(k string, v pcommon.Value) bool { + labelsAdapter = append(labelsAdapter, attributeToLabels(k, v, keyWithPrefix)...) + return true + }) + } else { + labelsAdapter = push.LabelsAdapter{ + push.LabelAdapter{Name: keyWithPrefix, Value: v.AsString()}, + } + } + + return labelsAdapter +} + func timestampFromLogRecord(lr plog.LogRecord) time.Time { if lr.Timestamp() != 0 { return time.Unix(0, int64(lr.Timestamp())) diff --git a/pkg/loghttp/push/otlp_config.go b/pkg/loghttp/push/otlp_config.go new file mode 100644 index 0000000000000..64120d4a6252e --- /dev/null +++ b/pkg/loghttp/push/otlp_config.go @@ -0,0 +1,166 @@ +package push + +import ( + "fmt" + + "github.com/prometheus/prometheus/model/relabel" +) + +var blessedAttributes = []string{ + "service.name", + "service.namespace", + "service.instance.id", + "deployment.environment", + "cloud.region", + "cloud.availability_zone", + "k8s.cluster.name", + "k8s.namespace.name", + "k8s.pod.name", + "k8s.container.name", + "container.name", + "k8s.replicaset.name", + "k8s.deployment.name", + "k8s.statefulset.name", + "k8s.daemonset.name", + "k8s.cronjob.name", + "k8s.job.name", +} + +// Action is the action to be performed on OTLP Resource Attribute. +type Action string + +const ( + // IndexLabel stores a Resource Attribute as a label in index to identify streams. + IndexLabel Action = "index_label" + // StructuredMetadata stores an Attribute as Structured Metadata with each log entry. + StructuredMetadata Action = "structured_metadata" + // Drop drops Attributes for which the Attribute name does match the regex. + Drop Action = "drop" +) + +var ( + errUnsupportedAction = fmt.Errorf("unsupported action, it must be one of: %s, %s, %s", Drop, IndexLabel, StructuredMetadata) + errAttributesAndRegexNotSet = fmt.Errorf("attributes or regex must be set") + errAttributesAndRegexBothSet = fmt.Errorf("only one of attributes or regex must be set") +) + +var DefaultOTLPConfig = OTLPConfig{ + ResourceAttributes: ResourceAttributesConfig{ + AttributesConfig: []AttributesConfig{ + { + Action: IndexLabel, + Attributes: blessedAttributes, + }, + }, + }, +} + +type OTLPConfig struct { + ResourceAttributes ResourceAttributesConfig `yaml:"resource_attributes,omitempty"` + ScopeAttributes []AttributesConfig `yaml:"scope_attributes,omitempty"` + LogAttributes []AttributesConfig `yaml:"log_attributes,omitempty"` +} + +func (c *OTLPConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { + *c = DefaultOTLPConfig + type plain OTLPConfig + if err := unmarshal((*plain)(c)); err != nil { + return err + } + + return nil +} + +func (c *OTLPConfig) actionForAttribute(attribute string, cfgs []AttributesConfig) Action { + for i := 0; i < len(cfgs); i++ { + if cfgs[i].Regex.Regexp != nil && cfgs[i].Regex.MatchString(attribute) { + return cfgs[i].Action + } + for _, cfgAttr := range cfgs[i].Attributes { + if cfgAttr == attribute { + return cfgs[i].Action + } + } + } + + return StructuredMetadata +} + +func (c *OTLPConfig) ActionForResourceAttribute(attribute string) Action { + return c.actionForAttribute(attribute, c.ResourceAttributes.AttributesConfig) +} + +func (c *OTLPConfig) ActionForScopeAttribute(attribute string) Action { + return c.actionForAttribute(attribute, c.ScopeAttributes) +} + +func (c *OTLPConfig) ActionForLogAttribute(attribute string) Action { + return c.actionForAttribute(attribute, c.LogAttributes) +} + +func (c *OTLPConfig) Validate() error { + for _, ac := range c.ScopeAttributes { + if ac.Action == IndexLabel { + return fmt.Errorf("%s action is only supported for resource_attributes", IndexLabel) + } + } + + for _, ac := range c.LogAttributes { + if ac.Action == IndexLabel { + return fmt.Errorf("%s action is only supported for resource_attributes", IndexLabel) + } + } + + return nil +} + +type AttributesConfig struct { + Action Action `yaml:"action,omitempty"` + Attributes []string `yaml:"attributes,omitempty"` + Regex relabel.Regexp `yaml:"regex,omitempty"` +} + +func (c *AttributesConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { + type plain AttributesConfig + if err := unmarshal((*plain)(c)); err != nil { + return err + } + + if c.Action == "" { + c.Action = StructuredMetadata + } + + if c.Action != IndexLabel && c.Action != StructuredMetadata && c.Action != Drop { + return errUnsupportedAction + } + + if len(c.Attributes) == 0 && c.Regex.Regexp == nil { + return errAttributesAndRegexNotSet + } + + if len(c.Attributes) != 0 && c.Regex.Regexp != nil { + return errAttributesAndRegexBothSet + } + + return nil +} + +type ResourceAttributesConfig struct { + IgnoreDefaults bool `yaml:"ignore_defaults,omitempty"` + AttributesConfig []AttributesConfig `yaml:"attributes,omitempty"` +} + +func (c *ResourceAttributesConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { + type plain ResourceAttributesConfig + if err := unmarshal((*plain)(c)); err != nil { + return err + } + + if !c.IgnoreDefaults { + c.AttributesConfig = append([]AttributesConfig{ + DefaultOTLPConfig.ResourceAttributes.AttributesConfig[0], + }, c.AttributesConfig...) + } + + return nil +} diff --git a/pkg/loghttp/push/otlp_config_test.go b/pkg/loghttp/push/otlp_config_test.go new file mode 100644 index 0000000000000..a1cfc15ff52c8 --- /dev/null +++ b/pkg/loghttp/push/otlp_config_test.go @@ -0,0 +1,355 @@ +package push + +import ( + "testing" + + "github.com/prometheus/prometheus/model/relabel" + "github.com/stretchr/testify/require" + "gopkg.in/yaml.v2" +) + +func TestUnmarshalOTLPConfig(t *testing.T) { + for _, tc := range []struct { + name string + yamlConfig []byte + expectedCfg OTLPConfig + expectedErr error + }{ + { + name: "only resource_attributes set", + yamlConfig: []byte(` +resource_attributes: + attributes: + - action: index_label + regex: foo`), + expectedCfg: OTLPConfig{ + ResourceAttributes: ResourceAttributesConfig{ + AttributesConfig: []AttributesConfig{ + DefaultOTLPConfig.ResourceAttributes.AttributesConfig[0], + { + Action: IndexLabel, + Regex: relabel.MustNewRegexp("foo"), + }, + }, + }, + }, + }, + { + name: "resource_attributes with defaults ignored", + yamlConfig: []byte(` +resource_attributes: + ignore_defaults: true + attributes: + - action: index_label + regex: foo`), + expectedCfg: OTLPConfig{ + ResourceAttributes: ResourceAttributesConfig{ + IgnoreDefaults: true, + AttributesConfig: []AttributesConfig{ + { + Action: IndexLabel, + Regex: relabel.MustNewRegexp("foo"), + }, + }, + }, + }, + }, + { + name: "resource_attributes not set", + yamlConfig: []byte(` +scope_attributes: + - action: drop + attributes: + - fizz`), + expectedCfg: OTLPConfig{ + ResourceAttributes: ResourceAttributesConfig{ + AttributesConfig: []AttributesConfig{ + { + Action: IndexLabel, + Attributes: blessedAttributes, + }, + }, + }, + ScopeAttributes: []AttributesConfig{ + { + Action: Drop, + Attributes: []string{"fizz"}, + }, + }, + }, + }, + { + name: "all 3 set", + yamlConfig: []byte(` +resource_attributes: + attributes: + - action: index_label + regex: foo +scope_attributes: + - action: drop + attributes: + - fizz +log_attributes: + - action: structured_metadata + attributes: + - buzz`), + expectedCfg: OTLPConfig{ + ResourceAttributes: ResourceAttributesConfig{ + AttributesConfig: []AttributesConfig{ + DefaultOTLPConfig.ResourceAttributes.AttributesConfig[0], + { + Action: IndexLabel, + Regex: relabel.MustNewRegexp("foo"), + }, + }, + }, + ScopeAttributes: []AttributesConfig{ + { + Action: Drop, + Attributes: []string{"fizz"}, + }, + }, + LogAttributes: []AttributesConfig{ + { + Action: StructuredMetadata, + Attributes: []string{"buzz"}, + }, + }, + }, + }, + { + name: "unsupported action should error", + yamlConfig: []byte(` +log_attributes: + - action: keep + attributes: + - fizz`), + expectedErr: errUnsupportedAction, + }, + { + name: "attributes and regex both not set should error", + yamlConfig: []byte(` +log_attributes: + - action: drop`), + expectedErr: errAttributesAndRegexNotSet, + }, + { + name: "attributes and regex both being set should error", + yamlConfig: []byte(` +log_attributes: + - action: drop + regex: foo + attributes: + - fizz`), + expectedErr: errAttributesAndRegexBothSet, + }, + } { + t.Run(tc.name, func(t *testing.T) { + cfg := OTLPConfig{} + err := yaml.UnmarshalStrict(tc.yamlConfig, &cfg) + if tc.expectedErr != nil { + require.ErrorIs(t, err, tc.expectedErr) + return + } + require.Equal(t, tc.expectedCfg, cfg) + }) + } +} + +func TestOTLPConfig(t *testing.T) { + type attrAndExpAction struct { + attr string + expectedAction Action + } + + for _, tc := range []struct { + name string + otlpConfig OTLPConfig + resAttrs []attrAndExpAction + scopeAttrs []attrAndExpAction + logAttrs []attrAndExpAction + }{ + { + name: "default OTLPConfig", + otlpConfig: DefaultOTLPConfig, + resAttrs: []attrAndExpAction{ + { + attr: attrServiceName, + expectedAction: IndexLabel, + }, + { + attr: "not_blessed", + expectedAction: StructuredMetadata, + }, + }, + scopeAttrs: []attrAndExpAction{ + { + attr: "method", + expectedAction: StructuredMetadata, + }, + }, + logAttrs: []attrAndExpAction{ + { + attr: "user_id", + expectedAction: StructuredMetadata, + }, + }, + }, + { + name: "drop everything except a few attrs", + otlpConfig: OTLPConfig{ + ResourceAttributes: ResourceAttributesConfig{ + AttributesConfig: []AttributesConfig{ + { + Action: IndexLabel, + Attributes: []string{attrServiceName}, + }, + { + Action: StructuredMetadata, + Regex: relabel.MustNewRegexp("^foo.*"), + }, + { + Action: Drop, + Regex: relabel.MustNewRegexp(".*"), + }, + }, + }, + ScopeAttributes: []AttributesConfig{ + { + Action: StructuredMetadata, + Attributes: []string{"method"}, + }, + { + Action: Drop, + Regex: relabel.MustNewRegexp(".*"), + }, + }, + LogAttributes: []AttributesConfig{ + { + Action: StructuredMetadata, + Attributes: []string{"user_id"}, + }, + { + Action: Drop, + Regex: relabel.MustNewRegexp(".*"), + }, + }, + }, + resAttrs: []attrAndExpAction{ + { + attr: attrServiceName, + expectedAction: IndexLabel, + }, + { + attr: "foo_bar", + expectedAction: StructuredMetadata, + }, + { + attr: "ping_foo", + expectedAction: Drop, + }, + }, + scopeAttrs: []attrAndExpAction{ + { + attr: "method", + expectedAction: StructuredMetadata, + }, + { + attr: "version", + expectedAction: Drop, + }, + }, + logAttrs: []attrAndExpAction{ + { + attr: "user_id", + expectedAction: StructuredMetadata, + }, + { + attr: "order_id", + expectedAction: Drop, + }, + }, + }, + { + name: "keep everything except a few attrs", + otlpConfig: OTLPConfig{ + ResourceAttributes: ResourceAttributesConfig{ + AttributesConfig: []AttributesConfig{ + { + Action: Drop, + Attributes: []string{attrServiceName}, + }, + { + Action: IndexLabel, + Regex: relabel.MustNewRegexp(".*"), + }, + }, + }, + ScopeAttributes: []AttributesConfig{ + { + Action: Drop, + Attributes: []string{"method"}, + }, + { + Action: StructuredMetadata, + Regex: relabel.MustNewRegexp(".*"), + }, + }, + LogAttributes: []AttributesConfig{ + { + Action: Drop, + Attributes: []string{"user_id"}, + }, + { + Action: StructuredMetadata, + Regex: relabel.MustNewRegexp(".*"), + }, + }, + }, + resAttrs: []attrAndExpAction{ + { + attr: attrServiceName, + expectedAction: Drop, + }, + { + attr: "foo_bar", + expectedAction: IndexLabel, + }, + }, + scopeAttrs: []attrAndExpAction{ + { + attr: "method", + expectedAction: Drop, + }, + { + attr: "version", + expectedAction: StructuredMetadata, + }, + }, + logAttrs: []attrAndExpAction{ + { + attr: "user_id", + expectedAction: Drop, + }, + { + attr: "order_id", + expectedAction: StructuredMetadata, + }, + }, + }, + } { + t.Run(tc.name, func(t *testing.T) { + for _, c := range tc.resAttrs { + require.Equal(t, c.expectedAction, tc.otlpConfig.ActionForResourceAttribute(c.attr)) + } + + for _, c := range tc.scopeAttrs { + require.Equal(t, c.expectedAction, tc.otlpConfig.ActionForScopeAttribute(c.attr)) + } + + for _, c := range tc.logAttrs { + require.Equal(t, c.expectedAction, tc.otlpConfig.ActionForLogAttribute(c.attr)) + } + }) + } +} diff --git a/pkg/loghttp/push/otlp_test.go b/pkg/loghttp/push/otlp_test.go index 8d02485833775..d817c933a43d5 100644 --- a/pkg/loghttp/push/otlp_test.go +++ b/pkg/loghttp/push/otlp_test.go @@ -7,6 +7,7 @@ import ( "time" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/relabel" "github.com/stretchr/testify/require" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/plog" @@ -23,6 +24,7 @@ func TestOTLPToLokiPushRequest(t *testing.T) { generateLogs func() plog.Logs expectedPushRequest logproto.PushRequest expectedStats Stats + otlpConfig OTLPConfig }{ { name: "no logs", @@ -31,6 +33,7 @@ func TestOTLPToLokiPushRequest(t *testing.T) { }, expectedPushRequest: logproto.PushRequest{}, expectedStats: *newPushStats(), + otlpConfig: DefaultOTLPConfig, }, { name: "resource with no logs", @@ -41,9 +44,11 @@ func TestOTLPToLokiPushRequest(t *testing.T) { }, expectedPushRequest: logproto.PushRequest{}, expectedStats: *newPushStats(), + otlpConfig: DefaultOTLPConfig, }, { - name: "resource with a log entry", + name: "resource with a log entry", + otlpConfig: DefaultOTLPConfig, generateLogs: func() plog.Logs { ld := plog.NewLogs() ld.ResourceLogs().AppendEmpty().Resource().Attributes().PutStr("service.name", "service-1") @@ -78,7 +83,8 @@ func TestOTLPToLokiPushRequest(t *testing.T) { }, }, { - name: "resource attributes and scope attributes stored as structured metadata", + name: "resource attributes and scope attributes stored as structured metadata", + otlpConfig: DefaultOTLPConfig, generateLogs: func() plog.Logs { ld := plog.NewLogs() ld.ResourceLogs().AppendEmpty() @@ -152,7 +158,8 @@ func TestOTLPToLokiPushRequest(t *testing.T) { }, }, { - name: "attributes with nested data", + name: "attributes with nested data", + otlpConfig: DefaultOTLPConfig, generateLogs: func() plog.Logs { ld := plog.NewLogs() ld.ResourceLogs().AppendEmpty() @@ -234,10 +241,153 @@ func TestOTLPToLokiPushRequest(t *testing.T) { mostRecentEntryTimestamp: now, }, }, + { + name: "custom otlp config", + otlpConfig: OTLPConfig{ + ResourceAttributes: ResourceAttributesConfig{ + AttributesConfig: []AttributesConfig{ + { + Action: IndexLabel, + Attributes: []string{"pod.name"}, + }, { + Action: IndexLabel, + Regex: relabel.MustNewRegexp("service.*"), + }, + { + Action: Drop, + Regex: relabel.MustNewRegexp("drop.*"), + }, + { + Action: StructuredMetadata, + Attributes: []string{"resource.nested"}, + }, + }, + }, + ScopeAttributes: []AttributesConfig{ + { + Action: Drop, + Attributes: []string{"drop.function"}, + }, + }, + LogAttributes: []AttributesConfig{ + { + Action: StructuredMetadata, + Regex: relabel.MustNewRegexp(".*_id"), + }, + { + Action: Drop, + Regex: relabel.MustNewRegexp(".*"), + }, + }, + }, + generateLogs: func() plog.Logs { + ld := plog.NewLogs() + ld.ResourceLogs().AppendEmpty() + ld.ResourceLogs().At(0).Resource().Attributes().PutStr("service.name", "service-1") + ld.ResourceLogs().At(0).Resource().Attributes().PutStr("pod.name", "service-1-abc") + ld.ResourceLogs().At(0).Resource().Attributes().PutStr("pod.ip", "10.200.200.200") + ld.ResourceLogs().At(0).Resource().Attributes().PutStr("drop.service.addr", "192.168.0.1") + ld.ResourceLogs().At(0).Resource().Attributes().PutStr("drop.service.version", "v1") + ld.ResourceLogs().At(0).Resource().Attributes().PutEmptyMap("resource.nested").PutStr("foo", "bar") + ld.ResourceLogs().At(0).ScopeLogs().AppendEmpty() + ld.ResourceLogs().At(0).ScopeLogs().At(0).Scope().SetName("fizz") + ld.ResourceLogs().At(0).ScopeLogs().At(0).Scope().Attributes().PutStr("drop.function", "login") + ld.ResourceLogs().At(0).ScopeLogs().At(0).Scope().Attributes().PutEmptyMap("scope.nested").PutStr("foo", "bar") + for i := 0; i < 2; i++ { + ld.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords().AppendEmpty() + ld.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords().At(i).Body().SetStr(fmt.Sprintf("test body - %d", i)) + ld.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords().At(i).SetTimestamp(pcommon.Timestamp(now.UnixNano())) + ld.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords().At(i).Attributes().PutStr("user_id", "u1") + ld.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords().At(i).Attributes().PutStr("order_id", "o1") + ld.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords().At(i).Attributes().PutEmptyMap("drop.log.nested").PutStr("foo", fmt.Sprintf("bar - %d", i)) + } + return ld + }, + expectedPushRequest: logproto.PushRequest{ + Streams: []logproto.Stream{ + { + Labels: `{pod_name="service-1-abc", service_name="service-1"}`, + Entries: []logproto.Entry{ + { + Timestamp: now, + Line: "test body - 0", + StructuredMetadata: push.LabelsAdapter{ + { + Name: "user_id", + Value: "u1", + }, + { + Name: "order_id", + Value: "o1", + }, + { + Name: "pod_ip", + Value: "10.200.200.200", + }, + { + Name: "resource_nested_foo", + Value: "bar", + }, + { + Name: "scope_nested_foo", + Value: "bar", + }, + { + Name: "scope_name", + Value: "fizz", + }, + }, + }, + { + Timestamp: now, + Line: "test body - 1", + StructuredMetadata: push.LabelsAdapter{ + { + Name: "user_id", + Value: "u1", + }, + { + Name: "order_id", + Value: "o1", + }, + { + Name: "pod_ip", + Value: "10.200.200.200", + }, + { + Name: "resource_nested_foo", + Value: "bar", + }, + { + Name: "scope_nested_foo", + Value: "bar", + }, + { + Name: "scope_name", + Value: "fizz", + }, + }, + }, + }, + }, + }, + }, + expectedStats: Stats{ + numLines: 2, + logLinesBytes: map[time.Duration]int64{ + time.Hour: 26, + }, + structuredMetadataBytes: map[time.Duration]int64{ + time.Hour: 113, + }, + streamLabelsSize: 42, + mostRecentEntryTimestamp: now, + }, + }, } { t.Run(tc.name, func(t *testing.T) { stats := newPushStats() - pushReq := otlpToLokiPushRequest(tc.generateLogs(), "foo", fakeRetention{}, stats) + pushReq := otlpToLokiPushRequest(tc.generateLogs(), "foo", fakeRetention{}, tc.otlpConfig, stats) require.Equal(t, tc.expectedPushRequest, *pushReq) require.Equal(t, tc.expectedStats, *stats) }) @@ -324,7 +474,7 @@ func TestOTLPLogToPushEntry(t *testing.T) { }, } { t.Run(tc.name, func(t *testing.T) { - require.Equal(t, tc.expectedResp, otlpLogToPushEntry(tc.buildLogRecord())) + require.Equal(t, tc.expectedResp, otlpLogToPushEntry(tc.buildLogRecord(), DefaultOTLPConfig)) }) } diff --git a/pkg/loghttp/push/push.go b/pkg/loghttp/push/push.go index dffa5ab1a05e3..15b7bba0a78c9 100644 --- a/pkg/loghttp/push/push.go +++ b/pkg/loghttp/push/push.go @@ -58,7 +58,11 @@ type TenantsRetention interface { RetentionPeriodFor(userID string, lbs labels.Labels) time.Duration } -type RequestParser func(userID string, r *http.Request, tenantsRetention TenantsRetention) (*logproto.PushRequest, *Stats, error) +type Limits interface { + OTLPConfig(userID string) OTLPConfig +} + +type RequestParser func(userID string, r *http.Request, tenantsRetention TenantsRetention, limits Limits) (*logproto.PushRequest, *Stats, error) type Stats struct { errs []error @@ -72,8 +76,8 @@ type Stats struct { bodySize int64 } -func ParseRequest(logger log.Logger, userID string, r *http.Request, tenantsRetention TenantsRetention, pushRequestParser RequestParser) (*logproto.PushRequest, error) { - req, pushStats, err := pushRequestParser(userID, r, tenantsRetention) +func ParseRequest(logger log.Logger, userID string, r *http.Request, tenantsRetention TenantsRetention, limits Limits, pushRequestParser RequestParser) (*logproto.PushRequest, error) { + req, pushStats, err := pushRequestParser(userID, r, tenantsRetention, limits) if err != nil { return nil, err } @@ -131,7 +135,7 @@ func ParseRequest(logger log.Logger, userID string, r *http.Request, tenantsRete return req, nil } -func ParseLokiRequest(userID string, r *http.Request, tenantsRetention TenantsRetention) (*logproto.PushRequest, *Stats, error) { +func ParseLokiRequest(userID string, r *http.Request, tenantsRetention TenantsRetention, _ Limits) (*logproto.PushRequest, *Stats, error) { // Body var body io.Reader // bodySize should always reflect the compressed size of the request body diff --git a/pkg/loghttp/push/push_test.go b/pkg/loghttp/push/push_test.go index 286b0e013a241..fa1e2fb28d115 100644 --- a/pkg/loghttp/push/push_test.go +++ b/pkg/loghttp/push/push_test.go @@ -200,7 +200,7 @@ func TestParseRequest(t *testing.T) { request.Header.Add("Content-Encoding", test.contentEncoding) } - data, err := ParseRequest(util_log.Logger, "fake", request, nil, ParseLokiRequest) + data, err := ParseRequest(util_log.Logger, "fake", request, nil, nil, ParseLokiRequest) structuredMetadataBytesReceived := int(structuredMetadataBytesReceivedStats.Value()["total"].(int64)) - previousStructuredMetadataBytesReceived previousStructuredMetadataBytesReceived += structuredMetadataBytesReceived diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index ac25798c33e31..e3052c1781b89 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -21,6 +21,7 @@ import ( "github.com/grafana/loki/pkg/compactor/deletionmode" "github.com/grafana/loki/pkg/distributor/shardstreams" + "github.com/grafana/loki/pkg/loghttp/push" "github.com/grafana/loki/pkg/logql/syntax" ruler_config "github.com/grafana/loki/pkg/ruler/config" "github.com/grafana/loki/pkg/ruler/util" @@ -199,6 +200,7 @@ type Limits struct { AllowStructuredMetadata bool `yaml:"allow_structured_metadata,omitempty" json:"allow_structured_metadata,omitempty" doc:"description=Allow user to send structured metadata in push payload."` MaxStructuredMetadataSize flagext.ByteSize `yaml:"max_structured_metadata_size" json:"max_structured_metadata_size" doc:"description=Maximum size accepted for structured metadata per log line."` MaxStructuredMetadataEntriesCount int `yaml:"max_structured_metadata_entries_count" json:"max_structured_metadata_entries_count" doc:"description=Maximum number of structured metadata entries per log line."` + OTLPConfig push.OTLPConfig `yaml:"otlp_config" json:"otlp_config" doc:"description=OTLP log ingestion configurations"` } type StreamRetention struct { @@ -341,7 +343,7 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { _ = l.MaxStructuredMetadataSize.Set(defaultMaxStructuredMetadataSize) f.Var(&l.MaxStructuredMetadataSize, "limits.max-structured-metadata-size", "Maximum size accepted for structured metadata per entry. Default: 64 kb. Any log line exceeding this limit will be discarded. There is no limit when unset or set to 0.") f.IntVar(&l.MaxStructuredMetadataEntriesCount, "limits.max-structured-metadata-entries-count", defaultMaxStructuredMetadataCount, "Maximum number of structured metadata entries per log line. Default: 128. Any log line exceeding this limit will be discarded. There is no limit when unset or set to 0.") - + l.OTLPConfig = push.DefaultOTLPConfig } // UnmarshalYAML implements the yaml.Unmarshaler interface. @@ -398,6 +400,10 @@ func (l *Limits) Validate() error { l.MaxQueryCapacity = 1 } + if err := l.OTLPConfig.Validate(); err != nil { + return err + } + return nil } @@ -892,6 +898,10 @@ func (o *Overrides) MaxStructuredMetadataCount(userID string) int { return o.getOverridesForUser(userID).MaxStructuredMetadataEntriesCount } +func (o *Overrides) OTLPConfig(userID string) push.OTLPConfig { + return o.getOverridesForUser(userID).OTLPConfig +} + func (o *Overrides) getOverridesForUser(userID string) *Limits { if o.tenantLimits != nil { l := o.tenantLimits.TenantLimits(userID) diff --git a/pkg/validation/limits_test.go b/pkg/validation/limits_test.go index 908531f9858f6..0fb6dcbae2ef0 100644 --- a/pkg/validation/limits_test.go +++ b/pkg/validation/limits_test.go @@ -12,6 +12,7 @@ import ( "gopkg.in/yaml.v2" "github.com/grafana/loki/pkg/compactor/deletionmode" + "github.com/grafana/loki/pkg/loghttp/push" ) func TestLimitsTagsYamlMatchJson(t *testing.T) { @@ -173,6 +174,18 @@ func TestLimitsDoesNotMutate(t *testing.T) { defaultLimits = initialDefault }() + defaultOTLPConfig := push.OTLPConfig{ + ResourceAttributes: push.ResourceAttributesConfig{ + IgnoreDefaults: true, + AttributesConfig: []push.AttributesConfig{ + { + Action: push.IndexLabel, + Attributes: []string{"pod"}, + }, + }, + }, + } + // Set new defaults with non-nil values for non-scalar types newDefaults := Limits{ RulerRemoteWriteHeaders: OverwriteMarshalingStringMap{map[string]string{"a": "b"}}, @@ -182,6 +195,7 @@ func TestLimitsDoesNotMutate(t *testing.T) { Selector: `{a="b"}`, }, }, + OTLPConfig: defaultOTLPConfig, } SetDefaultLimitsForYAMLUnmarshalling(newDefaults) @@ -206,6 +220,7 @@ ruler_remote_write_headers: Selector: `{a="b"}`, }, }, + OTLPConfig: defaultOTLPConfig, }, }, { @@ -222,6 +237,7 @@ ruler_remote_write_headers: Selector: `{a="b"}`, }, }, + OTLPConfig: defaultOTLPConfig, }, }, { @@ -241,6 +257,7 @@ retention_stream: // Rest from new defaults RulerRemoteWriteHeaders: OverwriteMarshalingStringMap{map[string]string{"a": "b"}}, + OTLPConfig: defaultOTLPConfig, }, }, { @@ -259,6 +276,7 @@ reject_old_samples: true Selector: `{a="b"}`, }, }, + OTLPConfig: defaultOTLPConfig, }, }, { @@ -277,6 +295,7 @@ query_timeout: 5m Selector: `{a="b"}`, }, }, + OTLPConfig: defaultOTLPConfig, }, }, } { From 6d66ea382cfdc90ca12b38e2ff6cb29a301a118e Mon Sep 17 00:00:00 2001 From: Poyzan <31743851+poyzannur@users.noreply.github.com> Date: Wed, 24 Jan 2024 15:41:34 +0000 Subject: [PATCH 02/23] [Fix] OR statements not being evaluated as part of nested line filters (#11735) **What this PR does / why we need it**: When nester line filters were being evaluated, the `OR` statements in right expressions were omitted and only `LineFilters` were returned. Resulting in only first value being returned. It can be reproduced in stringer unit test failing with, ``` Error Trace: /Users/poyzannur/workspace/loki/pkg/logql/syntax/ast_test.go:535 Error: Not equal: expected: "{app=\"foo\"} |= \"foo\" or \"bar\" |= \"baz\" or \"bal\"" actual : "{app=\"foo\"} |= \"foo\" or \"bar\" |= \"baz\"" ``` We now return the `OR` expression as part of nested line filters. Thanks a million to @ashwanthgoli for help with debugging, and extra unit test. **Which issue(s) this PR fixes**: Fixes https://github.com/grafana/support-escalations/issues/9042 **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --- pkg/logql/syntax/ast.go | 2 ++ pkg/logql/syntax/ast_test.go | 15 ++++++++++++++ pkg/logql/syntax/parser_test.go | 35 +++++++++++++++++++++++++++++++++ 3 files changed, 52 insertions(+) diff --git a/pkg/logql/syntax/ast.go b/pkg/logql/syntax/ast.go index 900802207c506..cea41f4d95c5d 100644 --- a/pkg/logql/syntax/ast.go +++ b/pkg/logql/syntax/ast.go @@ -353,6 +353,8 @@ func newNestedLineFilterExpr(left *LineFilterExpr, right *LineFilterExpr) *LineF return &LineFilterExpr{ Left: left, LineFilter: right.LineFilter, + Or: right.Or, + IsOrChild: right.IsOrChild, } } diff --git a/pkg/logql/syntax/ast_test.go b/pkg/logql/syntax/ast_test.go index b0c20005b74ad..ece470516eb45 100644 --- a/pkg/logql/syntax/ast_test.go +++ b/pkg/logql/syntax/ast_test.go @@ -383,6 +383,13 @@ func Test_FilterMatcher(t *testing.T) { }, []linecheck{{"foo", true}, {"bar", true}, {"none", false}}, }, + { + `{app="foo"} |= "foo" or "bar" |= "buzz" or "fizz"`, + []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "app", "foo"), + }, + []linecheck{{"foo buzz", true}, {"bar fizz", true}, {"foo", false}, {"bar", false}, {"none", false}}, + }, { `{app="foo"} != "foo" or "bar"`, []*labels.Matcher{ @@ -496,6 +503,14 @@ func TestStringer(t *testing.T) { in: `{app="foo"} |~ "foo" or "bar" or "baz"`, out: `{app="foo"} |~ "foo" or "bar" or "baz"`, }, + { + in: `{app="foo"} |= "foo" or "bar" |= "buzz" or "fizz"`, + out: `{app="foo"} |= "foo" or "bar" |= "buzz" or "fizz"`, + }, + { + out: `{app="foo"} |= "foo" or "bar" |~ "buzz|fizz"`, + in: `{app="foo"} |= "foo" or "bar" |~ "buzz|fizz"`, + }, { in: `{app="foo"} |= ip("127.0.0.1") or "foo"`, out: `{app="foo"} |= ip("127.0.0.1") or "foo"`, diff --git a/pkg/logql/syntax/parser_test.go b/pkg/logql/syntax/parser_test.go index cd45b6ec74c1e..7152d78adac12 100644 --- a/pkg/logql/syntax/parser_test.go +++ b/pkg/logql/syntax/parser_test.go @@ -3138,6 +3138,41 @@ var ParseTestCases = []struct { }, }, }, + { + in: `{app="foo"} |= "foo" or "bar" |= "buzz" or "fizz"`, + exp: &PipelineExpr{ + Left: newMatcherExpr([]*labels.Matcher{mustNewMatcher(labels.MatchEqual, "app", "foo")}), + MultiStages: MultiStageExpr{ + &LineFilterExpr{ + Left: newOrLineFilter( + &LineFilterExpr{ + LineFilter: LineFilter{ + Ty: labels.MatchEqual, + Match: "foo", + }, + }, + &LineFilterExpr{ + LineFilter: LineFilter{ + Ty: labels.MatchEqual, + Match: "bar", + }, + }), + LineFilter: LineFilter{ + Ty: labels.MatchEqual, + Match: "buzz", + }, + Or: &LineFilterExpr{ + LineFilter: LineFilter{ + Ty: labels.MatchEqual, + Match: "fizz", + }, + IsOrChild: true, + }, + IsOrChild: false, + }, + }, + }, + }, } func TestParse(t *testing.T) { From 06bb20914b027c96ac3af27ae36e91f35529524d Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 24 Jan 2024 11:30:02 -0800 Subject: [PATCH 03/23] [Blooms] Specs out BloomGenerator (#11755) This is my attempt at making more testable implementations of the work done by the `bloomcompactor` pkg. It's largely inspired by what already exists there, but without depending on things like object storage, chunk clients, etc, it's much easier to locally prove behavior. I'm planning to build on top of this, continuing to integrate work already done in this pkg. --- pkg/bloomcompactor/bloomcompactor.go | 5 +- pkg/bloomcompactor/v2spec.go | 142 +++++++++++++++++++ pkg/bloomcompactor/v2spec_test.go | 123 ++++++++++++++++ pkg/storage/bloom/v1/archive_test.go | 4 +- pkg/storage/bloom/v1/block.go | 24 ++-- pkg/storage/bloom/v1/bloom_tokenizer.go | 4 - pkg/storage/bloom/v1/bloom_tokenizer_test.go | 2 +- pkg/storage/bloom/v1/builder.go | 34 +++-- pkg/storage/bloom/v1/builder_test.go | 29 ++-- pkg/storage/bloom/v1/dedupe_test.go | 2 +- pkg/storage/bloom/v1/fuse_test.go | 8 +- pkg/storage/bloom/v1/index.go | 16 ++- pkg/storage/bloom/v1/merge_test.go | 6 +- pkg/storage/bloom/v1/test_util.go | 19 ++- 14 files changed, 354 insertions(+), 64 deletions(-) create mode 100644 pkg/bloomcompactor/v2spec.go create mode 100644 pkg/bloomcompactor/v2spec_test.go diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index dbe307ff18822..ab8724192bd76 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -376,6 +376,8 @@ func (c *Compactor) compactTenant(ctx context.Context, logger log.Logger, sc sto level.Debug(logger).Log("msg", "got token range for instance", "id", tr.Instance.Id, "min", tr.MinToken, "max", tr.MaxToken) } + // TODO(owen-d): can be optimized to only query for series within the fp range of the compactor shard(s) rather than scanning all series + // and filtering out the ones that don't belong to the compactor shard(s). _ = sc.indexShipper.ForEach(ctx, tableName, tenant, func(isMultiTenantIndex bool, idx shipperindex.Index) error { if isMultiTenantIndex { // Skip multi-tenant indexes @@ -406,9 +408,10 @@ func (c *Compactor) compactTenant(ctx context.Context, logger log.Logger, sc sto } temp := make([]tsdbindex.ChunkMeta, len(chksMetas)) + ls := labels.Copy() _ = copy(temp, chksMetas) //All seriesMetas given a table within fp of this compactor shard - seriesMetas = append(seriesMetas, seriesMeta{seriesFP: fingerprint, seriesLbs: labels, chunkRefs: temp}) + seriesMetas = append(seriesMetas, seriesMeta{seriesFP: fingerprint, seriesLbs: ls, chunkRefs: temp}) }, labels.MustNewMatcher(labels.MatchEqual, "", ""), ) diff --git a/pkg/bloomcompactor/v2spec.go b/pkg/bloomcompactor/v2spec.go new file mode 100644 index 0000000000000..9f7174b0f2831 --- /dev/null +++ b/pkg/bloomcompactor/v2spec.go @@ -0,0 +1,142 @@ +package bloomcompactor + +import ( + "context" + "fmt" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/chunk" +) + +// TODO(owen-d): add metrics +type Metrics struct { + bloomMetrics *v1.Metrics +} + +func NewMetrics(_ prometheus.Registerer, bloomMetrics *v1.Metrics) *Metrics { + return &Metrics{ + bloomMetrics: bloomMetrics, + } +} + +// inclusive range +type Keyspace struct { + min, max model.Fingerprint +} + +func (k Keyspace) Cmp(other Keyspace) v1.BoundsCheck { + if other.max < k.min { + return v1.Before + } else if other.min > k.max { + return v1.After + } + return v1.Overlap +} + +// Store is likely bound within. This allows specifying impls like ShardedStore +// to only request the shard-range needed from the existing store. +type BloomGenerator interface { + Generate(ctx context.Context) (skippedBlocks []*v1.Block, results v1.Iterator[*v1.Block], err error) +} + +// Simple implementation of a BloomGenerator. +type SimpleBloomGenerator struct { + store v1.Iterator[*v1.Series] + // TODO(owen-d): blocks need not be all downloaded prior. Consider implementing + // as an iterator of iterators, where each iterator is a batch of overlapping blocks. + blocks []*v1.Block + + // options to build blocks with + opts v1.BlockOptions + + metrics *Metrics + logger log.Logger + + readWriterFn func() (v1.BlockWriter, v1.BlockReader) + + tokenizer *v1.BloomTokenizer +} + +// SimpleBloomGenerator is a foundational implementation of BloomGenerator. +// It mainly wires up a few different components to generate bloom filters for a set of blocks +// and handles schema compatibility: +// Blocks which are incompatible with the schema are skipped and will have their chunks reindexed +func NewSimpleBloomGenerator( + opts v1.BlockOptions, + store v1.Iterator[*v1.Series], + blocks []*v1.Block, + readWriterFn func() (v1.BlockWriter, v1.BlockReader), + metrics *Metrics, + logger log.Logger, +) *SimpleBloomGenerator { + return &SimpleBloomGenerator{ + opts: opts, + store: store, + blocks: blocks, + logger: logger, + readWriterFn: readWriterFn, + metrics: metrics, + + tokenizer: v1.NewBloomTokenizer(opts.Schema.NGramLen(), opts.Schema.NGramSkip(), metrics.bloomMetrics), + } +} + +func (s *SimpleBloomGenerator) populate(series *v1.Series, bloom *v1.Bloom) error { + // TODO(owen-d): impl after threading in store + var chunkItr v1.Iterator[[]chunk.Chunk] = v1.NewEmptyIter[[]chunk.Chunk](nil) + + return s.tokenizer.PopulateSeriesWithBloom( + &v1.SeriesWithBloom{ + Series: series, + Bloom: bloom, + }, + chunkItr, + ) +} + +func (s *SimpleBloomGenerator) Generate(_ context.Context) (skippedBlocks []*v1.Block, results v1.Iterator[*v1.Block], err error) { + + blocksMatchingSchema := make([]v1.PeekingIterator[*v1.SeriesWithBloom], 0, len(s.blocks)) + for _, block := range s.blocks { + // TODO(owen-d): implement block naming so we can log the affected block in all these calls + logger := log.With(s.logger, "block", fmt.Sprintf("%#v", block)) + schema, err := block.Schema() + if err != nil { + level.Warn(logger).Log("msg", "failed to get schema for block", "err", err) + skippedBlocks = append(skippedBlocks, block) + } + + if !s.opts.Schema.Compatible(schema) { + level.Warn(logger).Log("msg", "block schema incompatible with options", "generator_schema", fmt.Sprintf("%#v", s.opts.Schema), "block_schema", fmt.Sprintf("%#v", schema)) + skippedBlocks = append(skippedBlocks, block) + } + + level.Debug(logger).Log("msg", "adding compatible block to bloom generation inputs") + itr := v1.NewPeekingIter[*v1.SeriesWithBloom](v1.NewBlockQuerier(block)) + blocksMatchingSchema = append(blocksMatchingSchema, itr) + } + + level.Debug(s.logger).Log("msg", "generating bloom filters for blocks", "num_blocks", len(blocksMatchingSchema), "skipped_blocks", len(skippedBlocks), "schema", fmt.Sprintf("%#v", s.opts.Schema)) + + // TODO(owen-d): implement bounded block sizes + + mergeBuilder := v1.NewMergeBuilder(blocksMatchingSchema, s.store, s.populate) + writer, reader := s.readWriterFn() + blockBuilder, err := v1.NewBlockBuilder(v1.NewBlockOptionsFromSchema(s.opts.Schema), writer) + if err != nil { + return skippedBlocks, nil, errors.Wrap(err, "failed to create bloom block builder") + } + _, err = mergeBuilder.Build(blockBuilder) + if err != nil { + return skippedBlocks, nil, errors.Wrap(err, "failed to build bloom block") + } + + return skippedBlocks, v1.NewSliceIter[*v1.Block]([]*v1.Block{v1.NewBlock(reader)}), nil + +} diff --git a/pkg/bloomcompactor/v2spec_test.go b/pkg/bloomcompactor/v2spec_test.go new file mode 100644 index 0000000000000..f5cdfc0580c3b --- /dev/null +++ b/pkg/bloomcompactor/v2spec_test.go @@ -0,0 +1,123 @@ +package bloomcompactor + +import ( + "bytes" + "context" + "testing" + + "github.com/go-kit/log" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" +) + +func blocksFromSchema(t *testing.T, n int, options v1.BlockOptions) (res []*v1.Block, data []v1.SeriesWithBloom) { + return blocksFromSchemaWithRange(t, n, options, 0, 0xffff) +} + +// splits 100 series across `n` non-overlapping blocks. +// uses options to build blocks with. +func blocksFromSchemaWithRange(t *testing.T, n int, options v1.BlockOptions, fromFP, throughFp model.Fingerprint) (res []*v1.Block, data []v1.SeriesWithBloom) { + if 100%n != 0 { + panic("100 series must be evenly divisible by n") + } + + numSeries := 100 + numKeysPerSeries := 10000 + data, _ = v1.MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, fromFP, throughFp, 0, 10000) + + seriesPerBlock := 100 / n + + for i := 0; i < n; i++ { + // references for linking in memory reader+writer + indexBuf := bytes.NewBuffer(nil) + bloomsBuf := bytes.NewBuffer(nil) + writer := v1.NewMemoryBlockWriter(indexBuf, bloomsBuf) + reader := v1.NewByteReader(indexBuf, bloomsBuf) + + builder, err := v1.NewBlockBuilder( + options, + writer, + ) + require.Nil(t, err) + + itr := v1.NewSliceIter[v1.SeriesWithBloom](data[i*seriesPerBlock : (i+1)*seriesPerBlock]) + _, err = builder.BuildFrom(itr) + require.Nil(t, err) + + res = append(res, v1.NewBlock(reader)) + } + + return res, data +} + +func dummyBloomGen(opts v1.BlockOptions, store v1.Iterator[*v1.Series], blocks []*v1.Block) *SimpleBloomGenerator { + return NewSimpleBloomGenerator( + opts, + store, + blocks, + func() (v1.BlockWriter, v1.BlockReader) { + indexBuf := bytes.NewBuffer(nil) + bloomsBuf := bytes.NewBuffer(nil) + return v1.NewMemoryBlockWriter(indexBuf, bloomsBuf), v1.NewByteReader(indexBuf, bloomsBuf) + }, + NewMetrics(nil, v1.NewMetrics(nil)), + log.NewNopLogger(), + ) +} + +func TestSimpleBloomGenerator(t *testing.T) { + for _, tc := range []struct { + desc string + fromSchema, toSchema v1.BlockOptions + sourceBlocks, numSkipped int + }{ + { + desc: "SkipsIncompatibleSchemas", + fromSchema: v1.NewBlockOptions(3, 0), + toSchema: v1.NewBlockOptions(4, 0), + sourceBlocks: 2, + numSkipped: 2, + }, + { + desc: "CombinesBlocks", + fromSchema: v1.NewBlockOptions(4, 0), + toSchema: v1.NewBlockOptions(4, 0), + sourceBlocks: 2, + numSkipped: 0, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + sourceBlocks, data := blocksFromSchema(t, tc.sourceBlocks, tc.fromSchema) + storeItr := v1.NewMapIter[v1.SeriesWithBloom, *v1.Series]( + v1.NewSliceIter[v1.SeriesWithBloom](data), + func(swb v1.SeriesWithBloom) *v1.Series { + return swb.Series + }, + ) + + gen := dummyBloomGen(tc.toSchema, storeItr, sourceBlocks) + skipped, results, err := gen.Generate(context.Background()) + require.Nil(t, err) + require.Equal(t, tc.numSkipped, len(skipped)) + + require.True(t, results.Next()) + block := results.At() + require.False(t, results.Next()) + + refs := v1.PointerSlice[v1.SeriesWithBloom](data) + + v1.EqualIterators[*v1.SeriesWithBloom]( + t, + func(a, b *v1.SeriesWithBloom) { + // TODO(owen-d): better equality check + // once chunk fetching is implemented + require.Equal(t, a.Series, b.Series) + }, + v1.NewSliceIter[*v1.SeriesWithBloom](refs), + block.Querier(), + ) + }) + } +} diff --git a/pkg/storage/bloom/v1/archive_test.go b/pkg/storage/bloom/v1/archive_test.go index 60620c8a5294f..e0c9407210f20 100644 --- a/pkg/storage/bloom/v1/archive_test.go +++ b/pkg/storage/bloom/v1/archive_test.go @@ -17,11 +17,11 @@ func TestArchive(t *testing.T) { numSeries := 100 numKeysPerSeries := 10000 - data, _ := mkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) + data, _ := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) builder, err := NewBlockBuilder( BlockOptions{ - schema: Schema{ + Schema: Schema{ version: DefaultSchemaVersion, encoding: chunkenc.EncSnappy, }, diff --git a/pkg/storage/bloom/v1/block.go b/pkg/storage/bloom/v1/block.go index 468f0d0315067..b1b08be008405 100644 --- a/pkg/storage/bloom/v1/block.go +++ b/pkg/storage/bloom/v1/block.go @@ -62,6 +62,11 @@ func (b *Block) LoadHeaders() error { } +// convenience method +func (b *Block) Querier() *BlockQuerier { + return NewBlockQuerier(b) +} + func (b *Block) Series() *LazySeriesIter { return NewLazySeriesIter(b) } @@ -70,31 +75,32 @@ func (b *Block) Blooms() *LazyBloomIter { return NewLazyBloomIter(b) } -type LazySchema func() (Schema, error) +func (b *Block) Schema() (Schema, error) { + if err := b.LoadHeaders(); err != nil { + return Schema{}, err + } + return b.index.schema, nil +} type BlockQuerier struct { series *LazySeriesIter blooms *LazyBloomIter - schema LazySchema + + block *Block // ref to underlying block cur *SeriesWithBloom } func NewBlockQuerier(b *Block) *BlockQuerier { return &BlockQuerier{ + block: b, series: NewLazySeriesIter(b), blooms: NewLazyBloomIter(b), - schema: func() (Schema, error) { - if err := b.LoadHeaders(); err != nil { - return Schema{}, err - } - return b.index.schema, nil - }, } } func (bq *BlockQuerier) Schema() (Schema, error) { - return bq.schema() + return bq.block.Schema() } func (bq *BlockQuerier) Seek(fp model.Fingerprint) error { diff --git a/pkg/storage/bloom/v1/bloom_tokenizer.go b/pkg/storage/bloom/v1/bloom_tokenizer.go index 946aeaf54495c..6008c6d1a65a7 100644 --- a/pkg/storage/bloom/v1/bloom_tokenizer.go +++ b/pkg/storage/bloom/v1/bloom_tokenizer.go @@ -49,10 +49,6 @@ func NewBloomTokenizer(nGramLen, nGramSkip int, metrics *Metrics) *BloomTokenize } } -func (bt *BloomTokenizer) SetLineTokenizer(t *NGramTokenizer) { - bt.lineTokenizer = t -} - func (bt *BloomTokenizer) GetNGramLength() uint64 { return uint64(bt.lineTokenizer.N) } diff --git a/pkg/storage/bloom/v1/bloom_tokenizer_test.go b/pkg/storage/bloom/v1/bloom_tokenizer_test.go index 0fad08e78f080..a1d09bab36b93 100644 --- a/pkg/storage/bloom/v1/bloom_tokenizer_test.go +++ b/pkg/storage/bloom/v1/bloom_tokenizer_test.go @@ -81,7 +81,7 @@ func TestSetLineTokenizer(t *testing.T) { require.Equal(t, bt.lineTokenizer.Skip, DefaultNGramSkip) // Set new tokenizer, and validate against that - bt.SetLineTokenizer(NewNGramTokenizer(6, 7)) + bt.lineTokenizer = NewNGramTokenizer(6, 7) require.Equal(t, bt.lineTokenizer.N, 6) require.Equal(t, bt.lineTokenizer.Skip, 7) } diff --git a/pkg/storage/bloom/v1/builder.go b/pkg/storage/bloom/v1/builder.go index 7b5d0dc3d73ff..fc4868bd0de6a 100644 --- a/pkg/storage/bloom/v1/builder.go +++ b/pkg/storage/bloom/v1/builder.go @@ -16,7 +16,12 @@ import ( ) type BlockOptions struct { - schema Schema + // Schema determines the Schema of the block and cannot be changed + Schema Schema + + // The following options can be changed on the fly. + // For instance, adding another page to a block with + // a different target page size is supported. // target size in bytes (decompressed) // of each page type @@ -31,14 +36,19 @@ type BlockBuilder struct { } func NewBlockOptions(NGramLength, NGramSkip uint64) BlockOptions { + return NewBlockOptionsFromSchema(Schema{ + version: byte(1), + nGramLength: NGramLength, + nGramSkip: NGramSkip, + }) +} + +func NewBlockOptionsFromSchema(s Schema) BlockOptions { return BlockOptions{ - schema: Schema{ - version: byte(1), - nGramLength: NGramLength, - nGramSkip: NGramSkip, - }, - SeriesPageSize: 100, - BloomPageSize: 10 << 10, // 0.01MB + Schema: s, + // TODO(owen-d): benchmark and find good defaults + SeriesPageSize: 4 << 10, // 4KB, typical page size + BloomPageSize: 256 << 10, // 256KB, no idea what to make this } } @@ -124,7 +134,7 @@ func NewBloomBlockBuilder(opts BlockOptions, writer io.WriteCloser) *BloomBlockB func (b *BloomBlockBuilder) WriteSchema() error { b.scratch.Reset() - b.opts.schema.Encode(b.scratch) + b.opts.Schema.Encode(b.scratch) if _, err := b.writer.Write(b.scratch.Get()); err != nil { return errors.Wrap(err, "writing schema") } @@ -191,7 +201,7 @@ func (b *BloomBlockBuilder) flushPage() error { decompressedLen, compressedLen, err := b.page.writePage( b.writer, - b.opts.schema.CompressorPool(), + b.opts.Schema.CompressorPool(), crc32Hash, ) if err != nil { @@ -300,7 +310,7 @@ func NewIndexBuilder(opts BlockOptions, writer io.WriteCloser) *IndexBuilder { func (b *IndexBuilder) WriteSchema() error { b.scratch.Reset() - b.opts.schema.Encode(b.scratch) + b.opts.Schema.Encode(b.scratch) if _, err := b.writer.Write(b.scratch.Get()); err != nil { return errors.Wrap(err, "writing schema") } @@ -381,7 +391,7 @@ func (b *IndexBuilder) flushPage() error { decompressedLen, compressedLen, err := b.page.writePage( b.writer, - b.opts.schema.CompressorPool(), + b.opts.Schema.CompressorPool(), crc32Hash, ) if err != nil { diff --git a/pkg/storage/bloom/v1/builder_test.go b/pkg/storage/bloom/v1/builder_test.go index e67dce4c97549..cb28f0cb53357 100644 --- a/pkg/storage/bloom/v1/builder_test.go +++ b/pkg/storage/bloom/v1/builder_test.go @@ -11,21 +11,10 @@ import ( "github.com/grafana/loki/pkg/chunkenc" ) -func EqualIterators[T any](t *testing.T, test func(a, b T), expected, actual Iterator[T]) { - for expected.Next() { - require.True(t, actual.Next()) - a, b := expected.At(), actual.At() - test(a, b) - } - require.False(t, actual.Next()) - require.Nil(t, expected.Err()) - require.Nil(t, actual.Err()) -} - func TestBlockBuilderRoundTrip(t *testing.T) { numSeries := 100 numKeysPerSeries := 10000 - data, keys := mkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) + data, keys := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) // references for linking in memory reader+writer indexBuf := bytes.NewBuffer(nil) @@ -59,7 +48,7 @@ func TestBlockBuilderRoundTrip(t *testing.T) { builder, err := NewBlockBuilder( BlockOptions{ - schema: schema, + Schema: schema, SeriesPageSize: 100, BloomPageSize: 10 << 10, }, @@ -115,9 +104,9 @@ func TestMergeBuilder(t *testing.T) { numSeries := 100 numKeysPerSeries := 100 blocks := make([]PeekingIterator[*SeriesWithBloom], 0, nBlocks) - data, _ := mkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) + data, _ := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) blockOpts := BlockOptions{ - schema: Schema{ + Schema: Schema{ version: DefaultSchemaVersion, encoding: chunkenc.EncSnappy, }, @@ -198,7 +187,7 @@ func TestMergeBuilder(t *testing.T) { func TestBlockReset(t *testing.T) { numSeries := 100 numKeysPerSeries := 10000 - data, _ := mkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 1, 0xffff, 0, 10000) + data, _ := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 1, 0xffff, 0, 10000) indexBuf := bytes.NewBuffer(nil) bloomsBuf := bytes.NewBuffer(nil) @@ -214,7 +203,7 @@ func TestBlockReset(t *testing.T) { builder, err := NewBlockBuilder( BlockOptions{ - schema: schema, + Schema: schema, SeriesPageSize: 100, BloomPageSize: 10 << 10, }, @@ -250,7 +239,7 @@ func TestMergeBuilder_Roundtrip(t *testing.T) { numSeries := 100 numKeysPerSeries := 100 minTs, maxTs := model.Time(0), model.Time(10000) - xs, _ := mkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, minTs, maxTs) + xs, _ := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, minTs, maxTs) var data [][]*SeriesWithBloom @@ -272,7 +261,7 @@ func TestMergeBuilder_Roundtrip(t *testing.T) { builder, err := NewBlockBuilder( BlockOptions{ - schema: Schema{ + Schema: Schema{ version: DefaultSchemaVersion, encoding: chunkenc.EncSnappy, }, @@ -345,7 +334,7 @@ func TestMergeBuilder_Roundtrip(t *testing.T) { checksum, err := mb.Build(builder) require.Nil(t, err) - require.Equal(t, uint32(0x779633b5), checksum) + require.Equal(t, uint32(0x2ec4fd6a), checksum) // ensure the new block contains one copy of all the data // by comparing it against an iterator over the source data diff --git a/pkg/storage/bloom/v1/dedupe_test.go b/pkg/storage/bloom/v1/dedupe_test.go index f30578c13cd1b..443d8e3e3750e 100644 --- a/pkg/storage/bloom/v1/dedupe_test.go +++ b/pkg/storage/bloom/v1/dedupe_test.go @@ -10,7 +10,7 @@ func TestMergeDedupeIter(t *testing.T) { var ( numSeries = 100 numKeysPerSeries = 10000 - data, _ = mkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) + data, _ = MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) dataPtr = PointerSlice(data) queriers = make([]PeekingIterator[*SeriesWithBloom], 4) ) diff --git a/pkg/storage/bloom/v1/fuse_test.go b/pkg/storage/bloom/v1/fuse_test.go index e784ac0168201..1b51320e1566b 100644 --- a/pkg/storage/bloom/v1/fuse_test.go +++ b/pkg/storage/bloom/v1/fuse_test.go @@ -20,11 +20,11 @@ func TestFusedQuerier(t *testing.T) { reader := NewByteReader(indexBuf, bloomsBuf) numSeries := 100 numKeysPerSeries := 10000 - data, _ := mkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) + data, _ := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) builder, err := NewBlockBuilder( BlockOptions{ - schema: Schema{ + Schema: Schema{ version: DefaultSchemaVersion, encoding: chunkenc.EncSnappy, }, @@ -112,11 +112,11 @@ func setupBlockForBenchmark(b *testing.B) (*BlockQuerier, [][]Request, []chan Ou reader := NewByteReader(indexBuf, bloomsBuf) numSeries := 10000 numKeysPerSeries := 100 - data, _ := mkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffffff, 0, 10000) + data, _ := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffffff, 0, 10000) builder, err := NewBlockBuilder( BlockOptions{ - schema: Schema{ + Schema: Schema{ version: DefaultSchemaVersion, encoding: chunkenc.EncSnappy, }, diff --git a/pkg/storage/bloom/v1/index.go b/pkg/storage/bloom/v1/index.go index 1ccc372248a8f..10c1e41fd1139 100644 --- a/pkg/storage/bloom/v1/index.go +++ b/pkg/storage/bloom/v1/index.go @@ -17,6 +17,18 @@ type Schema struct { nGramLength, nGramSkip uint64 } +func (s Schema) Compatible(other Schema) bool { + return s == other +} + +func (s Schema) NGramLen() int { + return int(s.nGramLength) +} + +func (s Schema) NGramSkip() int { + return int(s.nGramSkip) +} + // byte length func (s Schema) Len() int { // magic number + version + encoding + ngram length + ngram skip @@ -74,10 +86,6 @@ func (s *Schema) Decode(dec *encoding.Decbuf) error { return dec.Err() } -func (s Schema) NGramLen() int { - return int(s.nGramLength) -} - // Block index is a set of series pages along with // the headers for each page type BlockIndex struct { diff --git a/pkg/storage/bloom/v1/merge_test.go b/pkg/storage/bloom/v1/merge_test.go index 8f2bf02f4c2de..1b91366c2cb6d 100644 --- a/pkg/storage/bloom/v1/merge_test.go +++ b/pkg/storage/bloom/v1/merge_test.go @@ -12,7 +12,7 @@ func TestMergeBlockQuerier_NonOverlapping(t *testing.T) { numKeysPerSeries = 10000 numQueriers = 4 queriers []PeekingIterator[*SeriesWithBloom] - data, _ = mkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) + data, _ = MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) ) for i := 0; i < numQueriers; i++ { var ptrs []*SeriesWithBloom @@ -39,7 +39,7 @@ func TestMergeBlockQuerier_Duplicate(t *testing.T) { numKeysPerSeries = 10000 numQueriers = 2 queriers []PeekingIterator[*SeriesWithBloom] - data, _ = mkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) + data, _ = MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) ) for i := 0; i < numQueriers; i++ { queriers = append( @@ -69,7 +69,7 @@ func TestMergeBlockQuerier_Overlapping(t *testing.T) { numKeysPerSeries = 10000 numQueriers = 4 queriers []PeekingIterator[*SeriesWithBloom] - data, _ = mkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) + data, _ = MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) slices = make([][]*SeriesWithBloom, numQueriers) ) for i := 0; i < numSeries; i++ { diff --git a/pkg/storage/bloom/v1/test_util.go b/pkg/storage/bloom/v1/test_util.go index 3b706297171b3..fb9c8a0e4a389 100644 --- a/pkg/storage/bloom/v1/test_util.go +++ b/pkg/storage/bloom/v1/test_util.go @@ -13,6 +13,8 @@ import ( "github.com/grafana/loki/pkg/storage/bloom/v1/filter" ) +// TODO(owen-d): this should probably be in it's own testing-util package + func MakeBlockQuerier(t testing.TB, fromFp, throughFp model.Fingerprint, fromTs, throughTs model.Time) (*BlockQuerier, []SeriesWithBloom) { // references for linking in memory reader+writer indexBuf := bytes.NewBuffer(nil) @@ -21,11 +23,11 @@ func MakeBlockQuerier(t testing.TB, fromFp, throughFp model.Fingerprint, fromTs, reader := NewByteReader(indexBuf, bloomsBuf) numSeries := int(throughFp - fromFp) numKeysPerSeries := 1000 - data, _ := mkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, fromFp, throughFp, fromTs, throughTs) + data, _ := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, fromFp, throughFp, fromTs, throughTs) builder, err := NewBlockBuilder( BlockOptions{ - schema: Schema{ + Schema: Schema{ version: DefaultSchemaVersion, encoding: chunkenc.EncSnappy, nGramLength: 4, // see DefaultNGramLength in bloom_tokenizer_test.go @@ -44,7 +46,7 @@ func MakeBlockQuerier(t testing.TB, fromFp, throughFp model.Fingerprint, fromTs, return NewBlockQuerier(block), data } -func mkBasicSeriesWithBlooms(nSeries, keysPerSeries int, fromFp, throughFp model.Fingerprint, fromTs, throughTs model.Time) (seriesList []SeriesWithBloom, keysList [][][]byte) { +func MkBasicSeriesWithBlooms(nSeries, keysPerSeries int, fromFp, throughFp model.Fingerprint, fromTs, throughTs model.Time) (seriesList []SeriesWithBloom, keysList [][][]byte) { seriesList = make([]SeriesWithBloom, 0, nSeries) keysList = make([][][]byte, 0, nSeries) @@ -85,3 +87,14 @@ func mkBasicSeriesWithBlooms(nSeries, keysPerSeries int, fromFp, throughFp model } return } + +func EqualIterators[T any](t *testing.T, test func(a, b T), expected, actual Iterator[T]) { + for expected.Next() { + require.True(t, actual.Next()) + a, b := expected.At(), actual.At() + test(a, b) + } + require.False(t, actual.Next()) + require.Nil(t, expected.Err()) + require.Nil(t, actual.Err()) +} From 2c9482d8a28fc909555a53e0fd0bf20ad5b476f8 Mon Sep 17 00:00:00 2001 From: Travis Patterson Date: Wed, 24 Jan 2024 16:27:53 -0700 Subject: [PATCH 04/23] reset the basemap when the labelsbuilder is reset (#11771) This PR fixes an issue where the `label_format` stage sometimes doesn't populate labels when it should. The label `baz` exists on all log lines but is only populated on the first line. ![image](https://github.com/grafana/loki/assets/1413241/86585a94-937f-4e07-9657-b94699ce8586) The issue is this call to [IntoMap](https://github.com/grafana/loki/blob/06bb20914b027c96ac3af27ae36e91f35529524d/pkg/logql/log/fmt.go#L403). It tries to avoid copying the map multiple times [here](https://github.com/grafana/loki/blob/06bb20914b027c96ac3af27ae36e91f35529524d/pkg/logql/log/labels.go#L482) but the `baseMap` variable isn't reset when the labels builder is reset. As a result, calls to to `IntoMap` after the builder has been reset mean that `!b.hasDel() && !b.hasAdd() && !b.HasErr()` evaluates to `true` but `b.baseMap == nil` also evaluates to `true` so no items are copied. This PR makes it so that `b.baseMap` is reset along with the rest of the builder's state in `Reset` --- pkg/logql/log/labels.go | 9 +++--- pkg/logql/log/labels_test.go | 58 ++++++++++++++++++++++++++++++++++++ 2 files changed, 63 insertions(+), 4 deletions(-) diff --git a/pkg/logql/log/labels.go b/pkg/logql/log/labels.go index ea4570e652df6..10414a8ed3b88 100644 --- a/pkg/logql/log/labels.go +++ b/pkg/logql/log/labels.go @@ -135,6 +135,7 @@ type BaseLabelsBuilder struct { errDetails string groups []string + baseMap map[string]string parserKeyHints ParserHint // label key hints for metric queries that allows to limit parser extractions to only this list of labels. without, noLabels bool referencedStructuredMetadata bool @@ -146,7 +147,6 @@ type BaseLabelsBuilder struct { // LabelsBuilder is the same as labels.Builder but tailored for this package. type LabelsBuilder struct { base labels.Labels - baseMap map[string]string buf labels.Labels currentResult LabelsResult groupedResult LabelsResult @@ -211,6 +211,7 @@ func (b *BaseLabelsBuilder) Reset() { } b.err = "" b.errDetails = "" + b.baseMap = nil b.parserKeyHints.Reset() } @@ -481,9 +482,9 @@ func (b *LabelsBuilder) IntoMap(m map[string]string) { if !b.hasDel() && !b.hasAdd() && !b.HasErr() { if b.baseMap == nil { b.baseMap = b.base.Map() - for k, v := range b.baseMap { - m[k] = v - } + } + for k, v := range b.baseMap { + m[k] = v } return } diff --git a/pkg/logql/log/labels_test.go b/pkg/logql/log/labels_test.go index b4efaaa1b65bd..0f859c8125420 100644 --- a/pkg/logql/log/labels_test.go +++ b/pkg/logql/log/labels_test.go @@ -68,6 +68,64 @@ func TestLabelsBuilder_LabelsError(t *testing.T) { require.Equal(t, labels.FromStrings("already", "in"), lbs) } +func TestLabelsBuilder_IntoMap(t *testing.T) { + strs := []string{ + "namespace", "loki", + "job", "us-central1/loki", + "cluster", "us-central1", + "ToReplace", "text", + } + lbs := labels.FromStrings(strs...) + + t.Run("it still copies the map after a Reset", func(t *testing.T) { + b := NewBaseLabelsBuilder().ForLabels(lbs, lbs.Hash()) + + m := map[string]string{} + b.IntoMap(m) + + require.Equal(t, map[string]string{ + "namespace": "loki", + "job": "us-central1/loki", + "cluster": "us-central1", + "ToReplace": "text", + }, m) + + b.Reset() + + m2 := map[string]string{} + b.IntoMap(m2) + require.Equal(t, map[string]string{ + "namespace": "loki", + "job": "us-central1/loki", + "cluster": "us-central1", + "ToReplace": "text", + }, m2) + }) + + t.Run("it can copy the map several times", func(t *testing.T) { + b := NewBaseLabelsBuilder().ForLabels(lbs, lbs.Hash()) + + m := map[string]string{} + b.IntoMap(m) + + require.Equal(t, map[string]string{ + "namespace": "loki", + "job": "us-central1/loki", + "cluster": "us-central1", + "ToReplace": "text", + }, m) + + m2 := map[string]string{} + b.IntoMap(m2) + require.Equal(t, map[string]string{ + "namespace": "loki", + "job": "us-central1/loki", + "cluster": "us-central1", + "ToReplace": "text", + }, m2) + }) +} + func TestLabelsBuilder_LabelsResult(t *testing.T) { strs := []string{ "namespace", "loki", From 5acc29f2f63dc96dc1a6bcbad3e57d3aeefc9d83 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Thu, 25 Jan 2024 12:05:54 +0100 Subject: [PATCH 05/23] Bloom Gateway: Fix panic in chunk removal after receiving query results (#11770) **What this PR does / why we need it**: While removing not matching chunks from a fingerprint, the for loop over the chunks removes items, and therefore the index of the current iteration needs to be adjusted to accommodate for the removal, otherwise it may panic with `out of bounds`. Signed-off-by: Christian Haudum --- pkg/bloomgateway/bloomgateway.go | 9 +++-- pkg/bloomgateway/bloomgateway_test.go | 58 +++++++++++++++++++++++++++ 2 files changed, 63 insertions(+), 4 deletions(-) diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go index 5c2cc9dad003e..bb58f79288aee 100644 --- a/pkg/bloomgateway/bloomgateway.go +++ b/pkg/bloomgateway/bloomgateway.go @@ -355,7 +355,7 @@ outer: continue } // we must not remove items from req.Refs as long as the worker may iterater over them - g.removeNotMatchingChunks(req, o) + removeNotMatchingChunks(req, o, g.logger) } g.metrics.addUnfilteredCount(numChunksUnfiltered) @@ -365,7 +365,7 @@ outer: return &logproto.FilterChunkRefResponse{ChunkRefs: req.Refs}, nil } -func (g *Gateway) removeNotMatchingChunks(req *logproto.FilterChunkRefRequest, res v1.Output) { +func removeNotMatchingChunks(req *logproto.FilterChunkRefRequest, res v1.Output, logger log.Logger) { // binary search index of fingerprint idx := sort.Search(len(req.Refs), func(i int) bool { return req.Refs[i].Fingerprint >= uint64(res.Fp) @@ -373,7 +373,7 @@ func (g *Gateway) removeNotMatchingChunks(req *logproto.FilterChunkRefRequest, r // fingerprint not found if idx >= len(req.Refs) { - level.Error(g.logger).Log("msg", "index out of range", "idx", idx, "len", len(req.Refs), "fp", uint64(res.Fp)) + level.Error(logger).Log("msg", "index out of range", "idx", idx, "len", len(req.Refs), "fp", uint64(res.Fp)) return } @@ -387,10 +387,11 @@ func (g *Gateway) removeNotMatchingChunks(req *logproto.FilterChunkRefRequest, r for i := range res.Removals { toRemove := res.Removals[i] - for j := range req.Refs[idx].Refs { + for j := 0; j < len(req.Refs[idx].Refs); j++ { if toRemove.Checksum == req.Refs[idx].Refs[j].Checksum { req.Refs[idx].Refs[j] = nil // avoid leaking pointer req.Refs[idx].Refs = append(req.Refs[idx].Refs[:j], req.Refs[idx].Refs[j+1:]...) + j-- // since we removed the current item at index, we have to redo the same index } } } diff --git a/pkg/bloomgateway/bloomgateway_test.go b/pkg/bloomgateway/bloomgateway_test.go index 1e85e7d2089c0..e6d2b910c12f3 100644 --- a/pkg/bloomgateway/bloomgateway_test.go +++ b/pkg/bloomgateway/bloomgateway_test.go @@ -340,6 +340,64 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { }) } +func TestBloomGateway_RemoveNotMatchingChunks(t *testing.T) { + t.Run("removing chunks partially", func(t *testing.T) { + req := &logproto.FilterChunkRefRequest{ + Refs: []*logproto.GroupedChunkRefs{ + {Fingerprint: 0x00, Tenant: "fake", Refs: []*logproto.ShortRef{ + {Checksum: 0x1}, + {Checksum: 0x2}, + {Checksum: 0x3}, + {Checksum: 0x4}, + {Checksum: 0x5}, + }}, + }, + } + res := v1.Output{ + Fp: 0x00, Removals: v1.ChunkRefs{ + {Checksum: 0x2}, + {Checksum: 0x4}, + }, + } + expected := &logproto.FilterChunkRefRequest{ + Refs: []*logproto.GroupedChunkRefs{ + {Fingerprint: 0x00, Tenant: "fake", Refs: []*logproto.ShortRef{ + {Checksum: 0x1}, + {Checksum: 0x3}, + {Checksum: 0x5}, + }}, + }, + } + removeNotMatchingChunks(req, res, log.NewNopLogger()) + require.Equal(t, expected, req) + }) + + t.Run("removing all chunks removed fingerprint ref", func(t *testing.T) { + req := &logproto.FilterChunkRefRequest{ + Refs: []*logproto.GroupedChunkRefs{ + {Fingerprint: 0x00, Tenant: "fake", Refs: []*logproto.ShortRef{ + {Checksum: 0x1}, + {Checksum: 0x2}, + {Checksum: 0x3}, + }}, + }, + } + res := v1.Output{ + Fp: 0x00, Removals: v1.ChunkRefs{ + {Checksum: 0x1}, + {Checksum: 0x2}, + {Checksum: 0x2}, + }, + } + expected := &logproto.FilterChunkRefRequest{ + Refs: []*logproto.GroupedChunkRefs{}, + } + removeNotMatchingChunks(req, res, log.NewNopLogger()) + require.Equal(t, expected, req) + }) + +} + func createBlockQueriers(t *testing.T, numBlocks int, from, through model.Time, minFp, maxFp model.Fingerprint) ([]bloomshipper.BlockQuerierWithFingerprintRange, [][]v1.SeriesWithBloom) { t.Helper() step := (maxFp - minFp) / model.Fingerprint(numBlocks) From 600e390b16ba09d65d154438c2fbe68aad496bc5 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Thu, 25 Jan 2024 12:06:15 +0100 Subject: [PATCH 06/23] Extend `TestPartitionFingerprintRange` test case (#11750) **What this PR does / why we need it**: The test cases now also cover partially out of bounds blocks as well as inconsecutive block ranges. --------- Signed-off-by: Christian Haudum --- pkg/bloomgateway/util.go | 10 ---- pkg/bloomgateway/util_test.go | 88 +++++++++++++++++++++++------------ 2 files changed, 59 insertions(+), 39 deletions(-) diff --git a/pkg/bloomgateway/util.go b/pkg/bloomgateway/util.go index c45af6061e253..89d238864ac45 100644 --- a/pkg/bloomgateway/util.go +++ b/pkg/bloomgateway/util.go @@ -73,16 +73,6 @@ func convertToChunkRefs(refs []*logproto.ShortRef) v1.ChunkRefs { return result } -// getFirstLast returns the first and last item of a fingerprint slice -// It assumes an ascending sorted list of fingerprints. -func getFirstLast[T any](s []T) (T, T) { - var zero T - if len(s) == 0 { - return zero, zero - } - return s[0], s[len(s)-1] -} - type boundedTasks struct { blockRef bloomshipper.BlockRef tasks []Task diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index 08c6d2a1306a4..152309ddbde07 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -37,38 +37,68 @@ func mkBlockRef(minFp, maxFp uint64) bloomshipper.BlockRef { } func TestPartitionFingerprintRange(t *testing.T) { - seriesPerBound := 100 - bounds := []bloomshipper.BlockRef{ - mkBlockRef(0, 99), - mkBlockRef(100, 199), - mkBlockRef(200, 299), - mkBlockRef(300, 399), // one out of bounds block - } + t.Run("consecutive block ranges", func(t *testing.T) { + bounds := []bloomshipper.BlockRef{ + mkBlockRef(0, 99), // out of bounds block + mkBlockRef(100, 199), // contains partially [150..199] + mkBlockRef(200, 299), // contains fully [200..299] + mkBlockRef(300, 399), // contains partially [300..349] + mkBlockRef(400, 499), // out of bounds block + } - nTasks := 4 - nSeries := 300 - tasks := make([]Task, nTasks) - for i := 0; i < nSeries; i++ { - if tasks[i%4].Request == nil { - tasks[i%4].Request = &logproto.FilterChunkRefRequest{} + nTasks := 5 + nSeries := 200 + startFp := 150 + + tasks := make([]Task, nTasks) + for i := startFp; i < startFp+nSeries; i++ { + if tasks[i%nTasks].Request == nil { + tasks[i%nTasks].Request = &logproto.FilterChunkRefRequest{} + } + tasks[i%nTasks].Request.Refs = append(tasks[i%nTasks].Request.Refs, &logproto.GroupedChunkRefs{Fingerprint: uint64(i)}) } - tasks[i%4].Request.Refs = append(tasks[i%nTasks].Request.Refs, &logproto.GroupedChunkRefs{Fingerprint: uint64(i)}) - } - results := partitionFingerprintRange(tasks, bounds) - require.Equal(t, 3, len(results)) // ensure we only return bounds in range - for _, res := range results { - // ensure we have the right number of tasks per bound - for i := 0; i < nTasks; i++ { - require.Equal(t, seriesPerBound/nTasks, len(res.tasks[i].Request.Refs)) + results := partitionFingerprintRange(tasks, bounds) + require.Equal(t, 3, len(results)) // ensure we only return bounds in range + + actualFingerprints := make([]*logproto.GroupedChunkRefs, 0, nSeries) + expectedTaskRefs := []int{10, 20, 10} + for i, res := range results { + // ensure we have the right number of tasks per bound + require.Len(t, res.tasks, 5) + for _, task := range res.tasks { + require.Equal(t, expectedTaskRefs[i], len(task.Request.Refs)) + actualFingerprints = append(actualFingerprints, task.Request.Refs...) + } } - } - // ensure bound membership - for i := 0; i < nSeries; i++ { - require.Equal(t, - &logproto.GroupedChunkRefs{Fingerprint: uint64(i)}, - results[i/seriesPerBound].tasks[i%nTasks].Request.Refs[i%seriesPerBound/nTasks], - ) - } + // ensure bound membership + expectedFingerprints := make([]*logproto.GroupedChunkRefs, nSeries) + for i := 0; i < nSeries; i++ { + expectedFingerprints[i] = &logproto.GroupedChunkRefs{Fingerprint: uint64(startFp + i)} + } + + require.ElementsMatch(t, expectedFingerprints, actualFingerprints) + }) + + t.Run("inconsecutive block ranges", func(t *testing.T) { + bounds := []bloomshipper.BlockRef{ + mkBlockRef(0, 89), + mkBlockRef(100, 189), + mkBlockRef(200, 289), + } + + task := Task{Request: &logproto.FilterChunkRefRequest{}} + for i := 0; i < 300; i++ { + task.Request.Refs = append(task.Request.Refs, &logproto.GroupedChunkRefs{Fingerprint: uint64(i)}) + } + + results := partitionFingerprintRange([]Task{task}, bounds) + require.Equal(t, 3, len(results)) // ensure we only return bounds in range + for _, res := range results { + // ensure we have the right number of tasks per bound + require.Len(t, res.tasks, 1) + require.Len(t, res.tasks[0].Request.Refs, 90) + } + }) } From 5c8fd520d3a51333b1bdc3600af4b5d3b527a433 Mon Sep 17 00:00:00 2001 From: Ashwanth Date: Thu, 25 Jan 2024 16:44:12 +0530 Subject: [PATCH 07/23] fix(background cache): increment queue size exactly once (#11776) **What this PR does / why we need it**: While looking at the metrics from one of our cells, we noticed that the queue size is set to a very high value even when the length is pretty much 0. Background cache is incrementing the queue size twice for each enqueued key. This pr removes the additional increment call. `TestBackgroundSizeLimit` might be flaky because of the [writeBackLoop](https://github.com/grafana/loki/blob/8eb09c78c842b61f6619fb3755a43b180536b761/pkg/storage/chunk/cache/background.go#L193) which dequeues from the channel and reduces the queue size concurrently. To make the test predictable, I have set the `WriteBackGoroutines` to 0. **Which issue(s) this PR fixes**: Fixes # **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --- CHANGELOG.md | 1 + pkg/storage/chunk/cache/background.go | 1 - pkg/storage/chunk/cache/background_extra_test.go | 5 +++++ pkg/storage/chunk/cache/background_test.go | 10 +++++----- 4 files changed, 11 insertions(+), 6 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8c22d9126f9c6..37599ae8d347c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -64,6 +64,7 @@ * [11606](https://github.com/grafana/loki/pull/11606) **dannykopping** Fixed regression adding newlines to HTTP error response bodies which may break client integrations. * [11657](https://github.com/grafana/loki/pull/11657) **ashwanthgoli** Log results cache: compose empty response based on the request being served to avoid returning incorrect limit or direction. * [11587](https://github.com/grafana/loki/pull/11587) **trevorwhitney** Fix semantics of label parsing logic of metrics and logs queries. Both only parse the first label if multiple extractions into the same label are requested. +* [11776](https://github.com/grafana/loki/pull/11776) **ashwanthgoli** Background Cache: Fixes a bug that is causing the background queue size to be incremented twice for each enqueued item. ##### Changes diff --git a/pkg/storage/chunk/cache/background.go b/pkg/storage/chunk/cache/background.go index 299444c6a54e0..92995cf08d905 100644 --- a/pkg/storage/chunk/cache/background.go +++ b/pkg/storage/chunk/cache/background.go @@ -159,7 +159,6 @@ func (c *backgroundCache) Store(ctx context.Context, keys []string, bufs [][]byt select { case c.bgWrites <- bgWrite: - c.size.Add(int64(size)) c.queueBytes.Set(float64(c.size.Load())) c.queueLength.Add(float64(num)) c.enqueuedBytes.Add(float64(size)) diff --git a/pkg/storage/chunk/cache/background_extra_test.go b/pkg/storage/chunk/cache/background_extra_test.go index 07aa21c847f88..57f1eca455ab2 100644 --- a/pkg/storage/chunk/cache/background_extra_test.go +++ b/pkg/storage/chunk/cache/background_extra_test.go @@ -5,3 +5,8 @@ func Flush(c Cache) { close(b.bgWrites) b.wg.Wait() } + +func QueueSize(c Cache) int64 { + b := c.(*backgroundCache) + return b.size.Load() +} diff --git a/pkg/storage/chunk/cache/background_test.go b/pkg/storage/chunk/cache/background_test.go index af9827865ee7a..a718301957603 100644 --- a/pkg/storage/chunk/cache/background_test.go +++ b/pkg/storage/chunk/cache/background_test.go @@ -47,7 +47,7 @@ func TestBackgroundSizeLimit(t *testing.T) { require.NoError(t, err) c := cache.NewBackground("mock", cache.BackgroundConfig{ - WriteBackGoroutines: 1, + WriteBackGoroutines: 0, WriteBackBuffer: 100, WriteBackSizeLimit: flagext.ByteSize(limit), }, cache.NewMockCache(), nil) @@ -63,10 +63,10 @@ func TestBackgroundSizeLimit(t *testing.T) { // store the first 10KB require.NoError(t, c.Store(ctx, []string{firstKey}, [][]byte{first})) + require.Equal(t, cache.QueueSize(c), int64(10e3)) + // second key will not be stored because it will exceed the 15KB limit require.NoError(t, c.Store(ctx, []string{secondKey}, [][]byte{second})) - cache.Flush(c) - - found, _, _, _ := c.Fetch(ctx, []string{firstKey, secondKey}) - require.Equal(t, []string{firstKey}, found) + require.Equal(t, cache.QueueSize(c), int64(10e3)) + c.Stop() } From 6871f3c555601c849289d8eebb54935dd581e341 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Thu, 25 Jan 2024 14:44:22 +0100 Subject: [PATCH 08/23] Bloom Gateway: Partition requests into multiple tasks before enqueuing (#11768) **What this PR does / why we need it**: This PR changes at which point a request is split into multiple tasks in case the chunks boundaries of the request overlap multiple days. This solves the problem that the request handler needs to know for how many responses from the block querier needs to wait. Previously, a single task that spanned across multiple days was executed multiple times (once for each matching day). --------- Signed-off-by: Christian Haudum --- pkg/bloomgateway/bloomgateway.go | 80 +++++++++-- pkg/bloomgateway/multiplexing.go | 157 ++++++--------------- pkg/bloomgateway/multiplexing_test.go | 148 +++++--------------- pkg/bloomgateway/util.go | 76 ++++++++++- pkg/bloomgateway/util_test.go | 189 ++++++++++++++++++++++++-- pkg/bloomgateway/worker.go | 34 ++--- 6 files changed, 410 insertions(+), 274 deletions(-) diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go index bb58f79288aee..f9acd70a2ca21 100644 --- a/pkg/bloomgateway/bloomgateway.go +++ b/pkg/bloomgateway/bloomgateway.go @@ -299,6 +299,18 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk return nil, err } + // start time == end time --> empty response + if req.From.Equal(req.Through) { + return &logproto.FilterChunkRefResponse{ + ChunkRefs: []*logproto.GroupedChunkRefs{}, + }, nil + } + + // start time > end time --> error response + if req.Through.Before(req.From) { + return nil, errors.New("from time must not be after through time") + } + numChunksUnfiltered := len(req.Refs) // Shortcut if request does not contain filters @@ -315,21 +327,64 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk return req.Refs[i].Fingerprint < req.Refs[j].Fingerprint }) - task, resCh, errCh, err := NewTask(tenantID, req) + var expectedResponses int + seriesWithBloomsPerDay := partitionRequest(req) - if err != nil { - return nil, err + // no tasks --> empty response + if len(seriesWithBloomsPerDay) == 0 { + return &logproto.FilterChunkRefResponse{ + ChunkRefs: []*logproto.GroupedChunkRefs{}, + }, nil + } + + tasks := make([]Task, 0, len(seriesWithBloomsPerDay)) + for _, seriesWithBounds := range seriesWithBloomsPerDay { + task, err := NewTask(tenantID, seriesWithBounds, req.Filters) + if err != nil { + return nil, err + } + tasks = append(tasks, task) + expectedResponses += len(seriesWithBounds.series) } g.activeUsers.UpdateUserTimestamp(tenantID, time.Now()) - level.Info(g.logger).Log("msg", "enqueue task", "task", task.ID) - g.queue.Enqueue(tenantID, []string{}, task, func() { - // When enqueuing, we also add the task to the pending tasks - g.pendingTasks.Add(task.ID, task) - }) - requestCount := len(req.Refs) - responses := responsesPool.Get(requestCount) + errCh := make(chan error, 1) + resCh := make(chan v1.Output, 1) + + for _, task := range tasks { + level.Info(g.logger).Log("msg", "enqueue task", "task", task.ID, "day", task.day, "series", len(task.series)) + g.queue.Enqueue(tenantID, []string{}, task, func() { + // When enqueuing, we also add the task to the pending tasks + g.pendingTasks.Add(task.ID, task) + }) + + // Forward responses or error to the main channels + // TODO(chaudum): Refactor to make tasks cancelable + go func(t Task) { + for { + select { + case <-ctx.Done(): + return + case err := <-t.ErrCh: + if ctx.Err() != nil { + level.Warn(g.logger).Log("msg", "received err from channel, but context is already done", "err", ctx.Err()) + return + } + errCh <- err + case res := <-t.ResCh: + level.Debug(g.logger).Log("msg", "got partial result", "task", t.ID, "tenant", tenantID, "fp_int", uint64(res.Fp), "fp_hex", res.Fp, "chunks_to_remove", res.Removals.Len()) + if ctx.Err() != nil { + level.Warn(g.logger).Log("msg", "received res from channel, but context is already done", "err", ctx.Err()) + return + } + resCh <- res + } + } + }(task) + } + + responses := responsesPool.Get(expectedResponses) defer responsesPool.Put(responses) outer: @@ -342,9 +397,9 @@ outer: case res := <-resCh: responses = append(responses, res) // log line is helpful for debugging tests - level.Debug(g.logger).Log("msg", "got partial result", "task", task.ID, "tenant", tenantID, "fp_int", uint64(res.Fp), "fp_hex", res.Fp, "chunks_to_remove", res.Removals.Len(), "progress", fmt.Sprintf("%d/%d", len(responses), requestCount)) + level.Debug(g.logger).Log("msg", "got partial result", "progress", fmt.Sprintf("%d/%d", len(responses), expectedResponses)) // wait for all parts of the full response - if len(responses) == requestCount { + if len(responses) == expectedResponses { break outer } } @@ -354,7 +409,6 @@ outer: if o.Removals.Len() == 0 { continue } - // we must not remove items from req.Refs as long as the worker may iterater over them removeNotMatchingChunks(req, o, g.logger) } diff --git a/pkg/bloomgateway/multiplexing.go b/pkg/bloomgateway/multiplexing.go index 1afdc72c799ef..120e6da26f8b5 100644 --- a/pkg/bloomgateway/multiplexing.go +++ b/pkg/bloomgateway/multiplexing.go @@ -1,13 +1,13 @@ package bloomgateway import ( - "sort" "time" "github.com/oklog/ulid" "github.com/prometheus/common/model" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql/syntax" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" ) @@ -25,138 +25,63 @@ type Task struct { ID ulid.ULID // Tenant is the tenant ID Tenant string - // Request is the original request - Request *logproto.FilterChunkRefRequest + // ErrCh is a send-only channel to write an error to - ErrCh chan<- error + ErrCh chan error // ResCh is a send-only channel to write partial responses to - ResCh chan<- v1.Output + ResCh chan v1.Output + + // series of the original request + series []*logproto.GroupedChunkRefs + // filters of the original request + filters []syntax.LineFilter + // from..through date of the task's chunks + bounds model.Interval + + // TODO(chaudum): Investigate how to remove that. + day model.Time } // NewTask returns a new Task that can be enqueued to the task queue. // In addition, it returns a result and an error channel, as well // as an error if the instantiation fails. -func NewTask(tenantID string, req *logproto.FilterChunkRefRequest) (Task, chan v1.Output, chan error, error) { +func NewTask(tenantID string, refs seriesWithBounds, filters []syntax.LineFilter) (Task, error) { key, err := ulid.New(ulid.Now(), nil) if err != nil { - return Task{}, nil, nil, err + return Task{}, err } errCh := make(chan error, 1) - resCh := make(chan v1.Output, 1) + resCh := make(chan v1.Output, len(refs.series)) + task := Task{ ID: key, Tenant: tenantID, - Request: req, ErrCh: errCh, ResCh: resCh, + filters: filters, + series: refs.series, + bounds: refs.bounds, + day: refs.day, } - return task, resCh, errCh, nil -} - -// Copy returns a copy of the existing task but with a new slice of chunks -func (t Task) Copy(refs []*logproto.GroupedChunkRefs) Task { - return Task{ - ID: t.ID, - Tenant: t.Tenant, - Request: &logproto.FilterChunkRefRequest{ - From: t.Request.From, - Through: t.Request.Through, - Filters: t.Request.Filters, - Refs: refs, - }, - ErrCh: t.ErrCh, - ResCh: t.ResCh, - } -} - -// Bounds returns the day boundaries of the task -func (t Task) Bounds() (time.Time, time.Time) { - return getDayTime(t.Request.From), getDayTime(t.Request.Through) -} - -func (t Task) ChunkIterForDay(day time.Time) v1.Iterator[*logproto.GroupedChunkRefs] { - cf := filterGroupedChunkRefsByDay{day: day} - return &FilterIter[*logproto.GroupedChunkRefs]{ - iter: v1.NewSliceIter(t.Request.Refs), - matches: cf.contains, - transform: cf.filter, - } -} - -type filterGroupedChunkRefsByDay struct { - day time.Time -} - -func (cf filterGroupedChunkRefsByDay) contains(a *logproto.GroupedChunkRefs) bool { - from, through := getFromThrough(a.Refs) - if from.Time().After(cf.day.Add(Day)) || through.Time().Before(cf.day) { - return false - } - return true -} - -func (cf filterGroupedChunkRefsByDay) filter(a *logproto.GroupedChunkRefs) *logproto.GroupedChunkRefs { - minTs, maxTs := getFromThrough(a.Refs) - - // in most cases, all chunks are within day range - if minTs.Time().Compare(cf.day) >= 0 && maxTs.Time().Before(cf.day.Add(Day)) { - return a - } - - // case where certain chunks are outside of day range - // using binary search to get min and max index of chunks that fall into the day range - min := sort.Search(len(a.Refs), func(i int) bool { - start := a.Refs[i].From.Time() - end := a.Refs[i].Through.Time() - return start.Compare(cf.day) >= 0 || end.Compare(cf.day) >= 0 - }) - - max := sort.Search(len(a.Refs), func(i int) bool { - start := a.Refs[i].From.Time() - return start.Compare(cf.day.Add(Day)) > 0 - }) - - return &logproto.GroupedChunkRefs{ - Tenant: a.Tenant, - Fingerprint: a.Fingerprint, - Refs: a.Refs[min:max], - } + return task, nil } -type Predicate[T any] func(a T) bool -type Transform[T any] func(a T) T - -type FilterIter[T any] struct { - iter v1.Iterator[T] - matches Predicate[T] - transform Transform[T] - cache T - zero T // zero value of the return type of Next() +func (t Task) Bounds() (model.Time, model.Time) { + return t.bounds.Start, t.bounds.End } -func (it *FilterIter[T]) Next() bool { - next := it.iter.Next() - if !next { - it.cache = it.zero - return false - } - for next && !it.matches(it.iter.At()) { - next = it.iter.Next() - if !next { - it.cache = it.zero - return false - } +// Copy returns a copy of the existing task but with a new slice of grouped chunk refs +func (t Task) Copy(series []*logproto.GroupedChunkRefs) Task { + return Task{ + ID: ulid.ULID{}, // create emty ID to distinguish it as copied task + Tenant: t.Tenant, + ErrCh: t.ErrCh, + ResCh: t.ResCh, + filters: t.filters, + series: series, + bounds: t.bounds, + day: t.day, } - it.cache = it.transform(it.iter.At()) - return true -} - -func (it *FilterIter[T]) At() T { - return it.cache -} - -func (it *FilterIter[T]) Err() error { - return nil } // taskMergeIterator implements v1.Iterator @@ -164,12 +89,12 @@ type taskMergeIterator struct { curr v1.Request heap *v1.HeapIterator[v1.IndexedValue[*logproto.GroupedChunkRefs]] tasks []Task - day time.Time + day model.Time tokenizer *v1.NGramTokenizer err error } -func newTaskMergeIterator(day time.Time, tokenizer *v1.NGramTokenizer, tasks ...Task) v1.PeekingIterator[v1.Request] { +func newTaskMergeIterator(day model.Time, tokenizer *v1.NGramTokenizer, tasks ...Task) v1.PeekingIterator[v1.Request] { it := &taskMergeIterator{ tasks: tasks, curr: v1.Request{}, @@ -183,8 +108,8 @@ func newTaskMergeIterator(day time.Time, tokenizer *v1.NGramTokenizer, tasks ... func (it *taskMergeIterator) init() { sequences := make([]v1.PeekingIterator[v1.IndexedValue[*logproto.GroupedChunkRefs]], 0, len(it.tasks)) for i := range it.tasks { - iter := v1.NewIterWithIndex(it.tasks[i].ChunkIterForDay(it.day), i) - sequences = append(sequences, v1.NewPeekingIter(iter)) + iter := v1.NewSliceIterWithIndex(it.tasks[i].series, i) + sequences = append(sequences, iter) } it.heap = v1.NewHeapIterator( func(i, j v1.IndexedValue[*logproto.GroupedChunkRefs]) bool { @@ -207,7 +132,7 @@ func (it *taskMergeIterator) Next() bool { it.curr = v1.Request{ Fp: model.Fingerprint(group.Value().Fingerprint), Chks: convertToChunkRefs(group.Value().Refs), - Searches: convertToSearches(task.Request.Filters, it.tokenizer), + Searches: convertToSearches(task.filters, it.tokenizer), Response: task.ResCh, } return true diff --git a/pkg/bloomgateway/multiplexing_test.go b/pkg/bloomgateway/multiplexing_test.go index d414256f95268..67277d60f232c 100644 --- a/pkg/bloomgateway/multiplexing_test.go +++ b/pkg/bloomgateway/multiplexing_test.go @@ -5,62 +5,59 @@ import ( "time" "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/logproto" - "github.com/grafana/loki/pkg/logql/syntax" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" ) func TestTask(t *testing.T) { - t.Run("bounds returns request boundaries", func(t *testing.T) { - ts := model.Now() + ts := mktime("2024-01-24 12:00") + t.Run("bounds returns boundaries of chunks", func(t *testing.T) { req := &logproto.FilterChunkRefRequest{ - From: ts.Add(-1 * time.Hour), + From: ts.Add(-24 * time.Hour), Through: ts, + Refs: []*logproto.GroupedChunkRefs{ + { + Fingerprint: 0x00, + Refs: []*logproto.ShortRef{ + {From: ts.Add(-1 * time.Hour), Through: ts}, + }, + }, + }, } - task, _, _, err := NewTask("tenant", req) + swb := partitionRequest(req)[0] + task, err := NewTask("tenant", swb, nil) require.NoError(t, err) from, through := task.Bounds() - require.Equal(t, getDayTime(req.From), from) - require.Equal(t, getDayTime(req.Through), through) + require.Equal(t, ts.Add(-1*time.Hour), from) + require.Equal(t, ts, through) + require.Equal(t, truncateDay(ts), task.day) }) } +func createTasksForRequests(t *testing.T, tenant string, requests ...*logproto.FilterChunkRefRequest) []Task { + t.Helper() + + tasks := make([]Task, 0, len(requests)) + for _, r := range requests { + for _, swb := range partitionRequest(r) { + task, err := NewTask(tenant, swb, nil) + require.NoError(t, err) + tasks = append(tasks, task) + } + } + return tasks +} + func TestTaskMergeIterator(t *testing.T) { - // Thu Nov 09 2023 10:56:50 UTC - ts := model.TimeFromUnix(1699523810) - day := getDayTime(ts) + ts := mktime("2024-01-24 12:00") + day := truncateDay(ts) tenant := "fake" tokenizer := v1.NewNGramTokenizer(4, 0) t.Run("empty requests result in empty iterator", func(t *testing.T) { - r1 := &logproto.FilterChunkRefRequest{ - From: ts.Add(-3 * time.Hour), - Through: ts.Add(-2 * time.Hour), - Refs: []*logproto.GroupedChunkRefs{}, - } - t1, _, _, err := NewTask(tenant, r1) - require.NoError(t, err) - - r2 := &logproto.FilterChunkRefRequest{ - From: ts.Add(-1 * time.Hour), - Through: ts, - Refs: []*logproto.GroupedChunkRefs{}, - } - t2, _, _, err := NewTask(tenant, r2) - require.NoError(t, err) - - r3 := &logproto.FilterChunkRefRequest{ - From: ts.Add(-1 * time.Hour), - Through: ts, - Refs: []*logproto.GroupedChunkRefs{}, - } - t3, _, _, err := NewTask(tenant, r3) - require.NoError(t, err) - - it := newTaskMergeIterator(day, tokenizer, t1, t2, t3) + it := newTaskMergeIterator(day, tokenizer) // nothing to iterate over require.False(t, it.Next()) }) @@ -75,8 +72,6 @@ func TestTaskMergeIterator(t *testing.T) { }}, }, } - t1, _, _, err := NewTask(tenant, r1) - require.NoError(t, err) r2 := &logproto.FilterChunkRefRequest{ From: ts.Add(-1 * time.Hour), @@ -90,8 +85,6 @@ func TestTaskMergeIterator(t *testing.T) { }}, }, } - t2, _, _, err := NewTask(tenant, r2) - require.NoError(t, err) r3 := &logproto.FilterChunkRefRequest{ From: ts.Add(-1 * time.Hour), @@ -102,10 +95,9 @@ func TestTaskMergeIterator(t *testing.T) { }}, }, } - t3, _, _, err := NewTask(tenant, r3) - require.NoError(t, err) - it := newTaskMergeIterator(day, tokenizer, t1, t2, t3) + tasks := createTasksForRequests(t, tenant, r1, r2, r3) + it := newTaskMergeIterator(day, tokenizer, tasks...) // first item require.True(t, it.Next()) @@ -135,73 +127,3 @@ func TestTaskMergeIterator(t *testing.T) { require.False(t, it.Next()) }) } - -func TestChunkIterForDay(t *testing.T) { - tenant := "fake" - - // Thu Nov 09 2023 10:56:50 UTC - ts := model.TimeFromUnix(1699523810) - - t.Run("filter chunk refs that fall into the day range", func(t *testing.T) { - input := &logproto.FilterChunkRefRequest{ - From: ts.Add(-168 * time.Hour), // 1w ago - Through: ts, - Refs: []*logproto.GroupedChunkRefs{ - {Fingerprint: 100, Tenant: tenant, Refs: []*logproto.ShortRef{ - {From: ts.Add(-168 * time.Hour), Through: ts.Add(-167 * time.Hour), Checksum: 100}, - {From: ts.Add(-143 * time.Hour), Through: ts.Add(-142 * time.Hour), Checksum: 101}, - }}, - {Fingerprint: 200, Tenant: tenant, Refs: []*logproto.ShortRef{ - {From: ts.Add(-144 * time.Hour), Through: ts.Add(-143 * time.Hour), Checksum: 200}, - {From: ts.Add(-119 * time.Hour), Through: ts.Add(-118 * time.Hour), Checksum: 201}, - }}, - {Fingerprint: 300, Tenant: tenant, Refs: []*logproto.ShortRef{ - {From: ts.Add(-120 * time.Hour), Through: ts.Add(-119 * time.Hour), Checksum: 300}, - {From: ts.Add(-95 * time.Hour), Through: ts.Add(-94 * time.Hour), Checksum: 301}, - }}, - {Fingerprint: 400, Tenant: tenant, Refs: []*logproto.ShortRef{ - {From: ts.Add(-96 * time.Hour), Through: ts.Add(-95 * time.Hour), Checksum: 400}, - {From: ts.Add(-71 * time.Hour), Through: ts.Add(-70 * time.Hour), Checksum: 401}, - }}, - {Fingerprint: 500, Tenant: tenant, Refs: []*logproto.ShortRef{ - {From: ts.Add(-72 * time.Hour), Through: ts.Add(-71 * time.Hour), Checksum: 500}, - {From: ts.Add(-47 * time.Hour), Through: ts.Add(-46 * time.Hour), Checksum: 501}, - }}, - {Fingerprint: 600, Tenant: tenant, Refs: []*logproto.ShortRef{ - {From: ts.Add(-48 * time.Hour), Through: ts.Add(-47 * time.Hour), Checksum: 600}, - {From: ts.Add(-23 * time.Hour), Through: ts.Add(-22 * time.Hour), Checksum: 601}, - }}, - {Fingerprint: 700, Tenant: tenant, Refs: []*logproto.ShortRef{ - {From: ts.Add(-24 * time.Hour), Through: ts.Add(-23 * time.Hour), Checksum: 700}, - {From: ts.Add(-1 * time.Hour), Through: ts, Checksum: 701}, - }}, - }, - Filters: []syntax.LineFilter{ - {Ty: labels.MatchEqual, Match: "foo"}, - {Ty: labels.MatchEqual, Match: "bar"}, - }, - } - - // day ranges from ts-48h to ts-24h - day := getDayTime(ts.Add(-36 * time.Hour)) - - expected := []*logproto.GroupedChunkRefs{ - {Fingerprint: 500, Tenant: tenant, Refs: []*logproto.ShortRef{ - {From: ts.Add(-47 * time.Hour), Through: ts.Add(-46 * time.Hour), Checksum: 501}, - }}, - {Fingerprint: 600, Tenant: tenant, Refs: []*logproto.ShortRef{ - {From: ts.Add(-48 * time.Hour), Through: ts.Add(-47 * time.Hour), Checksum: 600}, - }}, - } - - task, _, _, _ := NewTask(tenant, input) - it := task.ChunkIterForDay(day) - - output := make([]*logproto.GroupedChunkRefs, 0, len(input.Refs)) - for it.Next() { - output = append(output, it.At()) - } - - require.Equal(t, expected, output) - }) -} diff --git a/pkg/bloomgateway/util.go b/pkg/bloomgateway/util.go index 89d238864ac45..cf72aec3b5b4b 100644 --- a/pkg/bloomgateway/util.go +++ b/pkg/bloomgateway/util.go @@ -15,7 +15,12 @@ import ( ) func getDayTime(ts model.Time) time.Time { - return time.Date(ts.Time().Year(), ts.Time().Month(), ts.Time().Day(), 0, 0, 0, 0, time.UTC) + return ts.Time().UTC().Truncate(Day) +} + +func truncateDay(ts model.Time) model.Time { + // model.minimumTick is time.Millisecond + return ts - (ts % model.Time(24*time.Hour/time.Millisecond)) } // getFromThrough assumes a list of ShortRefs sorted by From time @@ -24,6 +29,10 @@ func getFromThrough(refs []*logproto.ShortRef) (model.Time, model.Time) { return model.Earliest, model.Latest } + if len(refs) == 1 { + return refs[0].From, refs[0].Through + } + maxItem := slices.MaxFunc(refs, func(a, b *logproto.ShortRef) int { if a.Through > b.Through { return 1 @@ -85,7 +94,7 @@ func partitionFingerprintRange(tasks []Task, blocks []bloomshipper.BlockRef) (re } for _, task := range tasks { - refs := task.Request.Refs + refs := task.series min := sort.Search(len(refs), func(i int) bool { return block.Cmp(refs[i].Fingerprint) > v1.Before }) @@ -109,3 +118,66 @@ func partitionFingerprintRange(tasks []Task, blocks []bloomshipper.BlockRef) (re } return result } + +type seriesWithBounds struct { + bounds model.Interval + day model.Time + series []*logproto.GroupedChunkRefs +} + +func partitionRequest(req *logproto.FilterChunkRefRequest) []seriesWithBounds { + result := make([]seriesWithBounds, 0) + + fromDay, throughDay := truncateDay(req.From), truncateDay(req.Through) + + for day := fromDay; day.Equal(throughDay) || day.Before(throughDay); day = day.Add(Day) { + minTs, maxTs := model.Latest, model.Earliest + nextDay := day.Add(Day) + res := make([]*logproto.GroupedChunkRefs, 0, len(req.Refs)) + + for _, series := range req.Refs { + chunks := series.Refs + + min := sort.Search(len(chunks), func(i int) bool { + return chunks[i].Through >= day + }) + + max := sort.Search(len(chunks), func(i int) bool { + return chunks[i].From >= nextDay + }) + + // All chunks fall outside of the range + if min == len(chunks) || max == 0 { + continue + } + + if chunks[min].From < minTs { + minTs = chunks[min].From + } + if chunks[max-1].Through > maxTs { + maxTs = chunks[max-1].Through + } + // fmt.Println("day", day, "series", series.Fingerprint, "minTs", minTs, "maxTs", maxTs) + + res = append(res, &logproto.GroupedChunkRefs{ + Fingerprint: series.Fingerprint, + Tenant: series.Tenant, + Refs: chunks[min:max], + }) + + } + + if len(res) > 0 { + result = append(result, seriesWithBounds{ + bounds: model.Interval{ + Start: minTs, + End: maxTs, + }, + day: day, + series: res, + }) + } + } + + return result +} diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index 152309ddbde07..81f0720b9bf80 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -2,6 +2,7 @@ package bloomgateway import ( "testing" + "time" "github.com/prometheus/common/model" "github.com/stretchr/testify/require" @@ -27,6 +28,23 @@ func TestGetFromThrough(t *testing.T) { require.Equal(t, model.Time(4), chunks[len(chunks)-1].From) } +func TestTruncateDay(t *testing.T) { + expected := mktime("2024-01-24 00:00") + + for _, inp := range []string{ + "2024-01-24 00:00", + "2024-01-24 08:00", + "2024-01-24 16:00", + "2024-01-24 23:59", + } { + t.Run(inp, func(t *testing.T) { + ts := mktime(inp) + result := truncateDay(ts) + require.Equal(t, expected, result) + }) + } +} + func mkBlockRef(minFp, maxFp uint64) bloomshipper.BlockRef { return bloomshipper.BlockRef{ Ref: bloomshipper.Ref{ @@ -37,6 +55,7 @@ func mkBlockRef(minFp, maxFp uint64) bloomshipper.BlockRef { } func TestPartitionFingerprintRange(t *testing.T) { + t.Run("consecutive block ranges", func(t *testing.T) { bounds := []bloomshipper.BlockRef{ mkBlockRef(0, 99), // out of bounds block @@ -52,10 +71,7 @@ func TestPartitionFingerprintRange(t *testing.T) { tasks := make([]Task, nTasks) for i := startFp; i < startFp+nSeries; i++ { - if tasks[i%nTasks].Request == nil { - tasks[i%nTasks].Request = &logproto.FilterChunkRefRequest{} - } - tasks[i%nTasks].Request.Refs = append(tasks[i%nTasks].Request.Refs, &logproto.GroupedChunkRefs{Fingerprint: uint64(i)}) + tasks[i%nTasks].series = append(tasks[i%nTasks].series, &logproto.GroupedChunkRefs{Fingerprint: uint64(i)}) } results := partitionFingerprintRange(tasks, bounds) @@ -67,8 +83,8 @@ func TestPartitionFingerprintRange(t *testing.T) { // ensure we have the right number of tasks per bound require.Len(t, res.tasks, 5) for _, task := range res.tasks { - require.Equal(t, expectedTaskRefs[i], len(task.Request.Refs)) - actualFingerprints = append(actualFingerprints, task.Request.Refs...) + require.Equal(t, expectedTaskRefs[i], len(task.series)) + actualFingerprints = append(actualFingerprints, task.series...) } } @@ -88,9 +104,9 @@ func TestPartitionFingerprintRange(t *testing.T) { mkBlockRef(200, 289), } - task := Task{Request: &logproto.FilterChunkRefRequest{}} + task := Task{} for i := 0; i < 300; i++ { - task.Request.Refs = append(task.Request.Refs, &logproto.GroupedChunkRefs{Fingerprint: uint64(i)}) + task.series = append(task.series, &logproto.GroupedChunkRefs{Fingerprint: uint64(i)}) } results := partitionFingerprintRange([]Task{task}, bounds) @@ -98,7 +114,162 @@ func TestPartitionFingerprintRange(t *testing.T) { for _, res := range results { // ensure we have the right number of tasks per bound require.Len(t, res.tasks, 1) - require.Len(t, res.tasks[0].Request.Refs, 90) + require.Len(t, res.tasks[0].series, 90) } }) } + +func TestPartitionRequest(t *testing.T) { + ts := mktime("2024-01-24 12:00") + + testCases := map[string]struct { + inp *logproto.FilterChunkRefRequest + exp []seriesWithBounds + }{ + + "empty": { + inp: &logproto.FilterChunkRefRequest{ + From: ts.Add(-24 * time.Hour), + Through: ts, + }, + exp: []seriesWithBounds{}, + }, + + "all chunks within single day": { + inp: &logproto.FilterChunkRefRequest{ + From: ts.Add(-1 * time.Hour), + Through: ts, + Refs: []*logproto.GroupedChunkRefs{ + { + Fingerprint: 0x00, + Refs: []*logproto.ShortRef{ + {From: ts.Add(-60 * time.Minute), Through: ts.Add(-50 * time.Minute)}, + }, + }, + { + Fingerprint: 0x01, + Refs: []*logproto.ShortRef{ + {From: ts.Add(-55 * time.Minute), Through: ts.Add(-45 * time.Minute)}, + }, + }, + }, + }, + exp: []seriesWithBounds{ + { + bounds: model.Interval{Start: ts.Add(-60 * time.Minute), End: ts.Add(-45 * time.Minute)}, + day: mktime("2024-01-24 00:00"), + series: []*logproto.GroupedChunkRefs{ + { + Fingerprint: 0x00, + Refs: []*logproto.ShortRef{ + {From: ts.Add(-60 * time.Minute), Through: ts.Add(-50 * time.Minute)}, + }, + }, + { + Fingerprint: 0x01, + Refs: []*logproto.ShortRef{ + {From: ts.Add(-55 * time.Minute), Through: ts.Add(-45 * time.Minute)}, + }, + }, + }, + }, + }, + }, + + "chunks across multiple days - no overlap": { + inp: &logproto.FilterChunkRefRequest{ + From: ts.Add(-24 * time.Hour), + Through: ts, + Refs: []*logproto.GroupedChunkRefs{ + { + Fingerprint: 0x00, + Refs: []*logproto.ShortRef{ + {From: ts.Add(-23 * time.Hour), Through: ts.Add(-22 * time.Hour)}, + }, + }, + { + Fingerprint: 0x01, + Refs: []*logproto.ShortRef{ + {From: ts.Add(-2 * time.Hour), Through: ts.Add(-1 * time.Hour)}, + }, + }, + }, + }, + exp: []seriesWithBounds{ + { + bounds: model.Interval{Start: ts.Add(-23 * time.Hour), End: ts.Add(-22 * time.Hour)}, + day: mktime("2024-01-23 00:00"), + series: []*logproto.GroupedChunkRefs{ + { + Fingerprint: 0x00, + Refs: []*logproto.ShortRef{ + {From: ts.Add(-23 * time.Hour), Through: ts.Add(-22 * time.Hour)}, + }, + }, + }, + }, + { + bounds: model.Interval{Start: ts.Add(-2 * time.Hour), End: ts.Add(-1 * time.Hour)}, + day: mktime("2024-01-24 00:00"), + series: []*logproto.GroupedChunkRefs{ + { + Fingerprint: 0x01, + Refs: []*logproto.ShortRef{ + {From: ts.Add(-2 * time.Hour), Through: ts.Add(-1 * time.Hour)}, + }, + }, + }, + }, + }, + }, + + "chunks across multiple days - overlap": { + inp: &logproto.FilterChunkRefRequest{ + From: ts.Add(-24 * time.Hour), + Through: ts, + Refs: []*logproto.GroupedChunkRefs{ + { + Fingerprint: 0x00, + Refs: []*logproto.ShortRef{ + {From: ts.Add(-13 * time.Hour), Through: ts.Add(-11 * time.Hour)}, + }, + }, + }, + }, + exp: []seriesWithBounds{ + { + bounds: model.Interval{Start: ts.Add(-13 * time.Hour), End: ts.Add(-11 * time.Hour)}, + day: mktime("2024-01-23 00:00"), + series: []*logproto.GroupedChunkRefs{ + { + Fingerprint: 0x00, + Refs: []*logproto.ShortRef{ + {From: ts.Add(-13 * time.Hour), Through: ts.Add(-11 * time.Hour)}, + }, + }, + }, + }, + { + bounds: model.Interval{Start: ts.Add(-13 * time.Hour), End: ts.Add(-11 * time.Hour)}, + day: mktime("2024-01-24 00:00"), + series: []*logproto.GroupedChunkRefs{ + { + Fingerprint: 0x00, + Refs: []*logproto.ShortRef{ + {From: ts.Add(-13 * time.Hour), Through: ts.Add(-11 * time.Hour)}, + }, + }, + }, + }, + }, + }, + } + + for name, tc := range testCases { + t.Run(name, func(t *testing.T) { + result := partitionRequest(tc.inp) + require.Equal(t, tc.exp, result) + }) + } + +} diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go index 73100025a7431..e2146ed9fae72 100644 --- a/pkg/bloomgateway/worker.go +++ b/pkg/bloomgateway/worker.go @@ -79,18 +79,18 @@ type worker struct { cfg workerConfig queue *queue.RequestQueue shipper bloomshipper.Interface - tasks *pendingTasks + pending *pendingTasks logger log.Logger metrics *workerMetrics } -func newWorker(id string, cfg workerConfig, queue *queue.RequestQueue, shipper bloomshipper.Interface, tasks *pendingTasks, logger log.Logger, metrics *workerMetrics) *worker { +func newWorker(id string, cfg workerConfig, queue *queue.RequestQueue, shipper bloomshipper.Interface, pending *pendingTasks, logger log.Logger, metrics *workerMetrics) *worker { w := &worker{ id: id, cfg: cfg, queue: queue, shipper: shipper, - tasks: tasks, + pending: pending, logger: log.With(logger, "worker", id), metrics: metrics, } @@ -134,7 +134,7 @@ func (w *worker) running(ctx context.Context) error { } w.metrics.dequeuedTasks.WithLabelValues(w.id).Add(float64(len(items))) - tasksPerDay := make(map[time.Time][]Task) + tasksPerDay := make(map[model.Time][]Task) for _, item := range items { task, ok := item.(Task) @@ -144,17 +144,9 @@ func (w *worker) running(ctx context.Context) error { return errors.Errorf("failed to cast dequeued item to Task: %v", item) } level.Debug(w.logger).Log("msg", "dequeued task", "task", task.ID) - w.tasks.Delete(task.ID) + w.pending.Delete(task.ID) - fromDay, throughDay := task.Bounds() - - if fromDay.Equal(throughDay) { - tasksPerDay[fromDay] = append(tasksPerDay[fromDay], task) - } else { - for i := fromDay; i.Before(throughDay); i = i.Add(24 * time.Hour) { - tasksPerDay[i] = append(tasksPerDay[i], task) - } - } + tasksPerDay[task.day] = append(tasksPerDay[task.day], task) } for day, tasks := range tasksPerDay { @@ -162,7 +154,7 @@ func (w *worker) running(ctx context.Context) error { level.Debug(logger).Log("msg", "process tasks", "tasks", len(tasks)) storeFetchStart := time.Now() - blockRefs, err := w.shipper.GetBlockRefs(taskCtx, tasks[0].Tenant, toModelTime(day), toModelTime(day.Add(Day).Add(-1*time.Nanosecond))) + blockRefs, err := w.shipper.GetBlockRefs(taskCtx, tasks[0].Tenant, day, day.Add(Day).Add(-1*time.Nanosecond)) w.metrics.storeAccessLatency.WithLabelValues(w.id, "GetBlockRefs").Observe(time.Since(storeFetchStart).Seconds()) if err != nil { for _, t := range tasks { @@ -177,7 +169,7 @@ func (w *worker) running(ctx context.Context) error { if len(blockRefs) == 0 { level.Warn(logger).Log("msg", "no blocks found") for _, t := range tasks { - for _, ref := range t.Request.Refs { + for _, ref := range t.series { t.ResCh <- v1.Output{ Fp: model.Fingerprint(ref.Fingerprint), Removals: nil, @@ -188,13 +180,13 @@ func (w *worker) running(ctx context.Context) error { continue } - boundedRefs := partitionFingerprintRange(tasks, blockRefs) + tasksForBlocks := partitionFingerprintRange(tasks, blockRefs) blockRefs = blockRefs[:0] - for _, b := range boundedRefs { + for _, b := range tasksForBlocks { blockRefs = append(blockRefs, b.blockRef) } - err = w.processBlocksWithCallback(taskCtx, tasks[0].Tenant, day, blockRefs, boundedRefs) + err = w.processBlocksWithCallback(taskCtx, tasks[0].Tenant, day, blockRefs, tasksForBlocks) if err != nil { for _, t := range tasks { t.ErrCh <- err @@ -217,7 +209,7 @@ func (w *worker) stopping(err error) error { return nil } -func (w *worker) processBlocksWithCallback(taskCtx context.Context, tenant string, day time.Time, blockRefs []bloomshipper.BlockRef, boundedRefs []boundedTasks) error { +func (w *worker) processBlocksWithCallback(taskCtx context.Context, tenant string, day model.Time, blockRefs []bloomshipper.BlockRef, boundedRefs []boundedTasks) error { return w.shipper.Fetch(taskCtx, tenant, blockRefs, func(bq *v1.BlockQuerier, minFp, maxFp uint64) error { for _, b := range boundedRefs { if b.blockRef.MinFingerprint == minFp && b.blockRef.MaxFingerprint == maxFp { @@ -228,7 +220,7 @@ func (w *worker) processBlocksWithCallback(taskCtx context.Context, tenant strin }) } -func (w *worker) processBlock(blockQuerier *v1.BlockQuerier, day time.Time, tasks []Task) error { +func (w *worker) processBlock(blockQuerier *v1.BlockQuerier, day model.Time, tasks []Task) error { schema, err := blockQuerier.Schema() if err != nil { return err From 2a63d2472945cf9fcd8588eda3b82123b3bf317f Mon Sep 17 00:00:00 2001 From: Michel Hollands <42814411+MichelHollands@users.noreply.github.com> Date: Thu, 25 Jan 2024 15:13:35 +0100 Subject: [PATCH 09/23] Fix the ObjectExists method on FSObjectClient (#11747) **What this PR does / why we need it**: This method wasn't working. A test was added as well. **Special notes for your reviewer**: **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [X] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) Signed-off-by: Michel Hollands --- pkg/storage/chunk/client/local/fs_object_client.go | 3 ++- pkg/storage/chunk/client/local/fs_object_client_test.go | 6 +++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/pkg/storage/chunk/client/local/fs_object_client.go b/pkg/storage/chunk/client/local/fs_object_client.go index 0de470a42d565..deee987a27abe 100644 --- a/pkg/storage/chunk/client/local/fs_object_client.go +++ b/pkg/storage/chunk/client/local/fs_object_client.go @@ -66,7 +66,8 @@ func NewFSObjectClient(cfg FSConfig) (*FSObjectClient, error) { func (FSObjectClient) Stop() {} func (f *FSObjectClient) ObjectExists(_ context.Context, objectKey string) (bool, error) { - _, err := os.Lstat(objectKey) + fullPath := filepath.Join(f.cfg.Directory, filepath.FromSlash(objectKey)) + _, err := os.Lstat(fullPath) if err != nil { return false, err } diff --git a/pkg/storage/chunk/client/local/fs_object_client_test.go b/pkg/storage/chunk/client/local/fs_object_client_test.go index 1f8c976f82f34..09bf69530befb 100644 --- a/pkg/storage/chunk/client/local/fs_object_client_test.go +++ b/pkg/storage/chunk/client/local/fs_object_client_test.go @@ -57,7 +57,7 @@ func TestFSObjectClient_DeleteChunksBefore(t *testing.T) { require.Equal(t, 1, len(files), "Number of files should be 1 after enforcing retention") } -func TestFSObjectClient_List(t *testing.T) { +func TestFSObjectClient_List_and_ObjectExists(t *testing.T) { fsObjectsDir := t.TempDir() bucketClient, err := NewFSObjectClient(FSConfig{ @@ -152,6 +152,10 @@ func TestFSObjectClient_List(t *testing.T) { require.Len(t, storageObjects, 1) require.Equal(t, "outer-file1", storageObjects[0].Key) require.Empty(t, commonPrefixes) + + ok, err := bucketClient.ObjectExists(context.Background(), "outer-file2") + require.NoError(t, err) + require.True(t, ok) } func TestFSObjectClient_DeleteObject(t *testing.T) { From 9f3fd24958ea08fbbbf76a29f955517da4e0b20e Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Thu, 25 Jan 2024 17:44:41 +0100 Subject: [PATCH 10/23] Bloom Shipper: Fix bug in block boundary check (#11784) **What this PR does / why we need it**: The check `isOutsideRange()` returned incorrect results, because the end timestamp of the search interval is not inclusive. --------- Signed-off-by: Christian Haudum --- pkg/bloomgateway/bloomgateway_test.go | 2 +- pkg/bloomgateway/worker.go | 8 +- .../stores/shipper/bloomshipper/shipper.go | 86 ++++++++++++------- .../shipper/bloomshipper/shipper_test.go | 48 ++++++++--- 4 files changed, 98 insertions(+), 46 deletions(-) diff --git a/pkg/bloomgateway/bloomgateway_test.go b/pkg/bloomgateway/bloomgateway_test.go index e6d2b910c12f3..34dede40727c7 100644 --- a/pkg/bloomgateway/bloomgateway_test.go +++ b/pkg/bloomgateway/bloomgateway_test.go @@ -433,7 +433,7 @@ type mockBloomStore struct { var _ bloomshipper.Interface = &mockBloomStore{} // GetBlockRefs implements bloomshipper.Interface -func (s *mockBloomStore) GetBlockRefs(_ context.Context, tenant string, _, _ model.Time) ([]bloomshipper.BlockRef, error) { +func (s *mockBloomStore) GetBlockRefs(_ context.Context, tenant string, _ bloomshipper.Interval) ([]bloomshipper.BlockRef, error) { blocks := make([]bloomshipper.BlockRef, 0, len(s.bqs)) for i := range s.bqs { blocks = append(blocks, bloomshipper.BlockRef{ diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go index e2146ed9fae72..69f7859e64818 100644 --- a/pkg/bloomgateway/worker.go +++ b/pkg/bloomgateway/worker.go @@ -150,11 +150,17 @@ func (w *worker) running(ctx context.Context) error { } for day, tasks := range tasksPerDay { + // interval is [Start, End) + interval := bloomshipper.Interval{ + Start: day, // inclusive + End: day.Add(Day), // non-inclusive + } + logger := log.With(w.logger, "day", day) level.Debug(logger).Log("msg", "process tasks", "tasks", len(tasks)) storeFetchStart := time.Now() - blockRefs, err := w.shipper.GetBlockRefs(taskCtx, tasks[0].Tenant, day, day.Add(Day).Add(-1*time.Nanosecond)) + blockRefs, err := w.shipper.GetBlockRefs(taskCtx, tasks[0].Tenant, interval) w.metrics.storeAccessLatency.WithLabelValues(w.id, "GetBlockRefs").Observe(time.Since(storeFetchStart).Seconds()) if err != nil { for _, t := range tasks { diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index 36bfba913c98c..abcdcf370f813 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -15,6 +15,24 @@ import ( "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config" ) +type Interval struct { + Start model.Time + End model.Time +} + +func (i Interval) String() string { + return fmt.Sprintf("[%s, %s)", i.Start.Time(), i.End.Time()) +} + +func (i Interval) Cmp(other model.Time) v1.BoundsCheck { + if other.Before(i.Start) { + return v1.Before + } else if other.After(i.End) || other.Equal(i.End) { + return v1.After + } + return v1.Overlap +} + type fpRange [2]uint64 func (r fpRange) minFp() uint64 { @@ -25,6 +43,15 @@ func (r fpRange) maxFp() uint64 { return r[1] } +func (r fpRange) Cmp(other uint64) v1.BoundsCheck { + if other < r[0] { + return v1.Before + } else if other > r[1] { + return v1.After + } + return v1.Overlap +} + type BlockQuerierWithFingerprintRange struct { *v1.BlockQuerier MinFp, MaxFp model.Fingerprint @@ -33,7 +60,7 @@ type BlockQuerierWithFingerprintRange struct { type ForEachBlockCallback func(bq *v1.BlockQuerier, minFp, maxFp uint64) error type Interface interface { - GetBlockRefs(ctx context.Context, tenant string, from, through model.Time) ([]BlockRef, error) + GetBlockRefs(ctx context.Context, tenant string, interval Interval) ([]BlockRef, error) Fetch(ctx context.Context, tenant string, blocks []BlockRef, callback ForEachBlockCallback) error Stop() } @@ -63,10 +90,12 @@ func NewShipper(client Client, config config.Config, limits Limits, logger log.L }, nil } -func (s *Shipper) GetBlockRefs(ctx context.Context, tenantID string, from, through model.Time) ([]BlockRef, error) { - level.Debug(s.logger).Log("msg", "GetBlockRefs", "tenant", tenantID, "from", from, "through", through) +func (s *Shipper) GetBlockRefs(ctx context.Context, tenantID string, interval Interval) ([]BlockRef, error) { + level.Debug(s.logger).Log("msg", "GetBlockRefs", "tenant", tenantID, "[", interval.Start, "", interval.End) - blockRefs, err := s.getActiveBlockRefs(ctx, tenantID, from, through, []fpRange{{0, math.MaxUint64}}) + // TODO(chaudum): The bloom gateway should not fetch blocks for the complete key space + keyspaces := []fpRange{{0, math.MaxUint64}} + blockRefs, err := s.getActiveBlockRefs(ctx, tenantID, interval, keyspaces) if err != nil { return nil, fmt.Errorf("error fetching active block references : %w", err) } @@ -130,20 +159,20 @@ func getFirstLast[T any](s []T) (T, T) { return s[0], s[len(s)-1] } -func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, from, through model.Time, fingerprints []fpRange) ([]BlockRef, error) { - minFpRange, maxFpRange := getFirstLast(fingerprints) +func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, interval Interval, keyspaces []fpRange) ([]BlockRef, error) { + minFpRange, maxFpRange := getFirstLast(keyspaces) metas, err := s.client.GetMetas(ctx, MetaSearchParams{ TenantID: tenantID, MinFingerprint: model.Fingerprint(minFpRange.minFp()), MaxFingerprint: model.Fingerprint(maxFpRange.maxFp()), - StartTimestamp: from, - EndTimestamp: through, + StartTimestamp: interval.Start, + EndTimestamp: interval.End, }) if err != nil { return []BlockRef{}, fmt.Errorf("error fetching meta.json files: %w", err) } level.Debug(s.logger).Log("msg", "dowloaded metas", "count", len(metas)) - activeBlocks := s.findBlocks(metas, from, through, fingerprints) + activeBlocks := s.findBlocks(metas, interval, keyspaces) slices.SortStableFunc(activeBlocks, func(a, b BlockRef) int { if a.MinFingerprint < b.MinFingerprint { return -1 @@ -157,20 +186,22 @@ func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, from, return activeBlocks, nil } -func (s *Shipper) findBlocks(metas []Meta, startTimestamp, endTimestamp model.Time, fingerprints []fpRange) []BlockRef { - outdatedBlocks := make(map[string]interface{}) +func (s *Shipper) findBlocks(metas []Meta, interval Interval, keyspaces []fpRange) []BlockRef { + tombstones := make(map[string]interface{}) for _, meta := range metas { for _, tombstone := range meta.Tombstones { - outdatedBlocks[tombstone.BlockPath] = nil + tombstones[tombstone.BlockPath] = nil } } blocksSet := make(map[string]BlockRef) for _, meta := range metas { for _, block := range meta.Blocks { - if _, contains := outdatedBlocks[block.BlockPath]; contains { + if _, contains := tombstones[block.BlockPath]; contains { + // skip tombstoned blocks continue } - if isOutsideRange(&block, startTimestamp, endTimestamp, fingerprints) { + if isOutsideRange(block, interval, keyspaces) { + // skip block that are outside of interval or keyspaces continue } blocksSet[block.BlockPath] = block @@ -186,26 +217,21 @@ func (s *Shipper) findBlocks(metas []Meta, startTimestamp, endTimestamp model.Ti // isOutsideRange tests if a given BlockRef b is outside of search boundaries // defined by min/max timestamp and min/max fingerprint. // Fingerprint ranges must be sorted in ascending order. -func isOutsideRange(b *BlockRef, startTimestamp, endTimestamp model.Time, fingerprints []fpRange) bool { - // First, check time range - if b.EndTimestamp < startTimestamp || b.StartTimestamp > endTimestamp { +func isOutsideRange(b BlockRef, interval Interval, keyspaces []fpRange) bool { + // check time interval + if interval.Cmp(b.EndTimestamp) == v1.Before || interval.Cmp(b.StartTimestamp) == v1.After { return true } - // Then, check if outside of min/max of fingerprint slice - minFpRange, maxFpRange := getFirstLast(fingerprints) - if b.MaxFingerprint < minFpRange.minFp() || b.MinFingerprint > maxFpRange.maxFp() { - return true - } - - prev := fpRange{0, 0} - for i := 0; i < len(fingerprints); i++ { - fpr := fingerprints[i] - if b.MinFingerprint > prev.maxFp() && b.MaxFingerprint < fpr.minFp() { - return true + // check fingerprint ranges + for _, keyspace := range keyspaces { + if keyspace.Cmp(b.MinFingerprint) == v1.Before && keyspace.Cmp(b.MaxFingerprint) == v1.After { + return false + } + if keyspace.Cmp(b.MinFingerprint) == v1.Overlap || keyspace.Cmp(b.MaxFingerprint) == v1.Overlap { + return false } - prev = fpr } - return false + return true } diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go index 859aa38c82a61..755813c3a8479 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go @@ -10,6 +10,10 @@ import ( "github.com/stretchr/testify/require" ) +func interval(start, end model.Time) Interval { + return Interval{Start: start, End: end} +} + func Test_Shipper_findBlocks(t *testing.T) { t.Run("expected block that are specified in tombstones to be filtered out", func(t *testing.T) { metas := []Meta{ @@ -33,15 +37,19 @@ func Test_Shipper_findBlocks(t *testing.T) { createMatchingBlockRef("block3"), }, Blocks: []BlockRef{ - createMatchingBlockRef("block2"), - createMatchingBlockRef("block4"), createMatchingBlockRef("block5"), }, }, } + ts := model.Now() + shipper := &Shipper{} - blocks := shipper.findBlocks(metas, model.Now().Add(-2*time.Hour), model.Now().Add(-1*time.Hour), []fpRange{{100, 200}}) + interval := Interval{ + Start: ts.Add(-2 * time.Hour), + End: ts.Add(-1 * time.Hour), + } + blocks := shipper.findBlocks(metas, interval, []fpRange{{100, 200}}) expectedBlockRefs := []BlockRef{ createMatchingBlockRef("block2"), @@ -95,7 +103,7 @@ func Test_Shipper_findBlocks(t *testing.T) { t.Run(name, func(t *testing.T) { shipper := &Shipper{} ref := createBlockRef("fake-block", data.minFingerprint, data.maxFingerprint, data.startTimestamp, data.endTimestamp) - blocks := shipper.findBlocks([]Meta{{Blocks: []BlockRef{ref}}}, 300, 400, []fpRange{{100, 200}}) + blocks := shipper.findBlocks([]Meta{{Blocks: []BlockRef{ref}}}, interval(300, 400), []fpRange{{100, 200}}) if data.filtered { require.Empty(t, blocks) return @@ -112,61 +120,73 @@ func TestIsOutsideRange(t *testing.T) { t.Run("is outside if startTs > through", func(t *testing.T) { b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(&b, 0, 900, []fpRange{}) + isOutside := isOutsideRange(b, interval(0, 900), []fpRange{}) + require.True(t, isOutside) + }) + + t.Run("is outside if startTs == through ", func(t *testing.T) { + b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) + isOutside := isOutsideRange(b, interval(900, 1000), []fpRange{}) require.True(t, isOutside) }) t.Run("is outside if endTs < from", func(t *testing.T) { b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(&b, 2100, 3000, []fpRange{}) + isOutside := isOutsideRange(b, interval(2100, 3000), []fpRange{}) require.True(t, isOutside) }) t.Run("is outside if endFp < first fingerprint", func(t *testing.T) { b := createBlockRef("block", 0, 90, startTs, endTs) - isOutside := isOutsideRange(&b, startTs, endTs, []fpRange{{100, 199}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{100, 199}}) require.True(t, isOutside) }) t.Run("is outside if startFp > last fingerprint", func(t *testing.T) { b := createBlockRef("block", 200, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(&b, startTs, endTs, []fpRange{{0, 49}, {100, 149}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{0, 49}, {100, 149}}) require.True(t, isOutside) }) t.Run("is outside if within gaps in fingerprints", func(t *testing.T) { b := createBlockRef("block", 100, 199, startTs, endTs) - isOutside := isOutsideRange(&b, startTs, endTs, []fpRange{{0, 99}, {200, 299}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{0, 99}, {200, 299}}) require.True(t, isOutside) }) t.Run("is not outside if within fingerprints 1", func(t *testing.T) { b := createBlockRef("block", 10, 90, startTs, endTs) - isOutside := isOutsideRange(&b, startTs, endTs, []fpRange{{0, 99}, {200, 299}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{0, 99}, {200, 299}}) require.False(t, isOutside) }) t.Run("is not outside if within fingerprints 2", func(t *testing.T) { b := createBlockRef("block", 210, 290, startTs, endTs) - isOutside := isOutsideRange(&b, startTs, endTs, []fpRange{{0, 99}, {200, 299}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{0, 99}, {200, 299}}) require.False(t, isOutside) }) t.Run("is not outside if spans across multiple fingerprint ranges", func(t *testing.T) { b := createBlockRef("block", 50, 250, startTs, endTs) - isOutside := isOutsideRange(&b, startTs, endTs, []fpRange{{0, 99}, {200, 299}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{0, 99}, {200, 299}}) require.False(t, isOutside) }) t.Run("is not outside if fingerprint range and time range are larger than block", func(t *testing.T) { b := createBlockRef("block", math.MaxUint64/3, math.MaxUint64/3*2, startTs, endTs) - isOutside := isOutsideRange(&b, 0, 3000, []fpRange{{0, math.MaxUint64}}) + isOutside := isOutsideRange(b, interval(0, 3000), []fpRange{{0, math.MaxUint64}}) + require.False(t, isOutside) + }) + + t.Run("is not outside if block fingerprint range is bigger that search keyspace", func(t *testing.T) { + b := createBlockRef("block", 0x0000, 0xffff, model.Earliest, model.Latest) + isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{0x0100, 0xff00}}) require.False(t, isOutside) }) } func createMatchingBlockRef(blockPath string) BlockRef { - return createBlockRef(blockPath, 0, math.MaxUint64, model.Time(0), model.Now()) + return createBlockRef(blockPath, 0, math.MaxUint64, model.Time(0), model.Time(math.MaxInt64)) } func createBlockRef( From 191466523eb5ae9885bd724f66edd94758e854b9 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 25 Jan 2024 08:59:01 -0800 Subject: [PATCH 11/23] [Blooms] Fix panic when accessing page outside of page bounds (#11787) Move variable declaration to only access the desired page when it's valid. Also removes an unused (duplicate) test fn. --- pkg/storage/bloom/v1/index_querier.go | 3 +-- pkg/storage/bloom/v1/index_test.go | 20 -------------------- 2 files changed, 1 insertion(+), 22 deletions(-) diff --git a/pkg/storage/bloom/v1/index_querier.go b/pkg/storage/bloom/v1/index_querier.go index 2d653e35bad98..005f480e68e9c 100644 --- a/pkg/storage/bloom/v1/index_querier.go +++ b/pkg/storage/bloom/v1/index_querier.go @@ -52,8 +52,6 @@ func (it *LazySeriesIter) Seek(fp model.Fingerprint) error { return header.ThroughFp >= fp }) - page := it.b.index.pageHeaders[desiredPage] - switch { case desiredPage == len(it.b.index.pageHeaders): // no overlap exists, either because no page was found with a throughFP >= fp @@ -68,6 +66,7 @@ func (it *LazySeriesIter) Seek(fp model.Fingerprint) error { // on the right page, no action needed default: // need to load a new page + page := it.b.index.pageHeaders[desiredPage] r, err := it.b.reader.Index() if err != nil { return errors.Wrap(err, "getting index reader") diff --git a/pkg/storage/bloom/v1/index_test.go b/pkg/storage/bloom/v1/index_test.go index 08d2a9a216329..be8d807447dac 100644 --- a/pkg/storage/bloom/v1/index_test.go +++ b/pkg/storage/bloom/v1/index_test.go @@ -9,26 +9,6 @@ import ( "github.com/grafana/loki/pkg/util/encoding" ) -// does not include a real bloom offset -func mkBasicSeries(n int, fromFp, throughFp model.Fingerprint, fromTs, throughTs model.Time) []SeriesWithOffset { - var seriesList []SeriesWithOffset - for i := 0; i < n; i++ { - var series SeriesWithOffset - step := (throughFp - fromFp) / (model.Fingerprint(n)) - series.Fingerprint = fromFp + model.Fingerprint(i)*step - timeDelta := fromTs + (throughTs-fromTs)/model.Time(n)*model.Time(i) - series.Chunks = []ChunkRef{ - { - Start: fromTs + timeDelta*model.Time(i), - End: fromTs + timeDelta*model.Time(i), - Checksum: uint32(i), - }, - } - seriesList = append(seriesList, series) - } - return seriesList -} - func TestBloomOffsetEncoding(t *testing.T) { src := BloomOffset{Page: 1, ByteOffset: 2} enc := &encoding.Encbuf{} From 9964a30b62c05441eb8fba9fd03ac32a99756de2 Mon Sep 17 00:00:00 2001 From: Dylan Guedes Date: Thu, 25 Jan 2024 15:18:13 -0300 Subject: [PATCH 12/23] Jsonnet: Change by-pod source code to reuse the `latency` util function (#11783) **What this PR does / why we need it**: Modify our implementation of `p99LatencyByPod` to reuse the `latencyPanel` util function. --- .../dashboards/loki-reads.json | 50 ++---- .../dashboards/loki-reads.json | 150 +++++------------- .../dashboards/loki-reads.libsonnet | 66 +++----- 3 files changed, 81 insertions(+), 185 deletions(-) diff --git a/production/loki-mixin-compiled-ssd/dashboards/loki-reads.json b/production/loki-mixin-compiled-ssd/dashboards/loki-reads.json index fcf6c120fa606..79c0c82ba0175 100644 --- a/production/loki-mixin-compiled-ssd/dashboards/loki-reads.json +++ b/production/loki-mixin-compiled-ssd/dashboards/loki-reads.json @@ -208,19 +208,6 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "fieldConfig": { - "defaults": { - "custom": { - "fillOpacity": 50, - "showPoints": "never", - "stacking": { - "group": "A", - "mode": "normal" - } - }, - "unit": "s" - } - }, "fill": 1, "id": 3, "legend": { @@ -247,11 +234,13 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99,\n sum(\n rate(loki_request_duration_seconds_bucket{cluster=~\"$cluster\", job=~\"($namespace)/(loki|enterprise-logs)-read\", route=~\"loki_api_v1_series|api_prom_series|api_prom_query|api_prom_label|api_prom_label_name_values|loki_api_v1_query|loki_api_v1_query_range|loki_api_v1_labels|loki_api_v1_label_name_values\"}[$__rate_interval])\n ) by (pod, le)\n )\n", - "instant": false, + "expr": "histogram_quantile(0.99, sum(rate(loki_request_duration_seconds_bucket{cluster=~\"$cluster\", job=~\"($namespace)/(loki|enterprise-logs)-read\", route=~\"loki_api_v1_series|api_prom_series|api_prom_query|api_prom_label|api_prom_label_name_values|loki_api_v1_query|loki_api_v1_query_range|loki_api_v1_labels|loki_api_v1_label_name_values\"}[$__rate_interval])) by (le,pod)) * 1e3", + "format": "time_series", + "interval": "1m", + "intervalFactor": 2, "legendFormat": "__auto", - "range": true, - "refId": "A" + "refId": "A", + "step": 10 } ], "thresholds": [ ], @@ -273,7 +262,7 @@ }, "yaxes": [ { - "format": "short", + "format": "ms", "label": null, "logBase": 1, "max": null, @@ -484,19 +473,6 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "fieldConfig": { - "defaults": { - "custom": { - "fillOpacity": 50, - "showPoints": "never", - "stacking": { - "group": "A", - "mode": "normal" - } - }, - "unit": "s" - } - }, "fill": 1, "id": 6, "legend": { @@ -523,11 +499,13 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99,\n sum(\n rate(loki_boltdb_shipper_request_duration_seconds_bucket{cluster=~\"$cluster\",job=~\"($namespace)/(loki|enterprise-logs)-read\", operation=\"Shipper.Query\"}[$__rate_interval])\n ) by (pod, le)\n )\n", - "instant": false, + "expr": "histogram_quantile(0.99, sum(rate(loki_boltdb_shipper_request_duration_seconds_bucket{cluster=~\"$cluster\",job=~\"($namespace)/(loki|enterprise-logs)-read\", operation=\"Shipper.Query\"}[$__rate_interval])) by (le,pod)) * 1e3", + "format": "time_series", + "interval": "1m", + "intervalFactor": 2, "legendFormat": "__auto", - "range": true, - "refId": "A" + "refId": "A", + "step": 10 } ], "thresholds": [ ], @@ -549,7 +527,7 @@ }, "yaxes": [ { - "format": "short", + "format": "ms", "label": null, "logBase": 1, "max": null, diff --git a/production/loki-mixin-compiled/dashboards/loki-reads.json b/production/loki-mixin-compiled/dashboards/loki-reads.json index 085275b1e4fde..f8d38ba9d484d 100644 --- a/production/loki-mixin-compiled/dashboards/loki-reads.json +++ b/production/loki-mixin-compiled/dashboards/loki-reads.json @@ -208,19 +208,6 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "fieldConfig": { - "defaults": { - "custom": { - "fillOpacity": 50, - "showPoints": "never", - "stacking": { - "group": "A", - "mode": "normal" - } - }, - "unit": "s" - } - }, "fill": 1, "id": 3, "legend": { @@ -247,11 +234,13 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99,\n sum(\n rate(loki_request_duration_seconds_bucket{cluster=~\"$cluster\", job=~\"($namespace)/query-frontend\", route=~\"loki_api_v1_series|api_prom_series|api_prom_query|api_prom_label|api_prom_label_name_values|loki_api_v1_query|loki_api_v1_query_range|loki_api_v1_labels|loki_api_v1_label_name_values\"}[$__rate_interval])\n ) by (pod, le)\n )\n", - "instant": false, + "expr": "histogram_quantile(0.99, sum(rate(loki_request_duration_seconds_bucket{cluster=~\"$cluster\", job=~\"($namespace)/query-frontend\", route=~\"loki_api_v1_series|api_prom_series|api_prom_query|api_prom_label|api_prom_label_name_values|loki_api_v1_query|loki_api_v1_query_range|loki_api_v1_labels|loki_api_v1_label_name_values\"}[$__rate_interval])) by (le,pod)) * 1e3", + "format": "time_series", + "interval": "1m", + "intervalFactor": 2, "legendFormat": "__auto", - "range": true, - "refId": "A" + "refId": "A", + "step": 10 } ], "thresholds": [ ], @@ -273,7 +262,7 @@ }, "yaxes": [ { - "format": "short", + "format": "ms", "label": null, "logBase": 1, "max": null, @@ -484,19 +473,6 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "fieldConfig": { - "defaults": { - "custom": { - "fillOpacity": 50, - "showPoints": "never", - "stacking": { - "group": "A", - "mode": "normal" - } - }, - "unit": "s" - } - }, "fill": 1, "id": 6, "legend": { @@ -523,11 +499,13 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99,\n sum(\n rate(loki_request_duration_seconds_bucket{cluster=~\"$cluster\", job=~\"($namespace)/querier\", route=~\"loki_api_v1_series|api_prom_series|api_prom_query|api_prom_label|api_prom_label_name_values|loki_api_v1_query|loki_api_v1_query_range|loki_api_v1_labels|loki_api_v1_label_name_values\"}[$__rate_interval])\n ) by (pod, le)\n )\n", - "instant": false, + "expr": "histogram_quantile(0.99, sum(rate(loki_request_duration_seconds_bucket{cluster=~\"$cluster\", job=~\"($namespace)/querier\", route=~\"loki_api_v1_series|api_prom_series|api_prom_query|api_prom_label|api_prom_label_name_values|loki_api_v1_query|loki_api_v1_query_range|loki_api_v1_labels|loki_api_v1_label_name_values\"}[$__rate_interval])) by (le,pod)) * 1e3", + "format": "time_series", + "interval": "1m", + "intervalFactor": 2, "legendFormat": "__auto", - "range": true, - "refId": "A" + "refId": "A", + "step": 10 } ], "thresholds": [ ], @@ -549,7 +527,7 @@ }, "yaxes": [ { - "format": "short", + "format": "ms", "label": null, "logBase": 1, "max": null, @@ -760,19 +738,6 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "fieldConfig": { - "defaults": { - "custom": { - "fillOpacity": 50, - "showPoints": "never", - "stacking": { - "group": "A", - "mode": "normal" - } - }, - "unit": "s" - } - }, "fill": 1, "id": 9, "legend": { @@ -799,11 +764,13 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99,\n sum(\n rate(loki_request_duration_seconds_bucket{cluster=~\"$cluster\", job=~\"($namespace)/ingester\", route=~\"/logproto.Querier/Query|/logproto.Querier/Label|/logproto.Querier/Series|/logproto.Querier/QuerySample|/logproto.Querier/GetChunkIDs\"}[$__rate_interval])\n ) by (pod, le)\n )\n", - "instant": false, + "expr": "histogram_quantile(0.99, sum(rate(loki_request_duration_seconds_bucket{cluster=~\"$cluster\", job=~\"($namespace)/ingester\", route=~\"/logproto.Querier/Query|/logproto.Querier/Label|/logproto.Querier/Series|/logproto.Querier/QuerySample|/logproto.Querier/GetChunkIDs\"}[$__rate_interval])) by (le,pod)) * 1e3", + "format": "time_series", + "interval": "1m", + "intervalFactor": 2, "legendFormat": "__auto", - "range": true, - "refId": "A" + "refId": "A", + "step": 10 } ], "thresholds": [ ], @@ -825,7 +792,7 @@ }, "yaxes": [ { - "format": "short", + "format": "ms", "label": null, "logBase": 1, "max": null, @@ -1036,19 +1003,6 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "fieldConfig": { - "defaults": { - "custom": { - "fillOpacity": 50, - "showPoints": "never", - "stacking": { - "group": "A", - "mode": "normal" - } - }, - "unit": "s" - } - }, "fill": 1, "id": 12, "legend": { @@ -1075,11 +1029,13 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99,\n sum(\n rate(loki_request_duration_seconds_bucket{cluster=~\"$cluster\", job=~\"($namespace)/ingester-zone.*\", route=~\"/logproto.Querier/Query|/logproto.Querier/Label|/logproto.Querier/Series|/logproto.Querier/QuerySample|/logproto.Querier/GetChunkIDs\"}[$__rate_interval])\n ) by (pod, le)\n )\n", - "instant": false, + "expr": "histogram_quantile(0.99, sum(rate(loki_request_duration_seconds_bucket{cluster=~\"$cluster\", job=~\"($namespace)/ingester-zone.*\", route=~\"/logproto.Querier/Query|/logproto.Querier/Label|/logproto.Querier/Series|/logproto.Querier/QuerySample|/logproto.Querier/GetChunkIDs\"}[$__rate_interval])) by (le,pod)) * 1e3", + "format": "time_series", + "interval": "1m", + "intervalFactor": 2, "legendFormat": "__auto", - "range": true, - "refId": "A" + "refId": "A", + "step": 10 } ], "thresholds": [ ], @@ -1101,7 +1057,7 @@ }, "yaxes": [ { - "format": "short", + "format": "ms", "label": null, "logBase": 1, "max": null, @@ -1312,19 +1268,6 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "fieldConfig": { - "defaults": { - "custom": { - "fillOpacity": 50, - "showPoints": "never", - "stacking": { - "group": "A", - "mode": "normal" - } - }, - "unit": "s" - } - }, "fill": 1, "id": 15, "legend": { @@ -1351,11 +1294,13 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99,\n sum(\n rate(loki_index_request_duration_seconds_bucket{cluster=~\"$cluster\",job=~\"($namespace)/querier\", operation!=\"index_chunk\"}[$__rate_interval])\n ) by (pod, le)\n )\n", - "instant": false, + "expr": "histogram_quantile(0.99, sum(rate(loki_index_request_duration_seconds_bucket{cluster=~\"$cluster\",job=~\"($namespace)/querier\", operation!=\"index_chunk\"}[$__rate_interval])) by (le,pod)) * 1e3", + "format": "time_series", + "interval": "1m", + "intervalFactor": 2, "legendFormat": "__auto", - "range": true, - "refId": "A" + "refId": "A", + "step": 10 } ], "thresholds": [ ], @@ -1377,7 +1322,7 @@ }, "yaxes": [ { - "format": "short", + "format": "ms", "label": null, "logBase": 1, "max": null, @@ -1588,19 +1533,6 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "fieldConfig": { - "defaults": { - "custom": { - "fillOpacity": 50, - "showPoints": "never", - "stacking": { - "group": "A", - "mode": "normal" - } - }, - "unit": "s" - } - }, "fill": 1, "id": 18, "legend": { @@ -1627,11 +1559,13 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99,\n sum(\n rate(loki_boltdb_shipper_request_duration_seconds_bucket{cluster=~\"$cluster\",job=~\"($namespace)/(querier|index-gateway)\", operation=\"Shipper.Query\"}[$__rate_interval])\n ) by (pod, le)\n )\n", - "instant": false, + "expr": "histogram_quantile(0.99, sum(rate(loki_boltdb_shipper_request_duration_seconds_bucket{cluster=~\"$cluster\",job=~\"($namespace)/(querier|index-gateway)\", operation=\"Shipper.Query\"}[$__rate_interval])) by (le,pod)) * 1e3", + "format": "time_series", + "interval": "1m", + "intervalFactor": 2, "legendFormat": "__auto", - "range": true, - "refId": "A" + "refId": "A", + "step": 10 } ], "thresholds": [ ], @@ -1653,7 +1587,7 @@ }, "yaxes": [ { - "format": "short", + "format": "ms", "label": null, "logBase": 1, "max": null, diff --git a/production/loki-mixin/dashboards/loki-reads.libsonnet b/production/loki-mixin/dashboards/loki-reads.libsonnet index 538cade448fac..823b03126ccc5 100644 --- a/production/loki-mixin/dashboards/loki-reads.libsonnet +++ b/production/loki-mixin/dashboards/loki-reads.libsonnet @@ -8,39 +8,25 @@ local utils = import 'mixin-utils/utils.libsonnet'; local http_routes = 'loki_api_v1_series|api_prom_series|api_prom_query|api_prom_label|api_prom_label_name_values|loki_api_v1_query|loki_api_v1_query_range|loki_api_v1_labels|loki_api_v1_label_name_values', local grpc_routes = '/logproto.Querier/Query|/logproto.Querier/Label|/logproto.Querier/Series|/logproto.Querier/QuerySample|/logproto.Querier/GetChunkIDs', + local latencyPanelWithExtraGrouping(metricName, selector, multiplier='1e3', extra_grouping='') = { + nullPointMode: 'null as zero', + targets: [ + { + expr: 'histogram_quantile(0.99, sum(rate(%s_bucket%s[$__rate_interval])) by (le,%s)) * %s' % [metricName, selector, extra_grouping, multiplier], + format: 'time_series', + intervalFactor: 2, + refId: 'A', + step: 10, + interval: '1m', + legendFormat: '__auto', + }, + ], + yaxes: $.yaxes('ms'), + }, + local p99LatencyByPod(metric, selectorStr) = $.panel('Per Pod Latency (p99)') + - { - targets: [ - { - expr: - ||| - histogram_quantile(0.99, - sum( - rate(%s%s[$__rate_interval]) - ) by (pod, le) - ) - ||| % [metric, selectorStr], - instant: false, - legendFormat: '__auto', - range: true, - refId: 'A', - }, - ], - fieldConfig+: { - defaults+: { - custom+: { - fillOpacity: 50, - showPoints: 'never', - stacking: { - group: 'A', - mode: 'normal', - }, - }, - unit: 's', - }, - }, - }, + latencyPanelWithExtraGrouping(metric, selectorStr, '1e3', 'pod'), 'loki-reads.json': { local cfg = self, @@ -96,11 +82,9 @@ local utils = import 'mixin-utils/utils.libsonnet'; ) .addPanel( p99LatencyByPod( - 'loki_request_duration_seconds_bucket', + 'loki_request_duration_seconds', $.toPrometheusSelector( - dashboards['loki-reads.json'].clusterMatchers + - dashboards['loki-reads.json'].matchers.cortexgateway + - [utils.selector.re('route', http_routes)] + dashboards['loki-reads.json'].clusterMatchers + dashboards['loki-reads.json'].matchers.cortexgateway + [utils.selector.re('route', http_routes)] ), ) ) @@ -121,7 +105,7 @@ local utils = import 'mixin-utils/utils.libsonnet'; ) .addPanel( p99LatencyByPod( - 'loki_request_duration_seconds_bucket', + 'loki_request_duration_seconds', $.toPrometheusSelector( dashboards['loki-reads.json'].clusterMatchers + dashboards['loki-reads.json'].matchers.queryFrontend + @@ -147,7 +131,7 @@ local utils = import 'mixin-utils/utils.libsonnet'; ) .addPanel( p99LatencyByPod( - 'loki_request_duration_seconds_bucket', + 'loki_request_duration_seconds', $.toPrometheusSelector( dashboards['loki-reads.json'].clusterMatchers + dashboards['loki-reads.json'].matchers.querier + @@ -173,7 +157,7 @@ local utils = import 'mixin-utils/utils.libsonnet'; ) .addPanel( p99LatencyByPod( - 'loki_request_duration_seconds_bucket', + 'loki_request_duration_seconds', $.toPrometheusSelector( dashboards['loki-reads.json'].clusterMatchers + dashboards['loki-reads.json'].matchers.ingester + @@ -200,7 +184,7 @@ local utils = import 'mixin-utils/utils.libsonnet'; ) .addPanel( p99LatencyByPod( - 'loki_request_duration_seconds_bucket', + 'loki_request_duration_seconds', $.toPrometheusSelector( dashboards['loki-reads.json'].clusterMatchers + dashboards['loki-reads.json'].matchers.ingesterZoneAware + @@ -222,7 +206,7 @@ local utils = import 'mixin-utils/utils.libsonnet'; ) .addPanel( p99LatencyByPod( - 'loki_index_request_duration_seconds_bucket', + 'loki_index_request_duration_seconds', '{%s operation!="index_chunk"}' % dashboards['loki-reads.json'].querierSelector ) ) @@ -254,7 +238,7 @@ local utils = import 'mixin-utils/utils.libsonnet'; ) .addPanel( p99LatencyByPod( - 'loki_boltdb_shipper_request_duration_seconds_bucket', + 'loki_boltdb_shipper_request_duration_seconds', '{%s operation="Shipper.Query"}' % dashboards['loki-reads.json'].querierOrIndexGatewaySelector ) ) From a064f5d67009b314fac346818b14b27ecebd27ba Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Thu, 25 Jan 2024 12:02:40 -0700 Subject: [PATCH 13/23] feat: add release please config (#11333) **What this PR does / why we need it**: The new release process is being developed in another repo for now as it enables much faster feedback loops. We are at the stage where we are pulling the Loki repo and building images, so it's time to move the `release-please` config into Loki, and configure it for the various packages we will be releasing. This PR adds the two basic configs that `release-please` requires. This PR does not change anything about the release process, it just enables us to keep iterating out of band until we're ready to move over. **Which issue(s) this PR fixes**: Re #11332 --- .github/workflows/minor-release-pr.yml | 21 +++++++++++++++++++++ .github/workflows/patch-release-pr.yml | 21 +++++++++++++++++++++ .github/workflows/release.yml | 19 +++++++++++++++++++ .release-please-manifest.json | 6 ++++++ Makefile | 12 +++++++++++- pkg/querier/queryrange/limits_test.go | 3 +-- 6 files changed, 79 insertions(+), 3 deletions(-) create mode 100644 .github/workflows/minor-release-pr.yml create mode 100644 .github/workflows/patch-release-pr.yml create mode 100644 .github/workflows/release.yml create mode 100644 .release-please-manifest.json diff --git a/.github/workflows/minor-release-pr.yml b/.github/workflows/minor-release-pr.yml new file mode 100644 index 0000000000000..59581fa976d33 --- /dev/null +++ b/.github/workflows/minor-release-pr.yml @@ -0,0 +1,21 @@ +--- +name: 'create release PR for minor releases' +on: + push: + branches: + - 'k[0-9]*' + workflow_dispatch: {} +permissions: + contents: 'write' + issues: 'write' + pull-requests: 'write' +jobs: + create-release-pr: + uses: github/loki-release/.github/workflows/release-pr.yml@main + with: + release_repo: grafana/loki + skip_validation: false + versioning_strategy: always-bump-minor + secrets: + GCS_SERVICE_ACCOUNT_KEY: '${{ secrets.BACKEND_ENTERPRISE_DRONE }}' + GH_TOKEN: '${{ secrets.GITHUB_TOKEN }}' diff --git a/.github/workflows/patch-release-pr.yml b/.github/workflows/patch-release-pr.yml new file mode 100644 index 0000000000000..001b00d93b663 --- /dev/null +++ b/.github/workflows/patch-release-pr.yml @@ -0,0 +1,21 @@ +--- +name: 'create release PR for patch releases' +on: + push: + branches: + - 'release-[0-9].[0-9].x' + workflow_dispatch: {} +permissions: + contents: 'write' + issues: 'write' + pull-requests: 'write' +jobs: + create-release-pr: + uses: github/loki-release/.github/workflows/release-pr.yml@main + with: + release_repo: grafana/loki + skip_validation: false + versioning_strategy: always-bump-patch + secrets: + GCS_SERVICE_ACCOUNT_KEY: '${{ secrets.BACKEND_ENTERPRISE_DRONE }}' + GH_TOKEN: '${{ secrets.GITHUB_TOKEN }}' diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml new file mode 100644 index 0000000000000..cacdacf773a82 --- /dev/null +++ b/.github/workflows/release.yml @@ -0,0 +1,19 @@ +--- +name: 'create release' +on: + push: + branches: + - 'release-[0-9].[0-9].x' + - 'k[0-9]*' + workflow_dispatch: {} +permissions: + contents: write + pull-requests: write +jobs: + release: + uses: github/loki-release/.github/workflows/release.yml@main + with: + release_repo: grafana/loki + secrets: + GCS_SERVICE_ACCOUNT_KEY: '${{ secrets.BACKEND_ENTERPRISE_DRONE }}' + GH_TOKEN: '${{ secrets.GH_TOKEN }}' diff --git a/.release-please-manifest.json b/.release-please-manifest.json new file mode 100644 index 0000000000000..0e134950eab83 --- /dev/null +++ b/.release-please-manifest.json @@ -0,0 +1,6 @@ +{ + "cmd/loki": "2.9.4", + "cmd/loki-canary": "2.9.4", + "cmd/logcli": "2.9.4", + "clients/cmd/promtail": "2.9.4" +} diff --git a/Makefile b/Makefile index d311ed1c4f3c6..2acf8b4285047 100644 --- a/Makefile +++ b/Makefile @@ -281,7 +281,16 @@ cmd/migrate/migrate: ############# GOX = gox $(GO_FLAGS) -output="dist/{{.Dir}}-{{.OS}}-{{.Arch}}" CGO_GOX = gox $(DYN_GO_FLAGS) -cgo -output="dist/{{.Dir}}-{{.OS}}-{{.Arch}}" + +SKIP_ARM ?= false dist: clean +ifeq ($(SKIP_ARM),true) + CGO_ENABLED=0 $(GOX) -osarch="linux/amd64 darwin/amd64 windows/amd64 freebsd/amd64" ./cmd/loki + CGO_ENABLED=0 $(GOX) -osarch="linux/amd64 darwin/amd64 windows/amd64 freebsd/amd64" ./cmd/logcli + CGO_ENABLED=0 $(GOX) -osarch="linux/amd64 darwin/amd64 windows/amd64 freebsd/amd64" ./cmd/loki-canary + CGO_ENABLED=0 $(GOX) -osarch="darwin/amd64 windows/amd64 windows/386 freebsd/amd64" ./clients/cmd/promtail + CGO_ENABLED=1 $(CGO_GOX) -tags promtail_journal_enabled -osarch="linux/amd64" ./clients/cmd/promtail +else CGO_ENABLED=0 $(GOX) -osarch="linux/amd64 linux/arm64 linux/arm darwin/amd64 darwin/arm64 windows/amd64 freebsd/amd64" ./cmd/loki CGO_ENABLED=0 $(GOX) -osarch="linux/amd64 linux/arm64 linux/arm darwin/amd64 darwin/arm64 windows/amd64 freebsd/amd64" ./cmd/logcli CGO_ENABLED=0 $(GOX) -osarch="linux/amd64 linux/arm64 linux/arm darwin/amd64 darwin/arm64 windows/amd64 freebsd/amd64" ./cmd/loki-canary @@ -289,6 +298,7 @@ dist: clean PKG_CONFIG_PATH="/usr/lib/aarch64-linux-gnu/pkgconfig" CC="aarch64-linux-gnu-gcc" $(CGO_GOX) -tags promtail_journal_enabled -osarch="linux/arm64" ./clients/cmd/promtail PKG_CONFIG_PATH="/usr/lib/arm-linux-gnueabihf/pkgconfig" CC="arm-linux-gnueabihf-gcc" $(CGO_GOX) -tags promtail_journal_enabled -osarch="linux/arm" ./clients/cmd/promtail CGO_ENABLED=1 $(CGO_GOX) -tags promtail_journal_enabled -osarch="linux/amd64" ./clients/cmd/promtail +endif for i in dist/*; do zip -j -m $$i.zip $$i; done pushd dist && sha256sum * > SHA256SUMS && popd @@ -307,7 +317,7 @@ publish: packages lint: ## run linters go version golangci-lint version - GO111MODULE=on golangci-lint run -v + GO111MODULE=on golangci-lint run -v --timeout 15m faillint -paths "sync/atomic=go.uber.org/atomic" ./... ######## diff --git a/pkg/querier/queryrange/limits_test.go b/pkg/querier/queryrange/limits_test.go index a80cf96dde805..61e310faee4f6 100644 --- a/pkg/querier/queryrange/limits_test.go +++ b/pkg/querier/queryrange/limits_test.go @@ -821,7 +821,6 @@ func TestAcquireWithTiming(t *testing.T) { } } - // Check that the waiting time for the third request is larger than 0 milliseconds and less than or equal to 10-5=5 milliseconds - require.Greater(t, waiting3, 0*time.Millisecond) + require.GreaterOrEqual(t, waiting3, 0*time.Millisecond) require.LessOrEqual(t, waiting3, 5*time.Millisecond) } From 406b10c675d7522228ce05d06620a41ae6e589a9 Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Thu, 25 Jan 2024 13:34:56 -0700 Subject: [PATCH 14/23] accidentally merged this from ##11333 (#11788) **What this PR does / why we need it**: I did not intend to merge this change, reverting to what it was before merging #11333 --- pkg/querier/queryrange/limits_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/querier/queryrange/limits_test.go b/pkg/querier/queryrange/limits_test.go index 61e310faee4f6..a80cf96dde805 100644 --- a/pkg/querier/queryrange/limits_test.go +++ b/pkg/querier/queryrange/limits_test.go @@ -821,6 +821,7 @@ func TestAcquireWithTiming(t *testing.T) { } } - require.GreaterOrEqual(t, waiting3, 0*time.Millisecond) + // Check that the waiting time for the third request is larger than 0 milliseconds and less than or equal to 10-5=5 milliseconds + require.Greater(t, waiting3, 0*time.Millisecond) require.LessOrEqual(t, waiting3, 5*time.Millisecond) } From 1002ba00dff58ed588987169c0d3a0ddac2d022b Mon Sep 17 00:00:00 2001 From: George Wilson Date: Thu, 25 Jan 2024 23:56:18 +0000 Subject: [PATCH 15/23] Note the helm chart and docs do not agree (#11782) **What this PR does / why we need it**: Docs do not match the code in the helm chart - where the read deployment is made as a statefulset with ReadWriteOnce disks (https://github.com/grafana/loki/blob/main/production/helm/loki/templates/read/statefulset-read.yaml). I suspect the docs are correct that it's possible - but given it's the official helm chart - the deviation should be noted. **Which issue(s) this PR fixes**: N/A **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler --- docs/sources/get-started/deployment-modes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/get-started/deployment-modes.md b/docs/sources/get-started/deployment-modes.md index 5b4766f65253e..df4008ff332db 100644 --- a/docs/sources/get-started/deployment-modes.md +++ b/docs/sources/get-started/deployment-modes.md @@ -33,7 +33,7 @@ The three execution paths in simple scalable mode are each activated by appendin - `-target=write` - The write target is stateful and is controlled by a Kubernetes StatefulSet. It contains the following components: -- Distributor -- Ingester -- `-target=read` - The read target is stateless and can be run as a Kubernetes Deployment that can be scaled automatically. It contains the following components: +- `-target=read` - The read target is stateless and can be run as a Kubernetes Deployment that can be scaled automatically (Note that in the official helm chart it is currently deployed as a stateful set). It contains the following components: -- Query front end -- Queriers - `-target=backend` - The backend target is stateful, and is controlled by a Kubernetes StatefulSet. Contains the following components: From 88df82295d694919f18e4db4df884beb0aaa144d Mon Sep 17 00:00:00 2001 From: Periklis Tsirakidis Date: Fri, 26 Jan 2024 13:17:04 +0100 Subject: [PATCH 16/23] operator: Update Loki operand to v2.9.4 (#11778) **What this PR does / why we need it**: Update Loki operand to v2.9.4 **Which issue(s) this PR fixes**: Fixes # **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [ ] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --- .../manifests/loki-operator.clusterserviceversion.yaml | 6 +++--- .../manifests/loki-operator.clusterserviceversion.yaml | 6 +++--- .../manifests/loki-operator.clusterserviceversion.yaml | 6 +++--- .../community-openshift/manager_related_image_patch.yaml | 2 +- .../overlays/community/manager_related_image_patch.yaml | 2 +- .../overlays/development/manager_related_image_patch.yaml | 2 +- .../overlays/openshift/manager_related_image_patch.yaml | 2 +- operator/docs/operator/compatibility.md | 1 + operator/hack/addons_dev.yaml | 4 ++-- operator/hack/addons_ocp.yaml | 4 ++-- operator/internal/manifests/var.go | 2 +- 11 files changed, 19 insertions(+), 18 deletions(-) diff --git a/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml b/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml index 1e9f05cc372da..eef89deddb746 100644 --- a/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml +++ b/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml @@ -150,7 +150,7 @@ metadata: categories: OpenShift Optional, Logging & Tracing certified: "false" containerImage: docker.io/grafana/loki-operator:0.5.0 - createdAt: "2024-01-19T14:20:59Z" + createdAt: "2024-01-25T11:08:43Z" description: The Community Loki Operator provides Kubernetes native deployment and management of Loki and related logging components. features.operators.openshift.io/disconnected: "true" @@ -1703,7 +1703,7 @@ spec: - /manager env: - name: RELATED_IMAGE_LOKI - value: docker.io/grafana/loki:2.9.3 + value: docker.io/grafana/loki:2.9.4 - name: RELATED_IMAGE_GATEWAY value: quay.io/observatorium/api:latest - name: RELATED_IMAGE_OPA @@ -1826,7 +1826,7 @@ spec: provider: name: Grafana Loki SIG Operator relatedImages: - - image: docker.io/grafana/loki:2.9.3 + - image: docker.io/grafana/loki:2.9.4 name: loki - image: quay.io/observatorium/api:latest name: gateway diff --git a/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml b/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml index 4d88a93a92503..ecc294fddf367 100644 --- a/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml +++ b/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml @@ -150,7 +150,7 @@ metadata: categories: OpenShift Optional, Logging & Tracing certified: "false" containerImage: docker.io/grafana/loki-operator:0.5.0 - createdAt: "2024-01-19T14:20:57Z" + createdAt: "2024-01-25T11:08:41Z" description: The Community Loki Operator provides Kubernetes native deployment and management of Loki and related logging components. operators.operatorframework.io/builder: operator-sdk-unknown @@ -1683,7 +1683,7 @@ spec: - /manager env: - name: RELATED_IMAGE_LOKI - value: docker.io/grafana/loki:2.9.3 + value: docker.io/grafana/loki:2.9.4 - name: RELATED_IMAGE_GATEWAY value: quay.io/observatorium/api:latest - name: RELATED_IMAGE_OPA @@ -1794,7 +1794,7 @@ spec: provider: name: Grafana Loki SIG Operator relatedImages: - - image: docker.io/grafana/loki:2.9.3 + - image: docker.io/grafana/loki:2.9.4 name: loki - image: quay.io/observatorium/api:latest name: gateway diff --git a/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml b/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml index 2adb57379fc73..69e55579d125e 100644 --- a/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml +++ b/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml @@ -150,7 +150,7 @@ metadata: categories: OpenShift Optional, Logging & Tracing certified: "false" containerImage: quay.io/openshift-logging/loki-operator:0.1.0 - createdAt: "2024-01-19T14:21:01Z" + createdAt: "2024-01-25T11:08:45Z" description: | The Loki Operator for OCP provides a means for configuring and managing a Loki stack for cluster logging. ## Prerequisites and Requirements @@ -1688,7 +1688,7 @@ spec: - /manager env: - name: RELATED_IMAGE_LOKI - value: quay.io/openshift-logging/loki:v2.9.3 + value: quay.io/openshift-logging/loki:v2.9.4 - name: RELATED_IMAGE_GATEWAY value: quay.io/observatorium/api:latest - name: RELATED_IMAGE_OPA @@ -1811,7 +1811,7 @@ spec: provider: name: Red Hat relatedImages: - - image: quay.io/openshift-logging/loki:v2.9.3 + - image: quay.io/openshift-logging/loki:v2.9.4 name: loki - image: quay.io/observatorium/api:latest name: gateway diff --git a/operator/config/overlays/community-openshift/manager_related_image_patch.yaml b/operator/config/overlays/community-openshift/manager_related_image_patch.yaml index e9c5435287a80..0b11adb57bbeb 100644 --- a/operator/config/overlays/community-openshift/manager_related_image_patch.yaml +++ b/operator/config/overlays/community-openshift/manager_related_image_patch.yaml @@ -9,7 +9,7 @@ spec: - name: manager env: - name: RELATED_IMAGE_LOKI - value: docker.io/grafana/loki:2.9.3 + value: docker.io/grafana/loki:2.9.4 - name: RELATED_IMAGE_GATEWAY value: quay.io/observatorium/api:latest - name: RELATED_IMAGE_OPA diff --git a/operator/config/overlays/community/manager_related_image_patch.yaml b/operator/config/overlays/community/manager_related_image_patch.yaml index e9c5435287a80..0b11adb57bbeb 100644 --- a/operator/config/overlays/community/manager_related_image_patch.yaml +++ b/operator/config/overlays/community/manager_related_image_patch.yaml @@ -9,7 +9,7 @@ spec: - name: manager env: - name: RELATED_IMAGE_LOKI - value: docker.io/grafana/loki:2.9.3 + value: docker.io/grafana/loki:2.9.4 - name: RELATED_IMAGE_GATEWAY value: quay.io/observatorium/api:latest - name: RELATED_IMAGE_OPA diff --git a/operator/config/overlays/development/manager_related_image_patch.yaml b/operator/config/overlays/development/manager_related_image_patch.yaml index f9a2449bceed8..a9c7cf22f1d1e 100644 --- a/operator/config/overlays/development/manager_related_image_patch.yaml +++ b/operator/config/overlays/development/manager_related_image_patch.yaml @@ -9,6 +9,6 @@ spec: - name: manager env: - name: RELATED_IMAGE_LOKI - value: docker.io/grafana/loki:2.9.3 + value: docker.io/grafana/loki:2.9.4 - name: RELATED_IMAGE_GATEWAY value: quay.io/observatorium/api:latest diff --git a/operator/config/overlays/openshift/manager_related_image_patch.yaml b/operator/config/overlays/openshift/manager_related_image_patch.yaml index 5e64be8752f20..28c00163194a7 100644 --- a/operator/config/overlays/openshift/manager_related_image_patch.yaml +++ b/operator/config/overlays/openshift/manager_related_image_patch.yaml @@ -9,7 +9,7 @@ spec: - name: manager env: - name: RELATED_IMAGE_LOKI - value: quay.io/openshift-logging/loki:v2.9.3 + value: quay.io/openshift-logging/loki:v2.9.4 - name: RELATED_IMAGE_GATEWAY value: quay.io/observatorium/api:latest - name: RELATED_IMAGE_OPA diff --git a/operator/docs/operator/compatibility.md b/operator/docs/operator/compatibility.md index 36550f06a7062..2fc1e79662d1c 100644 --- a/operator/docs/operator/compatibility.md +++ b/operator/docs/operator/compatibility.md @@ -37,3 +37,4 @@ The versions of Loki compatible to be run with the Loki Operator are: * v2.9.1 * v2.9.2 * v2.9.3 +* v2.9.4 diff --git a/operator/hack/addons_dev.yaml b/operator/hack/addons_dev.yaml index adf6aa053add4..f2538cd9f5af1 100644 --- a/operator/hack/addons_dev.yaml +++ b/operator/hack/addons_dev.yaml @@ -29,7 +29,7 @@ spec: spec: containers: - name: logcli - image: docker.io/grafana/logcli:2.9.3-amd64 + image: docker.io/grafana/logcli:2.9.4-amd64 imagePullPolicy: IfNotPresent command: - /bin/sh @@ -73,7 +73,7 @@ spec: spec: containers: - name: promtail - image: docker.io/grafana/promtail:2.9.3 + image: docker.io/grafana/promtail:2.9.4 args: - -config.file=/etc/promtail/promtail.yaml - -log.level=info diff --git a/operator/hack/addons_ocp.yaml b/operator/hack/addons_ocp.yaml index 1a0ff7325a62a..5a09cbc0e6aa4 100644 --- a/operator/hack/addons_ocp.yaml +++ b/operator/hack/addons_ocp.yaml @@ -29,7 +29,7 @@ spec: spec: containers: - name: logcli - image: docker.io/grafana/logcli:2.9.3-amd64 + image: docker.io/grafana/logcli:2.9.4-amd64 imagePullPolicy: IfNotPresent command: - /bin/sh @@ -70,7 +70,7 @@ spec: spec: containers: - name: promtail - image: docker.io/grafana/promtail:2.9.3 + image: docker.io/grafana/promtail:2.9.4 args: - -config.file=/etc/promtail/promtail.yaml - -log.level=info diff --git a/operator/internal/manifests/var.go b/operator/internal/manifests/var.go index 9bbe8a12c0149..1395412b792da 100644 --- a/operator/internal/manifests/var.go +++ b/operator/internal/manifests/var.go @@ -59,7 +59,7 @@ const ( EnvRelatedImageGateway = "RELATED_IMAGE_GATEWAY" // DefaultContainerImage declares the default fallback for loki image. - DefaultContainerImage = "docker.io/grafana/loki:2.9.3" + DefaultContainerImage = "docker.io/grafana/loki:2.9.4" // DefaultLokiStackGatewayImage declares the default image for lokiStack-gateway. DefaultLokiStackGatewayImage = "quay.io/observatorium/api:latest" From d97ac48681d676486fa4d184043b0978ccf120d1 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Fri, 26 Jan 2024 13:51:16 +0100 Subject: [PATCH 17/23] Add timing metrics for compaction (#11785) **What this PR does / why we need it**: This PR modifies the metrics of the bloom compactor to: - Add new histogram metrics to track the time spent to process compaction cycles, tenants and jobs. - Replace `_succeeded` and `_failed` metrics with a new `_completed` metric with a status label. **Which issue(s) this PR fixes**: Fixes # **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --- .drone/drone.jsonnet | 4 +- .drone/drone.yml | 13 +----- integration/loki_micro_services_test.go | 13 ++++-- pkg/bloomcompactor/bloomcompactor.go | 29 +++++++++---- pkg/bloomcompactor/metrics.go | 58 ++++++++++++++----------- 5 files changed, 65 insertions(+), 52 deletions(-) diff --git a/.drone/drone.jsonnet b/.drone/drone.jsonnet index 49f67f06861a3..a6b1fbd9f9308 100644 --- a/.drone/drone.jsonnet +++ b/.drone/drone.jsonnet @@ -610,13 +610,13 @@ local build_image_tag = '0.33.0'; 'cd -', ]) { depends_on: ['clone'], when: onPRs }, make('test', container=false) { depends_on: ['clone-target-branch', 'check-generated-files'] }, - run('test-target-branch', commands=['cd ../loki-target-branch && BUILD_IN_CONTAINER=false make test']) { depends_on: ['clone-target-branch'], when: onPRs }, + //run('test-target-branch', commands=['cd ../loki-target-branch && BUILD_IN_CONTAINER=false make test']) { depends_on: ['clone-target-branch'], when: onPRs }, make('compare-coverage', container=false, args=[ 'old=../loki-target-branch/test_results.txt', 'new=test_results.txt', 'packages=ingester,distributor,querier,querier/queryrange,iter,storage,chunkenc,logql,loki', '> diff.txt', - ]) { depends_on: ['test', 'test-target-branch'], when: onPRs }, + ]) { depends_on: ['test' /*, 'test-target-branch' */], when: onPRs }, run('report-coverage', commands=[ "total_diff=$(sed 's/%//' diff.txt | awk '{sum+=$3;}END{print sum;}')", 'if [ $total_diff = 0 ]; then exit 0; fi', diff --git a/.drone/drone.yml b/.drone/drone.yml index 7a62b621262a8..5bc773dec971c 100644 --- a/.drone/drone.yml +++ b/.drone/drone.yml @@ -212,23 +212,12 @@ steps: environment: {} image: grafana/loki-build-image:0.33.0 name: test -- commands: - - cd ../loki-target-branch && BUILD_IN_CONTAINER=false make test - depends_on: - - clone-target-branch - environment: {} - image: grafana/loki-build-image:0.33.0 - name: test-target-branch - when: - event: - - pull_request - commands: - make BUILD_IN_CONTAINER=false compare-coverage old=../loki-target-branch/test_results.txt new=test_results.txt packages=ingester,distributor,querier,querier/queryrange,iter,storage,chunkenc,logql,loki > diff.txt depends_on: - test - - test-target-branch environment: {} image: grafana/loki-build-image:0.33.0 name: compare-coverage @@ -2113,6 +2102,6 @@ kind: secret name: gpg_private_key --- kind: signature -hmac: 457592d17208477ceb480f81dbdb88f7b95a5ad015c88d9d6fed06c2422a52f9 +hmac: 323c705d2c43805afcd46813fb9777c7d7d51f7d79aa018e7e78f208eb9f6713 ... diff --git a/integration/loki_micro_services_test.go b/integration/loki_micro_services_test.go index da09318fa837f..6d43052c37a2d 100644 --- a/integration/loki_micro_services_test.go +++ b/integration/loki_micro_services_test.go @@ -1174,7 +1174,7 @@ func TestBloomFiltersEndToEnd(t *testing.T) { ) require.NoError(t, clu.Run()) - now := time.Date(2024, time.January, 19, 12, 0, 0, 0, time.UTC) + now := time.Now() cliDistributor := client.New(tenantID, "", tDistributor.HTTPURL()) cliDistributor.Now = now @@ -1224,8 +1224,15 @@ func TestBloomFiltersEndToEnd(t *testing.T) { // verify metrics that observe usage of block for filtering metrics, err := cliBloomCompactor.Metrics() require.NoError(t, err) - successfulRunCount := getMetricValue(t, "loki_bloomcompactor_runs_completed_total", metrics) - t.Log("successful bloom compactor runs", successfulRunCount) + successfulRunCount, labels, err := extractMetric(`loki_bloomcompactor_runs_completed_total`, metrics) + if err != nil { + return false + } + t.Log("bloom compactor runs", successfulRunCount, labels) + if labels["status"] != "success" { + return false + } + return successfulRunCount == 1 }, 30*time.Second, time.Second) diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index ab8724192bd76..35dd19a22582a 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -210,14 +210,16 @@ func (c *Compactor) running(ctx context.Context) error { for { select { - case <-ticker.C: + case start := <-ticker.C: c.metrics.compactionRunsStarted.Inc() if err := c.runCompaction(ctx); err != nil { - c.metrics.compactionRunsFailed.Inc() + c.metrics.compactionRunsCompleted.WithLabelValues(statusFailure).Inc() + c.metrics.compactionRunTime.WithLabelValues(statusFailure).Observe(time.Since(start).Seconds()) level.Error(c.logger).Log("msg", "failed to run compaction", "err", err) continue } - c.metrics.compactionRunsCompleted.Inc() + c.metrics.compactionRunsCompleted.WithLabelValues(statusSuccess).Inc() + c.metrics.compactionRunTime.WithLabelValues(statusSuccess).Observe(time.Since(start).Seconds()) case <-ctx.Done(): return nil } @@ -330,6 +332,7 @@ func (c *Compactor) compactUsers(ctx context.Context, logger log.Logger, sc stor ownedTenants[tenant] = struct{}{} + start := time.Now() if err := c.compactTenantWithRetries(ctx, tenantLogger, sc, tableName, tenant); err != nil { switch { case errors.Is(err, context.Canceled): @@ -337,14 +340,16 @@ func (c *Compactor) compactUsers(ctx context.Context, logger log.Logger, sc stor level.Info(tenantLogger).Log("msg", "compaction for tenant was interrupted by a shutdown") return nil default: - c.metrics.compactionRunFailedTenants.Inc() + c.metrics.compactionRunTenantsCompleted.WithLabelValues(statusFailure).Inc() + c.metrics.compactionRunTenantsTime.WithLabelValues(statusFailure).Observe(time.Since(start).Seconds()) level.Error(tenantLogger).Log("msg", "failed to compact tenant", "err", err) errs.Add(err) } continue } - c.metrics.compactionRunSucceededTenants.Inc() + c.metrics.compactionRunTenantsCompleted.WithLabelValues(statusSuccess).Inc() + c.metrics.compactionRunTenantsTime.WithLabelValues(statusSuccess).Observe(time.Since(start).Seconds()) level.Info(tenantLogger).Log("msg", "successfully compacted tenant") } @@ -430,13 +435,19 @@ func (c *Compactor) compactTenant(ctx context.Context, logger log.Logger, sc sto jobLogger := log.With(logger, "job", job.String()) c.metrics.compactionRunJobStarted.Inc() + start := time.Now() err = c.runCompact(ctx, jobLogger, job, bt, sc) if err != nil { - c.metrics.compactionRunJobFailed.Inc() - errs.Add(errors.Wrap(err, "runBloomCompact failed")) - } else { - c.metrics.compactionRunJobSuceeded.Inc() + c.metrics.compactionRunJobCompleted.WithLabelValues(statusFailure).Inc() + c.metrics.compactionRunJobTime.WithLabelValues(statusFailure).Observe(time.Since(start).Seconds()) + errs.Add(errors.Wrap(err, fmt.Sprintf("runBloomCompact failed for job %s", job.String()))) + return nil } + + c.metrics.compactionRunJobCompleted.WithLabelValues(statusSuccess).Inc() + c.metrics.compactionRunJobTime.WithLabelValues(statusSuccess).Observe(time.Since(start).Seconds()) + level.Debug(logger).Log("msg", "compaction of job succeeded", "job", job.String(), "duration", time.Since(start)) + return nil }) diff --git a/pkg/bloomcompactor/metrics.go b/pkg/bloomcompactor/metrics.go index c043b8103c31d..ee2f1630ab5ec 100644 --- a/pkg/bloomcompactor/metrics.go +++ b/pkg/bloomcompactor/metrics.go @@ -8,19 +8,22 @@ import ( const ( metricsNamespace = "loki" metricsSubsystem = "bloomcompactor" + + statusSuccess = "success" + statusFailure = "failure" ) type metrics struct { compactionRunsStarted prometheus.Counter - compactionRunsCompleted prometheus.Counter - compactionRunsFailed prometheus.Counter + compactionRunsCompleted *prometheus.CounterVec + compactionRunTime *prometheus.HistogramVec compactionRunDiscoveredTenants prometheus.Counter compactionRunSkippedTenants prometheus.Counter - compactionRunSucceededTenants prometheus.Counter - compactionRunFailedTenants prometheus.Counter + compactionRunTenantsCompleted *prometheus.CounterVec + compactionRunTenantsTime *prometheus.HistogramVec compactionRunJobStarted prometheus.Counter - compactionRunJobSuceeded prometheus.Counter - compactionRunJobFailed prometheus.Counter + compactionRunJobCompleted *prometheus.CounterVec + compactionRunJobTime *prometheus.HistogramVec compactionRunInterval prometheus.Gauge compactorRunning prometheus.Gauge } @@ -33,18 +36,19 @@ func newMetrics(r prometheus.Registerer) *metrics { Name: "runs_started_total", Help: "Total number of compactions started", }), - compactionRunsCompleted: promauto.With(r).NewCounter(prometheus.CounterOpts{ + compactionRunsCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, Name: "runs_completed_total", Help: "Total number of compactions completed successfully", - }), - compactionRunsFailed: promauto.With(r).NewCounter(prometheus.CounterOpts{ + }, []string{"status"}), + compactionRunTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "runs_failed_total", - Help: "Total number of compaction runs failed", - }), + Name: "runs_time_seconds", + Help: "Time spent during a compaction cycle.", + Buckets: prometheus.DefBuckets, + }, []string{"status"}), compactionRunDiscoveredTenants: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, @@ -57,36 +61,38 @@ func newMetrics(r prometheus.Registerer) *metrics { Name: "tenants_skipped", Help: "Number of tenants skipped during the current compaction run", }), - compactionRunSucceededTenants: promauto.With(r).NewCounter(prometheus.CounterOpts{ + compactionRunTenantsCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "tenants_succeeded", + Name: "tenants_completed", Help: "Number of tenants successfully processed during the current compaction run", - }), - compactionRunFailedTenants: promauto.With(r).NewCounter(prometheus.CounterOpts{ + }, []string{"status"}), + compactionRunTenantsTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "tenants_failed", - Help: "Number of tenants failed processing during the current compaction run", - }), + Name: "tenants_time_seconds", + Help: "Time spent processing tenants.", + Buckets: prometheus.DefBuckets, + }, []string{"status"}), compactionRunJobStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, Name: "job_started", Help: "Number of jobs started processing during the current compaction run", }), - compactionRunJobSuceeded: promauto.With(r).NewCounter(prometheus.CounterOpts{ + compactionRunJobCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "job_succeeded", + Name: "job_completed", Help: "Number of jobs successfully processed during the current compaction run", - }), - compactionRunJobFailed: promauto.With(r).NewCounter(prometheus.CounterOpts{ + }, []string{"status"}), + compactionRunJobTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "job_failed", - Help: "Number of jobs failed processing during the current compaction run", - }), + Name: "job_time_seconds", + Help: "Time spent processing jobs.", + Buckets: prometheus.DefBuckets, + }, []string{"status"}), compactionRunInterval: promauto.With(r).NewGauge(prometheus.GaugeOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, From 3df253fdb730f40c035e4b19b98a3c353e52f056 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Fri, 26 Jan 2024 14:52:39 +0100 Subject: [PATCH 18/23] Add test-target-branch stage to drone again (#11799) **What this PR does / why we need it**: Adding the stage back after removing it in https://github.com/grafana/loki/pull/11785 to fix the test failing in main. --- .drone/drone.jsonnet | 4 ++-- .drone/drone.yml | 13 ++++++++++++- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/.drone/drone.jsonnet b/.drone/drone.jsonnet index a6b1fbd9f9308..49f67f06861a3 100644 --- a/.drone/drone.jsonnet +++ b/.drone/drone.jsonnet @@ -610,13 +610,13 @@ local build_image_tag = '0.33.0'; 'cd -', ]) { depends_on: ['clone'], when: onPRs }, make('test', container=false) { depends_on: ['clone-target-branch', 'check-generated-files'] }, - //run('test-target-branch', commands=['cd ../loki-target-branch && BUILD_IN_CONTAINER=false make test']) { depends_on: ['clone-target-branch'], when: onPRs }, + run('test-target-branch', commands=['cd ../loki-target-branch && BUILD_IN_CONTAINER=false make test']) { depends_on: ['clone-target-branch'], when: onPRs }, make('compare-coverage', container=false, args=[ 'old=../loki-target-branch/test_results.txt', 'new=test_results.txt', 'packages=ingester,distributor,querier,querier/queryrange,iter,storage,chunkenc,logql,loki', '> diff.txt', - ]) { depends_on: ['test' /*, 'test-target-branch' */], when: onPRs }, + ]) { depends_on: ['test', 'test-target-branch'], when: onPRs }, run('report-coverage', commands=[ "total_diff=$(sed 's/%//' diff.txt | awk '{sum+=$3;}END{print sum;}')", 'if [ $total_diff = 0 ]; then exit 0; fi', diff --git a/.drone/drone.yml b/.drone/drone.yml index 5bc773dec971c..7a62b621262a8 100644 --- a/.drone/drone.yml +++ b/.drone/drone.yml @@ -212,12 +212,23 @@ steps: environment: {} image: grafana/loki-build-image:0.33.0 name: test +- commands: + - cd ../loki-target-branch && BUILD_IN_CONTAINER=false make test + depends_on: + - clone-target-branch + environment: {} + image: grafana/loki-build-image:0.33.0 + name: test-target-branch + when: + event: + - pull_request - commands: - make BUILD_IN_CONTAINER=false compare-coverage old=../loki-target-branch/test_results.txt new=test_results.txt packages=ingester,distributor,querier,querier/queryrange,iter,storage,chunkenc,logql,loki > diff.txt depends_on: - test + - test-target-branch environment: {} image: grafana/loki-build-image:0.33.0 name: compare-coverage @@ -2102,6 +2113,6 @@ kind: secret name: gpg_private_key --- kind: signature -hmac: 323c705d2c43805afcd46813fb9777c7d7d51f7d79aa018e7e78f208eb9f6713 +hmac: 457592d17208477ceb480f81dbdb88f7b95a5ad015c88d9d6fed06c2422a52f9 ... From 215b5fd2fd71574e454529b1b620a295f1323dac Mon Sep 17 00:00:00 2001 From: William Dumont Date: Fri, 26 Jan 2024 16:51:30 +0100 Subject: [PATCH 19/23] : Fix flaky docker test (#11777) **What this PR does / why we need it**: We ported this fix a few days ago to the Agent via this [PR](https://github.com/grafana/agent/pull/6201) but the added test is often failing in our drone pipeline. I believe that there are two reasons why this test is flaky: - the test expects that the 5 last lines of logs are the expected ones but it seems that other lines might be logged as well (https://drone.grafana.net/grafana/agent/16045/4/2) - we saw that the simulated container did not always have enough time to stop before calling the tgt.startIfNotRunning() to restart it. (https://drone.grafana.net/grafana/agent/16077/4/2) Fix: - the test now uses "assert.EventuallyWithT" and checks that within 5 seconds the expected log lines will be amongst the logs without duplicate. - the test now stops the simulated container before restarting it **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) --------- Co-authored-by: Michel Hollands <42814411+MichelHollands@users.noreply.github.com> --- .../promtail/targets/docker/target_test.go | 65 ++++++++++--------- 1 file changed, 36 insertions(+), 29 deletions(-) diff --git a/clients/pkg/promtail/targets/docker/target_test.go b/clients/pkg/promtail/targets/docker/target_test.go index e9bbf15b55bb1..27a22871e4541 100644 --- a/clients/pkg/promtail/targets/docker/target_test.go +++ b/clients/pkg/promtail/targets/docker/target_test.go @@ -5,7 +5,6 @@ import ( "net/http" "net/http/httptest" "os" - "sort" "strings" "testing" "time" @@ -17,6 +16,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/relabel" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/grafana/loki/clients/pkg/promtail/client/fake" @@ -77,15 +77,6 @@ func Test_DockerTarget(t *testing.T) { ) require.NoError(t, err) - require.Eventually(t, func() bool { - return len(entryHandler.Received()) >= 5 - }, 5*time.Second, 100*time.Millisecond) - - received := entryHandler.Received() - sort.Slice(received, func(i, j int) bool { - return received[i].Timestamp.Before(received[j].Timestamp) - }) - expectedLines := []string{ "5.3.69.55 - - [09/Dec/2021:09:15:02 +0000] \"HEAD /brand/users/clicks-and-mortar/front-end HTTP/2.0\" 503 27087", "101.54.183.185 - - [09/Dec/2021:09:15:03 +0000] \"POST /next-generation HTTP/1.0\" 416 11468", @@ -93,27 +84,15 @@ func Test_DockerTarget(t *testing.T) { "28.104.242.74 - - [09/Dec/2021:09:15:03 +0000] \"PATCH /value-added/cultivate/systems HTTP/2.0\" 405 11843", "150.187.51.54 - satterfield1852 [09/Dec/2021:09:15:03 +0000] \"GET /incentivize/deliver/innovative/cross-platform HTTP/1.1\" 301 13032", } - actualLines := make([]string, 0, 5) - for _, entry := range received[:5] { - actualLines = append(actualLines, entry.Line) - } - require.ElementsMatch(t, actualLines, expectedLines) + assert.EventuallyWithT(t, func(c *assert.CollectT) { + assertExpectedLog(c, entryHandler, expectedLines) + }, 5*time.Second, 100*time.Millisecond, "Expected log lines were not found within the time limit.") + + target.Stop() + entryHandler.Clear() // restart target to simulate container restart target.startIfNotRunning() - entryHandler.Clear() - require.Eventually(t, func() bool { - return len(entryHandler.Received()) >= 5 - }, 5*time.Second, 100*time.Millisecond) - - receivedAfterRestart := entryHandler.Received() - sort.Slice(receivedAfterRestart, func(i, j int) bool { - return receivedAfterRestart[i].Timestamp.Before(receivedAfterRestart[j].Timestamp) - }) - actualLinesAfterRestart := make([]string, 0, 5) - for _, entry := range receivedAfterRestart[:5] { - actualLinesAfterRestart = append(actualLinesAfterRestart, entry.Line) - } expectedLinesAfterRestart := []string{ "243.115.12.215 - - [09/Dec/2023:09:16:57 +0000] \"DELETE /morph/exploit/granular HTTP/1.0\" 500 26468", "221.41.123.237 - - [09/Dec/2023:09:16:57 +0000] \"DELETE /user-centric/whiteboard HTTP/2.0\" 205 22487", @@ -121,5 +100,33 @@ func Test_DockerTarget(t *testing.T) { "62.180.191.187 - - [09/Dec/2023:09:16:57 +0000] \"DELETE /cultivate/integrate/technologies HTTP/2.0\" 302 12979", "156.249.2.192 - - [09/Dec/2023:09:16:57 +0000] \"POST /revolutionize/mesh/metrics HTTP/2.0\" 401 5297", } - require.ElementsMatch(t, actualLinesAfterRestart, expectedLinesAfterRestart) + assert.EventuallyWithT(t, func(c *assert.CollectT) { + assertExpectedLog(c, entryHandler, expectedLinesAfterRestart) + }, 5*time.Second, 100*time.Millisecond, "Expected log lines after restart were not found within the time limit.") +} + +// assertExpectedLog will verify that all expectedLines were received, in any order, without duplicates. +func assertExpectedLog(c *assert.CollectT, entryHandler *fake.Client, expectedLines []string) { + logLines := entryHandler.Received() + testLogLines := make(map[string]int) + for _, l := range logLines { + if containsString(expectedLines, l.Line) { + testLogLines[l.Line]++ + } + } + // assert that all log lines were received + assert.Len(c, testLogLines, len(expectedLines)) + // assert that there are no duplicated log lines + for _, v := range testLogLines { + assert.Equal(c, v, 1) + } +} + +func containsString(slice []string, str string) bool { + for _, item := range slice { + if item == str { + return true + } + } + return false } From de4f56e42d14eb25f22a249aca04dd0736e88d15 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 26 Jan 2024 10:32:26 -0800 Subject: [PATCH 20/23] [Blooms] Refactoring bloom compactor to isolate state from logic (#11793) This PR refactors a bunch of code to separate state+I/O related complexity from logic so we can test and extend it more easily. It also adds more logic for the bloom generator to use. --- integration/loki_micro_services_test.go | 1 + pkg/bloomcompactor/bloomcompactor.go | 4 +- pkg/bloomcompactor/mergecompactor.go | 15 +- pkg/bloomcompactor/v2spec.go | 230 +++++++++++++++++-- pkg/bloomcompactor/v2spec_test.go | 11 + pkg/storage/bloom/v1/bloom_tokenizer.go | 135 +++++------ pkg/storage/bloom/v1/bloom_tokenizer_test.go | 70 +++--- pkg/storage/bloom/v1/metrics.go | 6 - pkg/storage/bloom/v1/tokenizer_test.go | 6 +- pkg/storage/bloom/v1/util.go | 4 +- 10 files changed, 334 insertions(+), 148 deletions(-) diff --git a/integration/loki_micro_services_test.go b/integration/loki_micro_services_test.go index 6d43052c37a2d..3b888314cd68c 100644 --- a/integration/loki_micro_services_test.go +++ b/integration/loki_micro_services_test.go @@ -1061,6 +1061,7 @@ func TestCategorizedLabels(t *testing.T) { } func TestBloomFiltersEndToEnd(t *testing.T) { + t.Skip("skipping until blooms have settled") commonFlags := []string{ "-bloom-compactor.compaction-interval=10s", "-bloom-compactor.enable-compaction=true", diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index 35dd19a22582a..2880b3e0ab719 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -549,7 +549,9 @@ func (c *Compactor) runCompact(ctx context.Context, logger log.Logger, job Job, return err } - resultingBlock, err = compactNewChunks(ctx, logger, job, bt, storeClient.chunk, builder, c.limits) + // NB(owen-d): this panics/etc, but the code is being refactored and will be removed. I've replaced `bt` with `nil` + // to pass compiler checks while keeping this code around as reference + resultingBlock, err = compactNewChunks(ctx, logger, job, nil, storeClient.chunk, builder, c.limits) if err != nil { return level.Error(logger).Log("msg", "failed compacting new chunks", "err", err) } diff --git a/pkg/bloomcompactor/mergecompactor.go b/pkg/bloomcompactor/mergecompactor.go index 6e2143f75135c..3486e40846b8a 100644 --- a/pkg/bloomcompactor/mergecompactor.go +++ b/pkg/bloomcompactor/mergecompactor.go @@ -2,7 +2,6 @@ package bloomcompactor import ( "context" - "fmt" "github.com/grafana/dskit/concurrency" @@ -75,7 +74,7 @@ func makeBlockIterFromBlocks(ctx context.Context, logger log.Logger, return blockIters, blockPaths, nil } -func createPopulateFunc(ctx context.Context, job Job, storeClient storeClient, bt *v1.BloomTokenizer, limits Limits) func(series *v1.Series, bloom *v1.Bloom) error { +func createPopulateFunc(_ context.Context, job Job, _ storeClient, bt *v1.BloomTokenizer, _ Limits) func(series *v1.Series, bloom *v1.Bloom) error { return func(series *v1.Series, bloom *v1.Bloom) error { bloomForChks := v1.SeriesWithBloom{ Series: series, @@ -96,11 +95,13 @@ func createPopulateFunc(ctx context.Context, job Job, storeClient storeClient, b } } - batchesIterator, err := newChunkBatchesIterator(ctx, storeClient.chunk, chunkRefs, limits.BloomCompactorChunksBatchSize(job.tenantID)) - if err != nil { - return fmt.Errorf("error creating chunks batches iterator: %w", err) - } - err = bt.PopulateSeriesWithBloom(&bloomForChks, batchesIterator) + // batchesIterator, err := newChunkBatchesIterator(ctx, storeClient.chunk, chunkRefs, limits.BloomCompactorChunksBatchSize(job.tenantID)) + // if err != nil { + // return fmt.Errorf("error creating chunks batches iterator: %w", err) + // } + // NB(owen-d): this panics/etc, but the code is being refactored and will be removed. + // I've replaced `batchesIterator` with `emptyIter` to pass compiler checks while keeping this code around as reference + err := bt.Populate(&bloomForChks, v1.NewEmptyIter[v1.ChunkRefWithIter]()) if err != nil { return err } diff --git a/pkg/bloomcompactor/v2spec.go b/pkg/bloomcompactor/v2spec.go index 9f7174b0f2831..334b79f2ae746 100644 --- a/pkg/bloomcompactor/v2spec.go +++ b/pkg/bloomcompactor/v2spec.go @@ -3,25 +3,45 @@ package bloomcompactor import ( "context" "fmt" + "math" + "time" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/common/model" + "github.com/grafana/loki/pkg/chunkenc" + "github.com/grafana/loki/pkg/logproto" + logql_log "github.com/grafana/loki/pkg/logql/log" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" ) -// TODO(owen-d): add metrics +/* +This file maintains a number of things supporting bloom generation. Most notably, the `BloomGenerator` interface/implementation which builds bloom filters. + +- `BloomGenerator`: Builds blooms. Most other things in this file are supporting this in various ways. +- `SimpleBloomGenerator`: A foundational implementation of `BloomGenerator` which wires up a few different components to generate bloom filters for a set of blocks and handles schema compatibility: +- `chunkLoader`: Loads chunks w/ a specific fingerprint from the store, returns an iterator of chunk iterators. We return iterators rather than chunk implementations mainly for ease of testing. In practice, this will just be an iterator over `MemChunk`s. +*/ + type Metrics struct { bloomMetrics *v1.Metrics + chunkSize prometheus.Histogram // uncompressed size of all chunks summed per series } -func NewMetrics(_ prometheus.Registerer, bloomMetrics *v1.Metrics) *Metrics { +func NewMetrics(r prometheus.Registerer, bloomMetrics *v1.Metrics) *Metrics { return &Metrics{ bloomMetrics: bloomMetrics, + chunkSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Name: "bloom_chunk_series_size", + Help: "Uncompressed size of chunks in a series", + Buckets: prometheus.ExponentialBucketsRange(1024, 1073741824, 10), + }), } } @@ -47,7 +67,8 @@ type BloomGenerator interface { // Simple implementation of a BloomGenerator. type SimpleBloomGenerator struct { - store v1.Iterator[*v1.Series] + store v1.Iterator[*v1.Series] + chunkLoader ChunkLoader // TODO(owen-d): blocks need not be all downloaded prior. Consider implementing // as an iterator of iterators, where each iterator is a batch of overlapping blocks. blocks []*v1.Block @@ -70,14 +91,17 @@ type SimpleBloomGenerator struct { func NewSimpleBloomGenerator( opts v1.BlockOptions, store v1.Iterator[*v1.Series], + chunkLoader ChunkLoader, blocks []*v1.Block, readWriterFn func() (v1.BlockWriter, v1.BlockReader), metrics *Metrics, logger log.Logger, ) *SimpleBloomGenerator { return &SimpleBloomGenerator{ - opts: opts, + opts: opts, + // TODO(owen-d): implement Iterator[Series] against TSDB files to hook in here. store: store, + chunkLoader: chunkLoader, blocks: blocks, logger: logger, readWriterFn: readWriterFn, @@ -87,20 +111,25 @@ func NewSimpleBloomGenerator( } } -func (s *SimpleBloomGenerator) populate(series *v1.Series, bloom *v1.Bloom) error { - // TODO(owen-d): impl after threading in store - var chunkItr v1.Iterator[[]chunk.Chunk] = v1.NewEmptyIter[[]chunk.Chunk](nil) +func (s *SimpleBloomGenerator) populator(ctx context.Context) func(series *v1.Series, bloom *v1.Bloom) error { + return func(series *v1.Series, bloom *v1.Bloom) error { + chunkItersWithFP, err := s.chunkLoader.Load(ctx, series) + if err != nil { + return errors.Wrapf(err, "failed to load chunks for series: %#v", series) + } + + return s.tokenizer.Populate( + &v1.SeriesWithBloom{ + Series: series, + Bloom: bloom, + }, + chunkItersWithFP.itr, + ) + } - return s.tokenizer.PopulateSeriesWithBloom( - &v1.SeriesWithBloom{ - Series: series, - Bloom: bloom, - }, - chunkItr, - ) } -func (s *SimpleBloomGenerator) Generate(_ context.Context) (skippedBlocks []*v1.Block, results v1.Iterator[*v1.Block], err error) { +func (s *SimpleBloomGenerator) Generate(ctx context.Context) (skippedBlocks []*v1.Block, results v1.Iterator[*v1.Block], err error) { blocksMatchingSchema := make([]v1.PeekingIterator[*v1.SeriesWithBloom], 0, len(s.blocks)) for _, block := range s.blocks { @@ -126,7 +155,7 @@ func (s *SimpleBloomGenerator) Generate(_ context.Context) (skippedBlocks []*v1. // TODO(owen-d): implement bounded block sizes - mergeBuilder := v1.NewMergeBuilder(blocksMatchingSchema, s.store, s.populate) + mergeBuilder := v1.NewMergeBuilder(blocksMatchingSchema, s.store, s.populator(ctx)) writer, reader := s.readWriterFn() blockBuilder, err := v1.NewBlockBuilder(v1.NewBlockOptionsFromSchema(s.opts.Schema), writer) if err != nil { @@ -140,3 +169,172 @@ func (s *SimpleBloomGenerator) Generate(_ context.Context) (skippedBlocks []*v1. return skippedBlocks, v1.NewSliceIter[*v1.Block]([]*v1.Block{v1.NewBlock(reader)}), nil } + +// IndexLoader loads an index. This helps us do things like +// load TSDBs for a specific period excluding multitenant (pre-compacted) indices +type indexLoader interface { + Index() (tsdb.Index, error) +} + +// ChunkItersByFingerprint models the chunks belonging to a fingerprint +type ChunkItersByFingerprint struct { + fp model.Fingerprint + itr v1.Iterator[v1.ChunkRefWithIter] +} + +// ChunkLoader loads chunks from a store +type ChunkLoader interface { + Load(context.Context, *v1.Series) (*ChunkItersByFingerprint, error) +} + +// interface modeled from `pkg/storage/stores/composite_store.ChunkFetcherProvider` +type fetcherProvider interface { + GetChunkFetcher(model.Time) chunkFetcher +} + +// interface modeled from `pkg/storage/chunk/fetcher.Fetcher` +type chunkFetcher interface { + FetchChunks(ctx context.Context, chunks []chunk.Chunk) ([]chunk.Chunk, error) +} + +// StoreChunkLoader loads chunks from a store +type StoreChunkLoader struct { + userID string + fetcherProvider fetcherProvider + metrics *Metrics +} + +func NewStoreChunkLoader(userID string, fetcherProvider fetcherProvider, metrics *Metrics) *StoreChunkLoader { + return &StoreChunkLoader{ + userID: userID, + fetcherProvider: fetcherProvider, + metrics: metrics, + } +} + +func (s *StoreChunkLoader) Load(ctx context.Context, series *v1.Series) (*ChunkItersByFingerprint, error) { + // TODO(owen-d): This is probalby unnecessary as we should only have one fetcher + // because we'll only be working on a single index period at a time, but this should protect + // us in the case of refactoring/changing this and likely isn't a perf bottleneck. + chksByFetcher := make(map[chunkFetcher][]chunk.Chunk) + for _, chk := range series.Chunks { + fetcher := s.fetcherProvider.GetChunkFetcher(chk.Start) + chksByFetcher[fetcher] = append(chksByFetcher[fetcher], chunk.Chunk{ + ChunkRef: logproto.ChunkRef{ + Fingerprint: uint64(series.Fingerprint), + UserID: s.userID, + From: chk.Start, + Through: chk.End, + Checksum: chk.Checksum, + }, + }) + } + + work := make([]chunkWork, 0, len(chksByFetcher)) + for fetcher, chks := range chksByFetcher { + work = append(work, chunkWork{ + fetcher: fetcher, + chks: chks, + }) + } + + return &ChunkItersByFingerprint{ + fp: series.Fingerprint, + itr: newBatchedLoader(ctx, work, batchedLoaderDefaultBatchSize, s.metrics), + }, nil +} + +type chunkWork struct { + fetcher chunkFetcher + chks []chunk.Chunk +} + +// batchedLoader implements `v1.Iterator[v1.ChunkRefWithIter]` in batches +// to ensure memory is bounded while loading chunks +// TODO(owen-d): testware +type batchedLoader struct { + metrics *Metrics + batchSize int + ctx context.Context + work []chunkWork + + cur v1.ChunkRefWithIter + batch []chunk.Chunk + err error +} + +const batchedLoaderDefaultBatchSize = 50 + +func newBatchedLoader(ctx context.Context, work []chunkWork, batchSize int, metrics *Metrics) *batchedLoader { + return &batchedLoader{ + metrics: metrics, + batchSize: batchSize, + ctx: ctx, + work: work, + } +} + +func (b *batchedLoader) Next() bool { + if len(b.batch) > 0 { + b.cur, b.err = b.format(b.batch[0]) + b.batch = b.batch[1:] + return b.err == nil + } + + if len(b.work) == 0 { + return false + } + + // setup next batch + next := b.work[0] + batchSize := min(b.batchSize, len(next.chks)) + toFetch := next.chks[:batchSize] + // update work + b.work[0].chks = next.chks[batchSize:] + if len(b.work[0].chks) == 0 { + b.work = b.work[1:] + } + + b.batch, b.err = next.fetcher.FetchChunks(b.ctx, toFetch) + return b.err == nil +} + +func (b *batchedLoader) format(c chunk.Chunk) (v1.ChunkRefWithIter, error) { + chk := c.Data.(*chunkenc.Facade).LokiChunk() + b.metrics.chunkSize.Observe(float64(chk.UncompressedSize())) + itr, err := chk.Iterator( + b.ctx, + time.Unix(0, 0), // TODO: Parameterize/better handle the timestamps? + time.Unix(0, math.MaxInt64), + logproto.FORWARD, + logql_log.NewNoopPipeline().ForStream(c.Metric), + ) + + if err != nil { + return v1.ChunkRefWithIter{}, err + } + + return v1.ChunkRefWithIter{ + Ref: v1.ChunkRef{ + Start: c.From, + End: c.Through, + Checksum: c.Checksum, + }, + Itr: itr, + }, nil +} + +func (b *batchedLoader) At() v1.ChunkRefWithIter { + return b.cur +} + +func (b *batchedLoader) Err() error { + return b.err +} + +func min(a, b int) int { + if a < b { + return a + } + return b +} diff --git a/pkg/bloomcompactor/v2spec_test.go b/pkg/bloomcompactor/v2spec_test.go index f5cdfc0580c3b..08c722d06e5d4 100644 --- a/pkg/bloomcompactor/v2spec_test.go +++ b/pkg/bloomcompactor/v2spec_test.go @@ -52,10 +52,21 @@ func blocksFromSchemaWithRange(t *testing.T, n int, options v1.BlockOptions, fro return res, data } +// doesn't actually load any chunks +type dummyChunkLoader struct{} + +func (dummyChunkLoader) Load(_ context.Context, series *v1.Series) (*ChunkItersByFingerprint, error) { + return &ChunkItersByFingerprint{ + fp: series.Fingerprint, + itr: v1.NewEmptyIter[v1.ChunkRefWithIter](), + }, nil +} + func dummyBloomGen(opts v1.BlockOptions, store v1.Iterator[*v1.Series], blocks []*v1.Block) *SimpleBloomGenerator { return NewSimpleBloomGenerator( opts, store, + dummyChunkLoader{}, blocks, func() (v1.BlockWriter, v1.BlockReader) { indexBuf := bytes.NewBuffer(nil) diff --git a/pkg/storage/bloom/v1/bloom_tokenizer.go b/pkg/storage/bloom/v1/bloom_tokenizer.go index 6008c6d1a65a7..c9ff6f23cc0f1 100644 --- a/pkg/storage/bloom/v1/bloom_tokenizer.go +++ b/pkg/storage/bloom/v1/bloom_tokenizer.go @@ -1,18 +1,14 @@ package v1 import ( - "context" "fmt" "math" "time" "github.com/go-kit/log/level" - "github.com/grafana/loki/pkg/chunkenc" - "github.com/grafana/loki/pkg/logproto" - "github.com/grafana/loki/pkg/logql/log" + "github.com/grafana/loki/pkg/iter" - "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/util/encoding" util_log "github.com/grafana/loki/pkg/util/log" ) @@ -65,10 +61,10 @@ func clearCache(cache map[string]interface{}) { // of specific ngram length, along with the length of the prefix. // It ensures enough capacity for the prefix and the token so additional tokens can be created // without allocations by appending them to the prefix length -func prefixedToken(ngram int, chk logproto.ChunkRef) ([]byte, int) { +func prefixedToken(ngram int, chk ChunkRef) ([]byte, int) { var enc encoding.Encbuf - enc.PutBE64(uint64(chk.From)) - enc.PutBE64(uint64(chk.Through)) + enc.PutBE64(uint64(chk.Start)) + enc.PutBE64(uint64(chk.End)) enc.PutBE32(chk.Checksum) prefixLn := enc.Len() // record the length of the prefix @@ -78,94 +74,85 @@ func prefixedToken(ngram int, chk logproto.ChunkRef) ([]byte, int) { return enc.Get(), prefixLn } -// PopulateSeriesWithBloom is intended to be called on the write path, and is used to populate the bloom filter for a given series. -func (bt *BloomTokenizer) PopulateSeriesWithBloom(seriesWithBloom *SeriesWithBloom, chunks Iterator[[]chunk.Chunk]) error { +// ChunkRefWithIter is a wrapper around a ChunkRef and an EntryIterator. +type ChunkRefWithIter struct { + Ref ChunkRef + Itr iter.EntryIterator +} + +// Populate adds the tokens from the given chunks to the given seriesWithBloom. +func (bt *BloomTokenizer) Populate(swb *SeriesWithBloom, chks Iterator[ChunkRefWithIter]) error { startTime := time.Now().UnixMilli() - level.Debug(util_log.Logger).Log("msg", "PopulateSeriesWithBloom") clearCache(bt.cache) - chunkTotalUncompressedSize := 0 - - for chunks.Next() { - chunksBatch := chunks.At() - for idx := range chunksBatch { - lc := chunksBatch[idx].Data.(*chunkenc.Facade).LokiChunk() - tokenBuf, prefixLn := prefixedToken(bt.lineTokenizer.N, chunksBatch[idx].ChunkRef) - chunkTotalUncompressedSize += lc.UncompressedSize() - - itr, err := lc.Iterator( - context.Background(), - time.Unix(0, 0), // TODO: Parameterize/better handle the timestamps? - time.Unix(0, math.MaxInt64), - logproto.FORWARD, - log.NewNoopPipeline().ForStream(chunksBatch[idx].Metric), - ) - if err != nil { - level.Error(util_log.Logger).Log("msg", "chunk iterator cannot be created", "err", err) - return err - } - - defer itr.Close() - - for itr.Next() && itr.Error() == nil { - chunkTokenizer := NewPrefixedTokenIter(tokenBuf, prefixLn, bt.lineTokenizer.Tokens(itr.Entry().Line)) - for chunkTokenizer.Next() { - tok := chunkTokenizer.At() - if tok != nil { - str := string(tok) - _, found := bt.cache[str] // A cache is used ahead of the SBF, as it cuts out the costly operations of scaling bloom filters - if !found { - bt.cache[str] = nil - seriesWithBloom.Bloom.ScalableBloomFilter.TestAndAdd(tok) - - if len(bt.cache) >= cacheSize { // While crude, this has proven efficient in performance testing. This speaks to the similarity in log lines near each other - clearCache(bt.cache) - } + for chks.Err() == nil && chks.Next() { + chk := chks.At() + itr := chk.Itr + tokenBuf, prefixLn := prefixedToken(bt.lineTokenizer.N, chk.Ref) + + defer itr.Close() + + for itr.Next() && itr.Error() == nil { + // TODO(owen-d): rather than iterate over the line twice, once for prefixed tokenizer & once for + // raw tokenizer, we could iterate once and just return (prefix, token) pairs from the tokenizer. + // Double points for them being different-ln references to the same data. + chunkTokenizer := NewPrefixedTokenIter(tokenBuf, prefixLn, bt.lineTokenizer.Tokens(itr.Entry().Line)) + for chunkTokenizer.Next() { + tok := chunkTokenizer.At() + if tok != nil { + // TODO(owen-d): unsafe this? + str := string(tok) + _, found := bt.cache[str] // A cache is used ahead of the SBF, as it cuts out the costly operations of scaling bloom filters + if !found { + bt.cache[str] = nil + + swb.Bloom.ScalableBloomFilter.TestAndAdd(tok) + + if len(bt.cache) >= cacheSize { // While crude, this has proven efficient in performance testing. This speaks to the similarity in log lines near each other + clearCache(bt.cache) } } } - lineTokenizer := bt.lineTokenizer.Tokens(itr.Entry().Line) - for lineTokenizer.Next() { - tok := lineTokenizer.At() - if tok != nil { - str := string(tok) - _, found := bt.cache[str] // A cache is used ahead of the SBF, as it cuts out the costly operations of scaling bloom filters - if !found { - bt.cache[str] = nil - - seriesWithBloom.Bloom.ScalableBloomFilter.TestAndAdd(tok) - - if len(bt.cache) >= cacheSize { // While crude, this has proven efficient in performance testing. This speaks to the similarity in log lines near each other - clearCache(bt.cache) - } + } + lineTokenizer := bt.lineTokenizer.Tokens(itr.Entry().Line) + for lineTokenizer.Next() { + tok := lineTokenizer.At() + if tok != nil { + str := string(tok) + _, found := bt.cache[str] // A cache is used ahead of the SBF, as it cuts out the costly operations of scaling bloom filters + if !found { + bt.cache[str] = nil + + swb.Bloom.ScalableBloomFilter.TestAndAdd(tok) + + if len(bt.cache) >= cacheSize { // While crude, this has proven efficient in performance testing. This speaks to the similarity in log lines near each other + clearCache(bt.cache) } } } - } - seriesWithBloom.Series.Chunks = append(seriesWithBloom.Series.Chunks, ChunkRef{ - Start: chunksBatch[idx].From, - End: chunksBatch[idx].Through, - Checksum: chunksBatch[idx].Checksum, - }) - } // for each chunk + + } + if err := itr.Error(); err != nil { + return fmt.Errorf("error iterating chunk: %#v, %w", chk.Ref, err) + } + swb.Series.Chunks = append(swb.Series.Chunks, chk.Ref) } - if err := chunks.Err(); err != nil { + if err := chks.Err(); err != nil { level.Error(util_log.Logger).Log("msg", "error downloading chunks batch", "err", err) return fmt.Errorf("error downloading chunks batch: %w", err) } endTime := time.Now().UnixMilli() - fillRatio := seriesWithBloom.Bloom.ScalableBloomFilter.FillRatio() + fillRatio := swb.Bloom.ScalableBloomFilter.FillRatio() bt.metrics.hammingWeightRatio.Observe(fillRatio) bt.metrics.estimatedCount.Observe( - float64(estimatedCount(seriesWithBloom.Bloom.ScalableBloomFilter.Capacity(), fillRatio)), + float64(estimatedCount(swb.Bloom.ScalableBloomFilter.Capacity(), fillRatio)), ) - bt.metrics.bloomSize.Observe(float64(seriesWithBloom.Bloom.ScalableBloomFilter.Capacity() / eightBits)) + bt.metrics.bloomSize.Observe(float64(swb.Bloom.ScalableBloomFilter.Capacity() / eightBits)) bt.metrics.sbfCreationTime.Add(float64(endTime - startTime)) - bt.metrics.chunkSize.Observe(float64(chunkTotalUncompressedSize)) return nil } diff --git a/pkg/storage/bloom/v1/bloom_tokenizer_test.go b/pkg/storage/bloom/v1/bloom_tokenizer_test.go index a1d09bab36b93..c2b0481e2db58 100644 --- a/pkg/storage/bloom/v1/bloom_tokenizer_test.go +++ b/pkg/storage/bloom/v1/bloom_tokenizer_test.go @@ -1,7 +1,9 @@ package v1 import ( + "context" "fmt" + "math" "testing" "time" @@ -9,8 +11,8 @@ import ( "github.com/grafana/loki/pkg/chunkenc" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql/log" "github.com/grafana/loki/pkg/push" - "github.com/grafana/loki/pkg/storage/chunk" "github.com/prometheus/common/model" "github.com/stretchr/testify/require" @@ -33,9 +35,9 @@ var ( func TestPrefixedKeyCreation(t *testing.T) { var ones uint64 = 0xffffffffffffffff - ref := logproto.ChunkRef{ - From: 0, - Through: model.Time(int64(ones)), + ref := ChunkRef{ + Start: 0, + End: model.Time(int64(ones)), Checksum: 0xffffffff, } for _, tc := range []struct { @@ -86,7 +88,7 @@ func TestSetLineTokenizer(t *testing.T) { require.Equal(t, bt.lineTokenizer.Skip, 7) } -func TestPopulateSeriesWithBloom(t *testing.T) { +func TestTokenizerPopulate(t *testing.T) { var testLine = "this is a log line" bt := NewBloomTokenizer(DefaultNGramLength, DefaultNGramSkip, metrics) @@ -94,23 +96,19 @@ func TestPopulateSeriesWithBloom(t *testing.T) { var lbsList []labels.Labels lbsList = append(lbsList, labels.FromStrings("foo", "bar")) - var fpList []model.Fingerprint - for i := range lbsList { - fpList = append(fpList, model.Fingerprint(lbsList[i].Hash())) - } - - var memChunks = make([]*chunkenc.MemChunk, 0) - memChunk0 := chunkenc.NewMemChunk(chunkenc.ChunkFormatV4, chunkenc.EncSnappy, chunkenc.ChunkHeadFormatFor(chunkenc.ChunkFormatV4), 256000, 1500000) - _ = memChunk0.Append(&push.Entry{ + memChunk := chunkenc.NewMemChunk(chunkenc.ChunkFormatV4, chunkenc.EncSnappy, chunkenc.ChunkHeadFormatFor(chunkenc.ChunkFormatV4), 256000, 1500000) + _ = memChunk.Append(&push.Entry{ Timestamp: time.Unix(0, 1), Line: testLine, }) - memChunks = append(memChunks, memChunk0) - - var chunks = make([]chunk.Chunk, 0) - for i := range memChunks { - chunks = append(chunks, chunk.NewChunk("user", fpList[i], lbsList[i], chunkenc.NewFacade(memChunks[i], 256000, 1500000), model.TimeFromUnixNano(0), model.TimeFromUnixNano(1))) - } + itr, err := memChunk.Iterator( + context.Background(), + time.Unix(0, 0), // TODO: Parameterize/better handle the timestamps? + time.Unix(0, math.MaxInt64), + logproto.FORWARD, + log.NewNoopPipeline().ForStream(nil), + ) + require.Nil(t, err) bloom := Bloom{ ScalableBloomFilter: *sbf, @@ -123,12 +121,12 @@ func TestPopulateSeriesWithBloom(t *testing.T) { Series: &series, } - err := bt.PopulateSeriesWithBloom(&swb, NewSliceIter([][]chunk.Chunk{chunks})) + err = bt.Populate(&swb, NewSliceIter([]ChunkRefWithIter{{Ref: ChunkRef{}, Itr: itr}})) require.NoError(t, err) tokenizer := NewNGramTokenizer(DefaultNGramLength, DefaultNGramSkip) - itr := tokenizer.Tokens(testLine) - for itr.Next() { - token := itr.At() + toks := tokenizer.Tokens(testLine) + for toks.Next() { + token := toks.At() require.True(t, swb.Bloom.Test(token)) } } @@ -142,23 +140,19 @@ func BenchmarkPopulateSeriesWithBloom(b *testing.B) { var lbsList []labels.Labels lbsList = append(lbsList, labels.FromStrings("foo", "bar")) - var fpList []model.Fingerprint - for i := range lbsList { - fpList = append(fpList, model.Fingerprint(lbsList[i].Hash())) - } - - var memChunks = make([]*chunkenc.MemChunk, 0) - memChunk0 := chunkenc.NewMemChunk(chunkenc.ChunkFormatV4, chunkenc.EncSnappy, chunkenc.ChunkHeadFormatFor(chunkenc.ChunkFormatV4), 256000, 1500000) - _ = memChunk0.Append(&push.Entry{ + memChunk := chunkenc.NewMemChunk(chunkenc.ChunkFormatV4, chunkenc.EncSnappy, chunkenc.ChunkHeadFormatFor(chunkenc.ChunkFormatV4), 256000, 1500000) + _ = memChunk.Append(&push.Entry{ Timestamp: time.Unix(0, 1), Line: testLine, }) - memChunks = append(memChunks, memChunk0) - - var chunks = make([]chunk.Chunk, 0) - for i := range memChunks { - chunks = append(chunks, chunk.NewChunk("user", fpList[i], lbsList[i], chunkenc.NewFacade(memChunks[i], 256000, 1500000), model.TimeFromUnixNano(0), model.TimeFromUnixNano(1))) - } + itr, err := memChunk.Iterator( + context.Background(), + time.Unix(0, 0), // TODO: Parameterize/better handle the timestamps? + time.Unix(0, math.MaxInt64), + logproto.FORWARD, + log.NewNoopPipeline().ForStream(nil), + ) + require.Nil(b, err) bloom := Bloom{ ScalableBloomFilter: *sbf, @@ -171,7 +165,7 @@ func BenchmarkPopulateSeriesWithBloom(b *testing.B) { Series: &series, } - err := bt.PopulateSeriesWithBloom(&swb, NewSliceIter([][]chunk.Chunk{chunks})) + err = bt.Populate(&swb, NewSliceIter([]ChunkRefWithIter{{Ref: ChunkRef{}, Itr: itr}})) require.NoError(b, err) } } diff --git a/pkg/storage/bloom/v1/metrics.go b/pkg/storage/bloom/v1/metrics.go index d73af79f61b3c..aa604c29f1573 100644 --- a/pkg/storage/bloom/v1/metrics.go +++ b/pkg/storage/bloom/v1/metrics.go @@ -7,7 +7,6 @@ import ( type Metrics struct { sbfCreationTime prometheus.Counter // time spent creating sbfs - chunkSize prometheus.Histogram // uncompressed size of all chunks summed per series bloomSize prometheus.Histogram // size of the bloom filter in bytes hammingWeightRatio prometheus.Histogram // ratio of the hamming weight of the bloom filter to the number of bits in the bloom filter estimatedCount prometheus.Histogram // estimated number of elements in the bloom filter @@ -19,11 +18,6 @@ func NewMetrics(r prometheus.Registerer) *Metrics { Name: "bloom_creation_time", Help: "Time spent creating scalable bloom filters", }), - chunkSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Name: "bloom_chunk_series_size", - Help: "Uncompressed size of chunks in a series", - Buckets: prometheus.ExponentialBucketsRange(1024, 1073741824, 10), - }), bloomSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ Name: "bloom_size", Help: "Size of the bloom filter in bytes", diff --git a/pkg/storage/bloom/v1/tokenizer_test.go b/pkg/storage/bloom/v1/tokenizer_test.go index 471eaea74081b..b70d9610fab47 100644 --- a/pkg/storage/bloom/v1/tokenizer_test.go +++ b/pkg/storage/bloom/v1/tokenizer_test.go @@ -4,8 +4,6 @@ import ( "testing" "github.com/stretchr/testify/require" - - "github.com/grafana/loki/pkg/logproto" ) const BigFile = "../../../logql/sketch/testdata/war_peace.txt" @@ -173,7 +171,7 @@ func BenchmarkTokens(b *testing.B) { { desc: "v2", f: func() func() { - buf, prefixLn := prefixedToken(v2Three.N, logproto.ChunkRef{}) + buf, prefixLn := prefixedToken(v2Three.N, ChunkRef{}) return func() { itr := NewPrefixedTokenIter(buf, prefixLn, v2Three.Tokens(lorem)) for itr.Next() { @@ -190,7 +188,7 @@ func BenchmarkTokens(b *testing.B) { { desc: "v2", f: func() func() { - buf, prefixLn := prefixedToken(v2Three.N, logproto.ChunkRef{}) + buf, prefixLn := prefixedToken(v2Three.N, ChunkRef{}) return func() { itr := NewPrefixedTokenIter(buf, prefixLn, v2ThreeSkip1.Tokens(lorem)) for itr.Next() { diff --git a/pkg/storage/bloom/v1/util.go b/pkg/storage/bloom/v1/util.go index 15de62e9f9590..5a7046b5d0477 100644 --- a/pkg/storage/bloom/v1/util.go +++ b/pkg/storage/bloom/v1/util.go @@ -201,8 +201,8 @@ func (it *EmptyIter[T]) At() T { // noop func (it *EmptyIter[T]) Reset() {} -func NewEmptyIter[T any](zero T) *EmptyIter[T] { - return &EmptyIter[T]{zero: zero} +func NewEmptyIter[T any]() *EmptyIter[T] { + return &EmptyIter[T]{} } type CancellableIter[T any] struct { From fe4ba0c3ba4c611440bae9a82556a105480d3f8a Mon Sep 17 00:00:00 2001 From: Joao Marcal Date: Fri, 26 Jan 2024 19:33:27 +0100 Subject: [PATCH 21/23] operator: Add OpenShift CloudCredentials support for AWS STS (#11524) Co-authored-by: Periklis Tsirakidis --- operator/CHANGELOG.md | 1 + .../apis/config/v1/projectconfig_types.go | 7 + operator/apis/loki/v1/lokistack_types.go | 6 + .../loki-operator.clusterserviceversion.yaml | 12 +- .../loki-operator.clusterserviceversion.yaml | 10 ++ .../loki-operator.clusterserviceversion.yaml | 12 +- .../loki-operator.clusterserviceversion.yaml | 2 +- .../loki-operator.clusterserviceversion.yaml | 2 +- operator/config/rbac/role.yaml | 10 ++ .../loki/credentialsrequests_controller.go | 71 ++++++++ .../credentialsrequests_controller_test.go | 155 +++++++++++++++++ .../lokistack/credentialsrequest_discovery.go | 30 ++++ .../credentialsrequest_discovery_test.go | 98 +++++++++++ .../controllers/loki/lokistack_controller.go | 65 +++++-- .../loki/lokistack_controller_test.go | 20 ++- operator/docs/operator/api.md | 8 + operator/docs/operator/feature-gates.md | 11 ++ operator/go.mod | 16 +- operator/go.sum | 37 ++-- operator/hack/deploy-aws-storage-secret.sh | 46 ++++- .../handlers/credentialsrequest_create.go | 42 +++++ .../credentialsrequest_create_test.go | 50 ++++++ .../handlers/credentialsrequest_delete.go | 43 +++++ .../credentialsrequest_delete_test.go | 47 ++++++ .../handlers/internal/storage/secrets.go | 102 +++++++++-- .../handlers/internal/storage/secrets_test.go | 85 +++++++++- .../handlers/internal/storage/storage.go | 7 +- .../handlers/internal/storage/storage_test.go | 159 ++++++++++++++++++ operator/internal/manifests/compactor.go | 2 +- operator/internal/manifests/distributor.go | 2 +- operator/internal/manifests/gateway.go | 2 +- operator/internal/manifests/indexgateway.go | 2 +- operator/internal/manifests/ingester.go | 2 +- .../manifests/openshift/credentialsrequest.go | 96 +++++++++++ .../openshift/credentialsrequest_test.go | 119 +++++++++++++ .../internal/manifests/openshift/options.go | 9 + operator/internal/manifests/openshift/var.go | 2 + operator/internal/manifests/querier.go | 2 +- operator/internal/manifests/query-frontend.go | 2 +- operator/internal/manifests/ruler.go | 2 +- .../internal/manifests/storage/configure.go | 44 ++++- .../manifests/storage/configure_test.go | 145 ++++++++++++++-- .../internal/manifests/storage/options.go | 23 ++- operator/internal/manifests/storage/var.go | 19 ++- operator/internal/manifests/var.go | 22 ++- operator/main.go | 19 +++ 46 files changed, 1549 insertions(+), 119 deletions(-) create mode 100644 operator/controllers/loki/credentialsrequests_controller.go create mode 100644 operator/controllers/loki/credentialsrequests_controller_test.go create mode 100644 operator/controllers/loki/internal/lokistack/credentialsrequest_discovery.go create mode 100644 operator/controllers/loki/internal/lokistack/credentialsrequest_discovery_test.go create mode 100644 operator/internal/handlers/credentialsrequest_create.go create mode 100644 operator/internal/handlers/credentialsrequest_create_test.go create mode 100644 operator/internal/handlers/credentialsrequest_delete.go create mode 100644 operator/internal/handlers/credentialsrequest_delete_test.go create mode 100644 operator/internal/manifests/openshift/credentialsrequest.go create mode 100644 operator/internal/manifests/openshift/credentialsrequest_test.go diff --git a/operator/CHANGELOG.md b/operator/CHANGELOG.md index 2f57c42a78d71..039e37a32297f 100644 --- a/operator/CHANGELOG.md +++ b/operator/CHANGELOG.md @@ -1,5 +1,6 @@ ## Main +- [11524](https://github.com/grafana/loki/pull/11524) **JoaoBraveCoding**, **periklis**: Add OpenShift cloud credentials support for AWS STS - [11513](https://github.com/grafana/loki/pull/11513) **btaani**: Add a custom metric that collects Lokistacks requiring a schema upgrade - [11718](https://github.com/grafana/loki/pull/11718) **periklis**: Upgrade k8s.io, sigs.k8s.io and openshift deps - [11671](https://github.com/grafana/loki/pull/11671) **JoaoBraveCoding**: Update mixins to fix structured metadata dashboards diff --git a/operator/apis/config/v1/projectconfig_types.go b/operator/apis/config/v1/projectconfig_types.go index 488f7b2cb64f3..ba7cc703c5bb8 100644 --- a/operator/apis/config/v1/projectconfig_types.go +++ b/operator/apis/config/v1/projectconfig_types.go @@ -51,6 +51,13 @@ type OpenShiftFeatureGates struct { // Dashboards enables the loki-mixin dashboards into the OpenShift Console Dashboards bool `json:"dashboards,omitempty"` + + // ManagedAuthEnv enabled when the operator installation is on OpenShift STS clusters. + ManagedAuthEnv bool +} + +func (o OpenShiftFeatureGates) ManagedAuthEnabled() bool { + return o.Enabled && o.ManagedAuthEnv } // FeatureGates is the supported set of all operator feature gates. diff --git a/operator/apis/loki/v1/lokistack_types.go b/operator/apis/loki/v1/lokistack_types.go index 6124c65cd5217..a50fb48b187ea 100644 --- a/operator/apis/loki/v1/lokistack_types.go +++ b/operator/apis/loki/v1/lokistack_types.go @@ -1062,6 +1062,12 @@ const ( ReasonMissingObjectStorageSecret LokiStackConditionReason = "MissingObjectStorageSecret" // ReasonInvalidObjectStorageSecret when the format of the secret is invalid. ReasonInvalidObjectStorageSecret LokiStackConditionReason = "InvalidObjectStorageSecret" + // ReasonMissingCredentialsRequest when the required request for managed auth credentials to object + // storage is missing. + ReasonMissingCredentialsRequest LokiStackConditionReason = "MissingCredentialsRequest" + // ReasonMissingManagedAuthSecret when the required secret for managed auth credentials to object + // storage is missing. + ReasonMissingManagedAuthSecret LokiStackConditionReason = "MissingManagedAuthenticationSecret" // ReasonInvalidObjectStorageSchema when the spec contains an invalid schema(s). ReasonInvalidObjectStorageSchema LokiStackConditionReason = "InvalidObjectStorageSchema" // ReasonMissingObjectStorageCAConfigMap when the required configmap to verify object storage diff --git a/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml b/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml index eef89deddb746..4b20f814804a2 100644 --- a/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml +++ b/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml @@ -157,7 +157,7 @@ metadata: features.operators.openshift.io/fips-compliant: "false" features.operators.openshift.io/proxy-aware: "true" features.operators.openshift.io/tls-profiles: "true" - features.operators.openshift.io/token-auth-aws: "false" + features.operators.openshift.io/token-auth-aws: "true" features.operators.openshift.io/token-auth-azure: "false" features.operators.openshift.io/token-auth-gcp: "false" operators.operatorframework.io/builder: operator-sdk-unknown @@ -1463,6 +1463,16 @@ spec: - patch - update - watch + - apiGroups: + - cloudcredential.openshift.io + resources: + - credentialsrequests + verbs: + - create + - delete + - get + - list + - watch - apiGroups: - config.openshift.io resources: diff --git a/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml b/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml index ecc294fddf367..81575be404e82 100644 --- a/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml +++ b/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml @@ -1443,6 +1443,16 @@ spec: - patch - update - watch + - apiGroups: + - cloudcredential.openshift.io + resources: + - credentialsrequests + verbs: + - create + - delete + - get + - list + - watch - apiGroups: - config.openshift.io resources: diff --git a/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml b/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml index 69e55579d125e..b79f4ea7a2f49 100644 --- a/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml +++ b/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml @@ -164,7 +164,7 @@ metadata: features.operators.openshift.io/fips-compliant: "false" features.operators.openshift.io/proxy-aware: "true" features.operators.openshift.io/tls-profiles: "true" - features.operators.openshift.io/token-auth-aws: "false" + features.operators.openshift.io/token-auth-aws: "true" features.operators.openshift.io/token-auth-azure: "false" features.operators.openshift.io/token-auth-gcp: "false" olm.skipRange: '>=5.7.0-0 <5.9.0' @@ -1448,6 +1448,16 @@ spec: - patch - update - watch + - apiGroups: + - cloudcredential.openshift.io + resources: + - credentialsrequests + verbs: + - create + - delete + - get + - list + - watch - apiGroups: - config.openshift.io resources: diff --git a/operator/config/manifests/community-openshift/bases/loki-operator.clusterserviceversion.yaml b/operator/config/manifests/community-openshift/bases/loki-operator.clusterserviceversion.yaml index c7eb60e5a3e3b..a669b4da3da24 100644 --- a/operator/config/manifests/community-openshift/bases/loki-operator.clusterserviceversion.yaml +++ b/operator/config/manifests/community-openshift/bases/loki-operator.clusterserviceversion.yaml @@ -14,7 +14,7 @@ metadata: features.operators.openshift.io/fips-compliant: "false" features.operators.openshift.io/proxy-aware: "true" features.operators.openshift.io/tls-profiles: "true" - features.operators.openshift.io/token-auth-aws: "false" + features.operators.openshift.io/token-auth-aws: "true" features.operators.openshift.io/token-auth-azure: "false" features.operators.openshift.io/token-auth-gcp: "false" repository: https://github.com/grafana/loki/tree/main/operator diff --git a/operator/config/manifests/openshift/bases/loki-operator.clusterserviceversion.yaml b/operator/config/manifests/openshift/bases/loki-operator.clusterserviceversion.yaml index 5483709ad5d66..0e724292edbb6 100644 --- a/operator/config/manifests/openshift/bases/loki-operator.clusterserviceversion.yaml +++ b/operator/config/manifests/openshift/bases/loki-operator.clusterserviceversion.yaml @@ -20,7 +20,7 @@ metadata: features.operators.openshift.io/fips-compliant: "false" features.operators.openshift.io/proxy-aware: "true" features.operators.openshift.io/tls-profiles: "true" - features.operators.openshift.io/token-auth-aws: "false" + features.operators.openshift.io/token-auth-aws: "true" features.operators.openshift.io/token-auth-azure: "false" features.operators.openshift.io/token-auth-gcp: "false" olm.skipRange: '>=5.7.0-0 <5.9.0' diff --git a/operator/config/rbac/role.yaml b/operator/config/rbac/role.yaml index 09dc60b8c33b9..766a6d7d191e6 100644 --- a/operator/config/rbac/role.yaml +++ b/operator/config/rbac/role.yaml @@ -47,6 +47,16 @@ rules: - patch - update - watch +- apiGroups: + - cloudcredential.openshift.io + resources: + - credentialsrequests + verbs: + - create + - delete + - get + - list + - watch - apiGroups: - config.openshift.io resources: diff --git a/operator/controllers/loki/credentialsrequests_controller.go b/operator/controllers/loki/credentialsrequests_controller.go new file mode 100644 index 0000000000000..61d0b58423e90 --- /dev/null +++ b/operator/controllers/loki/credentialsrequests_controller.go @@ -0,0 +1,71 @@ +package controllers + +import ( + "context" + + "github.com/go-logr/logr" + apierrors "k8s.io/apimachinery/pkg/api/errors" + "k8s.io/apimachinery/pkg/runtime" + ctrl "sigs.k8s.io/controller-runtime" + "sigs.k8s.io/controller-runtime/pkg/client" + + lokiv1 "github.com/grafana/loki/operator/apis/loki/v1" + "github.com/grafana/loki/operator/controllers/loki/internal/lokistack" + "github.com/grafana/loki/operator/controllers/loki/internal/management/state" + "github.com/grafana/loki/operator/internal/external/k8s" + "github.com/grafana/loki/operator/internal/handlers" +) + +// CredentialsRequestsReconciler reconciles a single CredentialsRequest resource for each LokiStack request. +type CredentialsRequestsReconciler struct { + client.Client + Scheme *runtime.Scheme + Log logr.Logger +} + +// Reconcile creates a single CredentialsRequest per LokiStack for the OpenShift cloud-credentials-operator (CCO) to +// provide a managed cloud credentials Secret. On successful creation, the LokiStack resource is annotated +// with `loki.grafana.com/credentials-request-secret-ref` that refers to the secret provided by CCO. If the LokiStack +// resource is not found its accompanying CredentialsRequest resource is deleted. +func (r *CredentialsRequestsReconciler) Reconcile(ctx context.Context, req ctrl.Request) (ctrl.Result, error) { + var stack lokiv1.LokiStack + if err := r.Client.Get(ctx, req.NamespacedName, &stack); err != nil { + if apierrors.IsNotFound(err) { + return ctrl.Result{}, handlers.DeleteCredentialsRequest(ctx, r.Client, req.NamespacedName) + } + return ctrl.Result{}, err + } + + managed, err := state.IsManaged(ctx, req, r.Client) + if err != nil { + return ctrl.Result{}, err + } + if !managed { + r.Log.Info("Skipping reconciliation for unmanaged LokiStack resource", "name", req.String()) + // Stop requeueing for unmanaged LokiStack custom resources + return ctrl.Result{}, nil + } + + secretRef, err := handlers.CreateCredentialsRequest(ctx, r.Client, req.NamespacedName) + if err != nil { + return ctrl.Result{}, err + } + + if err := lokistack.AnnotateForCredentialsRequest(ctx, r.Client, req.NamespacedName, secretRef); err != nil { + return ctrl.Result{}, err + } + + return ctrl.Result{}, nil +} + +// SetupWithManager sets up the controller with the Manager. +func (r *CredentialsRequestsReconciler) SetupWithManager(mgr ctrl.Manager) error { + b := ctrl.NewControllerManagedBy(mgr) + return r.buildController(k8s.NewCtrlBuilder(b)) +} + +func (r *CredentialsRequestsReconciler) buildController(bld k8s.Builder) error { + return bld. + For(&lokiv1.LokiStack{}). + Complete(r) +} diff --git a/operator/controllers/loki/credentialsrequests_controller_test.go b/operator/controllers/loki/credentialsrequests_controller_test.go new file mode 100644 index 0000000000000..e6738c1d1796e --- /dev/null +++ b/operator/controllers/loki/credentialsrequests_controller_test.go @@ -0,0 +1,155 @@ +package controllers + +import ( + "context" + "testing" + + cloudcredentialsv1 "github.com/openshift/cloud-credential-operator/pkg/apis/cloudcredential/v1" + "github.com/stretchr/testify/require" + apierrors "k8s.io/apimachinery/pkg/api/errors" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime/schema" + "k8s.io/apimachinery/pkg/types" + ctrl "sigs.k8s.io/controller-runtime" + "sigs.k8s.io/controller-runtime/pkg/client" + + lokiv1 "github.com/grafana/loki/operator/apis/loki/v1" + "github.com/grafana/loki/operator/internal/external/k8s/k8sfakes" + "github.com/grafana/loki/operator/internal/manifests/storage" +) + +func TestCredentialsRequestController_RegistersCustomResource_WithDefaultPredicates(t *testing.T) { + b := &k8sfakes.FakeBuilder{} + k := &k8sfakes.FakeClient{} + c := &CredentialsRequestsReconciler{Client: k, Scheme: scheme} + + b.ForReturns(b) + b.OwnsReturns(b) + + err := c.buildController(b) + require.NoError(t, err) + + // Require only one For-Call for the custom resource + require.Equal(t, 1, b.ForCallCount()) + + // Require For-call with LokiStack resource + obj, _ := b.ForArgsForCall(0) + require.Equal(t, &lokiv1.LokiStack{}, obj) +} + +func TestCredentialsRequestController_DeleteCredentialsRequest_WhenLokiStackNotFound(t *testing.T) { + k := &k8sfakes.FakeClient{} + c := &CredentialsRequestsReconciler{Client: k, Scheme: scheme} + r := ctrl.Request{ + NamespacedName: types.NamespacedName{ + Name: "my-stack", + Namespace: "ns", + }, + } + + // Set managed auth environment + t.Setenv("ROLEARN", "a-role-arn") + + k.GetStub = func(_ context.Context, key types.NamespacedName, _ client.Object, _ ...client.GetOption) error { + if key.Name == r.Name && key.Namespace == r.Namespace { + return apierrors.NewNotFound(schema.GroupResource{}, "lokistack not found") + } + return nil + } + + res, err := c.Reconcile(context.Background(), r) + require.NoError(t, err) + require.Equal(t, ctrl.Result{}, res) + require.Equal(t, 1, k.DeleteCallCount()) +} + +func TestCredentialsRequestController_CreateCredentialsRequest_WhenLokiStackNotAnnotated(t *testing.T) { + k := &k8sfakes.FakeClient{} + c := &CredentialsRequestsReconciler{Client: k, Scheme: scheme} + r := ctrl.Request{ + NamespacedName: types.NamespacedName{ + Name: "my-stack", + Namespace: "ns", + }, + } + s := lokiv1.LokiStack{ + ObjectMeta: metav1.ObjectMeta{ + Name: "my-stack", + Namespace: "ns", + }, + Spec: lokiv1.LokiStackSpec{ + ManagementState: lokiv1.ManagementStateManaged, + }, + } + + // Set managed auth environment + t.Setenv("ROLEARN", "a-role-arn") + + k.GetStub = func(_ context.Context, key types.NamespacedName, out client.Object, _ ...client.GetOption) error { + if key.Name == r.Name && key.Namespace == r.Namespace { + k.SetClientObject(out, &s) + return nil + } + return apierrors.NewNotFound(schema.GroupResource{}, "lokistack not found") + } + + k.CreateStub = func(_ context.Context, o client.Object, _ ...client.CreateOption) error { + _, isCredReq := o.(*cloudcredentialsv1.CredentialsRequest) + if !isCredReq { + return apierrors.NewBadRequest("something went wrong creating a credentials request") + } + return nil + } + + k.UpdateStub = func(_ context.Context, o client.Object, _ ...client.UpdateOption) error { + stack, ok := o.(*lokiv1.LokiStack) + if !ok { + return apierrors.NewBadRequest("something went wrong creating a credentials request") + } + + _, hasSecretRef := stack.Annotations[storage.AnnotationCredentialsRequestsSecretRef] + if !hasSecretRef { + return apierrors.NewBadRequest("something went updating the lokistack annotations") + } + return nil + } + + res, err := c.Reconcile(context.Background(), r) + require.NoError(t, err) + require.Equal(t, ctrl.Result{}, res) + require.Equal(t, 1, k.CreateCallCount()) + require.Equal(t, 1, k.UpdateCallCount()) +} + +func TestCredentialsRequestController_SkipsUnmanaged(t *testing.T) { + k := &k8sfakes.FakeClient{} + c := &CredentialsRequestsReconciler{Client: k, Scheme: scheme} + r := ctrl.Request{ + NamespacedName: types.NamespacedName{ + Name: "my-stack", + Namespace: "ns", + }, + } + + s := lokiv1.LokiStack{ + ObjectMeta: metav1.ObjectMeta{ + Name: "my-stack", + Namespace: "ns", + }, + Spec: lokiv1.LokiStackSpec{ + ManagementState: lokiv1.ManagementStateUnmanaged, + }, + } + + k.GetStub = func(_ context.Context, key types.NamespacedName, out client.Object, _ ...client.GetOption) error { + if key.Name == s.Name && key.Namespace == s.Namespace { + k.SetClientObject(out, &s) + return nil + } + return apierrors.NewNotFound(schema.GroupResource{}, "something not found") + } + + res, err := c.Reconcile(context.Background(), r) + require.NoError(t, err) + require.Equal(t, ctrl.Result{}, res) +} diff --git a/operator/controllers/loki/internal/lokistack/credentialsrequest_discovery.go b/operator/controllers/loki/internal/lokistack/credentialsrequest_discovery.go new file mode 100644 index 0000000000000..c911c1196eed4 --- /dev/null +++ b/operator/controllers/loki/internal/lokistack/credentialsrequest_discovery.go @@ -0,0 +1,30 @@ +package lokistack + +import ( + "context" + + "github.com/ViaQ/logerr/v2/kverrors" + "sigs.k8s.io/controller-runtime/pkg/client" + + "github.com/grafana/loki/operator/internal/external/k8s" + "github.com/grafana/loki/operator/internal/manifests/storage" +) + +// AnnotateForCredentialsRequest adds the `loki.grafana.com/credentials-request-secret-ref` annotation +// to the named Lokistack. If no LokiStack is found, then skip reconciliation. Or else return an error. +func AnnotateForCredentialsRequest(ctx context.Context, k k8s.Client, key client.ObjectKey, secretRef string) error { + stack, err := getLokiStack(ctx, k, key) + if stack == nil || err != nil { + return err + } + + if val, ok := stack.Annotations[storage.AnnotationCredentialsRequestsSecretRef]; ok && val == secretRef { + return nil + } + + if err := updateAnnotation(ctx, k, stack, storage.AnnotationCredentialsRequestsSecretRef, secretRef); err != nil { + return kverrors.Wrap(err, "failed to update lokistack `credentialsRequestSecretRef` annotation", "key", key) + } + + return nil +} diff --git a/operator/controllers/loki/internal/lokistack/credentialsrequest_discovery_test.go b/operator/controllers/loki/internal/lokistack/credentialsrequest_discovery_test.go new file mode 100644 index 0000000000000..ef073ca853ba5 --- /dev/null +++ b/operator/controllers/loki/internal/lokistack/credentialsrequest_discovery_test.go @@ -0,0 +1,98 @@ +package lokistack + +import ( + "context" + "testing" + + "github.com/stretchr/testify/require" + apierrors "k8s.io/apimachinery/pkg/api/errors" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/types" + "sigs.k8s.io/controller-runtime/pkg/client" + + lokiv1 "github.com/grafana/loki/operator/apis/loki/v1" + "github.com/grafana/loki/operator/internal/external/k8s/k8sfakes" + "github.com/grafana/loki/operator/internal/manifests/storage" +) + +func TestAnnotateForCredentialsRequest_ReturnError_WhenLokiStackMissing(t *testing.T) { + k := &k8sfakes.FakeClient{} + annotationVal := "ns-my-stack-aws-creds" + stackKey := client.ObjectKey{Name: "my-stack", Namespace: "ns"} + + k.GetStub = func(_ context.Context, _ types.NamespacedName, out client.Object, _ ...client.GetOption) error { + return apierrors.NewBadRequest("failed to get lokistack") + } + + err := AnnotateForCredentialsRequest(context.Background(), k, stackKey, annotationVal) + require.Error(t, err) +} + +func TestAnnotateForCredentialsRequest_DoNothing_WhenAnnotationExists(t *testing.T) { + k := &k8sfakes.FakeClient{} + + annotationVal := "ns-my-stack-aws-creds" + s := &lokiv1.LokiStack{ + ObjectMeta: metav1.ObjectMeta{ + Name: "my-stack", + Namespace: "ns", + Annotations: map[string]string{ + storage.AnnotationCredentialsRequestsSecretRef: annotationVal, + }, + }, + } + stackKey := client.ObjectKeyFromObject(s) + + k.GetStub = func(_ context.Context, key types.NamespacedName, out client.Object, _ ...client.GetOption) error { + if key.Name == stackKey.Name && key.Namespace == stackKey.Namespace { + k.SetClientObject(out, s) + return nil + } + return nil + } + + err := AnnotateForCredentialsRequest(context.Background(), k, stackKey, annotationVal) + require.NoError(t, err) + require.Equal(t, 0, k.UpdateCallCount()) +} + +func TestAnnotateForCredentialsRequest_UpdateLokistack_WhenAnnotationMissing(t *testing.T) { + k := &k8sfakes.FakeClient{} + + annotationVal := "ns-my-stack-aws-creds" + s := &lokiv1.LokiStack{ + ObjectMeta: metav1.ObjectMeta{ + Name: "my-stack", + Namespace: "ns", + Annotations: map[string]string{}, + }, + } + stackKey := client.ObjectKeyFromObject(s) + + k.GetStub = func(_ context.Context, key types.NamespacedName, out client.Object, _ ...client.GetOption) error { + if key.Name == stackKey.Name && key.Namespace == stackKey.Namespace { + k.SetClientObject(out, s) + return nil + } + return nil + } + + k.UpdateStub = func(_ context.Context, o client.Object, _ ...client.UpdateOption) error { + stack, ok := o.(*lokiv1.LokiStack) + if !ok { + return apierrors.NewBadRequest("failed conversion to *lokiv1.LokiStack") + } + val, ok := stack.Annotations[storage.AnnotationCredentialsRequestsSecretRef] + if !ok { + return apierrors.NewBadRequest("missing annotation") + } + if val != annotationVal { + return apierrors.NewBadRequest("annotations does not match input") + } + return nil + } + + err := AnnotateForCredentialsRequest(context.Background(), k, stackKey, annotationVal) + require.NoError(t, err) + require.Equal(t, 1, k.UpdateCallCount()) +} diff --git a/operator/controllers/loki/lokistack_controller.go b/operator/controllers/loki/lokistack_controller.go index 629ee85d5edd7..40e7691bd1a2b 100644 --- a/operator/controllers/loki/lokistack_controller.go +++ b/operator/controllers/loki/lokistack_controller.go @@ -3,17 +3,20 @@ package controllers import ( "context" "errors" + "strings" "time" "github.com/go-logr/logr" "github.com/google/go-cmp/cmp" openshiftconfigv1 "github.com/openshift/api/config/v1" routev1 "github.com/openshift/api/route/v1" + cloudcredentialv1 "github.com/openshift/cloud-credential-operator/pkg/apis/cloudcredential/v1" monitoringv1 "github.com/prometheus-operator/prometheus-operator/pkg/apis/monitoring/v1" appsv1 "k8s.io/api/apps/v1" corev1 "k8s.io/api/core/v1" networkingv1 "k8s.io/api/networking/v1" rbacv1 "k8s.io/api/rbac/v1" + apierrors "k8s.io/apimachinery/pkg/api/errors" "k8s.io/apimachinery/pkg/runtime" "k8s.io/apimachinery/pkg/types" ctrl "sigs.k8s.io/controller-runtime" @@ -30,7 +33,7 @@ import ( "github.com/grafana/loki/operator/controllers/loki/internal/management/state" "github.com/grafana/loki/operator/internal/external/k8s" "github.com/grafana/loki/operator/internal/handlers" - "github.com/grafana/loki/operator/internal/manifests/openshift" + manifestsocp "github.com/grafana/loki/operator/internal/manifests/openshift" "github.com/grafana/loki/operator/internal/status" ) @@ -125,6 +128,7 @@ type LokiStackReconciler struct { // +kubebuilder:rbac:groups=policy,resources=poddisruptionbudgets,verbs=get;list;watch;create;update // +kubebuilder:rbac:groups=config.openshift.io,resources=dnses;apiservers;proxies,verbs=get;list;watch // +kubebuilder:rbac:groups=route.openshift.io,resources=routes,verbs=get;list;watch;create;update;delete +// +kubebuilder:rbac:groups=cloudcredential.openshift.io,resources=credentialsrequests,verbs=get;list;watch;create;delete // Reconcile is part of the main kubernetes reconciliation loop which aims to // move the current state of the cluster closer to the desired state. @@ -149,7 +153,7 @@ func (r *LokiStackReconciler) Reconcile(ctx context.Context, req ctrl.Request) ( err = r.updateResources(ctx, req) switch { case errors.As(err, °raded): - // degraded errors are handled by status.Refresh below + // degraded errors are handled by status.Refresh below case err != nil: return ctrl.Result{}, err } @@ -210,17 +214,19 @@ func (r *LokiStackReconciler) buildController(bld k8s.Builder) error { } if r.FeatureGates.OpenShift.Enabled { - bld = bld.Owns(&routev1.Route{}, updateOrDeleteOnlyPred) - } else { - bld = bld.Owns(&networkingv1.Ingress{}, updateOrDeleteOnlyPred) - } + bld = bld. + Owns(&routev1.Route{}, updateOrDeleteOnlyPred). + Watches(&cloudcredentialv1.CredentialsRequest{}, r.enqueueForCredentialsRequest(), updateOrDeleteOnlyPred) - if r.FeatureGates.OpenShift.ClusterTLSPolicy { - bld = bld.Watches(&openshiftconfigv1.APIServer{}, r.enqueueAllLokiStacksHandler(), updateOrDeleteOnlyPred) - } + if r.FeatureGates.OpenShift.ClusterTLSPolicy { + bld = bld.Watches(&openshiftconfigv1.APIServer{}, r.enqueueAllLokiStacksHandler(), updateOrDeleteOnlyPred) + } - if r.FeatureGates.OpenShift.ClusterProxy { - bld = bld.Watches(&openshiftconfigv1.Proxy{}, r.enqueueAllLokiStacksHandler(), updateOrDeleteOnlyPred) + if r.FeatureGates.OpenShift.ClusterProxy { + bld = bld.Watches(&openshiftconfigv1.Proxy{}, r.enqueueAllLokiStacksHandler(), updateOrDeleteOnlyPred) + } + } else { + bld = bld.Owns(&networkingv1.Ingress{}, updateOrDeleteOnlyPred) } return bld.Complete(r) @@ -271,9 +277,9 @@ func (r *LokiStackReconciler) enqueueForAlertManagerServices() handler.EventHand } var requests []reconcile.Request - if obj.GetName() == openshift.MonitoringSVCOperated && - (obj.GetNamespace() == openshift.MonitoringUserWorkloadNS || - obj.GetNamespace() == openshift.MonitoringNS) { + if obj.GetName() == manifestsocp.MonitoringSVCOperated && + (obj.GetNamespace() == manifestsocp.MonitoringUserWorkloadNS || + obj.GetNamespace() == manifestsocp.MonitoringNS) { for _, stack := range lokiStacks.Items { if stack.Spec.Tenants != nil && (stack.Spec.Tenants.Mode == lokiv1.OpenshiftLogging || @@ -352,3 +358,34 @@ func (r *LokiStackReconciler) enqueueForStorageCA() handler.EventHandler { return requests }) } + +func (r *LokiStackReconciler) enqueueForCredentialsRequest() handler.EventHandler { + return handler.EnqueueRequestsFromMapFunc(func(ctx context.Context, obj client.Object) []reconcile.Request { + a := obj.GetAnnotations() + owner, ok := a[manifestsocp.AnnotationCredentialsRequestOwner] + if !ok { + return nil + } + + var ( + ownerParts = strings.Split(owner, "/") + namespace = ownerParts[0] + name = ownerParts[1] + key = client.ObjectKey{Namespace: namespace, Name: name} + ) + + var stack lokiv1.LokiStack + if err := r.Client.Get(ctx, key, &stack); err != nil { + if !apierrors.IsNotFound(err) { + r.Log.Error(err, "failed retrieving CredentialsRequest owning Lokistack", "key", key) + } + return nil + } + + return []reconcile.Request{ + { + NamespacedName: key, + }, + } + }) +} diff --git a/operator/controllers/loki/lokistack_controller_test.go b/operator/controllers/loki/lokistack_controller_test.go index 7421b63331b5d..515d829766aa1 100644 --- a/operator/controllers/loki/lokistack_controller_test.go +++ b/operator/controllers/loki/lokistack_controller_test.go @@ -10,6 +10,7 @@ import ( "github.com/go-logr/logr" openshiftconfigv1 "github.com/openshift/api/config/v1" routev1 "github.com/openshift/api/route/v1" + cloudcredentialv1 "github.com/openshift/cloud-credential-operator/pkg/apis/cloudcredential/v1" "github.com/stretchr/testify/require" appsv1 "k8s.io/api/apps/v1" corev1 "k8s.io/api/core/v1" @@ -202,11 +203,23 @@ func TestLokiStackController_RegisterWatchedResources(t *testing.T) { } table := []test{ { - src: &openshiftconfigv1.APIServer{}, + src: &cloudcredentialv1.CredentialsRequest{}, index: 3, watchesCallsCount: 4, featureGates: configv1.FeatureGates{ OpenShift: configv1.OpenShiftFeatureGates{ + Enabled: true, + }, + }, + pred: updateOrDeleteOnlyPred, + }, + { + src: &openshiftconfigv1.APIServer{}, + index: 4, + watchesCallsCount: 5, + featureGates: configv1.FeatureGates{ + OpenShift: configv1.OpenShiftFeatureGates{ + Enabled: true, ClusterTLSPolicy: true, }, }, @@ -214,10 +227,11 @@ func TestLokiStackController_RegisterWatchedResources(t *testing.T) { }, { src: &openshiftconfigv1.Proxy{}, - index: 3, - watchesCallsCount: 4, + index: 4, + watchesCallsCount: 5, featureGates: configv1.FeatureGates{ OpenShift: configv1.OpenShiftFeatureGates{ + Enabled: true, ClusterProxy: true, }, }, diff --git a/operator/docs/operator/api.md b/operator/docs/operator/api.md index 989a6ef481649..92f93dd970224 100644 --- a/operator/docs/operator/api.md +++ b/operator/docs/operator/api.md @@ -1745,6 +1745,10 @@ with the select cluster size.

"InvalidTenantsConfiguration"

ReasonInvalidTenantsConfiguration when the tenant configuration provided is invalid.

+

"MissingCredentialsRequest"

+

ReasonMissingCredentialsRequest when the required request for managed auth credentials to object +storage is missing.

+

"MissingGatewayTenantAuthenticationConfig"

ReasonMissingGatewayAuthenticationConfig when the config for when a tenant is missing authentication config

@@ -1759,6 +1763,10 @@ for authentication is missing.

ReasonMissingGatewayTenantSecret when the required tenant secret for authentication is missing.

+

"MissingManagedAuthenticationSecret"

+

ReasonMissingManagedAuthSecret when the required secret for managed auth credentials to object +storage is missing.

+

"MissingObjectStorageCAConfigMap"

ReasonMissingObjectStorageCAConfigMap when the required configmap to verify object storage certificates is missing.

diff --git a/operator/docs/operator/feature-gates.md b/operator/docs/operator/feature-gates.md index 1d5c046be7755..34fbdf4b69a4d 100644 --- a/operator/docs/operator/feature-gates.md +++ b/operator/docs/operator/feature-gates.md @@ -409,6 +409,17 @@ bool

Dashboards enables the loki-mixin dashboards into the OpenShift Console

+ + +ManagedAuthEnv
+ +bool + + + +

ManagedAuthEnv enabled when the operator installation is on OpenShift STS clusters.

+ + diff --git a/operator/go.mod b/operator/go.mod index 6f02675596bdf..10104f11e38ed 100644 --- a/operator/go.mod +++ b/operator/go.mod @@ -12,6 +12,7 @@ require ( github.com/imdario/mergo v0.3.13 github.com/maxbrunsfeld/counterfeiter/v6 v6.7.0 github.com/openshift/api v0.0.0-20240116035456-11ed2fbcb805 // release-4.15 + github.com/openshift/cloud-credential-operator v0.0.0-20240122210451-67842c7839ac // release-4.15 github.com/openshift/library-go v0.0.0-20240117151256-95b334bccb5d // release-4.15 github.com/prometheus-operator/prometheus-operator/pkg/apis/monitoring v0.67.1 github.com/prometheus/client_golang v1.17.0 @@ -125,8 +126,8 @@ require ( go.etcd.io/etcd/api/v3 v3.5.9 // indirect go.etcd.io/etcd/client/pkg/v3 v3.5.9 // indirect go.etcd.io/etcd/client/v3 v3.5.9 // indirect - go.opentelemetry.io/otel v1.11.2 // indirect - go.opentelemetry.io/otel/trace v1.11.2 // indirect + go.opentelemetry.io/otel v1.14.0 // indirect + go.opentelemetry.io/otel/trace v1.14.0 // indirect go.uber.org/atomic v1.10.0 // indirect go.uber.org/goleak v1.2.1 // indirect go.uber.org/multierr v1.11.0 // indirect @@ -136,7 +137,7 @@ require ( golang.org/x/exp v0.0.0-20230124195608-d38c7dcee874 // indirect golang.org/x/mod v0.14.0 // indirect golang.org/x/net v0.19.0 // indirect - golang.org/x/oauth2 v0.8.0 // indirect + golang.org/x/oauth2 v0.10.0 // indirect golang.org/x/sync v0.5.0 // indirect golang.org/x/sys v0.15.0 // indirect golang.org/x/term v0.15.0 // indirect @@ -145,12 +146,13 @@ require ( golang.org/x/tools v0.16.1 // indirect gomodules.xyz/jsonpatch/v2 v2.4.0 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/genproto v0.0.0-20230526161137-0005af68ea54 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20230525234035-dd9d682886f9 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19 // indirect - google.golang.org/grpc v1.56.3 // indirect + google.golang.org/genproto v0.0.0-20230711160842-782d3b101e98 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20230711160842-782d3b101e98 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20230711160842-782d3b101e98 // indirect + google.golang.org/grpc v1.58.3 // indirect google.golang.org/protobuf v1.31.0 // indirect gopkg.in/inf.v0 v0.9.1 // indirect + gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect k8s.io/apiextensions-apiserver v0.28.3 // indirect k8s.io/klog/v2 v2.100.1 // indirect diff --git a/operator/go.sum b/operator/go.sum index 0d66cc79ec3b4..cfaf2c62e1c57 100644 --- a/operator/go.sum +++ b/operator/go.sum @@ -637,12 +637,12 @@ github.com/envoyproxy/go-control-plane v0.9.10-0.20210907150352-cf90f659a021/go. github.com/envoyproxy/go-control-plane v0.10.2-0.20220325020618-49ff273808a1/go.mod h1:KJwIaB5Mv44NWtYuAOFCVOjcI94vtpEz2JU/D2v6IjE= github.com/envoyproxy/go-control-plane v0.10.3/go.mod h1:fJJn/j26vwOu972OllsvAgJJM//w9BV6Fxbg2LuVd34= github.com/envoyproxy/go-control-plane v0.11.0/go.mod h1:VnHyVMpzcLvCFt9yUz1UnCwHLhwx1WguiVDV7pTG/tI= -github.com/envoyproxy/go-control-plane v0.11.1-0.20230524094728-9239064ad72f h1:7T++XKzy4xg7PKy+bM+Sa9/oe1OC88yz2hXQUISoXfA= +github.com/envoyproxy/go-control-plane v0.11.1 h1:wSUXTlLfiAQRWs2F+p+EKOY9rUyis1MyGqJ2DIk5HpM= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/envoyproxy/protoc-gen-validate v0.6.7/go.mod h1:dyJXwwfPK2VSqiB9Klm1J6romD608Ba7Hij42vrOBCo= github.com/envoyproxy/protoc-gen-validate v0.9.1/go.mod h1:OKNgG7TCp5pF4d6XftA0++PMirau2/yoOwVac3AbF2w= github.com/envoyproxy/protoc-gen-validate v0.10.0/go.mod h1:DRjgyB0I43LtJapqN6NiRwroiAU2PaFuvk/vjgh61ss= -github.com/envoyproxy/protoc-gen-validate v0.10.1 h1:c0g45+xCJhdgFGw7a5QAfdS4byAbud7miNWJ1WwEVf8= +github.com/envoyproxy/protoc-gen-validate v1.0.2 h1:QkIBuU5k+x7/QXPvPPnWXWlCdaBFApVqftFV6k087DA= github.com/evanphx/json-patch v5.6.0+incompatible h1:jBYDEEiFBPxA0v50tFdvOzQQTCvpL6mnFh5mB2/l16U= github.com/evanphx/json-patch/v5 v5.6.0 h1:b91NhWfaz02IuVxO9faSllyAtNXHMPkC5J8sJCLunww= github.com/evanphx/json-patch/v5 v5.6.0/go.mod h1:G79N1coSVB93tBe7j6PhzjmR3/2VvlbKOFpnXhI9Bw4= @@ -1020,6 +1020,8 @@ github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8 github.com/opencontainers/image-spec v1.0.2 h1:9yCKha/T5XdGtO0q9Q9a6T5NUCsTn/DrBg0D7ufOcFM= github.com/openshift/api v0.0.0-20240116035456-11ed2fbcb805 h1:5NjcOG5i+WH0F4FI8dKSf0fNgX0YQkrJ8w3YcsHx6KM= github.com/openshift/api v0.0.0-20240116035456-11ed2fbcb805/go.mod h1:qNtV0315F+f8ld52TLtPvrfivZpdimOzTi3kn9IVbtU= +github.com/openshift/cloud-credential-operator v0.0.0-20240122210451-67842c7839ac h1:ZYatLLVj5pYeNGi9xeebTLfVqdl31MoCa2Jenog1ecM= +github.com/openshift/cloud-credential-operator v0.0.0-20240122210451-67842c7839ac/go.mod h1:fUDZ7YKd5PC+wFYczavCyHJaw0H3m0WGXNdpFUuN47Q= github.com/openshift/library-go v0.0.0-20240117151256-95b334bccb5d h1:jDgYsLszzWSgxr0Tas9+L0F2pIu0mngCLv6BA5vubQ4= github.com/openshift/library-go v0.0.0-20240117151256-95b334bccb5d/go.mod h1:0q1UIvboZXfSlUaK+08wsXYw4N6OUo2b/z3a1EWNGyw= github.com/opentracing-contrib/go-grpc v0.0.0-20180928155321-4b5a12d3ff02/go.mod h1:JNdpVEzCpXBgIiv4ds+TzhN1hrtxq6ClLrTlT9OQRSc= @@ -1180,10 +1182,10 @@ go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/otel v1.11.2 h1:YBZcQlsVekzFsFbjygXMOXSs6pialIZxcjfO/mBDmR0= -go.opentelemetry.io/otel v1.11.2/go.mod h1:7p4EUV+AqgdlNV9gL97IgUZiVR3yrFXYo53f9BM3tRI= -go.opentelemetry.io/otel/trace v1.11.2 h1:Xf7hWSF2Glv0DE3MH7fBHvtpSBsjcBUe5MYAmZM/+y0= -go.opentelemetry.io/otel/trace v1.11.2/go.mod h1:4N+yC7QEz7TTsG9BSRLNAa63eg5E06ObSbKPmxQ/pKA= +go.opentelemetry.io/otel v1.14.0 h1:/79Huy8wbf5DnIPhemGB+zEPVwnN6fuQybr/SRXa6hM= +go.opentelemetry.io/otel v1.14.0/go.mod h1:o4buv+dJzx8rohcUeRmWUZhqupFvzWis188WlggnNeU= +go.opentelemetry.io/otel/trace v1.14.0 h1:wp2Mmvj41tDsyAJXiWDWpfNsOiIyd38fy85pyKcFq/M= +go.opentelemetry.io/otel/trace v1.14.0/go.mod h1:8avnQLK+CG77yNLUae4ea2JDQ6iT+gozhnZjy/rw9G8= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v0.15.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= @@ -1373,8 +1375,8 @@ golang.org/x/oauth2 v0.0.0-20221014153046-6fdb5e3db783/go.mod h1:h4gKUeWbJ4rQPri golang.org/x/oauth2 v0.4.0/go.mod h1:RznEsdpjGAINPTOF0UH/t+xJ75L18YO3Ho6Pyn+uRec= golang.org/x/oauth2 v0.5.0/go.mod h1:9/XBHVqLaWO3/BRHs5jbpYCnOZVjj5V0ndyaAM7KB4I= golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= -golang.org/x/oauth2 v0.8.0 h1:6dkIjl3j3LtZ/O3sTgZTMsLKSftL/B8Zgq4huOIIUu8= -golang.org/x/oauth2 v0.8.0/go.mod h1:yr7u4HXZRm1R1kBWqr/xKNqewf0plRYoB7sla+BCIXE= +golang.org/x/oauth2 v0.10.0 h1:zHCpF2Khkwy4mMB4bv0U37YtJdTGW8jI0glAApi0Kh8= +golang.org/x/oauth2 v0.10.0/go.mod h1:kTpgurOux7LqtuxjuyZa4Gj2gdezIt/jQtGnNFfypQI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1802,12 +1804,12 @@ google.golang.org/genproto v0.0.0-20230209215440-0dfe4f8abfcc/go.mod h1:RGgjbofJ google.golang.org/genproto v0.0.0-20230216225411-c8e22ba71e44/go.mod h1:8B0gmkoRebU8ukX6HP+4wrVQUY1+6PkQ44BSyIlflHA= google.golang.org/genproto v0.0.0-20230222225845-10f96fb3dbec/go.mod h1:3Dl5ZL0q0isWJt+FVcfpQyirqemEuLAK/iFvg1UP1Hw= google.golang.org/genproto v0.0.0-20230306155012-7f2fa6fef1f4/go.mod h1:NWraEVixdDnqcqQ30jipen1STv2r/n24Wb7twVTGR4s= -google.golang.org/genproto v0.0.0-20230526161137-0005af68ea54 h1:9NWlQfY2ePejTmfwUH1OWwmznFa+0kKcHGPDvcPza9M= -google.golang.org/genproto v0.0.0-20230526161137-0005af68ea54/go.mod h1:zqTuNwFlFRsw5zIts5VnzLQxSRqh+CGOTVMlYbY0Eyk= -google.golang.org/genproto/googleapis/api v0.0.0-20230525234035-dd9d682886f9 h1:m8v1xLLLzMe1m5P+gCTF8nJB9epwZQUBERm20Oy1poQ= -google.golang.org/genproto/googleapis/api v0.0.0-20230525234035-dd9d682886f9/go.mod h1:vHYtlOoi6TsQ3Uk2yxR7NI5z8uoV+3pZtR4jmHIkRig= -google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19 h1:0nDDozoAU19Qb2HwhXadU8OcsiO/09cnTqhUtq2MEOM= -google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= +google.golang.org/genproto v0.0.0-20230711160842-782d3b101e98 h1:Z0hjGZePRE0ZBWotvtrwxFNrNE9CUAGtplaDK5NNI/g= +google.golang.org/genproto v0.0.0-20230711160842-782d3b101e98/go.mod h1:S7mY02OqCJTD0E1OiQy1F72PWFB4bZJ87cAtLPYgDR0= +google.golang.org/genproto/googleapis/api v0.0.0-20230711160842-782d3b101e98 h1:FmF5cCW94Ij59cfpoLiwTgodWmm60eEV0CjlsVg2fuw= +google.golang.org/genproto/googleapis/api v0.0.0-20230711160842-782d3b101e98/go.mod h1:rsr7RhLuwsDKL7RmgDDCUc6yaGr1iqceVb5Wv6f6YvQ= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230711160842-782d3b101e98 h1:bVf09lpb+OJbByTj913DRJioFFAjf/ZGxEz7MajTp2U= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230711160842-782d3b101e98/go.mod h1:TUfxEVdsvPg18p6AslUXFoLdpED4oBnGwyqk3dV1XzM= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= @@ -1850,8 +1852,8 @@ google.golang.org/grpc v1.51.0/go.mod h1:wgNDFcnuBGmxLKI/qn4T+m5BtEBYXJPvibbUPsA google.golang.org/grpc v1.52.0/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5vorUY= google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= google.golang.org/grpc v1.55.0/go.mod h1:iYEXKGkEBhg1PjZQvoYEVPTDkHo1/bjTnfwTeGONTY8= -google.golang.org/grpc v1.56.3 h1:8I4C0Yq1EjstUzUJzpcRVbuYA2mODtEmpWiQoN/b2nc= -google.golang.org/grpc v1.56.3/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= +google.golang.org/grpc v1.58.3 h1:BjnpXut1btbtgN/6sp+brB2Kbm2LjNXnidYujAVbSoQ= +google.golang.org/grpc v1.58.3/go.mod h1:tgX3ZQDlNJGU96V6yHh1T/JeoBQ2TXdr43YbYSsCJk0= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= @@ -1881,7 +1883,8 @@ gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EV gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= -gopkg.in/ini.v1 v1.66.6 h1:LATuAqN/shcYAOkv3wl2L4rkaKqkcgTBQjOyYDvcPKI= +gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= +gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= diff --git a/operator/hack/deploy-aws-storage-secret.sh b/operator/hack/deploy-aws-storage-secret.sh index ecad7efc5537b..e9689321d0b22 100755 --- a/operator/hack/deploy-aws-storage-secret.sh +++ b/operator/hack/deploy-aws-storage-secret.sh @@ -1,4 +1,22 @@ #!/usr/bin/env bash +# +# usage: deploy-aws-storage-secret.sh () +# +# This scripts deploys a LokiStack Secret resource holding the +# authentication credentials to access AWS S3. It supports three +# modes: static authentication, managed with custom role_arn and +# fully managed by OpeShift's Cloud-Credentials-Operator. To use +# one of the managed you need to pass the environment variable +# STS=true. If you pass the second optional argument you can set +# your custom managed role_arn. +# +# bucket_name is the name of the bucket to be used in the LokiStack +# object storage secret. +# +# role_arn is the ARN value of the upfront manually provisioned AWS +# Role that grants access to the and it's object on +# AWS S3. +# set -euo pipefail @@ -12,15 +30,33 @@ fi readonly namespace=${NAMESPACE:-openshift-logging} region=${REGION:-$(aws configure get region)} readonly region + +# static authentication from the current select AWS CLI profile. access_key_id=${ACCESS_KEY_ID:-$(aws configure get aws_access_key_id)} readonly access_key_id secret_access_key=${SECRET_ACCESS_KEY:-$(aws configure get aws_secret_access_key)} readonly secret_access_key -kubectl --ignore-not-found=true -n "${namespace}" delete secret test -kubectl -n "${namespace}" create secret generic test \ +# Managed authentication with/without a manually provisioned AWS Role. +readonly sts=${STS:-false} +readonly role_arn=${2-} + +create_secret_args=( \ --from-literal=region="$(echo -n "${region}")" \ --from-literal=bucketnames="$(echo -n "${bucket_name}")" \ - --from-literal=access_key_id="$(echo -n "${access_key_id}")" \ - --from-literal=access_key_secret="$(echo -n "${secret_access_key}")" \ - --from-literal=endpoint="$(echo -n "https://s3.${region}.amazonaws.com")" +) + +if [[ "${sts}" = "true" ]]; then + if [[ -n "${role_arn}" ]]; then + create_secret_args+=(--from-literal=role_arn="$(echo -n "${role_arn}")") + fi +else + create_secret_args+=( \ + --from-literal=access_key_id="$(echo -n "${access_key_id}")" \ + --from-literal=access_key_secret="$(echo -n "${secret_access_key}")" \ + --from-literal=endpoint="$(echo -n "https://s3.${region}.amazonaws.com")" \ + ) +fi + +kubectl --ignore-not-found=true -n "${namespace}" delete secret test +kubectl -n "${namespace}" create secret generic test "${create_secret_args[@]}" diff --git a/operator/internal/handlers/credentialsrequest_create.go b/operator/internal/handlers/credentialsrequest_create.go new file mode 100644 index 0000000000000..477528326b9a5 --- /dev/null +++ b/operator/internal/handlers/credentialsrequest_create.go @@ -0,0 +1,42 @@ +package handlers + +import ( + "context" + + "github.com/ViaQ/logerr/v2/kverrors" + apierrors "k8s.io/apimachinery/pkg/api/errors" + "sigs.k8s.io/controller-runtime/pkg/client" + + "github.com/grafana/loki/operator/internal/external/k8s" + "github.com/grafana/loki/operator/internal/manifests/openshift" +) + +// CreateCredentialsRequest creates a new CredentialsRequest resource for a Lokistack +// to request a cloud credentials Secret resource from the OpenShift cloud-credentials-operator. +func CreateCredentialsRequest(ctx context.Context, k k8s.Client, stack client.ObjectKey) (string, error) { + managedAuthEnv := openshift.DiscoverManagedAuthEnv() + if managedAuthEnv == nil { + return "", nil + } + + opts := openshift.Options{ + BuildOpts: openshift.BuildOptions{ + LokiStackName: stack.Name, + LokiStackNamespace: stack.Namespace, + }, + ManagedAuthEnv: managedAuthEnv, + } + + credReq, err := openshift.BuildCredentialsRequest(opts) + if err != nil { + return "", err + } + + if err := k.Create(ctx, credReq); err != nil { + if !apierrors.IsAlreadyExists(err) { + return "", kverrors.Wrap(err, "failed to create credentialsrequest", "key", client.ObjectKeyFromObject(credReq)) + } + } + + return credReq.Spec.SecretRef.Name, nil +} diff --git a/operator/internal/handlers/credentialsrequest_create_test.go b/operator/internal/handlers/credentialsrequest_create_test.go new file mode 100644 index 0000000000000..f6bf9c0f1b526 --- /dev/null +++ b/operator/internal/handlers/credentialsrequest_create_test.go @@ -0,0 +1,50 @@ +package handlers + +import ( + "context" + "testing" + + "github.com/stretchr/testify/require" + "k8s.io/apimachinery/pkg/api/errors" + "k8s.io/apimachinery/pkg/runtime/schema" + "sigs.k8s.io/controller-runtime/pkg/client" + + "github.com/grafana/loki/operator/internal/external/k8s/k8sfakes" +) + +func TestCreateCredentialsRequest_DoNothing_WhenManagedAuthEnvMissing(t *testing.T) { + k := &k8sfakes.FakeClient{} + key := client.ObjectKey{Name: "my-stack", Namespace: "ns"} + + secretRef, err := CreateCredentialsRequest(context.Background(), k, key) + require.NoError(t, err) + require.Empty(t, secretRef) +} + +func TestCreateCredentialsRequest_CreateNewResource(t *testing.T) { + k := &k8sfakes.FakeClient{} + key := client.ObjectKey{Name: "my-stack", Namespace: "ns"} + + t.Setenv("ROLEARN", "a-role-arn") + + secretRef, err := CreateCredentialsRequest(context.Background(), k, key) + require.NoError(t, err) + require.NotEmpty(t, secretRef) + require.Equal(t, 1, k.CreateCallCount()) +} + +func TestCreateCredentialsRequest_DoNothing_WhenCredentialsRequestExist(t *testing.T) { + k := &k8sfakes.FakeClient{} + key := client.ObjectKey{Name: "my-stack", Namespace: "ns"} + + t.Setenv("ROLEARN", "a-role-arn") + + k.CreateStub = func(_ context.Context, _ client.Object, _ ...client.CreateOption) error { + return errors.NewAlreadyExists(schema.GroupResource{}, "credentialsrequest exists") + } + + secretRef, err := CreateCredentialsRequest(context.Background(), k, key) + require.NoError(t, err) + require.NotEmpty(t, secretRef) + require.Equal(t, 1, k.CreateCallCount()) +} diff --git a/operator/internal/handlers/credentialsrequest_delete.go b/operator/internal/handlers/credentialsrequest_delete.go new file mode 100644 index 0000000000000..edf05fcb205d0 --- /dev/null +++ b/operator/internal/handlers/credentialsrequest_delete.go @@ -0,0 +1,43 @@ +package handlers + +import ( + "context" + + "github.com/ViaQ/logerr/v2/kverrors" + "k8s.io/apimachinery/pkg/api/errors" + "sigs.k8s.io/controller-runtime/pkg/client" + + "github.com/grafana/loki/operator/internal/external/k8s" + "github.com/grafana/loki/operator/internal/manifests/openshift" +) + +// DeleteCredentialsRequest deletes a LokiStack's accompanying CredentialsRequest resource +// to trigger the OpenShift cloud-credentials-operator to wipe out any credentials related +// Secret resource on the LokiStack namespace. +func DeleteCredentialsRequest(ctx context.Context, k k8s.Client, stack client.ObjectKey) error { + managedAuthEnv := openshift.DiscoverManagedAuthEnv() + if managedAuthEnv == nil { + return nil + } + + opts := openshift.Options{ + BuildOpts: openshift.BuildOptions{ + LokiStackName: stack.Name, + LokiStackNamespace: stack.Namespace, + }, + ManagedAuthEnv: managedAuthEnv, + } + + credReq, err := openshift.BuildCredentialsRequest(opts) + if err != nil { + return kverrors.Wrap(err, "failed to build credentialsrequest", "key", stack) + } + + if err := k.Delete(ctx, credReq); err != nil { + if !errors.IsNotFound(err) { + return kverrors.Wrap(err, "failed to delete credentialsrequest", "key", client.ObjectKeyFromObject(credReq)) + } + } + + return nil +} diff --git a/operator/internal/handlers/credentialsrequest_delete_test.go b/operator/internal/handlers/credentialsrequest_delete_test.go new file mode 100644 index 0000000000000..57f1c005ee706 --- /dev/null +++ b/operator/internal/handlers/credentialsrequest_delete_test.go @@ -0,0 +1,47 @@ +package handlers + +import ( + "context" + "testing" + + "github.com/stretchr/testify/require" + "k8s.io/apimachinery/pkg/api/errors" + "k8s.io/apimachinery/pkg/runtime/schema" + "sigs.k8s.io/controller-runtime/pkg/client" + + "github.com/grafana/loki/operator/internal/external/k8s/k8sfakes" +) + +func TestDeleteCredentialsRequest_DoNothing_WhenManagedAuthEnvMissing(t *testing.T) { + k := &k8sfakes.FakeClient{} + key := client.ObjectKey{Name: "my-stack", Namespace: "ns"} + + err := DeleteCredentialsRequest(context.Background(), k, key) + require.NoError(t, err) +} + +func TestDeleteCredentialsRequest_DeleteExistingResource(t *testing.T) { + k := &k8sfakes.FakeClient{} + key := client.ObjectKey{Name: "my-stack", Namespace: "ns"} + + t.Setenv("ROLEARN", "a-role-arn") + + err := DeleteCredentialsRequest(context.Background(), k, key) + require.NoError(t, err) + require.Equal(t, 1, k.DeleteCallCount()) +} + +func TestDeleteCredentialsRequest_DoNothing_WhenCredentialsRequestNotExists(t *testing.T) { + k := &k8sfakes.FakeClient{} + key := client.ObjectKey{Name: "my-stack", Namespace: "ns"} + + t.Setenv("ROLEARN", "a-role-arn") + + k.DeleteStub = func(_ context.Context, _ client.Object, _ ...client.DeleteOption) error { + return errors.NewNotFound(schema.GroupResource{}, "credentials request not found") + } + + err := DeleteCredentialsRequest(context.Background(), k, key) + require.NoError(t, err) + require.Equal(t, 1, k.DeleteCallCount()) +} diff --git a/operator/internal/handlers/internal/storage/secrets.go b/operator/internal/handlers/internal/storage/secrets.go index 0ef5f197a625e..705cabb6cf5d9 100644 --- a/operator/internal/handlers/internal/storage/secrets.go +++ b/operator/internal/handlers/internal/storage/secrets.go @@ -11,6 +11,7 @@ import ( apierrors "k8s.io/apimachinery/pkg/api/errors" "sigs.k8s.io/controller-runtime/pkg/client" + configv1 "github.com/grafana/loki/operator/apis/config/v1" lokiv1 "github.com/grafana/loki/operator/apis/loki/v1" "github.com/grafana/loki/operator/internal/external/k8s" "github.com/grafana/loki/operator/internal/manifests/storage" @@ -19,47 +20,92 @@ import ( var hashSeparator = []byte(",") -func getSecret(ctx context.Context, k k8s.Client, stack *lokiv1.LokiStack) (*corev1.Secret, error) { - var storageSecret corev1.Secret +func getSecrets(ctx context.Context, k k8s.Client, stack *lokiv1.LokiStack, fg configv1.FeatureGates) (*corev1.Secret, *corev1.Secret, error) { + var ( + storageSecret corev1.Secret + managedAuthSecret corev1.Secret + ) + key := client.ObjectKey{Name: stack.Spec.Storage.Secret.Name, Namespace: stack.Namespace} if err := k.Get(ctx, key, &storageSecret); err != nil { if apierrors.IsNotFound(err) { - return nil, &status.DegradedError{ + return nil, nil, &status.DegradedError{ Message: "Missing object storage secret", Reason: lokiv1.ReasonMissingObjectStorageSecret, Requeue: false, } } - return nil, kverrors.Wrap(err, "failed to lookup lokistack storage secret", "name", key) + return nil, nil, kverrors.Wrap(err, "failed to lookup lokistack storage secret", "name", key) + } + + if fg.OpenShift.ManagedAuthEnv { + secretName, ok := stack.Annotations[storage.AnnotationCredentialsRequestsSecretRef] + if !ok { + return nil, nil, &status.DegradedError{ + Message: "Missing OpenShift cloud credentials request", + Reason: lokiv1.ReasonMissingCredentialsRequest, + Requeue: true, + } + } + + managedAuthCredsKey := client.ObjectKey{Name: secretName, Namespace: stack.Namespace} + if err := k.Get(ctx, managedAuthCredsKey, &managedAuthSecret); err != nil { + if apierrors.IsNotFound(err) { + return nil, nil, &status.DegradedError{ + Message: "Missing OpenShift cloud credentials secret", + Reason: lokiv1.ReasonMissingManagedAuthSecret, + Requeue: true, + } + } + return nil, nil, kverrors.Wrap(err, "failed to lookup OpenShift CCO managed authentication credentials secret", "name", stack) + } + + return &storageSecret, &managedAuthSecret, nil } - return &storageSecret, nil + return &storageSecret, nil, nil } -// extractSecret reads a k8s secret into a manifest object storage struct if valid. -func extractSecret(s *corev1.Secret, secretType lokiv1.ObjectStorageSecretType) (storage.Options, error) { - hash, err := hashSecretData(s) +// extractSecrets reads the k8s obj storage secret into a manifest object storage struct if valid. +// The managed auth is also read into the manifest object under the right circumstances. +func extractSecrets(secretType lokiv1.ObjectStorageSecretType, objStore, managedAuth *corev1.Secret, fg configv1.FeatureGates) (storage.Options, error) { + hash, err := hashSecretData(objStore) if err != nil { return storage.Options{}, kverrors.Wrap(err, "error calculating hash for secret", "type", secretType) } storageOpts := storage.Options{ - SecretName: s.Name, + SecretName: objStore.Name, SecretSHA1: hash, SharedStore: secretType, } + if fg.OpenShift.ManagedAuthEnabled() { + var managedAuthHash string + managedAuthHash, err = hashSecretData(managedAuth) + if err != nil { + return storage.Options{}, kverrors.Wrap(err, "error calculating hash for secret", "type", client.ObjectKeyFromObject(managedAuth)) + } + + storageOpts.OpenShift = storage.OpenShiftOptions{ + CloudCredentials: storage.CloudCredentials{ + SecretName: managedAuth.Name, + SHA1: managedAuthHash, + }, + } + } + switch secretType { case lokiv1.ObjectStorageSecretAzure: - storageOpts.Azure, err = extractAzureConfigSecret(s) + storageOpts.Azure, err = extractAzureConfigSecret(objStore) case lokiv1.ObjectStorageSecretGCS: - storageOpts.GCS, err = extractGCSConfigSecret(s) + storageOpts.GCS, err = extractGCSConfigSecret(objStore) case lokiv1.ObjectStorageSecretS3: - storageOpts.S3, err = extractS3ConfigSecret(s) + storageOpts.S3, err = extractS3ConfigSecret(objStore, fg) case lokiv1.ObjectStorageSecretSwift: - storageOpts.Swift, err = extractSwiftConfigSecret(s) + storageOpts.Swift, err = extractSwiftConfigSecret(objStore) case lokiv1.ObjectStorageSecretAlibabaCloud: - storageOpts.AlibabaCloud, err = extractAlibabaCloudConfigSecret(s) + storageOpts.AlibabaCloud, err = extractAlibabaCloudConfigSecret(objStore) default: return storage.Options{}, kverrors.New("unknown secret type", "type", secretType) } @@ -146,7 +192,7 @@ func extractGCSConfigSecret(s *corev1.Secret) (*storage.GCSStorageConfig, error) }, nil } -func extractS3ConfigSecret(s *corev1.Secret) (*storage.S3StorageConfig, error) { +func extractS3ConfigSecret(s *corev1.Secret, fg configv1.FeatureGates) (*storage.S3StorageConfig, error) { // Extract and validate mandatory fields buckets := s.Data[storage.KeyAWSBucketNames] if len(buckets) == 0 { @@ -176,8 +222,29 @@ func extractS3ConfigSecret(s *corev1.Secret) (*storage.S3StorageConfig, error) { SSE: sseCfg, } + var ( + isManagedAuthEnv = len(roleArn) != 0 + isStaticAuthEnv = !isManagedAuthEnv + ) + switch { - case len(roleArn) == 0: + case fg.OpenShift.ManagedAuthEnabled(): + cfg.STS = true + cfg.Audience = storage.AWSOpenShiftAudience + // Do not allow users overriding the role arn provided on Loki Operator installation + if len(roleArn) != 0 { + return nil, kverrors.New("extra secret field set", "field", storage.KeyAWSRoleArn) + } + if len(audience) != 0 { + return nil, kverrors.New("extra secret field set", "field", storage.KeyAWSAudience) + } + // In the STS case region is not an optional field + if len(region) == 0 { + return nil, kverrors.New("missing secret field", "field", storage.KeyAWSRegion) + } + + return cfg, nil + case isStaticAuthEnv: cfg.Endpoint = string(endpoint) if len(endpoint) == 0 { @@ -191,8 +258,7 @@ func extractS3ConfigSecret(s *corev1.Secret) (*storage.S3StorageConfig, error) { } return cfg, nil - // TODO(JoaoBraveCoding) For CCO integration here we will first check if we get a secret, OS use-case - case len(roleArn) != 0: // Extract STS from user provided values + case isManagedAuthEnv: // Extract STS from user provided values cfg.STS = true cfg.Audience = string(audience) diff --git a/operator/internal/handlers/internal/storage/secrets_test.go b/operator/internal/handlers/internal/storage/secrets_test.go index c72c63ea1ee12..535fd3a0aa141 100644 --- a/operator/internal/handlers/internal/storage/secrets_test.go +++ b/operator/internal/handlers/internal/storage/secrets_test.go @@ -7,6 +7,7 @@ import ( corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + configv1 "github.com/grafana/loki/operator/apis/config/v1" lokiv1 "github.com/grafana/loki/operator/apis/loki/v1" ) @@ -135,7 +136,7 @@ func TestAzureExtract(t *testing.T) { t.Run(tst.name, func(t *testing.T) { t.Parallel() - opts, err := extractSecret(tst.secret, lokiv1.ObjectStorageSecretAzure) + opts, err := extractSecrets(lokiv1.ObjectStorageSecretAzure, tst.secret, nil, configv1.FeatureGates{}) if !tst.wantErr { require.NoError(t, err) require.NotEmpty(t, opts.SecretName) @@ -186,7 +187,7 @@ func TestGCSExtract(t *testing.T) { t.Run(tst.name, func(t *testing.T) { t.Parallel() - _, err := extractSecret(tst.secret, lokiv1.ObjectStorageSecretGCS) + _, err := extractSecrets(lokiv1.ObjectStorageSecretGCS, tst.secret, nil, configv1.FeatureGates{}) if !tst.wantErr { require.NoError(t, err) } @@ -360,7 +361,7 @@ func TestS3Extract(t *testing.T) { t.Run(tst.name, func(t *testing.T) { t.Parallel() - opts, err := extractSecret(tst.secret, lokiv1.ObjectStorageSecretS3) + opts, err := extractSecrets(lokiv1.ObjectStorageSecretS3, tst.secret, nil, configv1.FeatureGates{}) if !tst.wantErr { require.NoError(t, err) require.NotEmpty(t, opts.SecretName) @@ -374,6 +375,80 @@ func TestS3Extract(t *testing.T) { } } +func TestS3Extract_WithOpenShiftManagedAuth(t *testing.T) { + fg := configv1.FeatureGates{ + OpenShift: configv1.OpenShiftFeatureGates{ + Enabled: true, + ManagedAuthEnv: true, + }, + } + type test struct { + name string + secret *corev1.Secret + managedAuthSecret *corev1.Secret + wantErr bool + } + table := []test{ + { + name: "missing role-arn", + secret: &corev1.Secret{}, + managedAuthSecret: &corev1.Secret{}, + wantErr: true, + }, + { + name: "missing region", + secret: &corev1.Secret{}, + managedAuthSecret: &corev1.Secret{}, + wantErr: true, + }, + { + name: "override role arn not allowed", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "role_arn": []byte("role-arn"), + }, + }, + managedAuthSecret: &corev1.Secret{}, + wantErr: true, + }, + { + name: "STS all set", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "bucketnames": []byte("this,that"), + "region": []byte("a-region"), + }, + }, + managedAuthSecret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "managed-auth"}, + }, + }, + } + for _, tst := range table { + tst := tst + t.Run(tst.name, func(t *testing.T) { + t.Parallel() + + opts, err := extractSecrets(lokiv1.ObjectStorageSecretS3, tst.secret, tst.managedAuthSecret, fg) + if !tst.wantErr { + require.NoError(t, err) + require.NotEmpty(t, opts.SecretName) + require.NotEmpty(t, opts.SecretSHA1) + require.Equal(t, opts.SharedStore, lokiv1.ObjectStorageSecretS3) + require.True(t, opts.S3.STS) + require.Equal(t, opts.S3.Audience, "openshift") + require.Equal(t, opts.OpenShift.CloudCredentials.SecretName, tst.managedAuthSecret.Name) + require.NotEmpty(t, opts.OpenShift.CloudCredentials.SHA1) + } + if tst.wantErr { + require.NotNil(t, err) + } + }) + } +} + func TestSwiftExtract(t *testing.T) { type test struct { name string @@ -509,7 +584,7 @@ func TestSwiftExtract(t *testing.T) { t.Run(tst.name, func(t *testing.T) { t.Parallel() - opts, err := extractSecret(tst.secret, lokiv1.ObjectStorageSecretSwift) + opts, err := extractSecrets(lokiv1.ObjectStorageSecretSwift, tst.secret, nil, configv1.FeatureGates{}) if !tst.wantErr { require.NoError(t, err) require.NotEmpty(t, opts.SecretName) @@ -583,7 +658,7 @@ func TestAlibabaCloudExtract(t *testing.T) { t.Run(tst.name, func(t *testing.T) { t.Parallel() - opts, err := extractSecret(tst.secret, lokiv1.ObjectStorageSecretAlibabaCloud) + opts, err := extractSecrets(lokiv1.ObjectStorageSecretAlibabaCloud, tst.secret, nil, configv1.FeatureGates{}) if !tst.wantErr { require.NoError(t, err) require.NotEmpty(t, opts.SecretName) diff --git a/operator/internal/handlers/internal/storage/storage.go b/operator/internal/handlers/internal/storage/storage.go index e1657121ccd6d..32b59522ef6a5 100644 --- a/operator/internal/handlers/internal/storage/storage.go +++ b/operator/internal/handlers/internal/storage/storage.go @@ -20,13 +20,14 @@ import ( // - The object storage schema config is invalid. // - The object storage CA ConfigMap is missing if one referenced. // - The object storage CA ConfigMap data is invalid. +// - The object storage managed auth secret is missing (Only on OpenShift STS-clusters) func BuildOptions(ctx context.Context, k k8s.Client, stack *lokiv1.LokiStack, fg configv1.FeatureGates) (storage.Options, error) { - storageSecret, err := getSecret(ctx, k, stack) + storageSecret, managedAuthSecret, err := getSecrets(ctx, k, stack, fg) if err != nil { return storage.Options{}, err } - objStore, err := extractSecret(storageSecret, stack.Spec.Storage.Secret.Type) + objStore, err := extractSecrets(stack.Spec.Storage.Secret.Type, storageSecret, managedAuthSecret, fg) if err != nil { return storage.Options{}, &status.DegradedError{ Message: fmt.Sprintf("Invalid object storage secret contents: %s", err), @@ -34,7 +35,7 @@ func BuildOptions(ctx context.Context, k k8s.Client, stack *lokiv1.LokiStack, fg Requeue: false, } } - objStore.OpenShiftEnabled = fg.OpenShift.Enabled + objStore.OpenShift.Enabled = fg.OpenShift.Enabled storageSchemas, err := storage.BuildSchemaConfig( time.Now().UTC(), diff --git a/operator/internal/handlers/internal/storage/storage_test.go b/operator/internal/handlers/internal/storage/storage_test.go index f56e446d6da8f..9bc73630b2dc2 100644 --- a/operator/internal/handlers/internal/storage/storage_test.go +++ b/operator/internal/handlers/internal/storage/storage_test.go @@ -2,6 +2,7 @@ package storage import ( "context" + "fmt" "testing" "github.com/stretchr/testify/require" @@ -16,6 +17,7 @@ import ( configv1 "github.com/grafana/loki/operator/apis/config/v1" lokiv1 "github.com/grafana/loki/operator/apis/loki/v1" "github.com/grafana/loki/operator/internal/external/k8s/k8sfakes" + "github.com/grafana/loki/operator/internal/manifests/storage" "github.com/grafana/loki/operator/internal/status" ) @@ -46,6 +48,16 @@ var ( }, } + defaultManagedAuthSecret = corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{ + Name: "some-stack-secret", + Namespace: "some-ns", + }, + Data: map[string][]byte{ + "region": []byte("a-region"), + }, + } + invalidSecret = corev1.Secret{ ObjectMeta: metav1.ObjectMeta{ Name: "some-stack-secret", @@ -123,6 +135,153 @@ func TestBuildOptions_WhenMissingSecret_SetDegraded(t *testing.T) { require.Equal(t, degradedErr, err) } +func TestBuildOptions_WhenMissingCloudCredentialsRequest_SetDegraded(t *testing.T) { + sw := &k8sfakes.FakeStatusWriter{} + k := &k8sfakes.FakeClient{} + r := ctrl.Request{ + NamespacedName: types.NamespacedName{ + Name: "my-stack", + Namespace: "some-ns", + }, + } + + fg := configv1.FeatureGates{ + OpenShift: configv1.OpenShiftFeatureGates{ + ManagedAuthEnv: true, + }, + } + + degradedErr := &status.DegradedError{ + Message: "Missing OpenShift cloud credentials request", + Reason: lokiv1.ReasonMissingCredentialsRequest, + Requeue: true, + } + + stack := &lokiv1.LokiStack{ + TypeMeta: metav1.TypeMeta{ + Kind: "LokiStack", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "my-stack", + Namespace: "some-ns", + UID: "b23f9a38-9672-499f-8c29-15ede74d3ece", + Annotations: map[string]string{}, + }, + Spec: lokiv1.LokiStackSpec{ + Size: lokiv1.SizeOneXExtraSmall, + Storage: lokiv1.ObjectStorageSpec{ + Schemas: []lokiv1.ObjectStorageSchema{ + { + Version: lokiv1.ObjectStorageSchemaV11, + EffectiveDate: "2020-10-11", + }, + }, + Secret: lokiv1.ObjectStorageSecretSpec{ + Name: defaultManagedAuthSecret.Name, + Type: lokiv1.ObjectStorageSecretS3, + }, + }, + }, + } + + k.GetStub = func(_ context.Context, name types.NamespacedName, object client.Object, _ ...client.GetOption) error { + _, isLokiStack := object.(*lokiv1.LokiStack) + if r.Name == name.Name && r.Namespace == name.Namespace && isLokiStack { + k.SetClientObject(object, stack) + return nil + } + if name.Name == defaultManagedAuthSecret.Name { + k.SetClientObject(object, &defaultManagedAuthSecret) + return nil + } + return apierrors.NewNotFound(schema.GroupResource{}, "something is not found") + } + + k.StatusStub = func() client.StatusWriter { return sw } + + _, err := BuildOptions(context.TODO(), k, stack, fg) + + // make sure error is returned + require.Error(t, err) + require.Equal(t, degradedErr, err) +} + +func TestBuildOptions_WhenMissingCloudCredentialsSecret_SetDegraded(t *testing.T) { + sw := &k8sfakes.FakeStatusWriter{} + k := &k8sfakes.FakeClient{} + r := ctrl.Request{ + NamespacedName: types.NamespacedName{ + Name: "my-stack", + Namespace: "some-ns", + }, + } + + fg := configv1.FeatureGates{ + OpenShift: configv1.OpenShiftFeatureGates{ + ManagedAuthEnv: true, + }, + } + + degradedErr := &status.DegradedError{ + Message: "Missing OpenShift cloud credentials secret", + Reason: lokiv1.ReasonMissingManagedAuthSecret, + Requeue: true, + } + + stack := &lokiv1.LokiStack{ + TypeMeta: metav1.TypeMeta{ + Kind: "LokiStack", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "my-stack", + Namespace: "some-ns", + UID: "b23f9a38-9672-499f-8c29-15ede74d3ece", + Annotations: map[string]string{ + storage.AnnotationCredentialsRequestsSecretRef: "my-stack-aws-creds", + }, + }, + Spec: lokiv1.LokiStackSpec{ + Size: lokiv1.SizeOneXExtraSmall, + Storage: lokiv1.ObjectStorageSpec{ + Schemas: []lokiv1.ObjectStorageSchema{ + { + Version: lokiv1.ObjectStorageSchemaV11, + EffectiveDate: "2020-10-11", + }, + }, + Secret: lokiv1.ObjectStorageSecretSpec{ + Name: defaultManagedAuthSecret.Name, + Type: lokiv1.ObjectStorageSecretS3, + }, + }, + }, + } + + k.GetStub = func(_ context.Context, name types.NamespacedName, object client.Object, _ ...client.GetOption) error { + _, isLokiStack := object.(*lokiv1.LokiStack) + if r.Name == name.Name && r.Namespace == name.Namespace && isLokiStack { + k.SetClientObject(object, stack) + return nil + } + if name.Name == defaultManagedAuthSecret.Name { + k.SetClientObject(object, &defaultManagedAuthSecret) + return nil + } + if name.Name == fmt.Sprintf("%s-aws-creds", stack.Name) { + return apierrors.NewNotFound(schema.GroupResource{}, "cloud credentials auth secret is not found") + } + return apierrors.NewNotFound(schema.GroupResource{}, "something is not found") + } + + k.StatusStub = func() client.StatusWriter { return sw } + + _, err := BuildOptions(context.TODO(), k, stack, fg) + + // make sure error is returned + require.Error(t, err) + require.Equal(t, degradedErr, err) +} + func TestBuildOptions_WhenInvalidSecret_SetDegraded(t *testing.T) { sw := &k8sfakes.FakeStatusWriter{} k := &k8sfakes.FakeClient{} diff --git a/operator/internal/manifests/compactor.go b/operator/internal/manifests/compactor.go index 0c5c6b038a1cf..24d95945cf0ac 100644 --- a/operator/internal/manifests/compactor.go +++ b/operator/internal/manifests/compactor.go @@ -67,7 +67,7 @@ func BuildCompactor(opts Options) ([]client.Object, error) { // NewCompactorStatefulSet creates a statefulset object for a compactor. func NewCompactorStatefulSet(opts Options) *appsv1.StatefulSet { l := ComponentLabels(LabelCompactorComponent, opts.Name) - a := commonAnnotations(opts.ConfigSHA1, opts.ObjectStorage.SecretSHA1, opts.CertRotationRequiredAt) + a := commonAnnotations(opts) podSpec := corev1.PodSpec{ ServiceAccountName: opts.Name, Affinity: configureAffinity(LabelCompactorComponent, opts.Name, opts.Gates.DefaultNodeAffinity, opts.Stack.Template.Compactor), diff --git a/operator/internal/manifests/distributor.go b/operator/internal/manifests/distributor.go index 7b5a0a033f19a..ca84da935982a 100644 --- a/operator/internal/manifests/distributor.go +++ b/operator/internal/manifests/distributor.go @@ -67,7 +67,7 @@ func BuildDistributor(opts Options) ([]client.Object, error) { // NewDistributorDeployment creates a deployment object for a distributor func NewDistributorDeployment(opts Options) *appsv1.Deployment { l := ComponentLabels(LabelDistributorComponent, opts.Name) - a := commonAnnotations(opts.ConfigSHA1, opts.ObjectStorage.SecretSHA1, opts.CertRotationRequiredAt) + a := commonAnnotations(opts) podSpec := corev1.PodSpec{ ServiceAccountName: opts.Name, Affinity: configureAffinity(LabelDistributorComponent, opts.Name, opts.Gates.DefaultNodeAffinity, opts.Stack.Template.Distributor), diff --git a/operator/internal/manifests/gateway.go b/operator/internal/manifests/gateway.go index 1ba3a9905e577..b3809ed9c296c 100644 --- a/operator/internal/manifests/gateway.go +++ b/operator/internal/manifests/gateway.go @@ -114,7 +114,7 @@ func BuildGateway(opts Options) ([]client.Object, error) { // NewGatewayDeployment creates a deployment object for a lokiStack-gateway func NewGatewayDeployment(opts Options, sha1C string) *appsv1.Deployment { l := ComponentLabels(LabelGatewayComponent, opts.Name) - a := commonAnnotations(sha1C, "", opts.CertRotationRequiredAt) + a := gatewayAnnotations(sha1C, opts.CertRotationRequiredAt) podSpec := corev1.PodSpec{ ServiceAccountName: GatewayName(opts.Name), Affinity: configureAffinity(LabelGatewayComponent, opts.Name, opts.Gates.DefaultNodeAffinity, opts.Stack.Template.Gateway), diff --git a/operator/internal/manifests/indexgateway.go b/operator/internal/manifests/indexgateway.go index f4dbbe8f6f248..171598cc2822e 100644 --- a/operator/internal/manifests/indexgateway.go +++ b/operator/internal/manifests/indexgateway.go @@ -73,7 +73,7 @@ func BuildIndexGateway(opts Options) ([]client.Object, error) { // NewIndexGatewayStatefulSet creates a statefulset object for an index-gateway func NewIndexGatewayStatefulSet(opts Options) *appsv1.StatefulSet { l := ComponentLabels(LabelIndexGatewayComponent, opts.Name) - a := commonAnnotations(opts.ConfigSHA1, opts.ObjectStorage.SecretSHA1, opts.CertRotationRequiredAt) + a := commonAnnotations(opts) podSpec := corev1.PodSpec{ ServiceAccountName: opts.Name, Affinity: configureAffinity(LabelIndexGatewayComponent, opts.Name, opts.Gates.DefaultNodeAffinity, opts.Stack.Template.IndexGateway), diff --git a/operator/internal/manifests/ingester.go b/operator/internal/manifests/ingester.go index 6e7a50af4806e..5aabb3abfe73a 100644 --- a/operator/internal/manifests/ingester.go +++ b/operator/internal/manifests/ingester.go @@ -73,7 +73,7 @@ func BuildIngester(opts Options) ([]client.Object, error) { // NewIngesterStatefulSet creates a deployment object for an ingester func NewIngesterStatefulSet(opts Options) *appsv1.StatefulSet { l := ComponentLabels(LabelIngesterComponent, opts.Name) - a := commonAnnotations(opts.ConfigSHA1, opts.ObjectStorage.SecretSHA1, opts.CertRotationRequiredAt) + a := commonAnnotations(opts) podSpec := corev1.PodSpec{ ServiceAccountName: opts.Name, Affinity: configureAffinity(LabelIngesterComponent, opts.Name, opts.Gates.DefaultNodeAffinity, opts.Stack.Template.Ingester), diff --git a/operator/internal/manifests/openshift/credentialsrequest.go b/operator/internal/manifests/openshift/credentialsrequest.go new file mode 100644 index 0000000000000..8fc2c5d3f5129 --- /dev/null +++ b/operator/internal/manifests/openshift/credentialsrequest.go @@ -0,0 +1,96 @@ +package openshift + +import ( + "fmt" + "os" + "path" + + "github.com/ViaQ/logerr/v2/kverrors" + cloudcredentialv1 "github.com/openshift/cloud-credential-operator/pkg/apis/cloudcredential/v1" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime" + "sigs.k8s.io/controller-runtime/pkg/client" + + "github.com/grafana/loki/operator/internal/manifests/storage" +) + +const ( + ccoNamespace = "openshift-cloud-credential-operator" +) + +func BuildCredentialsRequest(opts Options) (*cloudcredentialv1.CredentialsRequest, error) { + stack := client.ObjectKey{Name: opts.BuildOpts.LokiStackName, Namespace: opts.BuildOpts.LokiStackNamespace} + + providerSpec, secretName, err := encodeProviderSpec(opts.BuildOpts.LokiStackName, opts.ManagedAuthEnv) + if err != nil { + return nil, kverrors.Wrap(err, "failed encoding credentialsrequest provider spec") + } + + return &cloudcredentialv1.CredentialsRequest{ + ObjectMeta: metav1.ObjectMeta{ + Name: fmt.Sprintf("%s-%s", stack.Namespace, secretName), + Namespace: ccoNamespace, + Annotations: map[string]string{ + AnnotationCredentialsRequestOwner: stack.String(), + }, + }, + Spec: cloudcredentialv1.CredentialsRequestSpec{ + SecretRef: corev1.ObjectReference{ + Name: secretName, + Namespace: stack.Namespace, + }, + ProviderSpec: providerSpec, + ServiceAccountNames: []string{ + stack.Name, + }, + CloudTokenPath: path.Join(storage.SATokenVolumeOcpDirectory, "token"), + }, + }, nil +} + +func encodeProviderSpec(stackName string, env *ManagedAuthEnv) (*runtime.RawExtension, string, error) { + var ( + spec runtime.Object + secretName string + ) + + switch { + case env.AWS != nil: + spec = &cloudcredentialv1.AWSProviderSpec{ + StatementEntries: []cloudcredentialv1.StatementEntry{ + { + Action: []string{ + "s3:ListBucket", + "s3:PutObject", + "s3:GetObject", + "s3:DeleteObject", + }, + Effect: "Allow", + Resource: "arn:aws:s3:*:*:*", + }, + }, + STSIAMRoleARN: env.AWS.RoleARN, + } + secretName = fmt.Sprintf("%s-aws-creds", stackName) + } + + encodedSpec, err := cloudcredentialv1.Codec.EncodeProviderSpec(spec.DeepCopyObject()) + return encodedSpec, secretName, err +} + +func DiscoverManagedAuthEnv() *ManagedAuthEnv { + // AWS + roleARN := os.Getenv("ROLEARN") + + switch { + case roleARN != "": + return &ManagedAuthEnv{ + AWS: &AWSSTSEnv{ + RoleARN: roleARN, + }, + } + } + + return nil +} diff --git a/operator/internal/manifests/openshift/credentialsrequest_test.go b/operator/internal/manifests/openshift/credentialsrequest_test.go new file mode 100644 index 0000000000000..0672cadfc210f --- /dev/null +++ b/operator/internal/manifests/openshift/credentialsrequest_test.go @@ -0,0 +1,119 @@ +package openshift + +import ( + "strings" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/operator/internal/manifests/storage" +) + +func TestBuildCredentialsRequest_HasOwnerAnnotation(t *testing.T) { + opts := Options{ + BuildOpts: BuildOptions{ + LokiStackName: "a-stack", + LokiStackNamespace: "ns", + }, + ManagedAuthEnv: &ManagedAuthEnv{ + AWS: &AWSSTSEnv{ + RoleARN: "role-arn", + }, + }, + } + + credReq, err := BuildCredentialsRequest(opts) + require.NoError(t, err) + require.Contains(t, credReq.Annotations, AnnotationCredentialsRequestOwner) +} + +func TestBuildCredentialsRequest_HasSecretRef_MatchingLokiStackNamespace(t *testing.T) { + opts := Options{ + BuildOpts: BuildOptions{ + LokiStackName: "a-stack", + LokiStackNamespace: "ns", + }, + ManagedAuthEnv: &ManagedAuthEnv{ + AWS: &AWSSTSEnv{ + RoleARN: "role-arn", + }, + }, + } + + credReq, err := BuildCredentialsRequest(opts) + require.NoError(t, err) + require.Equal(t, opts.BuildOpts.LokiStackNamespace, credReq.Spec.SecretRef.Namespace) +} + +func TestBuildCredentialsRequest_HasServiceAccountNames_ContainsLokiStackName(t *testing.T) { + opts := Options{ + BuildOpts: BuildOptions{ + LokiStackName: "a-stack", + LokiStackNamespace: "ns", + }, + ManagedAuthEnv: &ManagedAuthEnv{ + AWS: &AWSSTSEnv{ + RoleARN: "role-arn", + }, + }, + } + + credReq, err := BuildCredentialsRequest(opts) + require.NoError(t, err) + require.Contains(t, credReq.Spec.ServiceAccountNames, opts.BuildOpts.LokiStackName) +} + +func TestBuildCredentialsRequest_CloudTokenPath_MatchinOpenShiftSADirectory(t *testing.T) { + opts := Options{ + BuildOpts: BuildOptions{ + LokiStackName: "a-stack", + LokiStackNamespace: "ns", + }, + ManagedAuthEnv: &ManagedAuthEnv{ + AWS: &AWSSTSEnv{ + RoleARN: "role-arn", + }, + }, + } + + credReq, err := BuildCredentialsRequest(opts) + require.NoError(t, err) + require.True(t, strings.HasPrefix(credReq.Spec.CloudTokenPath, storage.SATokenVolumeOcpDirectory)) +} + +func TestBuildCredentialsRequest_FollowsNamingConventions(t *testing.T) { + tests := []struct { + desc string + opts Options + wantName string + wantSecretName string + }{ + { + desc: "aws", + opts: Options{ + BuildOpts: BuildOptions{ + LokiStackName: "a-stack", + LokiStackNamespace: "ns", + }, + ManagedAuthEnv: &ManagedAuthEnv{ + AWS: &AWSSTSEnv{ + RoleARN: "role-arn", + }, + }, + }, + wantName: "ns-a-stack-aws-creds", + wantSecretName: "a-stack-aws-creds", + }, + } + for _, test := range tests { + test := test + t.Run(test.desc, func(t *testing.T) { + t.Parallel() + + credReq, err := BuildCredentialsRequest(test.opts) + require.NoError(t, err) + require.Equal(t, test.wantName, credReq.Name) + require.Equal(t, test.wantSecretName, credReq.Spec.SecretRef.Name) + }) + } +} diff --git a/operator/internal/manifests/openshift/options.go b/operator/internal/manifests/openshift/options.go index 2ebf5ebde1f46..e5d33a3355269 100644 --- a/operator/internal/manifests/openshift/options.go +++ b/operator/internal/manifests/openshift/options.go @@ -14,6 +14,7 @@ type Options struct { BuildOpts BuildOptions Authentication []AuthenticationSpec Authorization AuthorizationSpec + ManagedAuthEnv *ManagedAuthEnv } // AuthenticationSpec describes the authentication specification @@ -54,6 +55,14 @@ type TenantData struct { CookieSecret string } +type AWSSTSEnv struct { + RoleARN string +} + +type ManagedAuthEnv struct { + AWS *AWSSTSEnv +} + // NewOptions returns an openshift options struct. func NewOptions( stackName, stackNamespace string, diff --git a/operator/internal/manifests/openshift/var.go b/operator/internal/manifests/openshift/var.go index 5e3ac6300e3eb..84928c48d7e28 100644 --- a/operator/internal/manifests/openshift/var.go +++ b/operator/internal/manifests/openshift/var.go @@ -48,6 +48,8 @@ var ( MonitoringSVCUserWorkload = "alertmanager-user-workload" MonitoringUserWorkloadNS = "openshift-user-workload-monitoring" + + AnnotationCredentialsRequestOwner = "loki.grafana.com/credentialsrequest-owner" ) func authorizerRbacName(componentName string) string { diff --git a/operator/internal/manifests/querier.go b/operator/internal/manifests/querier.go index c807fe8ed1f0d..f98de94a060ea 100644 --- a/operator/internal/manifests/querier.go +++ b/operator/internal/manifests/querier.go @@ -73,7 +73,7 @@ func BuildQuerier(opts Options) ([]client.Object, error) { // NewQuerierDeployment creates a deployment object for a querier func NewQuerierDeployment(opts Options) *appsv1.Deployment { l := ComponentLabels(LabelQuerierComponent, opts.Name) - a := commonAnnotations(opts.ConfigSHA1, opts.ObjectStorage.SecretSHA1, opts.CertRotationRequiredAt) + a := commonAnnotations(opts) podSpec := corev1.PodSpec{ ServiceAccountName: opts.Name, Affinity: configureAffinity(LabelQuerierComponent, opts.Name, opts.Gates.DefaultNodeAffinity, opts.Stack.Template.Querier), diff --git a/operator/internal/manifests/query-frontend.go b/operator/internal/manifests/query-frontend.go index e1023e872371c..7786470020cb1 100644 --- a/operator/internal/manifests/query-frontend.go +++ b/operator/internal/manifests/query-frontend.go @@ -67,7 +67,7 @@ func BuildQueryFrontend(opts Options) ([]client.Object, error) { // NewQueryFrontendDeployment creates a deployment object for a query-frontend func NewQueryFrontendDeployment(opts Options) *appsv1.Deployment { l := ComponentLabels(LabelQueryFrontendComponent, opts.Name) - a := commonAnnotations(opts.ConfigSHA1, opts.ObjectStorage.SecretSHA1, opts.CertRotationRequiredAt) + a := commonAnnotations(opts) podSpec := corev1.PodSpec{ ServiceAccountName: opts.Name, Affinity: configureAffinity(LabelQueryFrontendComponent, opts.Name, opts.Gates.DefaultNodeAffinity, opts.Stack.Template.QueryFrontend), diff --git a/operator/internal/manifests/ruler.go b/operator/internal/manifests/ruler.go index c34adb765ee71..4e9eaf22b66b0 100644 --- a/operator/internal/manifests/ruler.go +++ b/operator/internal/manifests/ruler.go @@ -97,7 +97,7 @@ func NewRulerStatefulSet(opts Options) *appsv1.StatefulSet { } l := ComponentLabels(LabelRulerComponent, opts.Name) - a := commonAnnotations(opts.ConfigSHA1, opts.ObjectStorage.SecretSHA1, opts.CertRotationRequiredAt) + a := commonAnnotations(opts) podSpec := corev1.PodSpec{ Affinity: configureAffinity(LabelRulerComponent, opts.Name, opts.Gates.DefaultNodeAffinity, opts.Stack.Template.Ruler), Volumes: []corev1.Volume{ diff --git a/operator/internal/manifests/storage/configure.go b/operator/internal/manifests/storage/configure.go index 06956827db420..6f7b22c4bd8ce 100644 --- a/operator/internal/manifests/storage/configure.go +++ b/operator/internal/manifests/storage/configure.go @@ -131,6 +131,11 @@ func ensureObjectStoreCredentials(p *corev1.PodSpec, opts Options) corev1.PodSpe container.Env = append(container.Env, managedAuthCredentials(opts)...) volumes = append(volumes, saTokenVolume(opts)) container.VolumeMounts = append(container.VolumeMounts, saTokenVolumeMount(opts)) + + if opts.OpenShift.ManagedAuthEnabled() { + volumes = append(volumes, managedAuthVolume(opts)) + container.VolumeMounts = append(container.VolumeMounts, managedAuthVolumeMount(opts)) + } } else { container.Env = append(container.Env, staticAuthCredentials(opts)...) } @@ -179,9 +184,16 @@ func staticAuthCredentials(opts Options) []corev1.EnvVar { func managedAuthCredentials(opts Options) []corev1.EnvVar { switch opts.SharedStore { case lokiv1.ObjectStorageSecretS3: - return []corev1.EnvVar{ - envVarFromSecret(EnvAWSRoleArn, opts.SecretName, KeyAWSRoleArn), - envVarFromValue(EnvAWSWebIdentityTokenFile, path.Join(opts.S3.WebIdentityTokenFile, "token")), + if opts.OpenShift.ManagedAuthEnabled() { + return []corev1.EnvVar{ + envVarFromValue(EnvAWSCredentialsFile, path.Join(managedAuthSecretDirectory, KeyAWSCredentialsFilename)), + envVarFromValue(EnvAWSSdkLoadConfig, "true"), + } + } else { + return []corev1.EnvVar{ + envVarFromSecret(EnvAWSRoleArn, opts.SecretName, KeyAWSRoleArn), + envVarFromValue(EnvAWSWebIdentityTokenFile, path.Join(opts.S3.WebIdentityTokenFile, "token")), + } } default: return []corev1.EnvVar{} @@ -270,8 +282,8 @@ func setSATokenPath(opts *Options) { switch opts.SharedStore { case lokiv1.ObjectStorageSecretS3: opts.S3.WebIdentityTokenFile = saTokenVolumeK8sDirectory - if opts.OpenShiftEnabled { - opts.S3.WebIdentityTokenFile = saTokenVolumeOcpDirectory + if opts.OpenShift.Enabled { + opts.S3.WebIdentityTokenFile = SATokenVolumeOcpDirectory } } } @@ -297,8 +309,8 @@ func saTokenVolume(opts Options) corev1.Volume { if opts.S3.Audience != "" { audience = opts.S3.Audience } - if opts.OpenShiftEnabled { - audience = awsOpenShiftAudience + if opts.OpenShift.Enabled { + audience = AWSOpenShiftAudience } } return corev1.Volume{ @@ -318,3 +330,21 @@ func saTokenVolume(opts Options) corev1.Volume { }, } } + +func managedAuthVolumeMount(opts Options) corev1.VolumeMount { + return corev1.VolumeMount{ + Name: opts.OpenShift.CloudCredentials.SecretName, + MountPath: managedAuthSecretDirectory, + } +} + +func managedAuthVolume(opts Options) corev1.Volume { + return corev1.Volume{ + Name: opts.OpenShift.CloudCredentials.SecretName, + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{ + SecretName: opts.OpenShift.CloudCredentials.SecretName, + }, + }, + } +} diff --git a/operator/internal/manifests/storage/configure_test.go b/operator/internal/manifests/storage/configure_test.go index 220d0c6c701a6..3b3029733554d 100644 --- a/operator/internal/manifests/storage/configure_test.go +++ b/operator/internal/manifests/storage/configure_test.go @@ -393,13 +393,19 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { { desc: "object storage S3 in STS Mode in OpenShift", opts: Options{ - SecretName: "test", - OpenShiftEnabled: true, - SharedStore: lokiv1.ObjectStorageSecretS3, + SecretName: "test", + SharedStore: lokiv1.ObjectStorageSecretS3, S3: &S3StorageConfig{ STS: true, Audience: "test", }, + OpenShift: OpenShiftOptions{ + Enabled: true, + CloudCredentials: CloudCredentials{ + SecretName: "cloud-credentials", + SHA1: "deadbeef", + }, + }, }, dpl: &appsv1.Deployment{ Spec: appsv1.DeploymentSpec{ @@ -432,22 +438,20 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { ReadOnly: false, MountPath: "/var/run/secrets/openshift/serviceaccount", }, + { + Name: "cloud-credentials", + ReadOnly: false, + MountPath: "/etc/storage/managed-auth", + }, }, Env: []corev1.EnvVar{ { - Name: EnvAWSRoleArn, - ValueFrom: &corev1.EnvVarSource{ - SecretKeyRef: &corev1.SecretKeySelector{ - LocalObjectReference: corev1.LocalObjectReference{ - Name: "test", - }, - Key: KeyAWSRoleArn, - }, - }, + Name: "AWS_SHARED_CREDENTIALS_FILE", + Value: "/etc/storage/managed-auth/credentials", }, { - Name: "AWS_WEB_IDENTITY_TOKEN_FILE", - Value: "/var/run/secrets/openshift/serviceaccount/token", + Name: "AWS_SDK_LOAD_CONFIG", + Value: "true", }, }, }, @@ -477,6 +481,14 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { }, }, }, + { + Name: "cloud-credentials", + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{ + SecretName: "cloud-credentials", + }, + }, + }, }, }, }, @@ -948,6 +960,111 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) { }, }, }, + { + desc: "object storage S3 in STS Mode in OpenShift", + opts: Options{ + SecretName: "test", + SharedStore: lokiv1.ObjectStorageSecretS3, + S3: &S3StorageConfig{ + STS: true, + Audience: "test", + }, + OpenShift: OpenShiftOptions{ + Enabled: true, + CloudCredentials: CloudCredentials{ + SecretName: "cloud-credentials", + SHA1: "deadbeef", + }, + }, + }, + sts: &appsv1.StatefulSet{ + Spec: appsv1.StatefulSetSpec{ + Template: corev1.PodTemplateSpec{ + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Name: "loki-ingester", + }, + }, + }, + }, + }, + }, + want: &appsv1.StatefulSet{ + Spec: appsv1.StatefulSetSpec{ + Template: corev1.PodTemplateSpec{ + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Name: "loki-ingester", + VolumeMounts: []corev1.VolumeMount{ + { + Name: "test", + ReadOnly: false, + MountPath: "/etc/storage/secrets", + }, + { + Name: saTokenVolumeName, + ReadOnly: false, + MountPath: "/var/run/secrets/openshift/serviceaccount", + }, + { + Name: "cloud-credentials", + ReadOnly: false, + MountPath: "/etc/storage/managed-auth", + }, + }, + Env: []corev1.EnvVar{ + { + Name: "AWS_SHARED_CREDENTIALS_FILE", + Value: "/etc/storage/managed-auth/credentials", + }, + { + Name: "AWS_SDK_LOAD_CONFIG", + Value: "true", + }, + }, + }, + }, + Volumes: []corev1.Volume{ + { + Name: "test", + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{ + SecretName: "test", + }, + }, + }, + { + Name: saTokenVolumeName, + VolumeSource: corev1.VolumeSource{ + Projected: &corev1.ProjectedVolumeSource{ + Sources: []corev1.VolumeProjection{ + { + ServiceAccountToken: &corev1.ServiceAccountTokenProjection{ + Audience: "openshift", + ExpirationSeconds: ptr.To[int64](3600), + Path: corev1.ServiceAccountTokenKey, + }, + }, + }, + }, + }, + }, + { + Name: "cloud-credentials", + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{ + SecretName: "cloud-credentials", + }, + }, + }, + }, + }, + }, + }, + }, + }, { desc: "object storage S3 with SSE KMS encryption Context", opts: Options{ diff --git a/operator/internal/manifests/storage/options.go b/operator/internal/manifests/storage/options.go index 7ecf7f78b4258..80efb24f62c8b 100644 --- a/operator/internal/manifests/storage/options.go +++ b/operator/internal/manifests/storage/options.go @@ -16,10 +16,11 @@ type Options struct { Swift *SwiftStorageConfig AlibabaCloud *AlibabaCloudStorageConfig - SecretName string - SecretSHA1 string - TLS *TLSConfig - OpenShiftEnabled bool + SecretName string + SecretSHA1 string + TLS *TLSConfig + + OpenShift OpenShiftOptions } // AzureStorageConfig for Azure storage config @@ -86,3 +87,17 @@ type TLSConfig struct { CA string Key string } + +type OpenShiftOptions struct { + Enabled bool + CloudCredentials CloudCredentials +} + +type CloudCredentials struct { + SecretName string + SHA1 string +} + +func (o OpenShiftOptions) ManagedAuthEnabled() bool { + return o.CloudCredentials.SecretName != "" && o.CloudCredentials.SHA1 != "" +} diff --git a/operator/internal/manifests/storage/var.go b/operator/internal/manifests/storage/var.go index 16b7e10d3d1b5..d77de3262d314 100644 --- a/operator/internal/manifests/storage/var.go +++ b/operator/internal/manifests/storage/var.go @@ -15,6 +15,10 @@ const ( EnvAWSRoleArn = "AWS_ROLE_ARN" // EnvAWSWebIdentityToken is the environment variable to specify the path to the web identity token file used in the federated identity workflow. EnvAWSWebIdentityTokenFile = "AWS_WEB_IDENTITY_TOKEN_FILE" + // EnvAWSCredentialsFile is the environment variable to specify the path to the shared credentials file + EnvAWSCredentialsFile = "AWS_SHARED_CREDENTIALS_FILE" + // EnvAWSSdkLoadConfig is the environment that enabled the AWS SDK to enable the shared credentials file to be loaded + EnvAWSSdkLoadConfig = "AWS_SDK_LOAD_CONFIG" // EnvAzureStorageAccountName is the environment variable to specify the Azure storage account name to access the container. EnvAzureStorageAccountName = "AZURE_STORAGE_ACCOUNT_NAME" // EnvAzureStorageAccountKey is the environment variable to specify the Azure storage account key to access the container. @@ -55,6 +59,8 @@ const ( KeyAWSRoleArn = "role_arn" // KeyAWSAudience is the audience for the AWS STS workflow. KeyAWSAudience = "audience" + // KeyAWSCredentialsFilename is the config filename containing the AWS authentication credentials. + KeyAWSCredentialsFilename = "credentials" // KeyAzureStorageAccountKey is the secret data key for the Azure storage account key. KeyAzureStorageAccountKey = "account_key" @@ -102,14 +108,17 @@ const ( KeySwiftUsername = "username" saTokenVolumeK8sDirectory = "/var/run/secrets/kubernetes.io/serviceaccount" - saTokenVolumeOcpDirectory = "/var/run/secrets/openshift/serviceaccount" + SATokenVolumeOcpDirectory = "/var/run/secrets/openshift/serviceaccount" saTokenVolumeName = "bound-sa-token" saTokenExpiration int64 = 3600 - secretDirectory = "/etc/storage/secrets" - storageTLSVolume = "storage-tls" - caDirectory = "/etc/storage/ca" + secretDirectory = "/etc/storage/secrets" + managedAuthSecretDirectory = "/etc/storage/managed-auth" + storageTLSVolume = "storage-tls" + caDirectory = "/etc/storage/ca" awsDefaultAudience = "sts.amazonaws.com" - awsOpenShiftAudience = "openshift" + AWSOpenShiftAudience = "openshift" + + AnnotationCredentialsRequestsSecretRef = "loki.grafana.com/credentials-request-secret-ref" ) diff --git a/operator/internal/manifests/var.go b/operator/internal/manifests/var.go index 1395412b792da..cffb30d0cf900 100644 --- a/operator/internal/manifests/var.go +++ b/operator/internal/manifests/var.go @@ -78,6 +78,8 @@ const ( AnnotationLokiConfigHash string = "loki.grafana.com/config-hash" // AnnotationLokiObjectStoreHash stores the last SHA1 hash of the loki object storage credetials. AnnotationLokiObjectStoreHash string = "loki.grafana.com/object-store-hash" + // AnnotationLokiManagedAuthHash stores the last SHA1 hash of the loki managed auth credentials. + AnnotationLokiManagedAuthHash string = "loki.grafana.com/managed-auth-hash" // LabelCompactorComponent is the label value for the compactor component LabelCompactorComponent string = "compactor" @@ -133,20 +135,30 @@ var ( volumeFileSystemMode = corev1.PersistentVolumeFilesystem ) -func commonAnnotations(configHash, objStoreHash, rotationRequiredAt string) map[string]string { +func commonAnnotations(opts Options) map[string]string { a := map[string]string{ - AnnotationLokiConfigHash: configHash, + AnnotationLokiConfigHash: opts.ConfigSHA1, + AnnotationCertRotationRequiredAt: opts.CertRotationRequiredAt, + } - AnnotationCertRotationRequiredAt: rotationRequiredAt, + if opts.ObjectStorage.SecretSHA1 != "" { + a[AnnotationLokiObjectStoreHash] = opts.ObjectStorage.SecretSHA1 } - if objStoreHash != "" { - a[AnnotationLokiObjectStoreHash] = objStoreHash + if opts.ObjectStorage.OpenShift.CloudCredentials.SHA1 != "" { + a[AnnotationLokiManagedAuthHash] = opts.ObjectStorage.OpenShift.CloudCredentials.SHA1 } return a } +func gatewayAnnotations(configSHA1, certRotationRequiredAt string) map[string]string { + return map[string]string{ + AnnotationLokiConfigHash: configSHA1, + AnnotationCertRotationRequiredAt: certRotationRequiredAt, + } +} + func commonLabels(stackName string) map[string]string { return map[string]string{ "app.kubernetes.io/name": "lokistack", diff --git a/operator/main.go b/operator/main.go index 654cd11d9f0a7..a88a857bcee44 100644 --- a/operator/main.go +++ b/operator/main.go @@ -8,6 +8,7 @@ import ( "github.com/ViaQ/logerr/v2/log" configv1 "github.com/openshift/api/config/v1" routev1 "github.com/openshift/api/route/v1" + cloudcredentialv1 "github.com/openshift/cloud-credential-operator/pkg/apis/cloudcredential/v1" monitoringv1 "github.com/prometheus-operator/prometheus-operator/pkg/apis/monitoring/v1" "k8s.io/apimachinery/pkg/runtime" utilruntime "k8s.io/apimachinery/pkg/util/runtime" @@ -20,6 +21,7 @@ import ( lokiv1beta1 "github.com/grafana/loki/operator/apis/loki/v1beta1" lokictrl "github.com/grafana/loki/operator/controllers/loki" "github.com/grafana/loki/operator/internal/config" + manifestsocp "github.com/grafana/loki/operator/internal/manifests/openshift" "github.com/grafana/loki/operator/internal/metrics" "github.com/grafana/loki/operator/internal/operator" "github.com/grafana/loki/operator/internal/validation" @@ -83,6 +85,7 @@ func main() { if ctrlCfg.Gates.OpenShift.Enabled { utilruntime.Must(routev1.AddToScheme(scheme)) + utilruntime.Must(cloudcredentialv1.AddToScheme(scheme)) } } @@ -92,6 +95,11 @@ func main() { os.Exit(1) } + if ctrlCfg.Gates.OpenShift.Enabled && manifestsocp.DiscoverManagedAuthEnv() != nil { + logger.Info("discovered OpenShift Cluster within a managed authentication environment") + ctrlCfg.Gates.OpenShift.ManagedAuthEnv = true + } + if err = (&lokictrl.LokiStackReconciler{ Client: mgr.GetClient(), Log: logger.WithName("controllers").WithName("lokistack"), @@ -121,6 +129,17 @@ func main() { } } + if ctrlCfg.Gates.OpenShift.ManagedAuthEnabled() { + if err = (&lokictrl.CredentialsRequestsReconciler{ + Client: mgr.GetClient(), + Scheme: mgr.GetScheme(), + Log: logger.WithName("controllers").WithName("lokistack-credentialsrequest"), + }).SetupWithManager(mgr); err != nil { + logger.Error(err, "unable to create controller", "controller", "lokistack-credentialsrequest") + os.Exit(1) + } + } + if ctrlCfg.Gates.LokiStackWebhook { v := &validation.LokiStackValidator{} if err = v.SetupWebhookWithManager(mgr); err != nil { From 541b109c3d030c98d228ad45327ad31db1ba881d Mon Sep 17 00:00:00 2001 From: Vladislav <144857440+suregrop@users.noreply.github.com> Date: Sat, 27 Jan 2024 01:14:28 +0500 Subject: [PATCH 22/23] Docs: Fixed docker_sd_configs top level key name (#11803) **What this PR does / why we need it**: Misspelling in top level key name, promtail accept docker_sd_config**s** but not docker_sd_config **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) Co-authored-by: J Stickler --- docs/sources/send-data/promtail/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/send-data/promtail/configuration.md b/docs/sources/send-data/promtail/configuration.md index 68042664bb13b..cbd5c7fc717e2 100644 --- a/docs/sources/send-data/promtail/configuration.md +++ b/docs/sources/send-data/promtail/configuration.md @@ -1940,7 +1940,7 @@ users with thousands of services it can be more efficient to use the Consul API directly which has basic support for filtering nodes (currently by node metadata and a single tag). -### docker_sd_config +### docker_sd_configs Docker service discovery allows retrieving targets from a Docker daemon. It will only watch containers of the Docker daemon referenced with the host parameter. Docker From 012cf92dd2c57e7c6e5209ab0479ace521e91427 Mon Sep 17 00:00:00 2001 From: Sheikh-Abubaker Date: Sat, 27 Jan 2024 22:12:18 +0530 Subject: [PATCH 23/23] area/grafana-agent-operator: Bumped grafana-agent-operator version (#11576) **What this PR does / why we need it**: **Which issue(s) this PR fixes**: Fixes #11572 **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Sheikh-Abubaker Co-authored-by: Michel Hollands <42814411+MichelHollands@users.noreply.github.com> --- production/helm/loki/CHANGELOG.md | 3 +++ production/helm/loki/Chart.lock | 6 +++--- production/helm/loki/Chart.yaml | 4 ++-- production/helm/loki/README.md | 4 ++-- 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/production/helm/loki/CHANGELOG.md b/production/helm/loki/CHANGELOG.md index db040dc196710..42972ff61cef4 100644 --- a/production/helm/loki/CHANGELOG.md +++ b/production/helm/loki/CHANGELOG.md @@ -13,6 +13,9 @@ Entries should include a reference to the pull request that introduced the chang [//]: # ( : do not remove this line. This locator is used by the CI pipeline to automatically create a changelog entry for each new Loki release. Add other chart versions and respective changelog entries bellow this line.) +## 5.42.0 + +- [ENHANCEMENT] Bumped "grafana-agent-operator" depenency chart version to it's latest version ## 5.41.8 diff --git a/production/helm/loki/Chart.lock b/production/helm/loki/Chart.lock index 17f1dafad7ae9..e8c779c503304 100644 --- a/production/helm/loki/Chart.lock +++ b/production/helm/loki/Chart.lock @@ -4,6 +4,6 @@ dependencies: version: 4.0.15 - name: grafana-agent-operator repository: https://grafana.github.io/helm-charts - version: 0.2.16 -digest: sha256:56eeb13a669bc816c1452cde5d6dddc61f6893f8aff3da1d2b56ce3bdcbcf84d -generated: "2023-11-09T12:22:25.317696-03:00" + version: 0.3.15 +digest: sha256:b7a42cd0e56544f6168a586fde03e26c801bb20cf69bc004a8f6000d93b98100 +generated: "2024-01-27T21:57:28.190462917+05:30" diff --git a/production/helm/loki/Chart.yaml b/production/helm/loki/Chart.yaml index 8b5402bc9dbd8..812e724def1ee 100644 --- a/production/helm/loki/Chart.yaml +++ b/production/helm/loki/Chart.yaml @@ -3,7 +3,7 @@ name: loki description: Helm chart for Grafana Loki in simple, scalable mode type: application appVersion: 2.9.3 -version: 5.41.8 +version: 5.42.0 home: https://grafana.github.io/helm-charts sources: - https://github.com/grafana/loki @@ -18,7 +18,7 @@ dependencies: condition: minio.enabled - name: grafana-agent-operator alias: grafana-agent-operator - version: 0.2.16 + version: 0.3.15 repository: https://grafana.github.io/helm-charts condition: monitoring.selfMonitoring.grafanaAgent.installOperator maintainers: diff --git a/production/helm/loki/README.md b/production/helm/loki/README.md index 33730c361d31e..a89eaf0cd942f 100644 --- a/production/helm/loki/README.md +++ b/production/helm/loki/README.md @@ -1,6 +1,6 @@ # loki -![Version: 5.41.8](https://img.shields.io/badge/Version-5.41.8-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 2.9.3](https://img.shields.io/badge/AppVersion-2.9.3-informational?style=flat-square) +![Version: 5.42.0](https://img.shields.io/badge/Version-5.42.0-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 2.9.3](https://img.shields.io/badge/AppVersion-2.9.3-informational?style=flat-square) Helm chart for Grafana Loki in simple, scalable mode @@ -15,6 +15,6 @@ Helm chart for Grafana Loki in simple, scalable mode | Repository | Name | Version | |------------|------|---------| | https://charts.min.io/ | minio(minio) | 4.0.15 | -| https://grafana.github.io/helm-charts | grafana-agent-operator(grafana-agent-operator) | 0.2.16 | +| https://grafana.github.io/helm-charts | grafana-agent-operator(grafana-agent-operator) | 0.3.15 | Find more information in the Loki Helm Chart [documentation](https://grafana.com/docs/loki/next/installation/helm).