diff --git a/.gitignore b/.gitignore index 66eb0a8cefeb2..83ab9c808d348 100644 --- a/.gitignore +++ b/.gitignore @@ -27,8 +27,8 @@ cmd/querytee/querytee dlv rootfs/ dist -coverage.txt -test_results.txt +*coverage.txt +*test_results.txt .DS_Store .aws-sam .idea diff --git a/CHANGELOG.md b/CHANGELOG.md index ca45604012430..fa8861228407f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,6 +6,7 @@ ##### Enhancements +* [11814](https://github.com/grafana/loki/pull/11814) **kavirajk**: feat: Support split align and caching for instant metric query results * [11851](https://github.com/grafana/loki/pull/11851) **elcomtik**: Helm: Allow the definition of resources for GrafanaAgent pods. * [11819](https://github.com/grafana/loki/pull/11819) **jburnham**: Ruler: Add the ability to disable the `X-Scope-OrgId` tenant identification header in remote write requests. * [11633](https://github.com/grafana/loki/pull/11633) **cyriltovena**: Add profiling integrations to tracing instrumentation. @@ -57,6 +58,7 @@ * [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. * [11817](https://github.com/grafana/loki/pull/11817) **ashwanthgoli** Ruler: Add support for filtering results of `/prometheus/api/v1/rules` endpoint by rule_name, rule_group, file and type. * [11897](https://github.com/grafana/loki/pull/11897) **ashwanthgoli** Metadata: Introduces a separate split interval of `split_recent_metadata_queries_by_interval` for `recent_metadata_query_window` to help with caching recent metadata query results. +* [11970](https://github.com/grafana/loki/pull/11897) **masslessparticle** Ksonnet: Introduces memory limits to the compactor configuration to avoid unbounded memory usage. ##### Fixes * [11074](https://github.com/grafana/loki/pull/11074) **hainenber** Fix panic in lambda-promtail due to mishandling of empty DROP_LABELS env var. @@ -69,7 +71,7 @@ * [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. -* [11921](https://github.com/grafana/loki/pull/11921) **paul1r**: Parsing: String array elements were not being parsed correctly in JSON processing +* [11921](https://github.com/grafana/loki/pull/11921) **paul1r**: Parsing: String array elements were not being parsed correctly in JSON processing ##### Changes diff --git a/cmd/loki/loki-local-with-memcached.yaml b/cmd/loki/loki-local-with-memcached.yaml index d1b0ae1c2493c..a2f4336cdd484 100644 --- a/cmd/loki/loki-local-with-memcached.yaml +++ b/cmd/loki/loki-local-with-memcached.yaml @@ -22,6 +22,17 @@ query_range: cache_results: true cache_volume_results: true cache_series_results: true + cache_instant_metric_results: true + instant_metric_query_split_align: true + instant_metric_results_cache: + cache: + default_validity: 12h + memcached_client: + consistent_hash: true + addresses: "dns+localhost:11211" + max_idle_conns: 16 + timeout: 500ms + update_interval: 1m series_results_cache: cache: default_validity: 12h diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index c30f8da01fa23..70891a0448419 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -886,6 +886,28 @@ volume_results_cache: # CLI flag: -frontend.volume-results-cache.compression [compression: | default = ""] +# Cache instant metric query results. +# CLI flag: -querier.cache-instant-metric-results +[cache_instant_metric_results: | default = false] + +# If a cache config is not specified and cache_instant_metric_results is true, +# the config for the results cache is used. +instant_metric_results_cache: + # The cache block configures the cache backend. + # The CLI flags prefix for this block configuration is: + # frontend.instant-metric-results-cache + [cache: ] + + # Use compression in cache. The default is an empty value '', which disables + # compression. Supported values are: 'snappy' and ''. + # CLI flag: -frontend.instant-metric-results-cache.compression + [compression: | default = ""] + +# Whether to align the splits of instant metric query with splitByInterval and +# query's exec time. Useful when instant_metric_cache is enabled +# CLI flag: -querier.instant-metric-query-split-align +[instant_metric_query_split_align: | default = false] + # Cache series query results. # CLI flag: -querier.cache-series-results [cache_series_results: | default = false] @@ -2935,6 +2957,13 @@ The `limits_config` block configures global and per-tenant limits in Loki. # CLI flag: -experimental.querier.recent-metadata-query-window [recent_metadata_query_window: | default = 0s] +# Split instant metric queries by a time interval and execute in parallel. The +# value 0 disables splitting instant metric queries by time. This also +# determines how cache keys are chosen when instant metric query result caching +# is enabled. +# CLI flag: -querier.split-instant-metric-queries-by-interval +[split_instant_metric_queries_by_interval: | default = 1h] + # Interval to use for time-based splitting when a request is within the # `query_ingesters_within` window; defaults to `split-queries-by-interval` by # setting to 0. @@ -3155,7 +3184,7 @@ shard_streams: # Skip factor for the n-grams created when computing blooms from log lines. # CLI flag: -bloom-compactor.ngram-skip -[bloom_ngram_skip: | default = 0] +[bloom_ngram_skip: | default = 1] # Scalable Bloom Filter desired false-positive rate. # CLI flag: -bloom-compactor.false-positive-rate @@ -4403,6 +4432,7 @@ The cache block configures the cache backend. The supported CLI flags `` - `bloom.metas-cache` - `frontend` - `frontend.index-stats-results-cache` +- `frontend.instant-metric-results-cache` - `frontend.label-results-cache` - `frontend.series-results-cache` - `frontend.volume-results-cache` diff --git a/docs/sources/release-notes/cadence.md b/docs/sources/release-notes/cadence.md index f13781cf1c5f3..ef6fbcaf072fd 100644 --- a/docs/sources/release-notes/cadence.md +++ b/docs/sources/release-notes/cadence.md @@ -8,7 +8,7 @@ weight: 1 ## Stable Releases -Loki releases (this includes [Promtail](/clients/promtail), [Loki Canary](/operations/loki-canary/), etc) use the following +Loki releases (this includes [Promtail](https://grafana.com/docs/loki//send-data/promtail/), [Loki Canary](https://grafana.com/docs/loki//operations/loki-canary/), etc.) use the following naming scheme: `MAJOR`.`MINOR`.`PATCH`. - `MAJOR` (roughly once a year): these releases include large new features and possible backwards-compatibility breaks. @@ -18,14 +18,14 @@ naming scheme: `MAJOR`.`MINOR`.`PATCH`. {{% admonition type="note" %}} While our naming scheme resembles [Semantic Versioning](https://semver.org/), at this time we do not strictly follow its guidelines to the letter. Our goal is to provide regular releases that are as stable as possible, and we take backwards-compatibility -seriously. As with any software, always read the [release notes](/release-notes) and the [upgrade guide](/upgrading) whenever +seriously. As with any software, always read the [release notes](https://grafana.com/docs/loki//release-notes/) and the [upgrade guide](https://grafana.com/docs/loki//setup/upgrade/) whenever choosing a new version of Loki to install. {{% /admonition %}} New releases are based of a [weekly release](#weekly-releases) which we have vetted for stability over a number of weeks. We strongly recommend keeping up-to-date with patch releases as they are released. We post updates of new releases in the `#loki` channel -of our [Slack community](/community/getting-in-touch). +of our [Slack community](https://grafana.com/docs/loki//community/getting-in-touch/). You can find all of our releases [on GitHub](https://github.com/grafana/loki/releases) and on [Docker Hub](https://hub.docker.com/r/grafana/loki). diff --git a/go.mod b/go.mod index 6235582406d50..5b4c32ddb214b 100644 --- a/go.mod +++ b/go.mod @@ -118,7 +118,7 @@ require ( github.com/DmitriyVTitov/size v1.5.0 github.com/IBM/go-sdk-core/v5 v5.13.1 github.com/IBM/ibm-cos-sdk-go v1.10.0 - github.com/axiomhq/hyperloglog v0.0.0-20230201085229-3ddf4bad03dc + github.com/axiomhq/hyperloglog v0.0.0-20240124082744-24bca3a5b39b github.com/d4l3k/messagediff v1.2.1 github.com/efficientgo/core v1.0.0-rc.2 github.com/fsnotify/fsnotify v1.6.0 diff --git a/go.sum b/go.sum index 8ab729e928055..744c904e823c2 100644 --- a/go.sum +++ b/go.sum @@ -368,8 +368,8 @@ github.com/aws/aws-sdk-go-v2/service/sts v1.16.1 h1:xsOtPAvHqhvQvBza5ohaUcfq1Lce github.com/aws/aws-sdk-go-v2/service/sts v1.16.1/go.mod h1:Aq2/Qggh2oemSfyHH+EO4UBbgWG6zFCXLHYI4ILTY7w= github.com/aws/smithy-go v1.11.1 h1:IQ+lPZVkSM3FRtyaDox41R8YS6iwPMYIreejOgPW49g= github.com/aws/smithy-go v1.11.1/go.mod h1:3xHYmszWVx2c0kIwQeEVf9uSm4fYZt67FBJnwub1bgM= -github.com/axiomhq/hyperloglog v0.0.0-20230201085229-3ddf4bad03dc h1:Keo7wQ7UODUaHcEi7ltENhbAK2VgZjfat6mLy03tQzo= -github.com/axiomhq/hyperloglog v0.0.0-20230201085229-3ddf4bad03dc/go.mod h1:k08r+Yj1PRAmuayFiRK6MYuR5Ve4IuZtTfxErMIh0+c= +github.com/axiomhq/hyperloglog v0.0.0-20240124082744-24bca3a5b39b h1:F3yMzKumBUQ6Fn0sYI1YQ16vQRucpZOfBQ9HXWl5+XI= +github.com/axiomhq/hyperloglog v0.0.0-20240124082744-24bca3a5b39b/go.mod h1:k08r+Yj1PRAmuayFiRK6MYuR5Ve4IuZtTfxErMIh0+c= github.com/baidubce/bce-sdk-go v0.9.141 h1:EV5BH5lfymIGPSmYDo9xYdsVlvWAW6nFeiA6t929zBE= github.com/baidubce/bce-sdk-go v0.9.141/go.mod h1:zbYJMQwE4IZuyrJiFO8tO8NbtYiKTFTbwh4eIsqjVdg= github.com/baiyubin/aliyun-sts-go-sdk v0.0.0-20180326062324-cfa1a18b161f/go.mod h1:AuiFmCCPBSrqvVMvuqFuk0qogytodnVFVSN5CeJB8Gc= diff --git a/operator/CHANGELOG.md b/operator/CHANGELOG.md index 59afb29708782..d504e4ee31b52 100644 --- a/operator/CHANGELOG.md +++ b/operator/CHANGELOG.md @@ -1,5 +1,6 @@ ## Main +- [11964](https://github.com/grafana/loki/pull/11964) **xperimental**: Provide Azure region for managed credentials using environment variable - [11920](https://github.com/grafana/loki/pull/11920) **xperimental**: Refactor handling of credentials in managed-auth mode - [11869](https://github.com/grafana/loki/pull/11869) **periklis**: Add support for running with Google Workload Identity - [11868](https://github.com/grafana/loki/pull/11868) **xperimental**: Integrate support for OpenShift-managed credentials in Azure diff --git a/operator/internal/config/managed_auth.go b/operator/internal/config/managed_auth.go index 73598e7032f8f..76f9d72f3c262 100644 --- a/operator/internal/config/managed_auth.go +++ b/operator/internal/config/managed_auth.go @@ -26,6 +26,7 @@ func discoverManagedAuthConfig() *ManagedAuthConfig { clientID := os.Getenv("CLIENTID") tenantID := os.Getenv("TENANTID") subscriptionID := os.Getenv("SUBSCRIPTIONID") + region := os.Getenv("REGION") switch { case roleARN != "": @@ -40,6 +41,7 @@ func discoverManagedAuthConfig() *ManagedAuthConfig { ClientID: clientID, SubscriptionID: subscriptionID, TenantID: tenantID, + Region: region, }, } } diff --git a/operator/internal/handlers/credentialsrequest_create.go b/operator/internal/handlers/credentialsrequest.go similarity index 67% rename from operator/internal/handlers/credentialsrequest_create.go rename to operator/internal/handlers/credentialsrequest.go index 50e06375ffd8b..0d562332dc9d5 100644 --- a/operator/internal/handlers/credentialsrequest_create.go +++ b/operator/internal/handlers/credentialsrequest.go @@ -2,12 +2,10 @@ package handlers import ( "context" - "errors" "fmt" "github.com/ViaQ/logerr/v2/kverrors" "github.com/go-logr/logr" - corev1 "k8s.io/api/core/v1" apierrors "k8s.io/apimachinery/pkg/api/errors" "k8s.io/apimachinery/pkg/runtime" ctrl "sigs.k8s.io/controller-runtime" @@ -19,11 +17,8 @@ import ( "github.com/grafana/loki/operator/internal/external/k8s" "github.com/grafana/loki/operator/internal/manifests" "github.com/grafana/loki/operator/internal/manifests/openshift" - "github.com/grafana/loki/operator/internal/manifests/storage" ) -var errAzureNoRegion = errors.New("can not create CredentialsRequest: missing secret field: region") - // 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, log logr.Logger, scheme *runtime.Scheme, managedAuth *config.ManagedAuthConfig, k k8s.Client, req ctrl.Request) error { @@ -39,32 +34,6 @@ func CreateCredentialsRequest(ctx context.Context, log logr.Logger, scheme *runt return kverrors.Wrap(err, "failed to lookup LokiStack", "name", req.String()) } - if managedAuth.Azure != nil && managedAuth.Azure.Region == "" { - // Managed environment for Azure does not provide Region, but we need this for the CredentialsRequest. - // This looks like an oversight when creating the UI in OpenShift, but for now we need to pull this data - // from somewhere else -> the Azure Storage Secret - storageSecretName := client.ObjectKey{ - Namespace: stack.Namespace, - Name: stack.Spec.Storage.Secret.Name, - } - storageSecret := &corev1.Secret{} - if err := k.Get(ctx, storageSecretName, storageSecret); err != nil { - if apierrors.IsNotFound(err) { - // Skip this error here as it will be picked up by the LokiStack handler instead - ll.Error(err, "could not find secret for LokiStack", "name", req.String()) - return nil - } - return err - } - - region := storageSecret.Data[storage.KeyAzureRegion] - if len(region) == 0 { - return errAzureNoRegion - } - - managedAuth.Azure.Region = string(region) - } - opts := openshift.Options{ BuildOpts: openshift.BuildOptions{ LokiStackName: stack.Name, diff --git a/operator/internal/handlers/credentialsrequest_create_test.go b/operator/internal/handlers/credentialsrequest_test.go similarity index 71% rename from operator/internal/handlers/credentialsrequest_create_test.go rename to operator/internal/handlers/credentialsrequest_test.go index 626302a113274..dd6dfb50d77dc 100644 --- a/operator/internal/handlers/credentialsrequest_create_test.go +++ b/operator/internal/handlers/credentialsrequest_test.go @@ -6,7 +6,6 @@ import ( cloudcredentialv1 "github.com/openshift/cloud-credential-operator/pkg/apis/cloudcredential/v1" "github.com/stretchr/testify/require" - corev1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/runtime/schema" @@ -19,7 +18,7 @@ import ( "github.com/grafana/loki/operator/internal/external/k8s/k8sfakes" ) -func credentialsRequestFakeClient(cr *cloudcredentialv1.CredentialsRequest, lokistack *lokiv1.LokiStack, secret *corev1.Secret) *k8sfakes.FakeClient { +func credentialsRequestFakeClient(cr *cloudcredentialv1.CredentialsRequest, lokistack *lokiv1.LokiStack) *k8sfakes.FakeClient { k := &k8sfakes.FakeClient{} k.GetStub = func(_ context.Context, name types.NamespacedName, object client.Object, _ ...client.GetOption) error { switch object.(type) { @@ -33,11 +32,6 @@ func credentialsRequestFakeClient(cr *cloudcredentialv1.CredentialsRequest, loki return errors.NewNotFound(schema.GroupResource{}, name.Name) } k.SetClientObject(object, lokistack) - case *corev1.Secret: - if secret == nil { - return errors.NewNotFound(schema.GroupResource{}, name.Name) - } - k.SetClientObject(object, secret) } return nil } @@ -58,7 +52,7 @@ func TestCreateCredentialsRequest_CreateNewResource(t *testing.T) { }, } - k := credentialsRequestFakeClient(nil, lokistack, nil) + k := credentialsRequestFakeClient(nil, lokistack) req := ctrl.Request{ NamespacedName: client.ObjectKey{Name: "my-stack", Namespace: "ns"}, } @@ -89,13 +83,8 @@ func TestCreateCredentialsRequest_CreateNewResourceAzure(t *testing.T) { Namespace: "ns", }, } - secret := &corev1.Secret{ - Data: map[string][]byte{ - "region": []byte(wantRegion), - }, - } - k := credentialsRequestFakeClient(nil, lokistack, secret) + k := credentialsRequestFakeClient(nil, lokistack) req := ctrl.Request{ NamespacedName: client.ObjectKey{Name: "my-stack", Namespace: "ns"}, } @@ -105,6 +94,7 @@ func TestCreateCredentialsRequest_CreateNewResourceAzure(t *testing.T) { ClientID: "test-client-id", SubscriptionID: "test-tenant-id", TenantID: "test-subscription-id", + Region: "test-region", }, } @@ -122,47 +112,6 @@ func TestCreateCredentialsRequest_CreateNewResourceAzure(t *testing.T) { require.Equal(t, wantRegion, providerSpec.AzureRegion) } -func TestCreateCredentialsRequest_CreateNewResourceAzure_Errors(t *testing.T) { - lokistack := &lokiv1.LokiStack{ - ObjectMeta: metav1.ObjectMeta{ - Name: "my-stack", - Namespace: "ns", - }, - } - req := ctrl.Request{ - NamespacedName: client.ObjectKey{Name: "my-stack", Namespace: "ns"}, - } - - tt := []struct { - secret *corev1.Secret - wantError string - }{ - { - secret: &corev1.Secret{}, - wantError: errAzureNoRegion.Error(), - }, - } - - for _, tc := range tt { - tc := tc - t.Run(tc.wantError, func(t *testing.T) { - t.Parallel() - - managedAuth := &config.ManagedAuthConfig{ - Azure: &config.AzureEnvironment{ - ClientID: "test-client-id", - SubscriptionID: "test-tenant-id", - TenantID: "test-subscription-id", - }, - } - k := credentialsRequestFakeClient(nil, lokistack, tc.secret) - - err := CreateCredentialsRequest(context.Background(), logger, scheme, managedAuth, k, req) - require.EqualError(t, err, tc.wantError) - }) - } -} - func TestCreateCredentialsRequest_DoNothing_WhenCredentialsRequestExist(t *testing.T) { req := ctrl.Request{ NamespacedName: client.ObjectKey{Name: "my-stack", Namespace: "ns"}, @@ -187,7 +136,7 @@ func TestCreateCredentialsRequest_DoNothing_WhenCredentialsRequestExist(t *testi }, } - k := credentialsRequestFakeClient(cr, lokistack, nil) + k := credentialsRequestFakeClient(cr, lokistack) err := CreateCredentialsRequest(context.Background(), logger, scheme, managedAuth, k, req) require.NoError(t, err) diff --git a/operator/internal/handlers/internal/storage/secrets.go b/operator/internal/handlers/internal/storage/secrets.go index 99bafb911ec26..2492eea4d4191 100644 --- a/operator/internal/handlers/internal/storage/secrets.go +++ b/operator/internal/handlers/internal/storage/secrets.go @@ -182,18 +182,11 @@ func extractAzureConfigSecret(s *corev1.Secret, fg configv1.FeatureGates) (*stor // Extract and validate optional fields endpointSuffix := s.Data[storage.KeyAzureStorageEndpointSuffix] audience := s.Data[storage.KeyAzureAudience] - region := s.Data[storage.KeyAzureRegion] if !workloadIdentity && len(audience) > 0 { return nil, fmt.Errorf("%w: %s", errSecretFieldNotAllowed, storage.KeyAzureAudience) } - if fg.OpenShift.ManagedAuthEnv { - if len(region) == 0 { - return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureRegion) - } - } - return &storage.AzureStorageConfig{ Env: string(env), Container: string(container), diff --git a/operator/internal/handlers/internal/storage/secrets_test.go b/operator/internal/handlers/internal/storage/secrets_test.go index 1363cd4a660a6..ca3623b718c1b 100644 --- a/operator/internal/handlers/internal/storage/secrets_test.go +++ b/operator/internal/handlers/internal/storage/secrets_test.go @@ -156,27 +156,6 @@ func TestAzureExtract(t *testing.T) { }, wantError: "missing secret field: subscription_id", }, - { - name: "managed auth - no region", - secret: &corev1.Secret{ - ObjectMeta: metav1.ObjectMeta{Name: "test"}, - Data: map[string][]byte{ - "environment": []byte("here"), - "account_name": []byte("test-account-name"), - "container": []byte("this,that"), - }, - }, - managedSecret: &corev1.Secret{ - Data: map[string][]byte{}, - }, - featureGates: configv1.FeatureGates{ - OpenShift: configv1.OpenShiftFeatureGates{ - Enabled: true, - ManagedAuthEnv: true, - }, - }, - wantError: "missing secret field: region", - }, { name: "managed auth - no auth override", secret: &corev1.Secret{ diff --git a/operator/internal/manifests/openshift/credentialsrequest.go b/operator/internal/manifests/openshift/credentialsrequest.go index 0e97dd97c2b19..0c0a19adc98d3 100644 --- a/operator/internal/manifests/openshift/credentialsrequest.go +++ b/operator/internal/manifests/openshift/credentialsrequest.go @@ -12,6 +12,8 @@ import ( "github.com/grafana/loki/operator/internal/manifests/storage" ) +const azureFallbackRegion = "centralus" + func BuildCredentialsRequest(opts Options) (*cloudcredentialv1.CredentialsRequest, error) { stack := client.ObjectKey{Name: opts.BuildOpts.LokiStackName, Namespace: opts.BuildOpts.LokiStackNamespace} @@ -62,6 +64,15 @@ func encodeProviderSpec(env *config.ManagedAuthConfig) (*runtime.RawExtension, e } case env.Azure != nil: azure := env.Azure + if azure.Region == "" { + // The OpenShift Console currently does not provide a UI to configure the Azure Region + // for an operator using managed credentials. Because the CredentialsRequest is currently + // not used to create a Managed Identity, the region is actually never used. + // We default to the US region if nothing is set, so that the CredentialsRequest can be + // created. This should have no effect on the generated credential secret. + // The region can be configured by setting an environment variable on the operator Subscription. + azure.Region = azureFallbackRegion + } spec = &cloudcredentialv1.AzureProviderSpec{ Permissions: []string{ diff --git a/operator/internal/manifests/storage/options.go b/operator/internal/manifests/storage/options.go index 6693d2261e978..56e2b8e870df1 100644 --- a/operator/internal/manifests/storage/options.go +++ b/operator/internal/manifests/storage/options.go @@ -63,7 +63,6 @@ type AzureStorageConfig struct { Container string EndpointSuffix string Audience string - Region string WorkloadIdentity bool } diff --git a/operator/internal/manifests/storage/var.go b/operator/internal/manifests/storage/var.go index cbd944a821c34..1f236406bdd09 100644 --- a/operator/internal/manifests/storage/var.go +++ b/operator/internal/manifests/storage/var.go @@ -88,8 +88,6 @@ const ( KeyAzureStorageEndpointSuffix = "endpoint_suffix" // KeyAzureEnvironmentName is the secret data key for the Azure cloud environment name. KeyAzureEnvironmentName = "environment" - // KeyAzureRegion is the secret data key for storing the Azure cloud region. - KeyAzureRegion = "region" // KeyAzureAudience is the secret data key for customizing the audience used for the ServiceAccount token. KeyAzureAudience = "audience" diff --git a/pkg/bloomcompactor/batch.go b/pkg/bloomcompactor/batch.go index 2d43f83219df9..e9fae9f9df0f0 100644 --- a/pkg/bloomcompactor/batch.go +++ b/pkg/bloomcompactor/batch.go @@ -2,94 +2,360 @@ package bloomcompactor import ( "context" + "io" + "math" + "time" "github.com/grafana/dskit/multierror" + "golang.org/x/exp/slices" + "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/bloomshipper" ) -// interface modeled from `pkg/storage/stores/shipper/bloomshipper.Fetcher` -type blocksFetcher interface { - FetchBlocks(context.Context, []bloomshipper.BlockRef) ([]*bloomshipper.CloseableBlockQuerier, error) +type Fetcher[A, B any] interface { + Fetch(ctx context.Context, inputs []A) ([]B, error) } -func newBatchedBlockLoader(ctx context.Context, fetcher blocksFetcher, blocks []bloomshipper.BlockRef) (*batchedBlockLoader, error) { - return &batchedBlockLoader{ - ctx: ctx, - batchSize: 10, // make configurable? - source: blocks, - fetcher: fetcher, - }, nil +type FetchFunc[A, B any] func(ctx context.Context, inputs []A) ([]B, error) + +func (f FetchFunc[A, B]) Fetch(ctx context.Context, inputs []A) ([]B, error) { + return f(ctx, inputs) } -type batchedBlockLoader struct { - ctx context.Context +// batchedLoader implements `v1.Iterator[C]` in batches +type batchedLoader[A, B, C any] struct { + metrics *Metrics batchSize int + ctx context.Context + fetchers []Fetcher[A, B] + work [][]A + + mapper func(B) (C, error) + cur C + batch []B + err error +} + +const batchedLoaderDefaultBatchSize = 50 + +func newBatchedLoader[A, B, C any]( + ctx context.Context, + fetchers []Fetcher[A, B], + inputs [][]A, + mapper func(B) (C, error), + batchSize int, +) *batchedLoader[A, B, C] { + return &batchedLoader[A, B, C]{ + batchSize: max(batchSize, 1), + ctx: ctx, + fetchers: fetchers, + work: inputs, + mapper: mapper, + } +} + +func (b *batchedLoader[A, B, C]) Next() bool { + + // iterate work until we have non-zero length batch + for len(b.batch) == 0 { + + // empty batch + no work remaining = we're done + if len(b.work) == 0 { + return false + } - source []bloomshipper.BlockRef - fetcher blocksFetcher + // setup next batch + next := b.work[0] + batchSize := min(b.batchSize, len(next)) + toFetch := next[:batchSize] + fetcher := b.fetchers[0] - batch []*bloomshipper.CloseableBlockQuerier - cur *bloomshipper.CloseableBlockQuerier - err error + // update work + b.work[0] = b.work[0][batchSize:] + if len(b.work[0]) == 0 { + // if we've exhausted work from this set of inputs, + // set pointer to next set of inputs + // and their respective fetcher + b.work = b.work[1:] + b.fetchers = b.fetchers[1:] + } + + // there was no work in this batch; continue (should not happen) + if len(toFetch) == 0 { + continue + } + + b.batch, b.err = fetcher.Fetch(b.ctx, toFetch) + // error fetching, short-circuit iteration + if b.err != nil { + return false + } + } + + return b.prepNext() } -// At implements v1.CloseableIterator. -func (b *batchedBlockLoader) At() *bloomshipper.CloseableBlockQuerier { +func (b *batchedLoader[_, B, C]) prepNext() bool { + b.cur, b.err = b.mapper(b.batch[0]) + b.batch = b.batch[1:] + return b.err == nil +} + +func (b *batchedLoader[_, _, C]) At() C { return b.cur } -// Close implements v1.CloseableIterator. -func (b *batchedBlockLoader) Close() error { - if b.cur != nil { - return b.cur.Close() +func (b *batchedLoader[_, _, _]) Err() error { + return b.err +} + +// to ensure memory is bounded while loading chunks +// TODO(owen-d): testware +func newBatchedChunkLoader( + ctx context.Context, + fetchers []Fetcher[chunk.Chunk, chunk.Chunk], + inputs [][]chunk.Chunk, + metrics *Metrics, + batchSize int, +) *batchedLoader[chunk.Chunk, chunk.Chunk, v1.ChunkRefWithIter] { + + mapper := func(c chunk.Chunk) (v1.ChunkRefWithIter, error) { + chk := c.Data.(*chunkenc.Facade).LokiChunk() + metrics.chunkSize.Observe(float64(chk.UncompressedSize())) + itr, err := chk.Iterator( + ctx, + time.Unix(0, 0), + time.Unix(0, math.MaxInt64), + logproto.FORWARD, + logql_log.NewNoopPipeline().ForStream(nil), + ) + + 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 } - return nil + return newBatchedLoader(ctx, fetchers, inputs, mapper, batchSize) } -// CloseBatch closes the remaining items from the current batch -func (b *batchedBlockLoader) CloseBatch() error { - var err multierror.MultiError - for _, cur := range b.batch { - err.Add(cur.Close()) +func newBatchedBlockLoader( + ctx context.Context, + fetcher Fetcher[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier], + blocks []bloomshipper.BlockRef, + batchSize int, +) *batchedLoader[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier, *bloomshipper.CloseableBlockQuerier] { + + fetchers := []Fetcher[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier]{fetcher} + inputs := [][]bloomshipper.BlockRef{blocks} + mapper := func(a *bloomshipper.CloseableBlockQuerier) (*bloomshipper.CloseableBlockQuerier, error) { + return a, nil } - if len(b.batch) > 0 { - b.batch = b.batch[:0] + + return newBatchedLoader(ctx, fetchers, inputs, mapper, batchSize) +} + +// compiler checks +var _ v1.Iterator[*v1.SeriesWithBloom] = &blockLoadingIter{} +var _ v1.CloseableIterator[*v1.SeriesWithBloom] = &blockLoadingIter{} +var _ v1.ResettableIterator[*v1.SeriesWithBloom] = &blockLoadingIter{} + +// TODO(chaudum): testware +func newBlockLoadingIter(ctx context.Context, blocks []bloomshipper.BlockRef, fetcher FetchFunc[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier], batchSize int) *blockLoadingIter { + + return &blockLoadingIter{ + ctx: ctx, + fetcher: fetcher, + inputs: blocks, + batchSize: batchSize, + loaded: make(map[io.Closer]struct{}), + } +} + +type blockLoadingIter struct { + // constructor arguments + ctx context.Context + fetcher Fetcher[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier] + inputs []bloomshipper.BlockRef + overlapping v1.Iterator[[]bloomshipper.BlockRef] + batchSize int + // optional arguments + filter func(*bloomshipper.CloseableBlockQuerier) bool + // internals + initialized bool + err error + iter v1.Iterator[*v1.SeriesWithBloom] + loader *batchedLoader[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier, *bloomshipper.CloseableBlockQuerier] + loaded map[io.Closer]struct{} +} + +// At implements v1.Iterator. +func (i *blockLoadingIter) At() *v1.SeriesWithBloom { + if !i.initialized { + panic("iterator not initialized") + } + return i.iter.At() +} + +// Err implements v1.Iterator. +func (i *blockLoadingIter) Err() error { + if !i.initialized { + panic("iterator not initialized") + } + if i.err != nil { + return i.err + } + return i.iter.Err() +} + +// Next implements v1.Iterator. +func (i *blockLoadingIter) Next() bool { + i.init() + // next from current batch + hasNext := i.iter.Next() + if !hasNext && !i.loadNext() { + return false + } + // next from next batch + return i.iter.Next() +} + +// Close implements v1.CloseableIterator. +func (i *blockLoadingIter) Close() error { + var err multierror.MultiError + for k := range i.loaded { + err.Add(k.Close()) } return err.Err() } -// Err implements v1.CloseableIterator. -func (b *batchedBlockLoader) Err() error { - return b.err +// Reset implements v1.ResettableIterator. +// TODO(chaudum) Cache already fetched blocks to to avoid the overhead of +// creating the reader. +func (i *blockLoadingIter) Reset() error { + if !i.initialized { + return nil + } + // close loaded queriers + err := i.Close() + i.initialized = false + clear(i.loaded) + return err } -// Next implements v1.CloseableIterator. -func (b *batchedBlockLoader) Next() bool { - if len(b.batch) > 0 { - return b.setNext() +func (i *blockLoadingIter) init() { + if i.initialized { + return } - if len(b.source) == 0 { + // group overlapping blocks + i.overlapping = overlappingBlocksIter(i.inputs) + + // set "match all" filter function if not present + if i.filter == nil { + i.filter = func(cbq *bloomshipper.CloseableBlockQuerier) bool { return true } + } + + // load first batch + i.loadNext() + + // done + i.initialized = true +} + +func (i *blockLoadingIter) Filter(filter func(*bloomshipper.CloseableBlockQuerier) bool) { + if i.initialized { + panic("iterator already initialized") + } + i.filter = filter +} + +func (i *blockLoadingIter) loadNext() bool { + // check if there are more overlapping groups to load + if !i.overlapping.Next() { + i.iter = v1.NewEmptyIter[*v1.SeriesWithBloom]() + if i.overlapping.Err() != nil { + i.err = i.overlapping.Err() + } + return false } - // setup next batch - batchSize := min(b.batchSize, len(b.source)) - toFetch := b.source[:batchSize] + blockRefs := i.overlapping.At() - // update source - b.source = b.source[batchSize:] + loader := newBatchedBlockLoader(i.ctx, i.fetcher, blockRefs, i.batchSize) + filtered := v1.NewFilterIter[*bloomshipper.CloseableBlockQuerier](loader, i.filter) - b.batch, b.err = b.fetcher.FetchBlocks(b.ctx, toFetch) - if b.err != nil { + iters := make([]v1.PeekingIterator[*v1.SeriesWithBloom], 0, len(blockRefs)) + for filtered.Next() { + bq := loader.At() + if _, ok := i.loaded[bq]; !ok { + i.loaded[bq] = struct{}{} + } + iter, _ := bq.SeriesIter() + iters = append(iters, iter) + } + + if err := filtered.Err(); err != nil { + i.err = err + i.iter = v1.NewEmptyIter[*v1.SeriesWithBloom]() return false } - return b.setNext() -} -func (b *batchedBlockLoader) setNext() bool { - b.cur, b.err = b.batch[0], nil - b.batch = b.batch[1:] + if len(iters) == 0 { + i.iter = v1.NewEmptyIter[*v1.SeriesWithBloom]() + return true + } + + // Turn the list of blocks into a single iterator that returns the next series + mergedBlocks := v1.NewHeapIterForSeriesWithBloom(iters...) + // two overlapping blocks can conceivably have the same series, so we need to dedupe, + // preferring the one with the most chunks already indexed since we'll have + // to add fewer chunks to the bloom + i.iter = v1.NewDedupingIter[*v1.SeriesWithBloom, *v1.SeriesWithBloom]( + func(a, b *v1.SeriesWithBloom) bool { + return a.Series.Fingerprint == b.Series.Fingerprint + }, + v1.Identity[*v1.SeriesWithBloom], + func(a, b *v1.SeriesWithBloom) *v1.SeriesWithBloom { + if len(a.Series.Chunks) > len(b.Series.Chunks) { + return a + } + return b + }, + v1.NewPeekingIter(mergedBlocks), + ) return true } + +func overlappingBlocksIter(inputs []bloomshipper.BlockRef) v1.Iterator[[]bloomshipper.BlockRef] { + // can we assume sorted blocks? + peekIter := v1.NewPeekingIter(v1.NewSliceIter(inputs)) + + return v1.NewDedupingIter[bloomshipper.BlockRef, []bloomshipper.BlockRef]( + func(a bloomshipper.BlockRef, b []bloomshipper.BlockRef) bool { + minFp := b[0].Bounds.Min + maxFp := slices.MaxFunc(b, func(a, b bloomshipper.BlockRef) int { return int(a.Bounds.Max - b.Bounds.Max) }).Bounds.Max + return a.Bounds.Overlaps(v1.NewBounds(minFp, maxFp)) + }, + func(a bloomshipper.BlockRef) []bloomshipper.BlockRef { + return []bloomshipper.BlockRef{a} + }, + func(a bloomshipper.BlockRef, b []bloomshipper.BlockRef) []bloomshipper.BlockRef { + return append(b, a) + }, + peekIter, + ) +} diff --git a/pkg/bloomcompactor/batch_test.go b/pkg/bloomcompactor/batch_test.go index a1922bf931b86..bd2cb3378cfba 100644 --- a/pkg/bloomcompactor/batch_test.go +++ b/pkg/bloomcompactor/batch_test.go @@ -2,36 +2,209 @@ package bloomcompactor import ( "context" + "errors" "testing" "github.com/stretchr/testify/require" - "go.uber.org/atomic" + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" ) -type dummyBlocksFetcher struct { - count *atomic.Int32 -} +func TestBatchedLoader(t *testing.T) { + t.Parallel() -func (f *dummyBlocksFetcher) FetchBlocks(_ context.Context, blocks []bloomshipper.BlockRef) ([]*bloomshipper.CloseableBlockQuerier, error) { - f.count.Inc() - return make([]*bloomshipper.CloseableBlockQuerier, len(blocks)), nil -} + errMapper := func(i int) (int, error) { + return 0, errors.New("bzzt") + } + successMapper := func(i int) (int, error) { + return i, nil + } -func TestBatchedBlockLoader(t *testing.T) { - ctx := context.Background() - f := &dummyBlocksFetcher{count: atomic.NewInt32(0)} + expired, cancel := context.WithCancel(context.Background()) + cancel() - blocks := make([]bloomshipper.BlockRef, 25) - blocksIter, err := newBatchedBlockLoader(ctx, f, blocks) - require.NoError(t, err) + for _, tc := range []struct { + desc string + ctx context.Context + batchSize int + mapper func(int) (int, error) + err bool + inputs [][]int + exp []int + }{ + { + desc: "OneBatch", + ctx: context.Background(), + batchSize: 2, + mapper: successMapper, + err: false, + inputs: [][]int{{0, 1}}, + exp: []int{0, 1}, + }, + { + desc: "ZeroBatchSizeStillWorks", + ctx: context.Background(), + batchSize: 0, + mapper: successMapper, + err: false, + inputs: [][]int{{0, 1}}, + exp: []int{0, 1}, + }, + { + desc: "OneBatchLessThanFull", + ctx: context.Background(), + batchSize: 2, + mapper: successMapper, + err: false, + inputs: [][]int{{0}}, + exp: []int{0}, + }, + { + desc: "TwoBatches", + ctx: context.Background(), + batchSize: 2, + mapper: successMapper, + err: false, + inputs: [][]int{{0, 1, 2, 3}}, + exp: []int{0, 1, 2, 3}, + }, + { + desc: "MultipleBatchesMultipleLoaders", + ctx: context.Background(), + batchSize: 2, + mapper: successMapper, + err: false, + inputs: [][]int{{0, 1}, {2}, {3, 4, 5}}, + exp: []int{0, 1, 2, 3, 4, 5}, + }, + { + desc: "HandlesEmptyInputs", + ctx: context.Background(), + batchSize: 2, + mapper: successMapper, + err: false, + inputs: [][]int{{0, 1, 2, 3}, nil, {4}}, + exp: []int{0, 1, 2, 3, 4}, + }, + { + desc: "Timeout", + ctx: expired, + batchSize: 2, + mapper: successMapper, + err: true, + inputs: [][]int{{0}}, + }, + { + desc: "MappingFailure", + ctx: context.Background(), + batchSize: 2, + mapper: errMapper, + err: true, + inputs: [][]int{{0}}, + }, + } { + tc := tc + t.Run(tc.desc, func(t *testing.T) { + fetchers := make([]Fetcher[int, int], 0, len(tc.inputs)) + for range tc.inputs { + fetchers = append( + fetchers, + FetchFunc[int, int](func(ctx context.Context, xs []int) ([]int, error) { + if ctx.Err() != nil { + return nil, ctx.Err() + } + return xs, nil + }), + ) + } - var count int - for blocksIter.Next() && blocksIter.Err() == nil { - count++ + loader := newBatchedLoader[int, int, int]( + tc.ctx, + fetchers, + tc.inputs, + tc.mapper, + tc.batchSize, + ) + + got, err := v1.Collect[int](loader) + if tc.err { + require.Error(t, err) + return + } + require.NoError(t, err) + require.Equal(t, tc.exp, got) + + }) } +} - require.Equal(t, len(blocks), count) - require.Equal(t, int32(len(blocks)/blocksIter.batchSize+1), f.count.Load()) +func TestOverlappingBlocksIter(t *testing.T) { + t.Parallel() + for _, tc := range []struct { + desc string + inp []bloomshipper.BlockRef + exp int // expected groups + }{ + { + desc: "Empty", + inp: []bloomshipper.BlockRef{}, + exp: 0, + }, + { + desc: "NonOverlapping", + inp: []bloomshipper.BlockRef{ + genBlockRef(0x0000, 0x00ff), + genBlockRef(0x0100, 0x01ff), + genBlockRef(0x0200, 0x02ff), + }, + exp: 3, + }, + { + desc: "AllOverlapping", + inp: []bloomshipper.BlockRef{ + genBlockRef(0x0000, 0x02ff), // |-----------| + genBlockRef(0x0100, 0x01ff), // |---| + genBlockRef(0x0200, 0x02ff), // |---| + }, + exp: 1, + }, + { + desc: "PartialOverlapping", + inp: []bloomshipper.BlockRef{ + genBlockRef(0x0000, 0x01ff), // group 1 |-------| + genBlockRef(0x0100, 0x02ff), // group 1 |-------| + genBlockRef(0x0200, 0x03ff), // group 1 |-------| + genBlockRef(0x0200, 0x02ff), // group 1 |---| + }, + exp: 1, + }, + { + desc: "PartialOverlapping", + inp: []bloomshipper.BlockRef{ + genBlockRef(0x0000, 0x01ff), // group 1 |-------| + genBlockRef(0x0100, 0x02ff), // group 1 |-------| + genBlockRef(0x0100, 0x01ff), // group 1 |---| + genBlockRef(0x0300, 0x03ff), // group 2 |---| + genBlockRef(0x0310, 0x03ff), // group 2 |-| + }, + exp: 2, + }, + } { + tc := tc + t.Run(tc.desc, func(t *testing.T) { + it := overlappingBlocksIter(tc.inp) + var overlapping [][]bloomshipper.BlockRef + var i int + for it.Next() && it.Err() == nil { + require.NotNil(t, it.At()) + overlapping = append(overlapping, it.At()) + for _, r := range it.At() { + t.Log(i, r) + } + i++ + } + require.Equal(t, tc.exp, len(overlapping)) + }) + } } diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index 566b836609d10..dd5a9c96ca811 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -2,7 +2,10 @@ package bloomcompactor import ( "context" + "fmt" "math" + "slices" + "sort" "sync" "time" @@ -11,16 +14,23 @@ import ( "github.com/grafana/dskit/backoff" "github.com/grafana/dskit/concurrency" "github.com/grafana/dskit/multierror" + "github.com/grafana/dskit/ring" "github.com/grafana/dskit/services" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" + "github.com/grafana/loki/pkg/bloomutils" "github.com/grafana/loki/pkg/storage" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores" "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" + util_ring "github.com/grafana/loki/pkg/util/ring" +) + +var ( + RingOp = ring.NewOp([]ring.InstanceState{ring.JOINING, ring.ACTIVE}, nil) ) /* @@ -47,7 +57,7 @@ type Compactor struct { // temporary workaround until bloomStore has implemented read/write shipper interface bloomStore bloomshipper.Store - sharding ShardingStrategy + sharding util_ring.TenantSharding metrics *Metrics btMetrics *v1.Metrics @@ -59,7 +69,7 @@ func New( storeCfg storage.Config, clientMetrics storage.ClientMetrics, fetcherProvider stores.ChunkFetcherProvider, - sharding ShardingStrategy, + sharding util_ring.TenantSharding, limits Limits, logger log.Logger, r prometheus.Registerer, @@ -86,7 +96,7 @@ func New( c.bloomStore = bloomStore // initialize metrics - c.btMetrics = v1.NewMetrics(prometheus.WrapRegistererWithPrefix("loki_bloom_tokenizer", r)) + c.btMetrics = v1.NewMetrics(prometheus.WrapRegistererWithPrefix("loki_bloom_tokenizer_", r)) c.metrics = NewMetrics(r, c.btMetrics) chunkLoader := NewStoreChunkLoader( @@ -103,9 +113,7 @@ func New( c.logger, ) - c.metrics.compactionRunInterval.Set(cfg.CompactionInterval.Seconds()) c.Service = services.NewBasicService(c.starting, c.running, c.stopping) - return c, nil } @@ -132,11 +140,17 @@ func (c *Compactor) running(ctx context.Context) error { case <-ctx.Done(): return ctx.Err() - case <-ticker.C: + case start := <-ticker.C: + c.metrics.compactionsStarted.Inc() if err := c.runOne(ctx); err != nil { - level.Error(c.logger).Log("msg", "compaction iteration failed", "err", err) + level.Error(c.logger).Log("msg", "compaction iteration failed", "err", err, "duration", time.Since(start)) + c.metrics.compactionCompleted.WithLabelValues(statusFailure).Inc() + c.metrics.compactionTime.WithLabelValues(statusFailure).Observe(time.Since(start).Seconds()) return err } + level.Info(c.logger).Log("msg", "compaction iteration completed", "duration", time.Since(start)) + c.metrics.compactionCompleted.WithLabelValues(statusSuccess).Inc() + c.metrics.compactionTime.WithLabelValues(statusSuccess).Observe(time.Since(start).Seconds()) } } } @@ -169,11 +183,11 @@ func runWithRetries( type tenantTable struct { tenant string - table config.DayTime + table config.DayTable ownershipRange v1.FingerprintBounds } -func (c *Compactor) tenants(ctx context.Context, table config.DayTime) (v1.Iterator[string], error) { +func (c *Compactor) tenants(ctx context.Context, table config.DayTable) (v1.Iterator[string], error) { tenants, err := c.tsdbStore.UsersForPeriod(ctx, table) if err != nil { return nil, errors.Wrap(err, "getting tenants") @@ -182,13 +196,126 @@ func (c *Compactor) tenants(ctx context.Context, table config.DayTime) (v1.Itera return v1.NewSliceIter(tenants), nil } -// TODO(owen-d): implement w/ subrings -func (c *Compactor) ownsTenant(_ string) (ownershipRange v1.FingerprintBounds, owns bool) { - return v1.NewBounds(0, math.MaxUint64), true +// ownsTenant returns the ownership range for the tenant, if the compactor owns the tenant, and an error. +func (c *Compactor) ownsTenant(tenant string) ([]v1.FingerprintBounds, bool, error) { + tenantRing, owned := c.sharding.OwnsTenant(tenant) + if !owned { + return nil, false, nil + } + + // TOOD(owen-d): use .GetTokenRangesForInstance() + // when it's supported for non zone-aware rings + // instead of doing all this manually + + rs, err := tenantRing.GetAllHealthy(RingOp) + if err != nil { + return nil, false, errors.Wrap(err, "getting ring healthy instances") + } + + ranges, err := tokenRangesForInstance(c.cfg.Ring.InstanceID, rs.Instances) + if err != nil { + return nil, false, errors.Wrap(err, "getting token ranges for instance") + } + + keyspaces := bloomutils.KeyspacesFromTokenRanges(ranges) + return keyspaces, true, nil +} + +func tokenRangesForInstance(id string, instances []ring.InstanceDesc) (ranges ring.TokenRanges, err error) { + var ownedTokens map[uint32]struct{} + + // lifted from grafana/dskit/ring/model.go <*Desc>.GetTokens() + toks := make([][]uint32, 0, len(instances)) + for _, instance := range instances { + if instance.Id == id { + ranges = make(ring.TokenRanges, 0, 2*(len(instance.Tokens)+1)) + ownedTokens = make(map[uint32]struct{}, len(instance.Tokens)) + for _, tok := range instance.Tokens { + ownedTokens[tok] = struct{}{} + } + } + + // Tokens may not be sorted for an older version which, so we enforce sorting here. + tokens := instance.Tokens + if !sort.IsSorted(ring.Tokens(tokens)) { + sort.Sort(ring.Tokens(tokens)) + } + + toks = append(toks, tokens) + } + + if cap(ranges) == 0 { + return nil, fmt.Errorf("instance %s not found", id) + } + + allTokens := ring.MergeTokens(toks) + if len(allTokens) == 0 { + return nil, errors.New("no tokens in the ring") + } + + // mostly lifted from grafana/dskit/ring/token_range.go <*Ring>.GetTokenRangesForInstance() + + // non-zero value means we're now looking for start of the range. Zero value means we're looking for next end of range (ie. token owned by this instance). + rangeEnd := uint32(0) + + // if this instance claimed the first token, it owns the wrap-around range, which we'll break into two separate ranges + firstToken := allTokens[0] + _, ownsFirstToken := ownedTokens[firstToken] + + if ownsFirstToken { + // we'll start by looking for the beginning of the range that ends with math.MaxUint32 + rangeEnd = math.MaxUint32 + } + + // walk the ring backwards, alternating looking for ends and starts of ranges + for i := len(allTokens) - 1; i > 0; i-- { + token := allTokens[i] + _, owned := ownedTokens[token] + + if rangeEnd == 0 { + // we're looking for the end of the next range + if owned { + rangeEnd = token - 1 + } + } else { + // we have a range end, and are looking for the start of the range + if !owned { + ranges = append(ranges, rangeEnd, token) + rangeEnd = 0 + } + } + } + + // finally look at the first token again + // - if we have a range end, check if we claimed token 0 + // - if we don't, we have our start + // - if we do, the start is 0 + // - if we don't have a range end, check if we claimed token 0 + // - if we don't, do nothing + // - if we do, add the range of [0, token-1] + // - BUT, if the token itself is 0, do nothing, because we don't own the tokens themselves (we should be covered by the already added range that ends with MaxUint32) + + if rangeEnd == 0 { + if ownsFirstToken && firstToken != 0 { + ranges = append(ranges, firstToken-1, 0) + } + } else { + if ownsFirstToken { + ranges = append(ranges, rangeEnd, 0) + } else { + ranges = append(ranges, rangeEnd, firstToken) + } + } + + // Ensure returned ranges are sorted. + slices.Sort(ranges) + + return ranges, nil } // runs a single round of compaction for all relevant tenants and tables func (c *Compactor) runOne(ctx context.Context) error { + level.Info(c.logger).Log("msg", "running bloom compaction", "workers", c.cfg.WorkerParallelism) var workersErr error var wg sync.WaitGroup ch := make(chan tenantTable) @@ -201,7 +328,11 @@ func (c *Compactor) runOne(ctx context.Context) error { err := c.loadWork(ctx, ch) wg.Wait() - return multierror.New(workersErr, err, ctx.Err()).Err() + err = multierror.New(workersErr, err, ctx.Err()).Err() + if err != nil { + level.Error(c.logger).Log("msg", "compaction iteration failed", "err", err) + } + return err } func (c *Compactor) tables(ts time.Time) *dayRangeIterator { @@ -216,41 +347,60 @@ func (c *Compactor) tables(ts time.Time) *dayRangeIterator { fromDay := config.NewDayTime(model.TimeFromUnixNano(from)) throughDay := config.NewDayTime(model.TimeFromUnixNano(through)) - return newDayRangeIterator(fromDay, throughDay) + level.Debug(c.logger).Log("msg", "loaded tables for compaction", "from", fromDay, "through", throughDay) + return newDayRangeIterator(fromDay, throughDay, c.schemaCfg) } func (c *Compactor) loadWork(ctx context.Context, ch chan<- tenantTable) error { tables := c.tables(time.Now()) for tables.Next() && tables.Err() == nil && ctx.Err() == nil { - table := tables.At() + + level.Debug(c.logger).Log("msg", "loading work for table", "table", table) + tenants, err := c.tenants(ctx, table) if err != nil { return errors.Wrap(err, "getting tenants") } for tenants.Next() && tenants.Err() == nil && ctx.Err() == nil { + c.metrics.tenantsDiscovered.Inc() tenant := tenants.At() - ownershipRange, owns := c.ownsTenant(tenant) + ownershipRanges, owns, err := c.ownsTenant(tenant) + if err != nil { + return errors.Wrap(err, "checking tenant ownership") + } + level.Debug(c.logger).Log("msg", "enqueueing work for tenant", "tenant", tenant, "table", table, "ranges", len(ownershipRanges), "owns", owns) if !owns { + c.metrics.tenantsSkipped.Inc() continue } - - select { - case ch <- tenantTable{tenant: tenant, table: table, ownershipRange: ownershipRange}: - case <-ctx.Done(): - return ctx.Err() + c.metrics.tenantsOwned.Inc() + + for _, ownershipRange := range ownershipRanges { + + select { + case ch <- tenantTable{ + tenant: tenant, + table: table, + ownershipRange: ownershipRange, + }: + case <-ctx.Done(): + return ctx.Err() + } } } if err := tenants.Err(); err != nil { + level.Error(c.logger).Log("msg", "error iterating tenants", "err", err) return errors.Wrap(err, "iterating tenants") } } if err := tables.Err(); err != nil { + level.Error(c.logger).Log("msg", "error iterating tables", "err", err) return errors.Wrap(err, "iterating tables") } @@ -272,7 +422,11 @@ func (c *Compactor) runWorkers(ctx context.Context, ch <-chan tenantTable) error return nil } + start := time.Now() + c.metrics.tenantsStarted.Inc() if err := c.compactTenantTable(ctx, tt); err != nil { + c.metrics.tenantsCompleted.WithLabelValues(statusFailure).Inc() + c.metrics.tenantsCompletedTime.WithLabelValues(statusFailure).Observe(time.Since(start).Seconds()) return errors.Wrapf( err, "compacting tenant table (%s) for tenant (%s) with ownership (%s)", @@ -281,6 +435,8 @@ func (c *Compactor) runWorkers(ctx context.Context, ch <-chan tenantTable) error tt.ownershipRange, ) } + c.metrics.tenantsCompleted.WithLabelValues(statusSuccess).Inc() + c.metrics.tenantsCompletedTime.WithLabelValues(statusSuccess).Observe(time.Since(start).Seconds()) } } @@ -289,25 +445,39 @@ func (c *Compactor) runWorkers(ctx context.Context, ch <-chan tenantTable) error } func (c *Compactor) compactTenantTable(ctx context.Context, tt tenantTable) error { - level.Info(c.logger).Log("msg", "compacting", "org_id", tt.tenant, "table", tt.table, "ownership", tt.ownershipRange) - return c.controller.buildBlocks(ctx, tt.table, tt.tenant, tt.ownershipRange) + level.Info(c.logger).Log("msg", "compacting", "org_id", tt.tenant, "table", tt.table, "ownership", tt.ownershipRange.String()) + return c.controller.compactTenant(ctx, tt.table, tt.tenant, tt.ownershipRange) } type dayRangeIterator struct { min, max, cur config.DayTime + curPeriod config.PeriodConfig + schemaCfg config.SchemaConfig + err error } -func newDayRangeIterator(min, max config.DayTime) *dayRangeIterator { - return &dayRangeIterator{min: min, max: max, cur: min.Dec()} +func newDayRangeIterator(min, max config.DayTime, schemaCfg config.SchemaConfig) *dayRangeIterator { + return &dayRangeIterator{min: min, max: max, cur: min.Dec(), schemaCfg: schemaCfg} } func (r *dayRangeIterator) Next() bool { r.cur = r.cur.Inc() - return r.cur.Before(r.max) + if !r.cur.Before(r.max) { + return false + } + + period, err := r.schemaCfg.SchemaForTime(r.cur.ModelTime()) + if err != nil { + r.err = errors.Wrapf(err, "getting schema for time (%s)", r.cur) + return false + } + r.curPeriod = period + + return true } -func (r *dayRangeIterator) At() config.DayTime { - return r.cur +func (r *dayRangeIterator) At() config.DayTable { + return config.NewDayTable(r.cur, r.curPeriod.IndexTables.Prefix) } func (r *dayRangeIterator) Err() error { diff --git a/pkg/bloomcompactor/bloomcompactor_test.go b/pkg/bloomcompactor/bloomcompactor_test.go new file mode 100644 index 0000000000000..097e04d2a39a6 --- /dev/null +++ b/pkg/bloomcompactor/bloomcompactor_test.go @@ -0,0 +1,264 @@ +package bloomcompactor + +import ( + "context" + "flag" + "fmt" + "math" + "testing" + "time" + + "github.com/grafana/dskit/ring" + "github.com/grafana/dskit/services" + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/require" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + util_log "github.com/grafana/loki/pkg/util/log" + lokiring "github.com/grafana/loki/pkg/util/ring" + util_ring "github.com/grafana/loki/pkg/util/ring" + "github.com/grafana/loki/pkg/validation" +) + +func TestCompactor_ownsTenant(t *testing.T) { + for _, tc := range []struct { + name string + limits Limits + compactors int + + expectedCompactorsOwningTenant int + }{ + { + name: "no sharding with one instance", + limits: mockLimits{ + shardSize: 0, + }, + compactors: 1, + expectedCompactorsOwningTenant: 1, + }, + { + name: "no sharding with multiple instances", + limits: mockLimits{ + shardSize: 0, + }, + compactors: 10, + expectedCompactorsOwningTenant: 10, + }, + { + name: "sharding with one instance", + limits: mockLimits{ + shardSize: 5, + }, + compactors: 1, + expectedCompactorsOwningTenant: 1, + }, + { + name: "sharding with multiple instances", + limits: mockLimits{ + shardSize: 5, + }, + compactors: 10, + expectedCompactorsOwningTenant: 5, + }, + } { + t.Run(tc.name, func(t *testing.T) { + var ringManagers []*lokiring.RingManager + var compactors []*Compactor + for i := 0; i < tc.compactors; i++ { + var ringCfg lokiring.RingConfig + ringCfg.RegisterFlagsWithPrefix("", "", flag.NewFlagSet("ring", flag.PanicOnError)) + ringCfg.KVStore.Store = "inmemory" + ringCfg.InstanceID = fmt.Sprintf("bloom-compactor-%d", i) + ringCfg.InstanceAddr = fmt.Sprintf("localhost-%d", i) + + ringManager, err := lokiring.NewRingManager("bloom-compactor", lokiring.ServerMode, ringCfg, 1, 1, util_log.Logger, prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, ringManager.StartAsync(context.Background())) + + shuffleSharding := util_ring.NewTenantShuffleSharding(ringManager.Ring, ringManager.RingLifecycler, tc.limits.BloomCompactorShardSize) + + compactor := &Compactor{ + cfg: Config{ + Ring: ringCfg, + }, + sharding: shuffleSharding, + limits: tc.limits, + } + + ringManagers = append(ringManagers, ringManager) + compactors = append(compactors, compactor) + } + defer func() { + // Stop all rings and wait for them to stop. + for _, ringManager := range ringManagers { + ringManager.StopAsync() + require.Eventually(t, func() bool { + return ringManager.State() == services.Terminated + }, 1*time.Minute, 100*time.Millisecond) + } + }() + + // Wait for all rings to see each other. + for _, ringManager := range ringManagers { + require.Eventually(t, func() bool { + running := ringManager.State() == services.Running + discovered := ringManager.Ring.InstancesCount() == tc.compactors + return running && discovered + }, 1*time.Minute, 100*time.Millisecond) + } + + var compactorOwnsTenant int + var compactorOwnershipRange []v1.FingerprintBounds + for _, compactor := range compactors { + ownershipRange, ownsTenant, err := compactor.ownsTenant("tenant") + require.NoError(t, err) + if ownsTenant { + compactorOwnsTenant++ + compactorOwnershipRange = append(compactorOwnershipRange, ownershipRange...) + } + } + require.Equal(t, tc.expectedCompactorsOwningTenant, compactorOwnsTenant) + + coveredKeySpace := v1.NewBounds(math.MaxUint64, 0) + for i, boundsA := range compactorOwnershipRange { + for j, boundsB := range compactorOwnershipRange { + if i == j { + continue + } + // Assert that the fingerprint key-space is not overlapping + require.False(t, boundsA.Overlaps(boundsB)) + } + + if boundsA.Min < coveredKeySpace.Min { + coveredKeySpace.Min = boundsA.Min + } + if boundsA.Max > coveredKeySpace.Max { + coveredKeySpace.Max = boundsA.Max + } + + } + // Assert that the fingerprint key-space is complete + require.True(t, coveredKeySpace.Equal(v1.NewBounds(0, math.MaxUint64))) + }) + } +} + +type mockLimits struct { + shardSize int +} + +func (m mockLimits) AllByUserID() map[string]*validation.Limits { + panic("implement me") +} + +func (m mockLimits) DefaultLimits() *validation.Limits { + panic("implement me") +} + +func (m mockLimits) VolumeMaxSeries(_ string) int { + panic("implement me") +} + +func (m mockLimits) BloomCompactorShardSize(_ string) int { + return m.shardSize +} + +func (m mockLimits) BloomCompactorChunksBatchSize(_ string) int { + panic("implement me") +} + +func (m mockLimits) BloomCompactorMaxTableAge(_ string) time.Duration { + panic("implement me") +} + +func (m mockLimits) BloomCompactorEnabled(_ string) bool { + panic("implement me") +} + +func (m mockLimits) BloomNGramLength(_ string) int { + panic("implement me") +} + +func (m mockLimits) BloomNGramSkip(_ string) int { + panic("implement me") +} + +func (m mockLimits) BloomFalsePositiveRate(_ string) float64 { + panic("implement me") +} + +func (m mockLimits) BloomCompactorMaxBlockSize(_ string) int { + panic("implement me") +} + +func TestTokenRangesForInstance(t *testing.T) { + desc := func(id int, tokens ...uint32) ring.InstanceDesc { + return ring.InstanceDesc{Id: fmt.Sprintf("%d", id), Tokens: tokens} + } + + tests := map[string]struct { + input []ring.InstanceDesc + exp map[string]ring.TokenRanges + err bool + }{ + "no nodes": { + input: []ring.InstanceDesc{}, + exp: map[string]ring.TokenRanges{ + "0": {0, math.MaxUint32}, // have to put one in here to trigger test + }, + err: true, + }, + "one node": { + input: []ring.InstanceDesc{ + desc(0, 0, 100), + }, + exp: map[string]ring.TokenRanges{ + "0": {0, math.MaxUint32}, + }, + }, + "two nodes": { + input: []ring.InstanceDesc{ + desc(0, 25, 75), + desc(1, 10, 50, 100), + }, + exp: map[string]ring.TokenRanges{ + "0": {10, 24, 50, 74}, + "1": {0, 9, 25, 49, 75, math.MaxUint32}, + }, + }, + "consecutive tokens": { + input: []ring.InstanceDesc{ + desc(0, 99), + desc(1, 100), + }, + exp: map[string]ring.TokenRanges{ + "0": {0, 98, 100, math.MaxUint32}, + "1": {99, 99}, + }, + }, + "extremes": { + input: []ring.InstanceDesc{ + desc(0, 0), + desc(1, math.MaxUint32), + }, + exp: map[string]ring.TokenRanges{ + "0": {math.MaxUint32, math.MaxUint32}, + "1": {0, math.MaxUint32 - 1}, + }, + }, + } + + for desc, test := range tests { + t.Run(desc, func(t *testing.T) { + for id := range test.exp { + ranges, err := tokenRangesForInstance(id, test.input) + if test.err { + require.Error(t, err) + continue + } + require.NoError(t, err) + require.Equal(t, test.exp[id], ranges) + } + }) + } +} diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go index 8470fd9ad7082..2a4ff6cd45242 100644 --- a/pkg/bloomcompactor/controller.go +++ b/pkg/bloomcompactor/controller.go @@ -4,12 +4,10 @@ import ( "bytes" "context" "fmt" - "io" "sort" "github.com/go-kit/log" "github.com/go-kit/log/level" - "github.com/grafana/dskit/multierror" "github.com/pkg/errors" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" @@ -25,7 +23,6 @@ type SimpleBloomController struct { metrics *Metrics limits Limits - // TODO(owen-d): add metrics logger log.Logger } @@ -54,31 +51,34 @@ func (s *SimpleBloomController) rwFn() (v1.BlockWriter, v1.BlockReader) { return v1.NewMemoryBlockWriter(indexBuf, bloomsBuf), v1.NewByteReader(indexBuf, bloomsBuf) } -func (s *SimpleBloomController) buildBlocks( +/* +Compaction works as follows, split across many functions for clarity: + 1. Fetch all meta.jsons for the given tenant and table which overlap the ownership range of this compactor. + 2. Load current TSDBs for this tenant/table. + 3. For each live TSDB (there should be only 1, but this works with multiple), find any gaps + (fingerprint ranges) which are not up date, determined by checking other meta.jsons and comparing + the tsdbs they were generated from + their ownership ranges. + 4. Build new bloom blocks for each gap, using the series and chunks from the TSDBs and any existing + blocks which overlap the gaps to accelerate bloom generation. + 5. Write the new blocks and metas to the store. + 6. Determine if any meta.jsons overlap the ownership range but are outdated, and remove them and + their associated blocks if so. +*/ +func (s *SimpleBloomController) compactTenant( ctx context.Context, - table config.DayTime, + table config.DayTable, tenant string, ownershipRange v1.FingerprintBounds, ) error { - logger := log.With(s.logger, "ownership", ownershipRange, "org_id", tenant, "table", table) + logger := log.With(s.logger, "org_id", tenant, "table", table.Addr(), "ownership", ownershipRange.String()) - // 1. Resolve TSDBs - tsdbs, err := s.tsdbStore.ResolveTSDBs(ctx, table, tenant) + client, err := s.bloomStore.Client(table.ModelTime()) if err != nil { - level.Error(logger).Log("msg", "failed to resolve tsdbs", "err", err) - return errors.Wrap(err, "failed to resolve tsdbs") - } - - if len(tsdbs) == 0 { - return nil + level.Error(logger).Log("msg", "failed to get client", "err", err) + return errors.Wrap(err, "failed to get client") } - ids := make([]tsdb.Identifier, 0, len(tsdbs)) - for _, id := range tsdbs { - ids = append(ids, id) - } - - // 2. Fetch metas + // Fetch source metas to be used in both compaction and cleanup of out-of-date metas+blooms metas, err := s.bloomStore.FetchMetas( ctx, bloomshipper.MetaSearchParams{ @@ -92,31 +92,221 @@ func (s *SimpleBloomController) buildBlocks( return errors.Wrap(err, "failed to get metas") } - // 3. Determine which TSDBs have gaps in the ownership range and need to + level.Debug(logger).Log("msg", "found relevant metas", "metas", len(metas)) + + // fetch all metas overlapping our ownership range so we can safely + // check which metas can be deleted even if they only partially overlap out ownership range + superset, err := s.fetchSuperSet(ctx, tenant, table, ownershipRange, metas, logger) + if err != nil { + return errors.Wrap(err, "failed to fetch superset") + } + + // build compaction plans + work, err := s.findOutdatedGaps(ctx, tenant, table, ownershipRange, metas, logger) + if err != nil { + return errors.Wrap(err, "failed to find outdated gaps") + } + + // build new blocks + built, err := s.buildGaps(ctx, tenant, table, client, work, logger) + if err != nil { + return errors.Wrap(err, "failed to build gaps") + } + + // combine built and superset metas + // in preparation for removing outdated ones + combined := append(superset, built...) + + outdated := outdatedMetas(combined) + level.Debug(logger).Log("msg", "found outdated metas", "outdated", len(outdated)) + + var ( + deletedMetas int + deletedBlocks int + ) + defer func() { + s.metrics.metasDeleted.Add(float64(deletedMetas)) + s.metrics.blocksDeleted.Add(float64(deletedBlocks)) + }() + + for _, meta := range outdated { + for _, block := range meta.Blocks { + err := client.DeleteBlocks(ctx, []bloomshipper.BlockRef{block}) + if err != nil { + if client.IsObjectNotFoundErr(err) { + level.Debug(logger).Log("msg", "block not found while attempting delete, continuing", "block", block.String()) + } else { + level.Error(logger).Log("msg", "failed to delete block", "err", err, "block", block.String()) + return errors.Wrap(err, "failed to delete block") + } + } + deletedBlocks++ + level.Debug(logger).Log("msg", "removed outdated block", "block", block.String()) + } + + err = client.DeleteMetas(ctx, []bloomshipper.MetaRef{meta.MetaRef}) + if err != nil { + if client.IsObjectNotFoundErr(err) { + level.Debug(logger).Log("msg", "meta not found while attempting delete, continuing", "meta", meta.MetaRef.String()) + } else { + level.Error(logger).Log("msg", "failed to delete meta", "err", err, "meta", meta.MetaRef.String()) + return errors.Wrap(err, "failed to delete meta") + } + } + deletedMetas++ + level.Debug(logger).Log("msg", "removed outdated meta", "meta", meta.MetaRef.String()) + } + + level.Debug(logger).Log("msg", "finished compaction") + return nil +} + +// fetchSuperSet fetches all metas which overlap the ownership range of the first set of metas we've resolved +func (s *SimpleBloomController) fetchSuperSet( + ctx context.Context, + tenant string, + table config.DayTable, + ownershipRange v1.FingerprintBounds, + metas []bloomshipper.Meta, + logger log.Logger, +) ([]bloomshipper.Meta, error) { + // in order to delete outdates metas which only partially fall within the ownership range, + // we need to fetcha all metas in the entire bound range of the first set of metas we've resolved + /* + For instance, we have the following ownership range and we resolve `meta1` in our first Fetch call + because it overlaps the ownership range, we'll need to fetch newer metas that may overlap it in order + to check if it safely can be deleted. This falls partially outside our specific ownership range, but + we can safely run multiple deletes by treating their removal as idempotent. + |-------------ownership range-----------------| + |-------meta1-------| + + we fetch this before possibly deleting meta1 |------| + */ + superset := ownershipRange + for _, meta := range metas { + union := superset.Union(meta.Bounds) + if len(union) > 1 { + level.Error(logger).Log("msg", "meta bounds union is not a single range", "union", union) + return nil, errors.New("meta bounds union is not a single range") + } + superset = union[0] + } + + within := superset.Within(ownershipRange) + level.Debug(logger).Log( + "msg", "looking for superset metas", + "superset", superset.String(), + "superset_within", within, + ) + + if within { + // we don't need to fetch any more metas + // NB(owen-d): here we copy metas into the output. This is slightly inefficient, but + // helps prevent mutability bugs by returning the same slice as the input. + results := make([]bloomshipper.Meta, len(metas)) + copy(results, metas) + return results, nil + } + + supersetMetas, err := s.bloomStore.FetchMetas( + ctx, + bloomshipper.MetaSearchParams{ + TenantID: tenant, + Interval: bloomshipper.NewInterval(table.Bounds()), + Keyspace: superset, + }, + ) + + if err != nil { + level.Error(logger).Log("msg", "failed to get meta superset range", "err", err, "superset", superset) + return nil, errors.Wrap(err, "failed to get meta supseret range") + } + + level.Debug(logger).Log( + "msg", "found superset metas", + "metas", len(metas), + "fresh_metas", len(supersetMetas), + "delta", len(supersetMetas)-len(metas), + ) + + return supersetMetas, nil +} + +func (s *SimpleBloomController) findOutdatedGaps( + ctx context.Context, + tenant string, + table config.DayTable, + ownershipRange v1.FingerprintBounds, + metas []bloomshipper.Meta, + logger log.Logger, +) ([]blockPlan, error) { + // Resolve TSDBs + tsdbs, err := s.tsdbStore.ResolveTSDBs(ctx, table, tenant) + if err != nil { + level.Error(logger).Log("msg", "failed to resolve tsdbs", "err", err) + return nil, errors.Wrap(err, "failed to resolve tsdbs") + } + + if len(tsdbs) == 0 { + return nil, nil + } + + // Determine which TSDBs have gaps in the ownership range and need to // be processed. - tsdbsWithGaps, err := gapsBetweenTSDBsAndMetas(ownershipRange, ids, metas) + tsdbsWithGaps, err := gapsBetweenTSDBsAndMetas(ownershipRange, tsdbs, metas) if err != nil { level.Error(logger).Log("msg", "failed to find gaps", "err", err) - return errors.Wrap(err, "failed to find gaps") + return nil, errors.Wrap(err, "failed to find gaps") } if len(tsdbsWithGaps) == 0 { level.Debug(logger).Log("msg", "blooms exist for all tsdbs") - return nil + return nil, nil } work, err := blockPlansForGaps(tsdbsWithGaps, metas) if err != nil { level.Error(logger).Log("msg", "failed to create plan", "err", err) - return errors.Wrap(err, "failed to create plan") + return nil, errors.Wrap(err, "failed to create plan") } - nGramSize := uint64(s.limits.BloomNGramLength(tenant)) - nGramSkip := uint64(s.limits.BloomNGramSkip(tenant)) - maxBlockSize := uint64(s.limits.BloomCompactorMaxBlockSize(tenant)) - blockOpts := v1.NewBlockOptions(nGramSize, nGramSkip, maxBlockSize) + return work, nil +} - // 4. Generate Blooms +func (s *SimpleBloomController) loadWorkForGap( + ctx context.Context, + table config.DayTable, + tenant string, + id tsdb.Identifier, + gap gapWithBlocks, +) (v1.CloseableIterator[*v1.Series], v1.CloseableResettableIterator[*v1.SeriesWithBloom], error) { + // load a series iterator for the gap + seriesItr, err := s.tsdbStore.LoadTSDB(ctx, table, tenant, id, gap.bounds) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to load tsdb") + } + + // load a blocks iterator for the gap + fetcher, err := s.bloomStore.Fetcher(table.ModelTime()) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to get fetcher") + } + + f := FetchFunc[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier](fetcher.FetchBlocks) + blocksIter := newBlockLoadingIter(ctx, gap.blocks, f, 10) + + return seriesItr, blocksIter, nil +} + +func (s *SimpleBloomController) buildGaps( + ctx context.Context, + tenant string, + table config.DayTable, + client bloomshipper.Client, + work []blockPlan, + logger log.Logger, +) ([]bloomshipper.Meta, error) { + // Generate Blooms // Now that we have the gaps, we will generate a bloom block for each gap. // We can accelerate this by using existing blocks which may already contain // needed chunks in their blooms, for instance after a new TSDB version is generated @@ -127,56 +317,75 @@ func (s *SimpleBloomController) buildBlocks( // accelerate bloom generation for the new blocks. var ( - blockCt int - tsdbCt = len(work) + blockCt int + tsdbCt = len(work) + nGramSize = uint64(s.limits.BloomNGramLength(tenant)) + nGramSkip = uint64(s.limits.BloomNGramSkip(tenant)) + maxBlockSize = uint64(s.limits.BloomCompactorMaxBlockSize(tenant)) + blockOpts = v1.NewBlockOptions(nGramSize, nGramSkip, maxBlockSize) + created []bloomshipper.Meta + totalSeries uint64 ) for _, plan := range work { - for _, gap := range plan.gaps { + for i := range plan.gaps { + gap := plan.gaps[i] + logger := log.With(logger, "gap", gap.bounds.String(), "tsdb", plan.tsdb.Name()) + + meta := bloomshipper.Meta{ + MetaRef: bloomshipper.MetaRef{ + Ref: bloomshipper.Ref{ + TenantID: tenant, + TableName: table.Addr(), + Bounds: gap.bounds, + }, + }, + Sources: []tsdb.SingleTenantTSDBIdentifier{plan.tsdb}, + } + // Fetch blocks that aren't up to date but are in the desired fingerprint range // to try and accelerate bloom creation seriesItr, blocksIter, err := s.loadWorkForGap(ctx, table, tenant, plan.tsdb, gap) if err != nil { level.Error(logger).Log("msg", "failed to get series and blocks", "err", err) - return errors.Wrap(err, "failed to get series and blocks") + return nil, errors.Wrap(err, "failed to get series and blocks") } + // Blocks are built consuming the series iterator. For observability, we wrap the series iterator + // with a counter iterator to count the number of times Next() is called on it. + // This is used to observe the number of series that are being processed. + seriesItrWithCounter := v1.NewCounterIter[*v1.Series](seriesItr) + gen := NewSimpleBloomGenerator( tenant, blockOpts, - seriesItr, + seriesItrWithCounter, s.chunkLoader, blocksIter, s.rwFn, s.metrics, - log.With(logger, "tsdb", plan.tsdb.Name(), "ownership", gap), + logger, ) - _, loaded, newBlocks, err := gen.Generate(ctx) + level.Debug(logger).Log("msg", "generating blocks", "overlapping_blocks", len(gap.blocks)) + newBlocks := gen.Generate(ctx) if err != nil { - // TODO(owen-d): metrics level.Error(logger).Log("msg", "failed to generate bloom", "err", err) - s.closeLoadedBlocks(loaded, blocksIter) - return errors.Wrap(err, "failed to generate bloom") - } - - client, err := s.bloomStore.Client(table.ModelTime()) - if err != nil { - level.Error(logger).Log("msg", "failed to get client", "err", err) - s.closeLoadedBlocks(loaded, blocksIter) - return errors.Wrap(err, "failed to get client") + blocksIter.Close() + return nil, errors.Wrap(err, "failed to generate bloom") } for newBlocks.Next() && newBlocks.Err() == nil { blockCt++ blk := newBlocks.At() - built, err := bloomshipper.BlockFrom(tenant, table.Table(), blk) + built, err := bloomshipper.BlockFrom(tenant, table.Addr(), blk) if err != nil { level.Error(logger).Log("msg", "failed to build block", "err", err) - return errors.Wrap(err, "failed to build block") + blocksIter.Close() + return nil, errors.Wrap(err, "failed to build block") } if err := client.PutBlock( @@ -184,80 +393,168 @@ func (s *SimpleBloomController) buildBlocks( built, ); err != nil { level.Error(logger).Log("msg", "failed to write block", "err", err) - s.closeLoadedBlocks(loaded, blocksIter) - return errors.Wrap(err, "failed to write block") + blocksIter.Close() + return nil, errors.Wrap(err, "failed to write block") } + s.metrics.blocksCreated.Inc() + + totalGapKeyspace := (gap.bounds.Max - gap.bounds.Min) + progress := (built.Bounds.Max - gap.bounds.Min) + pct := float64(progress) / float64(totalGapKeyspace) * 100 + level.Debug(logger).Log( + "msg", "uploaded block", + "block", built.BlockRef.String(), + "progress_pct", fmt.Sprintf("%.2f", pct), + ) + + meta.Blocks = append(meta.Blocks, built.BlockRef) } if err := newBlocks.Err(); err != nil { - // TODO(owen-d): metrics level.Error(logger).Log("msg", "failed to generate bloom", "err", err) - s.closeLoadedBlocks(loaded, blocksIter) - return errors.Wrap(err, "failed to generate bloom") + return nil, errors.Wrap(err, "failed to generate bloom") } // Close pre-existing blocks - s.closeLoadedBlocks(loaded, blocksIter) + blocksIter.Close() + + // Write the new meta + // TODO(owen-d): put total size in log, total time in metrics+log + ref, err := bloomshipper.MetaRefFrom(tenant, table.Addr(), gap.bounds, meta.Sources, meta.Blocks) + if err != nil { + level.Error(logger).Log("msg", "failed to checksum meta", "err", err) + return nil, errors.Wrap(err, "failed to checksum meta") + } + meta.MetaRef = ref + + if err := client.PutMeta(ctx, meta); err != nil { + level.Error(logger).Log("msg", "failed to write meta", "err", err) + return nil, errors.Wrap(err, "failed to write meta") + } + s.metrics.metasCreated.Inc() + level.Debug(logger).Log("msg", "uploaded meta", "meta", meta.MetaRef.String()) + + created = append(created, meta) + totalSeries += uint64(seriesItrWithCounter.Count()) } } - // TODO(owen-d): build meta from blocks - // TODO(owen-d): reap tombstones, old metas - + s.metrics.tenantsSeries.Observe(float64(totalSeries)) level.Debug(logger).Log("msg", "finished bloom generation", "blocks", blockCt, "tsdbs", tsdbCt) - return nil - + return created, nil } -func (s *SimpleBloomController) loadWorkForGap( - ctx context.Context, - table config.DayTime, - tenant string, - id tsdb.Identifier, - gap gapWithBlocks, -) (v1.CloseableIterator[*v1.Series], v1.CloseableIterator[*bloomshipper.CloseableBlockQuerier], error) { - // load a series iterator for the gap - seriesItr, err := s.tsdbStore.LoadTSDB(ctx, table, tenant, id, gap.bounds) - if err != nil { - return nil, nil, errors.Wrap(err, "failed to load tsdb") - } +// outdatedMetas returns metas that are outdated and need to be removed, +// determined by if their entire ownership range is covered by other metas with newer +// TSDBs +func outdatedMetas(metas []bloomshipper.Meta) (outdated []bloomshipper.Meta) { + // first, ensure data is sorted so we can take advantage of that + sort.Slice(metas, func(i, j int) bool { + return metas[i].Bounds.Less(metas[j].Bounds) + }) + + // NB(owen-d): time complexity shouldn't be a problem + // given the number of metas should be low (famous last words, i know). + for i := range metas { + a := metas[i] + + var overlaps []v1.FingerprintBounds + + for j := range metas { + if j == i { + continue + } - // load a blocks iterator for the gap - fetcher, err := s.bloomStore.Fetcher(table.ModelTime()) - if err != nil { - return nil, nil, errors.Wrap(err, "failed to get fetcher") - } + b := metas[j] + intersection := a.Bounds.Intersection(b.Bounds) + if intersection == nil { + if a.Bounds.Cmp(b.Bounds.Min) == v1.After { + // All subsequent metas will be newer, so we can break + break + } + // otherwise, just check the next meta + continue + } - blocksIter, err := newBatchedBlockLoader(ctx, fetcher, gap.blocks) - if err != nil { - return nil, nil, errors.Wrap(err, "failed to load blocks") - } + // we can only remove older data, not data which may be newer + if !tsdbsStrictlyNewer(b.Sources, a.Sources) { + continue + } - return seriesItr, blocksIter, nil + // because we've sorted the metas, we only have to test overlaps against the last + // overlap we found (if any) + if len(overlaps) == 0 { + overlaps = append(overlaps, *intersection) + continue + } + + // best effort at merging overlaps first pass + last := overlaps[len(overlaps)-1] + overlaps = append(overlaps[:len(overlaps)-1], last.Union(*intersection)...) + + } + + if coversFullRange(a.Bounds, overlaps) { + outdated = append(outdated, a) + } + } + return } -func (s *SimpleBloomController) closeLoadedBlocks(toClose []io.Closer, it v1.CloseableIterator[*bloomshipper.CloseableBlockQuerier]) { - // close loaded blocks - var err multierror.MultiError - for _, closer := range toClose { - err.Add(closer.Close()) +func coversFullRange(bounds v1.FingerprintBounds, overlaps []v1.FingerprintBounds) bool { + // if there are no overlaps, the range is not covered + if len(overlaps) == 0 { + return false } - switch itr := it.(type) { - case *batchedBlockLoader: - // close remaining loaded blocks from batch - err.Add(itr.CloseBatch()) - default: - // close remaining loaded blocks - for itr.Next() && itr.Err() == nil { - err.Add(itr.At().Close()) + // keep track of bounds which need to be filled in order + // for the overlaps to cover the full range + missing := []v1.FingerprintBounds{bounds} + ignores := make(map[int]bool) + for _, overlap := range overlaps { + var i int + for { + if i >= len(missing) { + break + } + + if ignores[i] { + i++ + continue + } + + remaining := missing[i].Unless(overlap) + switch len(remaining) { + case 0: + // this range is covered, ignore it + ignores[i] = true + case 1: + // this range is partially covered, updated it + missing[i] = remaining[0] + case 2: + // this range has been partially covered in the middle, + // split it into two ranges and append + ignores[i] = true + missing = append(missing, remaining...) + } + i++ } + } - // log error - if err.Err() != nil { - level.Error(s.logger).Log("msg", "failed to close blocks", "err", err) + return len(ignores) == len(missing) +} + +// tsdbStrictlyNewer returns if all of the tsdbs in a are newer than all of the tsdbs in b +func tsdbsStrictlyNewer(as, bs []tsdb.SingleTenantTSDBIdentifier) bool { + for _, a := range as { + for _, b := range bs { + if a.TS.Before(b.TS) { + return false + } + } } + return true } type gapWithBlocks struct { @@ -275,7 +572,7 @@ type gapWithBlocks struct { // of the same chunks we need to ensure are indexed, just from previous tsdb iterations. // This is a performance optimization to avoid expensive re-reindexing type blockPlan struct { - tsdb tsdb.Identifier + tsdb tsdb.SingleTenantTSDBIdentifier gaps []gapWithBlocks } @@ -353,7 +650,7 @@ func blockPlansForGaps(tsdbs []tsdbGaps, metas []bloomshipper.Meta) ([]blockPlan // Used to signal the gaps that need to be populated for a tsdb type tsdbGaps struct { - tsdb tsdb.Identifier + tsdb tsdb.SingleTenantTSDBIdentifier gaps []v1.FingerprintBounds } @@ -361,7 +658,7 @@ type tsdbGaps struct { // that for each TSDB, there are metas covering the entire ownership range which were generated from that specific TSDB. func gapsBetweenTSDBsAndMetas( ownershipRange v1.FingerprintBounds, - tsdbs []tsdb.Identifier, + tsdbs []tsdb.SingleTenantTSDBIdentifier, metas []bloomshipper.Meta, ) (res []tsdbGaps, err error) { for _, db := range tsdbs { diff --git a/pkg/bloomcompactor/controller_test.go b/pkg/bloomcompactor/controller_test.go index 0660a5b601eea..72653c292b18b 100644 --- a/pkg/bloomcompactor/controller_test.go +++ b/pkg/bloomcompactor/controller_test.go @@ -142,14 +142,14 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { err bool exp []tsdbGaps ownershipRange v1.FingerprintBounds - tsdbs []tsdb.Identifier + tsdbs []tsdb.SingleTenantTSDBIdentifier metas []bloomshipper.Meta }{ { desc: "non-overlapping tsdbs and metas", err: true, ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(11, 20, []int{0}, nil), }, @@ -157,7 +157,7 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { { desc: "single tsdb", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(4, 8, []int{0}, nil), }, @@ -174,7 +174,7 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { { desc: "multiple tsdbs with separate blocks", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0), tsdbID(1)}, metas: []bloomshipper.Meta{ genMeta(0, 5, []int{0}, nil), genMeta(6, 10, []int{1}, nil), @@ -197,7 +197,7 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { { desc: "multiple tsdbs with the same blocks", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0), tsdbID(1)}, metas: []bloomshipper.Meta{ genMeta(0, 5, []int{0, 1}, nil), genMeta(6, 8, []int{1}, nil), @@ -242,7 +242,7 @@ func Test_blockPlansForGaps(t *testing.T) { for _, tc := range []struct { desc string ownershipRange v1.FingerprintBounds - tsdbs []tsdb.Identifier + tsdbs []tsdb.SingleTenantTSDBIdentifier metas []bloomshipper.Meta err bool exp []blockPlan @@ -250,7 +250,7 @@ func Test_blockPlansForGaps(t *testing.T) { { desc: "single overlapping meta+no overlapping block", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(11, 20)}), }, @@ -268,7 +268,7 @@ func Test_blockPlansForGaps(t *testing.T) { { desc: "single overlapping meta+one overlapping block", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), }, @@ -290,7 +290,7 @@ func Test_blockPlansForGaps(t *testing.T) { // but we can trim the range needing generation desc: "trims up to date area", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(9, 20, []int{0}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), // block for same tsdb genMeta(9, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), // block for different tsdb @@ -309,7 +309,7 @@ func Test_blockPlansForGaps(t *testing.T) { { desc: "uses old block for overlapping range", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(9, 20, []int{0}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), // block for same tsdb genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(5, 20)}), // block for different tsdb @@ -329,7 +329,7 @@ func Test_blockPlansForGaps(t *testing.T) { { desc: "multi case", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, // generate for both tsdbs + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0), tsdbID(1)}, // generate for both tsdbs metas: []bloomshipper.Meta{ genMeta(0, 2, []int{0}, []bloomshipper.BlockRef{ genBlockRef(0, 1), @@ -377,7 +377,7 @@ func Test_blockPlansForGaps(t *testing.T) { { desc: "dedupes block refs", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(9, 20, []int{1}, []bloomshipper.BlockRef{ genBlockRef(1, 4), @@ -421,3 +421,144 @@ func Test_blockPlansForGaps(t *testing.T) { }) } } + +func Test_coversFullRange(t *testing.T) { + for _, tc := range []struct { + desc string + src v1.FingerprintBounds + overlaps []v1.FingerprintBounds + exp bool + }{ + { + desc: "empty", + src: v1.NewBounds(0, 10), + overlaps: []v1.FingerprintBounds{}, + exp: false, + }, + { + desc: "single_full_range", + src: v1.NewBounds(0, 10), + overlaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 10), + }, + exp: true, + }, + { + desc: "single_partial_range", + src: v1.NewBounds(0, 10), + overlaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 5), + }, + exp: false, + }, + { + desc: "multiple_full_ranges", + src: v1.NewBounds(0, 10), + overlaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 5), + v1.NewBounds(6, 10), + }, + exp: true, + }, + { + desc: "multiple_partial_ranges", + src: v1.NewBounds(0, 10), + overlaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 5), + v1.NewBounds(7, 8), + }, + exp: false, + }, + { + desc: "wraps_partial_range", + src: v1.NewBounds(10, 20), + overlaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 12), + v1.NewBounds(13, 15), + v1.NewBounds(19, 21), + }, + exp: false, + }, + { + desc: "wraps_full_range", + src: v1.NewBounds(10, 20), + overlaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 12), + v1.NewBounds(13, 15), + v1.NewBounds(16, 25), + }, + exp: true, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + require.Equal(t, tc.exp, coversFullRange(tc.src, tc.overlaps)) + }) + } +} + +func Test_OutdatedMetas(t *testing.T) { + gen := func(bounds v1.FingerprintBounds, tsdbTimes ...model.Time) (meta bloomshipper.Meta) { + for _, tsdbTime := range tsdbTimes { + meta.Sources = append(meta.Sources, tsdb.SingleTenantTSDBIdentifier{TS: tsdbTime.Time()}) + } + meta.Bounds = bounds + return meta + } + + for _, tc := range []struct { + desc string + metas []bloomshipper.Meta + exp []bloomshipper.Meta + }{ + { + desc: "no metas", + metas: nil, + exp: nil, + }, + { + desc: "single meta", + metas: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 10), 0), + }, + exp: nil, + }, + { + desc: "single outdated meta", + metas: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 10), 0), + gen(v1.NewBounds(0, 10), 1), + }, + exp: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 10), 0), + }, + }, + { + desc: "single outdated via partitions", + metas: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 5), 0), + gen(v1.NewBounds(6, 10), 0), + gen(v1.NewBounds(0, 10), 1), + }, + exp: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 5), 0), + gen(v1.NewBounds(6, 10), 0), + }, + }, + { + desc: "multi tsdbs", + metas: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 5), 0, 1), + gen(v1.NewBounds(6, 10), 0, 1), + gen(v1.NewBounds(0, 10), 2, 3), + }, + exp: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 5), 0, 1), + gen(v1.NewBounds(6, 10), 0, 1), + }, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + require.Equal(t, tc.exp, outdatedMetas(tc.metas)) + }) + } +} diff --git a/pkg/bloomcompactor/metrics.go b/pkg/bloomcompactor/metrics.go index b02ac32aca727..74378cb786429 100644 --- a/pkg/bloomcompactor/metrics.go +++ b/pkg/bloomcompactor/metrics.go @@ -16,105 +16,133 @@ const ( ) type Metrics struct { - bloomMetrics *v1.Metrics - chunkSize prometheus.Histogram // uncompressed size of all chunks summed per series + bloomMetrics *v1.Metrics + compactorRunning prometheus.Gauge + chunkSize prometheus.Histogram // uncompressed size of all chunks summed per series - compactionRunsStarted prometheus.Counter - compactionRunsCompleted *prometheus.CounterVec - compactionRunTime *prometheus.HistogramVec - compactionRunDiscoveredTenants prometheus.Counter - compactionRunSkippedTenants prometheus.Counter - compactionRunTenantsCompleted *prometheus.CounterVec - compactionRunTenantsTime *prometheus.HistogramVec - compactionRunJobStarted prometheus.Counter - compactionRunJobCompleted *prometheus.CounterVec - compactionRunJobTime *prometheus.HistogramVec - compactionRunInterval prometheus.Gauge - compactorRunning prometheus.Gauge + compactionsStarted prometheus.Counter + compactionCompleted *prometheus.CounterVec + compactionTime *prometheus.HistogramVec + + tenantsDiscovered prometheus.Counter + tenantsOwned prometheus.Counter + tenantsSkipped prometheus.Counter + tenantsStarted prometheus.Counter + tenantsCompleted *prometheus.CounterVec + tenantsCompletedTime *prometheus.HistogramVec + tenantsSeries prometheus.Histogram + + blocksCreated prometheus.Counter + blocksDeleted prometheus.Counter + metasCreated prometheus.Counter + metasDeleted prometheus.Counter } func NewMetrics(r prometheus.Registerer, bloomMetrics *v1.Metrics) *Metrics { m := Metrics{ bloomMetrics: bloomMetrics, + compactorRunning: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "running", + Help: "Value will be 1 if compactor is currently running on this instance", + }), 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), + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "chunk_series_size", + Help: "Uncompressed size of chunks in a series", + Buckets: prometheus.ExponentialBucketsRange(1024, 1073741824, 10), }), - compactionRunsStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{ + + compactionsStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "runs_started_total", + Name: "compactions_started_total", Help: "Total number of compactions started", }), - compactionRunsCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + compactionCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "runs_completed_total", - Help: "Total number of compactions completed successfully", + Name: "compactions_completed_total", + Help: "Total number of compactions completed", }, []string{"status"}), - compactionRunTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + compactionTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "runs_time_seconds", + Name: "compactions_time_seconds", Help: "Time spent during a compaction cycle.", Buckets: prometheus.DefBuckets, }, []string{"status"}), - compactionRunDiscoveredTenants: promauto.With(r).NewCounter(prometheus.CounterOpts{ + + tenantsDiscovered: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "tenants_discovered", + Name: "tenants_discovered_total", Help: "Number of tenants discovered during the current compaction run", }), - compactionRunSkippedTenants: promauto.With(r).NewCounter(prometheus.CounterOpts{ + tenantsOwned: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "tenants_owned", + Help: "Number of tenants owned by this instance", + }), + tenantsSkipped: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "tenants_skipped", - Help: "Number of tenants skipped during the current compaction run", + Name: "tenants_skipped_total", + Help: "Number of tenants skipped since they are not owned by this instance", }), - compactionRunTenantsCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + tenantsStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "tenants_completed", + Name: "tenants_started_total", + Help: "Number of tenants started to process during the current compaction run", + }), + tenantsCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "tenants_completed_total", Help: "Number of tenants successfully processed during the current compaction run", }, []string{"status"}), - compactionRunTenantsTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + tenantsCompletedTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, Name: "tenants_time_seconds", Help: "Time spent processing tenants.", Buckets: prometheus.DefBuckets, }, []string{"status"}), - compactionRunJobStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{ + tenantsSeries: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "job_started", - Help: "Number of jobs started processing during the current compaction run", + Name: "tenants_series", + Help: "Number of series processed per tenant in the owned fingerprint-range.", + // Up to 10M series per tenant, way more than what we expect given our max_global_streams_per_user limits + Buckets: prometheus.ExponentialBucketsRange(1, 10000000, 10), }), - compactionRunJobCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + blocksCreated: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "job_completed", - Help: "Number of jobs successfully processed during the current compaction run", - }, []string{"status"}), - compactionRunJobTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + Name: "blocks_created_total", + Help: "Number of blocks created", + }), + blocksDeleted: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "job_time_seconds", - Help: "Time spent processing jobs.", - Buckets: prometheus.DefBuckets, - }, []string{"status"}), - compactionRunInterval: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Name: "blocks_deleted_total", + Help: "Number of blocks deleted", + }), + metasCreated: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "compaction_interval_seconds", - Help: "The configured interval on which compaction is run in seconds", + Name: "metas_created_total", + Help: "Number of metas created", }), - compactorRunning: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + metasDeleted: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "running", - Help: "Value will be 1 if compactor is currently running on this instance", + Name: "metas_deleted_total", + Help: "Number of metas deleted", }), } diff --git a/pkg/bloomcompactor/sharding.go b/pkg/bloomcompactor/sharding.go deleted file mode 100644 index 9b3009bd50652..0000000000000 --- a/pkg/bloomcompactor/sharding.go +++ /dev/null @@ -1,58 +0,0 @@ -package bloomcompactor - -import ( - "github.com/grafana/dskit/ring" - - util_ring "github.com/grafana/loki/pkg/util/ring" -) - -var ( - // TODO: Should we include LEAVING instances in the replication set? - RingOp = ring.NewOp([]ring.InstanceState{ring.JOINING, ring.ACTIVE}, nil) -) - -// ShardingStrategy describes whether compactor "owns" given user or job. -type ShardingStrategy interface { - util_ring.TenantSharding - OwnsFingerprint(tenantID string, fp uint64) (bool, error) -} - -type ShuffleShardingStrategy struct { - util_ring.TenantSharding - ringLifeCycler *ring.BasicLifecycler -} - -func NewShuffleShardingStrategy(r *ring.Ring, ringLifecycler *ring.BasicLifecycler, limits Limits) *ShuffleShardingStrategy { - s := ShuffleShardingStrategy{ - TenantSharding: util_ring.NewTenantShuffleSharding(r, ringLifecycler, limits.BloomCompactorShardSize), - ringLifeCycler: ringLifecycler, - } - - return &s -} - -// OwnsFingerprint makes sure only a single compactor processes the fingerprint. -func (s *ShuffleShardingStrategy) OwnsFingerprint(tenantID string, fp uint64) (bool, error) { - if !s.OwnsTenant(tenantID) { - return false, nil - } - - tenantRing := s.GetTenantSubRing(tenantID) - fpSharding := util_ring.NewFingerprintShuffleSharding(tenantRing, s.ringLifeCycler, RingOp) - return fpSharding.OwnsFingerprint(fp) -} - -// NoopStrategy is an implementation of the ShardingStrategy that does not -// filter anything. -type NoopStrategy struct { - util_ring.NoopStrategy -} - -// OwnsFingerprint implements TenantShuffleSharding. -func (s *NoopStrategy) OwnsFingerprint(_ string, _ uint64) (bool, error) { - return true, nil -} - -func NewNoopStrategy() *NoopStrategy { - return &NoopStrategy{NoopStrategy: util_ring.NoopStrategy{}} -} diff --git a/pkg/bloomcompactor/sharding_test.go b/pkg/bloomcompactor/sharding_test.go deleted file mode 100644 index 4e79752279fb9..0000000000000 --- a/pkg/bloomcompactor/sharding_test.go +++ /dev/null @@ -1,149 +0,0 @@ -package bloomcompactor - -import ( - "context" - "flag" - "fmt" - "testing" - "time" - - "github.com/grafana/dskit/services" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" - "github.com/stretchr/testify/require" - - util_log "github.com/grafana/loki/pkg/util/log" - lokiring "github.com/grafana/loki/pkg/util/ring" - "github.com/grafana/loki/pkg/validation" -) - -func TestShuffleSharding(t *testing.T) { - const shardSize = 2 - const rings = 4 - const tenants = 2000 - const jobsPerTenant = 200 - - var limits validation.Limits - limits.RegisterFlags(flag.NewFlagSet("limits", flag.PanicOnError)) - overrides, err := validation.NewOverrides(limits, nil) - require.NoError(t, err) - - var ringManagers []*lokiring.RingManager - var shards []*ShuffleShardingStrategy - for i := 0; i < rings; i++ { - var ringCfg lokiring.RingConfig - ringCfg.RegisterFlagsWithPrefix("", "", flag.NewFlagSet("ring", flag.PanicOnError)) - ringCfg.KVStore.Store = "inmemory" - ringCfg.InstanceID = fmt.Sprintf("bloom-compactor-%d", i) - ringCfg.InstanceAddr = fmt.Sprintf("localhost-%d", i) - - ringManager, err := lokiring.NewRingManager("bloom-compactor", lokiring.ServerMode, ringCfg, 1, 1, util_log.Logger, prometheus.NewRegistry()) - require.NoError(t, err) - require.NoError(t, ringManager.StartAsync(context.Background())) - - sharding := NewShuffleShardingStrategy(ringManager.Ring, ringManager.RingLifecycler, mockLimits{ - Overrides: overrides, - bloomCompactorShardSize: shardSize, - }) - - ringManagers = append(ringManagers, ringManager) - shards = append(shards, sharding) - } - - // Wait for all rings to see each other. - for i := 0; i < rings; i++ { - require.Eventually(t, func() bool { - running := ringManagers[i].State() == services.Running - discovered := ringManagers[i].Ring.InstancesCount() == rings - return running && discovered - }, 1*time.Minute, 100*time.Millisecond) - } - - // This is kind of an un-deterministic test, because sharding is random - // and the seed is initialized by the ring lib. - // Here we'll generate a bunch of tenants and test that if the sharding doesn't own the tenant, - // that's because the tenant is owned by other ring instances. - shard := shards[0] - otherShards := shards[1:] - var ownedTenants, ownedJobs int - for i := 0; i < tenants; i++ { - tenant := fmt.Sprintf("tenant-%d", i) - ownsTenant := shard.OwnsTenant(tenant) - - var tenantOwnedByOther int - for _, other := range otherShards { - otherOwns := other.OwnsTenant(tenant) - if otherOwns { - tenantOwnedByOther++ - } - } - - // If this shard owns the tenant, shardSize-1 other members should also own the tenant. - // Otherwise, shardSize other members should own the tenant. - if ownsTenant { - require.Equal(t, shardSize-1, tenantOwnedByOther) - ownedTenants++ - } else { - require.Equal(t, shardSize, tenantOwnedByOther) - } - - for j := 0; j < jobsPerTenant; j++ { - lbls := labels.FromStrings("namespace", fmt.Sprintf("namespace-%d", j)) - fp := model.Fingerprint(lbls.Hash()) - ownsFingerprint, err := shard.OwnsFingerprint(tenant, uint64(fp)) - require.NoError(t, err) - - var jobOwnedByOther int - for _, other := range otherShards { - otherOwns, err := other.OwnsFingerprint(tenant, uint64(fp)) - require.NoError(t, err) - if otherOwns { - jobOwnedByOther++ - } - } - - // If this shard owns the job, no one else should own the job. - // And if this shard doesn't own the job, only one of the other shards should own the job. - if ownsFingerprint { - require.Equal(t, 0, jobOwnedByOther) - ownedJobs++ - } else { - require.Equal(t, 1, jobOwnedByOther) - } - } - } - - t.Logf("owned tenants: %d (out of %d)", ownedTenants, tenants) - t.Logf("owned jobs: %d (out of %d)", ownedJobs, tenants*jobsPerTenant) - - // Stop all rings and wait for them to stop. - for i := 0; i < rings; i++ { - ringManagers[i].StopAsync() - require.Eventually(t, func() bool { - return ringManagers[i].State() == services.Terminated - }, 1*time.Minute, 100*time.Millisecond) - } -} - -type mockLimits struct { - *validation.Overrides - bloomCompactorShardSize int - chunksDownloadingBatchSize int - fpRate float64 -} - -func (m mockLimits) BloomFalsePositiveRate(_ string) float64 { - return m.fpRate -} - -func (m mockLimits) BloomCompactorShardSize(_ string) int { - return m.bloomCompactorShardSize -} - -func (m mockLimits) BloomCompactorChunksBatchSize(_ string) int { - if m.chunksDownloadingBatchSize != 0 { - return m.chunksDownloadingBatchSize - } - return 1 -} diff --git a/pkg/bloomcompactor/spec.go b/pkg/bloomcompactor/spec.go index 4a1125082ca54..cb030dfb59131 100644 --- a/pkg/bloomcompactor/spec.go +++ b/pkg/bloomcompactor/spec.go @@ -4,17 +4,13 @@ import ( "context" "fmt" "io" - "math" - "time" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/pkg/errors" "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/chunk/fetcher" @@ -48,7 +44,8 @@ type SimpleBloomGenerator struct { userID string store v1.Iterator[*v1.Series] chunkLoader ChunkLoader - blocksIter v1.CloseableIterator[*bloomshipper.CloseableBlockQuerier] + blocksIter v1.ResettableIterator[*v1.SeriesWithBloom] + skipped []v1.BlockMetadata // options to build blocks with opts v1.BlockOptions @@ -70,7 +67,7 @@ func NewSimpleBloomGenerator( opts v1.BlockOptions, store v1.Iterator[*v1.Series], chunkLoader ChunkLoader, - blocksIter v1.CloseableIterator[*bloomshipper.CloseableBlockQuerier], + blocksIter v1.ResettableIterator[*v1.SeriesWithBloom], readWriterFn func() (v1.BlockWriter, v1.BlockReader), metrics *Metrics, logger log.Logger, @@ -107,44 +104,41 @@ func (s *SimpleBloomGenerator) populator(ctx context.Context) func(series *v1.Se } -func (s *SimpleBloomGenerator) Generate(ctx context.Context) ([]v1.BlockMetadata, []io.Closer, v1.Iterator[*v1.Block], error) { - skippedBlocks := make([]v1.BlockMetadata, 0) - toClose := make([]io.Closer, 0) - blocksMatchingSchema := make([]*bloomshipper.CloseableBlockQuerier, 0) +func (s *SimpleBloomGenerator) Generate(ctx context.Context) v1.Iterator[*v1.Block] { + level.Debug(s.logger).Log("msg", "generating bloom filters for blocks", "schema", fmt.Sprintf("%+v", s.opts.Schema)) - for s.blocksIter.Next() && s.blocksIter.Err() == nil { - block := s.blocksIter.At() - toClose = append(toClose, block) - - logger := log.With(s.logger, "block", block.BlockRef) - md, err := block.Metadata() - schema := md.Options.Schema - if err != nil { - level.Warn(logger).Log("msg", "failed to get schema for block", "err", err) - skippedBlocks = append(skippedBlocks, md) - continue - } - - 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, md) - continue - } - - level.Debug(logger).Log("msg", "adding compatible block to bloom generation inputs") - blocksMatchingSchema = append(blocksMatchingSchema, block) - } + series := v1.NewPeekingIter(s.store) - if s.blocksIter.Err() != nil { - // should we ignore the error and continue with the blocks we got? - return skippedBlocks, toClose, v1.NewSliceIter([]*v1.Block{}), s.blocksIter.Err() + // TODO: Use interface + impl, ok := s.blocksIter.(*blockLoadingIter) + if ok { + impl.Filter( + func(bq *bloomshipper.CloseableBlockQuerier) bool { + + logger := log.With(s.logger, "block", bq.BlockRef) + md, err := bq.Metadata() + schema := md.Options.Schema + if err != nil { + level.Warn(logger).Log("msg", "failed to get schema for block", "err", err) + s.skipped = append(s.skipped, md) + bq.Close() // close unused querier + return false + } + + 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)) + s.skipped = append(s.skipped, md) + bq.Close() // close unused querier + return false + } + + level.Debug(logger).Log("msg", "adding compatible block to bloom generation inputs") + return true + }, + ) } - 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)) - - series := v1.NewPeekingIter(s.store) - blockIter := NewLazyBlockBuilderIterator(ctx, s.opts, s.populator(ctx), s.readWriterFn, series, blocksMatchingSchema) - return skippedBlocks, toClose, blockIter, nil + return NewLazyBlockBuilderIterator(ctx, s.opts, s.metrics, s.populator(ctx), s.readWriterFn, series, s.blocksIter) } // LazyBlockBuilderIterator is a lazy iterator over blocks that builds @@ -152,36 +146,34 @@ func (s *SimpleBloomGenerator) Generate(ctx context.Context) ([]v1.BlockMetadata type LazyBlockBuilderIterator struct { ctx context.Context opts v1.BlockOptions + metrics *Metrics populate func(*v1.Series, *v1.Bloom) error readWriterFn func() (v1.BlockWriter, v1.BlockReader) series v1.PeekingIterator[*v1.Series] - blocks []*bloomshipper.CloseableBlockQuerier + blocks v1.ResettableIterator[*v1.SeriesWithBloom] - blocksAsPeekingIter []v1.PeekingIterator[*v1.SeriesWithBloom] - curr *v1.Block - err error + curr *v1.Block + err error } func NewLazyBlockBuilderIterator( ctx context.Context, opts v1.BlockOptions, + metrics *Metrics, populate func(*v1.Series, *v1.Bloom) error, readWriterFn func() (v1.BlockWriter, v1.BlockReader), series v1.PeekingIterator[*v1.Series], - blocks []*bloomshipper.CloseableBlockQuerier, + blocks v1.ResettableIterator[*v1.SeriesWithBloom], ) *LazyBlockBuilderIterator { - it := &LazyBlockBuilderIterator{ + return &LazyBlockBuilderIterator{ ctx: ctx, opts: opts, + metrics: metrics, populate: populate, readWriterFn: readWriterFn, series: series, blocks: blocks, - - blocksAsPeekingIter: make([]v1.PeekingIterator[*v1.SeriesWithBloom], len(blocks)), } - - return it } func (b *LazyBlockBuilderIterator) Next() bool { @@ -190,21 +182,17 @@ func (b *LazyBlockBuilderIterator) Next() bool { return false } - // reset all the blocks to the start - for i, block := range b.blocks { - if err := block.Reset(); err != nil { - b.err = errors.Wrapf(err, "failed to reset block iterator %d", i) - return false - } - b.blocksAsPeekingIter[i] = v1.NewPeekingIter[*v1.SeriesWithBloom](block) - } - if err := b.ctx.Err(); err != nil { b.err = errors.Wrap(err, "context canceled") return false } - mergeBuilder := v1.NewMergeBuilder(b.blocksAsPeekingIter, b.series, b.populate) + if err := b.blocks.Reset(); err != nil { + b.err = errors.Wrap(err, "reset blocks iterator") + return false + } + + mergeBuilder := v1.NewMergeBuilder(b.blocks, b.series, b.populate, b.metrics.bloomMetrics) writer, reader := b.readWriterFn() blockBuilder, err := v1.NewBlockBuilder(b.opts, writer) if err != nil { @@ -292,137 +280,3 @@ func (s *StoreChunkLoader) Load(ctx context.Context, userID string, series *v1.S itr: newBatchedChunkLoader(ctx, fetchers, inputs, s.metrics, batchedLoaderDefaultBatchSize), }, nil } - -type Fetcher[A, B any] interface { - Fetch(ctx context.Context, inputs []A) ([]B, error) -} - -type FetchFunc[A, B any] func(ctx context.Context, inputs []A) ([]B, error) - -func (f FetchFunc[A, B]) Fetch(ctx context.Context, inputs []A) ([]B, error) { - return f(ctx, inputs) -} - -// batchedLoader implements `v1.Iterator[v1.ChunkRefWithIter]` in batches -// to ensure memory is bounded while loading chunks -// TODO(owen-d): testware -type batchedLoader[A, B, C any] struct { - metrics *Metrics - batchSize int - ctx context.Context - fetchers []Fetcher[A, B] - work [][]A - - mapper func(B) (C, error) - cur C - batch []B - err error -} - -const batchedLoaderDefaultBatchSize = 50 - -func newBatchedLoader[A, B, C any]( - ctx context.Context, - fetchers []Fetcher[A, B], - inputs [][]A, - mapper func(B) (C, error), - batchSize int, -) *batchedLoader[A, B, C] { - return &batchedLoader[A, B, C]{ - batchSize: max(batchSize, 1), - ctx: ctx, - fetchers: fetchers, - work: inputs, - mapper: mapper, - } -} - -func (b *batchedLoader[A, B, C]) Next() bool { - - // iterate work until we have non-zero length batch - for len(b.batch) == 0 { - - // empty batch + no work remaining = we're done - if len(b.work) == 0 { - return false - } - - // setup next batch - next := b.work[0] - batchSize := min(b.batchSize, len(next)) - toFetch := next[:batchSize] - fetcher := b.fetchers[0] - - // update work - b.work[0] = b.work[0][batchSize:] - if len(b.work[0]) == 0 { - // if we've exhausted work from this set of inputs, - // set pointer to next set of inputs - // and their respective fetcher - b.work = b.work[1:] - b.fetchers = b.fetchers[1:] - } - - // there was no work in this batch; continue (should not happen) - if len(toFetch) == 0 { - continue - } - - b.batch, b.err = fetcher.Fetch(b.ctx, toFetch) - // error fetching, short-circuit iteration - if b.err != nil { - return false - } - } - - return b.prepNext() -} - -func (b *batchedLoader[_, B, C]) prepNext() bool { - b.cur, b.err = b.mapper(b.batch[0]) - b.batch = b.batch[1:] - return b.err == nil -} - -func newBatchedChunkLoader( - ctx context.Context, - fetchers []Fetcher[chunk.Chunk, chunk.Chunk], - inputs [][]chunk.Chunk, - metrics *Metrics, - batchSize int, -) *batchedLoader[chunk.Chunk, chunk.Chunk, v1.ChunkRefWithIter] { - - mapper := func(c chunk.Chunk) (v1.ChunkRefWithIter, error) { - chk := c.Data.(*chunkenc.Facade).LokiChunk() - metrics.chunkSize.Observe(float64(chk.UncompressedSize())) - itr, err := chk.Iterator( - ctx, - time.Unix(0, 0), - 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 - } - return newBatchedLoader(ctx, fetchers, inputs, mapper, batchSize) -} - -func (b *batchedLoader[_, _, C]) At() C { - return b.cur -} - -func (b *batchedLoader[_, _, _]) Err() error { - return b.err -} diff --git a/pkg/bloomcompactor/spec_test.go b/pkg/bloomcompactor/spec_test.go index bb4fde6cc2359..f278948fed7a6 100644 --- a/pkg/bloomcompactor/spec_test.go +++ b/pkg/bloomcompactor/spec_test.go @@ -3,7 +3,6 @@ package bloomcompactor import ( "bytes" "context" - "errors" "testing" "github.com/go-kit/log" @@ -14,20 +13,19 @@ import ( "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" ) -func blocksFromSchema(t *testing.T, n int, options v1.BlockOptions) (res []*v1.Block, data []v1.SeriesWithBloom) { +func blocksFromSchema(t *testing.T, n int, options v1.BlockOptions) (res []*v1.Block, data []v1.SeriesWithBloom, refs []bloomshipper.BlockRef) { 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) { +func blocksFromSchemaWithRange(t *testing.T, n int, options v1.BlockOptions, fromFP, throughFp model.Fingerprint) (res []*v1.Block, data []v1.SeriesWithBloom, refs []bloomshipper.BlockRef) { 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) + data, _ = v1.MkBasicSeriesWithBlooms(numSeries, 0, fromFP, throughFp, 0, 10000) seriesPerBlock := numSeries / n @@ -44,14 +42,19 @@ func blocksFromSchemaWithRange(t *testing.T, n int, options v1.BlockOptions, fro ) require.Nil(t, err) - itr := v1.NewSliceIter[v1.SeriesWithBloom](data[i*seriesPerBlock : (i+1)*seriesPerBlock]) + minIdx, maxIdx := i*seriesPerBlock, (i+1)*seriesPerBlock + + itr := v1.NewSliceIter[v1.SeriesWithBloom](data[minIdx:maxIdx]) _, err = builder.BuildFrom(itr) require.Nil(t, err) res = append(res, v1.NewBlock(reader)) + ref := genBlockRef(data[minIdx].Series.Fingerprint, data[maxIdx-1].Series.Fingerprint) + t.Log("create block", ref) + refs = append(refs, ref) } - return res, data + return res, data, refs } // doesn't actually load any chunks @@ -64,14 +67,30 @@ func (dummyChunkLoader) Load(_ context.Context, _ string, series *v1.Series) (*C }, nil } -func dummyBloomGen(opts v1.BlockOptions, store v1.Iterator[*v1.Series], blocks []*v1.Block) *SimpleBloomGenerator { +func dummyBloomGen(t *testing.T, opts v1.BlockOptions, store v1.Iterator[*v1.Series], blocks []*v1.Block, refs []bloomshipper.BlockRef) *SimpleBloomGenerator { bqs := make([]*bloomshipper.CloseableBlockQuerier, 0, len(blocks)) - for _, b := range blocks { + for i, b := range blocks { bqs = append(bqs, &bloomshipper.CloseableBlockQuerier{ + BlockRef: refs[i], BlockQuerier: v1.NewBlockQuerier(b), }) } - blocksIter := v1.NewCloseableIterator(v1.NewSliceIter(bqs)) + + fetcher := func(_ context.Context, refs []bloomshipper.BlockRef) ([]*bloomshipper.CloseableBlockQuerier, error) { + res := make([]*bloomshipper.CloseableBlockQuerier, 0, len(refs)) + for _, ref := range refs { + for _, bq := range bqs { + if ref.Bounds.Equal(bq.Bounds) { + res = append(res, bq) + } + } + } + t.Log("req", refs) + t.Log("res", res) + return res, nil + } + + blocksIter := newBlockLoadingIter(context.Background(), refs, FetchFunc[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier](fetcher), 1) return NewSimpleBloomGenerator( "fake", @@ -95,6 +114,7 @@ func TestSimpleBloomGenerator(t *testing.T) { desc string fromSchema, toSchema v1.BlockOptions sourceBlocks, numSkipped, outputBlocks int + overlapping bool }{ { desc: "SkipsIncompatibleSchemas", @@ -118,11 +138,11 @@ func TestSimpleBloomGenerator(t *testing.T) { toSchema: v1.NewBlockOptions(4, 0, 1<<10), // 1KB sourceBlocks: 2, numSkipped: 0, - outputBlocks: 3, + outputBlocks: 6, }, } { t.Run(tc.desc, func(t *testing.T) { - sourceBlocks, data := blocksFromSchema(t, tc.sourceBlocks, tc.fromSchema) + sourceBlocks, data, refs := blocksFromSchemaWithRange(t, tc.sourceBlocks, tc.fromSchema, 0x00000, 0x6ffff) storeItr := v1.NewMapIter[v1.SeriesWithBloom, *v1.Series]( v1.NewSliceIter[v1.SeriesWithBloom](data), func(swb v1.SeriesWithBloom) *v1.Series { @@ -130,16 +150,15 @@ func TestSimpleBloomGenerator(t *testing.T) { }, ) - gen := dummyBloomGen(tc.toSchema, storeItr, sourceBlocks) - skipped, _, results, err := gen.Generate(context.Background()) - require.Nil(t, err) - require.Equal(t, tc.numSkipped, len(skipped)) + gen := dummyBloomGen(t, tc.toSchema, storeItr, sourceBlocks, refs) + results := gen.Generate(context.Background()) var outputBlocks []*v1.Block for results.Next() { outputBlocks = append(outputBlocks, results.At()) } require.Equal(t, tc.outputBlocks, len(outputBlocks)) + require.Equal(t, tc.numSkipped, len(gen.skipped)) // Check all the input series are present in the output blocks. expectedRefs := v1.PointerSlice(data) @@ -157,129 +176,3 @@ func TestSimpleBloomGenerator(t *testing.T) { }) } } - -func TestBatchedLoader(t *testing.T) { - errMapper := func(i int) (int, error) { - return 0, errors.New("bzzt") - } - successMapper := func(i int) (int, error) { - return i, nil - } - - expired, cancel := context.WithCancel(context.Background()) - cancel() - - for _, tc := range []struct { - desc string - ctx context.Context - batchSize int - mapper func(int) (int, error) - err bool - inputs [][]int - exp []int - }{ - { - desc: "OneBatch", - ctx: context.Background(), - batchSize: 2, - mapper: successMapper, - err: false, - inputs: [][]int{{0, 1}}, - exp: []int{0, 1}, - }, - { - desc: "ZeroBatchSizeStillWorks", - ctx: context.Background(), - batchSize: 0, - mapper: successMapper, - err: false, - inputs: [][]int{{0, 1}}, - exp: []int{0, 1}, - }, - { - desc: "OneBatchLessThanFull", - ctx: context.Background(), - batchSize: 2, - mapper: successMapper, - err: false, - inputs: [][]int{{0}}, - exp: []int{0}, - }, - { - desc: "TwoBatches", - ctx: context.Background(), - batchSize: 2, - mapper: successMapper, - err: false, - inputs: [][]int{{0, 1, 2, 3}}, - exp: []int{0, 1, 2, 3}, - }, - { - desc: "MultipleBatchesMultipleLoaders", - ctx: context.Background(), - batchSize: 2, - mapper: successMapper, - err: false, - inputs: [][]int{{0, 1}, {2}, {3, 4, 5}}, - exp: []int{0, 1, 2, 3, 4, 5}, - }, - { - desc: "HandlesEmptyInputs", - ctx: context.Background(), - batchSize: 2, - mapper: successMapper, - err: false, - inputs: [][]int{{0, 1, 2, 3}, nil, {4}}, - exp: []int{0, 1, 2, 3, 4}, - }, - { - desc: "Timeout", - ctx: expired, - batchSize: 2, - mapper: successMapper, - err: true, - inputs: [][]int{{0}}, - }, - { - desc: "MappingFailure", - ctx: context.Background(), - batchSize: 2, - mapper: errMapper, - err: true, - inputs: [][]int{{0}}, - }, - } { - t.Run(tc.desc, func(t *testing.T) { - fetchers := make([]Fetcher[int, int], 0, len(tc.inputs)) - for range tc.inputs { - fetchers = append( - fetchers, - FetchFunc[int, int](func(ctx context.Context, xs []int) ([]int, error) { - if ctx.Err() != nil { - return nil, ctx.Err() - } - return xs, nil - }), - ) - } - - loader := newBatchedLoader[int, int, int]( - tc.ctx, - fetchers, - tc.inputs, - tc.mapper, - tc.batchSize, - ) - - got, err := v1.Collect[int](loader) - if tc.err { - require.Error(t, err) - return - } - require.NoError(t, err) - require.Equal(t, tc.exp, got) - - }) - } - -} diff --git a/pkg/bloomcompactor/table_utils.go b/pkg/bloomcompactor/table_utils.go deleted file mode 100644 index 55bc2e9a328f1..0000000000000 --- a/pkg/bloomcompactor/table_utils.go +++ /dev/null @@ -1,16 +0,0 @@ -package bloomcompactor - -import ( - "github.com/prometheus/common/model" - - "github.com/grafana/loki/pkg/compactor/retention" -) - -func getIntervalsForTables(tables []string) map[string]model.Interval { - tablesIntervals := make(map[string]model.Interval, len(tables)) - for _, table := range tables { - tablesIntervals[table] = retention.ExtractIntervalFromTableName(table) - } - - return tablesIntervals -} diff --git a/pkg/bloomcompactor/tsdb.go b/pkg/bloomcompactor/tsdb.go index ad7b2eafac4cd..7f5ec5eab81a3 100644 --- a/pkg/bloomcompactor/tsdb.go +++ b/pkg/bloomcompactor/tsdb.go @@ -26,11 +26,11 @@ const ( ) type TSDBStore interface { - UsersForPeriod(ctx context.Context, table config.DayTime) ([]string, error) - ResolveTSDBs(ctx context.Context, table config.DayTime, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) + UsersForPeriod(ctx context.Context, table config.DayTable) ([]string, error) + ResolveTSDBs(ctx context.Context, table config.DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) LoadTSDB( ctx context.Context, - table config.DayTime, + table config.DayTable, tenant string, id tsdb.Identifier, bounds v1.FingerprintBounds, @@ -49,13 +49,13 @@ func NewBloomTSDBStore(storage storage.Client) *BloomTSDBStore { } } -func (b *BloomTSDBStore) UsersForPeriod(ctx context.Context, table config.DayTime) ([]string, error) { - _, users, err := b.storage.ListFiles(ctx, table.Table(), true) // bypass cache for ease of testing +func (b *BloomTSDBStore) UsersForPeriod(ctx context.Context, table config.DayTable) ([]string, error) { + _, users, err := b.storage.ListFiles(ctx, table.Addr(), true) // bypass cache for ease of testing return users, err } -func (b *BloomTSDBStore) ResolveTSDBs(ctx context.Context, table config.DayTime, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) { - indices, err := b.storage.ListUserFiles(ctx, table.Table(), tenant, true) // bypass cache for ease of testing +func (b *BloomTSDBStore) ResolveTSDBs(ctx context.Context, table config.DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) { + indices, err := b.storage.ListUserFiles(ctx, table.Addr(), tenant, true) // bypass cache for ease of testing if err != nil { return nil, errors.Wrap(err, "failed to list user files") } @@ -80,14 +80,14 @@ func (b *BloomTSDBStore) ResolveTSDBs(ctx context.Context, table config.DayTime, func (b *BloomTSDBStore) LoadTSDB( ctx context.Context, - table config.DayTime, + table config.DayTable, tenant string, id tsdb.Identifier, bounds v1.FingerprintBounds, ) (v1.CloseableIterator[*v1.Series], error) { withCompression := id.Name() + gzipExtension - data, err := b.storage.GetUserFile(ctx, table.Table(), tenant, withCompression) + data, err := b.storage.GetUserFile(ctx, table.Addr(), tenant, withCompression) if err != nil { return nil, errors.Wrap(err, "failed to get file") } @@ -236,8 +236,7 @@ func NewTSDBStores( if err != nil { return nil, errors.Wrap(err, "failed to create object client") } - prefix := path.Join(cfg.IndexTables.PathPrefix, cfg.IndexTables.Prefix) - res.stores[i] = NewBloomTSDBStore(storage.NewIndexStorageClient(c, prefix)) + res.stores[i] = NewBloomTSDBStore(storage.NewIndexStorageClient(c, cfg.IndexTables.PathPrefix)) } } @@ -272,8 +271,8 @@ func (s *TSDBStores) storeForPeriod(table config.DayTime) (TSDBStore, error) { ) } -func (s *TSDBStores) UsersForPeriod(ctx context.Context, table config.DayTime) ([]string, error) { - store, err := s.storeForPeriod(table) +func (s *TSDBStores) UsersForPeriod(ctx context.Context, table config.DayTable) ([]string, error) { + store, err := s.storeForPeriod(table.DayTime) if err != nil { return nil, err } @@ -281,8 +280,8 @@ func (s *TSDBStores) UsersForPeriod(ctx context.Context, table config.DayTime) ( return store.UsersForPeriod(ctx, table) } -func (s *TSDBStores) ResolveTSDBs(ctx context.Context, table config.DayTime, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) { - store, err := s.storeForPeriod(table) +func (s *TSDBStores) ResolveTSDBs(ctx context.Context, table config.DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) { + store, err := s.storeForPeriod(table.DayTime) if err != nil { return nil, err } @@ -292,12 +291,12 @@ func (s *TSDBStores) ResolveTSDBs(ctx context.Context, table config.DayTime, ten func (s *TSDBStores) LoadTSDB( ctx context.Context, - table config.DayTime, + table config.DayTable, tenant string, id tsdb.Identifier, bounds v1.FingerprintBounds, ) (v1.CloseableIterator[*v1.Series], error) { - store, err := s.storeForPeriod(table) + store, err := s.storeForPeriod(table.DayTime) if err != nil { return nil, err } diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go index 58f709f0be2f8..4e36e5ce3018e 100644 --- a/pkg/bloomgateway/bloomgateway.go +++ b/pkg/bloomgateway/bloomgateway.go @@ -82,10 +82,9 @@ var ( ) type metrics struct { - queueDuration prometheus.Histogram - inflightRequests prometheus.Summary - chunkRefsUnfiltered prometheus.Counter - chunkRefsFiltered prometheus.Counter + queueDuration prometheus.Histogram + inflightRequests prometheus.Summary + chunkRemovals *prometheus.CounterVec } func newMetrics(registerer prometheus.Registerer, namespace, subsystem string) *metrics { @@ -106,29 +105,15 @@ func newMetrics(registerer prometheus.Registerer, namespace, subsystem string) * MaxAge: time.Minute, AgeBuckets: 6, }), - chunkRefsUnfiltered: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + chunkRemovals: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{ Namespace: namespace, Subsystem: subsystem, - Name: "chunkrefs_pre_filtering", - Help: "Total amount of chunk refs pre filtering. Does not count chunk refs in failed requests.", - }), - chunkRefsFiltered: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "chunkrefs_post_filtering", - Help: "Total amount of chunk refs post filtering.", - }), + Name: "chunk_removals_total", + Help: "Total amount of removals received from the block querier partitioned by state. The state 'accepted' means that the removals are processed, the state 'dropped' means that the removals were received after the task context was done (e.g. client timeout, etc).", + }, []string{"state"}), } } -func (m *metrics) addUnfilteredCount(n int) { - m.chunkRefsUnfiltered.Add(float64(n)) -} - -func (m *metrics) addFilteredCount(n int) { - m.chunkRefsFiltered.Add(float64(n)) -} - // SyncMap is a map structure which can be synchronized using the RWMutex type SyncMap[k comparable, v any] struct { sync.RWMutex @@ -324,12 +309,8 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk return nil, errors.New("from time must not be after through time") } - numChunksUnfiltered := len(req.Refs) - // Shortcut if request does not contain filters if len(req.Filters) == 0 { - g.metrics.addUnfilteredCount(numChunksUnfiltered) - g.metrics.addFilteredCount(len(req.Refs)) return &logproto.FilterChunkRefResponse{ ChunkRefs: req.Refs, }, nil @@ -374,15 +355,14 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk // When enqueuing, we also add the task to the pending tasks g.pendingTasks.Add(task.ID, task) }) - go consumeTask(ctx, task, tasksCh, logger) + go g.consumeTask(ctx, task, tasksCh) } responses := responsesPool.Get(numSeries) defer responsesPool.Put(responses) remaining := len(tasks) -outer: - for { + for remaining > 0 { select { case <-ctx.Done(): return nil, errors.Wrap(ctx.Err(), "request failed") @@ -393,23 +373,17 @@ outer: } responses = append(responses, task.responses...) remaining-- - if remaining == 0 { - break outer - } } } - for _, o := range responses { - if o.Removals.Len() == 0 { - continue - } - removeNotMatchingChunks(req, o, g.logger) - } + preFilterSeries := len(req.Refs) - g.metrics.addUnfilteredCount(numChunksUnfiltered) - g.metrics.addFilteredCount(len(req.Refs)) + // TODO(chaudum): Don't wait for all responses before starting to filter chunks. + filtered := g.processResponses(req, responses) - level.Info(logger).Log("msg", "return filtered chunk refs", "unfiltered", numChunksUnfiltered, "filtered", len(req.Refs)) + postFilterSeries := len(req.Refs) + + level.Info(logger).Log("msg", "return filtered chunk refs", "pre_filter_series", preFilterSeries, "post_filter_series", postFilterSeries, "filtered_chunks", filtered) return &logproto.FilterChunkRefResponse{ChunkRefs: req.Refs}, nil } @@ -419,16 +393,18 @@ outer: // task is closed by the worker. // Once the tasks is closed, it will send the task with the results from the // block querier to the supplied task channel. -func consumeTask(ctx context.Context, task Task, tasksCh chan<- Task, logger log.Logger) { - logger = log.With(logger, "task", task.ID) +func (g *Gateway) consumeTask(ctx context.Context, task Task, tasksCh chan<- Task) { + logger := log.With(g.logger, "task", task.ID) for res := range task.resCh { select { case <-ctx.Done(): level.Debug(logger).Log("msg", "drop partial result", "fp_int", uint64(res.Fp), "fp_hex", res.Fp, "chunks_to_remove", res.Removals.Len()) + g.metrics.chunkRemovals.WithLabelValues("dropped").Add(float64(res.Removals.Len())) default: level.Debug(logger).Log("msg", "accept partial result", "fp_int", uint64(res.Fp), "fp_hex", res.Fp, "chunks_to_remove", res.Removals.Len()) task.responses = append(task.responses, res) + g.metrics.chunkRemovals.WithLabelValues("accepted").Add(float64(res.Removals.Len())) } } @@ -441,7 +417,18 @@ func consumeTask(ctx context.Context, task Task, tasksCh chan<- Task, logger log } } -func removeNotMatchingChunks(req *logproto.FilterChunkRefRequest, res v1.Output, logger log.Logger) { +func (g *Gateway) processResponses(req *logproto.FilterChunkRefRequest, responses []v1.Output) (filtered int) { + for _, o := range responses { + if o.Removals.Len() == 0 { + continue + } + filtered += g.removeNotMatchingChunks(req, o) + } + return +} + +func (g *Gateway) removeNotMatchingChunks(req *logproto.FilterChunkRefRequest, res v1.Output) (filtered int) { + // binary search index of fingerprint idx := sort.Search(len(req.Refs), func(i int) bool { return req.Refs[i].Fingerprint >= uint64(res.Fp) @@ -449,13 +436,15 @@ func removeNotMatchingChunks(req *logproto.FilterChunkRefRequest, res v1.Output, // fingerprint not found if idx >= len(req.Refs) { - level.Error(logger).Log("msg", "index out of range", "idx", idx, "len", len(req.Refs), "fp", uint64(res.Fp)) + level.Error(g.logger).Log("msg", "index out of range", "idx", idx, "len", len(req.Refs), "fp", uint64(res.Fp)) return } // if all chunks of a fingerprint are are removed // then remove the whole group from the response if len(req.Refs[idx].Refs) == res.Removals.Len() { + filtered += len(req.Refs[idx].Refs) + req.Refs[idx] = nil // avoid leaking pointer req.Refs = append(req.Refs[:idx], req.Refs[idx+1:]...) return @@ -465,10 +454,13 @@ func removeNotMatchingChunks(req *logproto.FilterChunkRefRequest, res v1.Output, toRemove := res.Removals[i] for j := 0; j < len(req.Refs[idx].Refs); j++ { if toRemove.Checksum == req.Refs[idx].Refs[j].Checksum { + filtered += 1 + 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 } } } + return } diff --git a/pkg/bloomgateway/bloomgateway_test.go b/pkg/bloomgateway/bloomgateway_test.go index f07e014b84dc3..fede86484a96b 100644 --- a/pkg/bloomgateway/bloomgateway_test.go +++ b/pkg/bloomgateway/bloomgateway_test.go @@ -423,6 +423,9 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { } func TestBloomGateway_RemoveNotMatchingChunks(t *testing.T) { + g := &Gateway{ + logger: log.NewNopLogger(), + } t.Run("removing chunks partially", func(t *testing.T) { req := &logproto.FilterChunkRefRequest{ Refs: []*logproto.GroupedChunkRefs{ @@ -450,7 +453,8 @@ func TestBloomGateway_RemoveNotMatchingChunks(t *testing.T) { }}, }, } - removeNotMatchingChunks(req, res, log.NewNopLogger()) + n := g.removeNotMatchingChunks(req, res) + require.Equal(t, 2, n) require.Equal(t, expected, req) }) @@ -474,7 +478,8 @@ func TestBloomGateway_RemoveNotMatchingChunks(t *testing.T) { expected := &logproto.FilterChunkRefRequest{ Refs: []*logproto.GroupedChunkRefs{}, } - removeNotMatchingChunks(req, res, log.NewNopLogger()) + n := g.removeNotMatchingChunks(req, res) + require.Equal(t, 3, n) require.Equal(t, expected, req) }) diff --git a/pkg/bloomgateway/client.go b/pkg/bloomgateway/client.go index 9a75e4e87c26b..721e0c35ca50e 100644 --- a/pkg/bloomgateway/client.go +++ b/pkg/bloomgateway/client.go @@ -7,7 +7,6 @@ import ( "io" "math" "math/rand" - "sort" "sync" "github.com/go-kit/log" @@ -20,6 +19,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/common/model" + "golang.org/x/exp/slices" "google.golang.org/grpc" "google.golang.org/grpc/health/grpc_health_v1" @@ -291,77 +291,86 @@ func (c *GatewayClient) doForAddrs(addrs []string, fn func(logproto.BloomGateway return err } -func groupFingerprintsByServer(groups []*logproto.GroupedChunkRefs, servers []addrsWithTokenRange) []instanceWithFingerprints { +func groupFingerprintsByServer(groups []*logproto.GroupedChunkRefs, servers []addrsWithBounds) []instanceWithFingerprints { boundedFingerprints := partitionFingerprintsByAddresses(groups, servers) return groupByInstance(boundedFingerprints) } -func serverAddressesWithTokenRanges(subRing ring.ReadRing, instances []ring.InstanceDesc) ([]addrsWithTokenRange, error) { +func mapTokenRangeToFingerprintRange(r bloomutils.Range[uint32]) v1.FingerprintBounds { + minFp := uint64(r.Min) << 32 + maxFp := uint64(r.Max) << 32 + return v1.NewBounds( + model.Fingerprint(minFp), + model.Fingerprint(maxFp|math.MaxUint32), + ) +} + +func serverAddressesWithTokenRanges(subRing ring.ReadRing, instances []ring.InstanceDesc) ([]addrsWithBounds, error) { bufDescs, bufHosts, bufZones := ring.MakeBuffersForGet() - servers := make([]addrsWithTokenRange, 0, len(instances)) + servers := make([]addrsWithBounds, 0, len(instances)) it := bloomutils.NewInstanceSortMergeIterator(instances) + for it.Next() { // We can use on of the tokens from the token range // to obtain all addresses for that token. - rs, err := subRing.Get(it.At().MaxToken, BlocksOwnerRead, bufDescs, bufHosts, bufZones) + rs, err := subRing.Get(it.At().TokenRange.Max, BlocksOwnerRead, bufDescs, bufHosts, bufZones) if err != nil { return nil, errors.Wrap(err, "bloom gateway get ring") } - servers = append(servers, addrsWithTokenRange{ - id: it.At().Instance.Id, - addrs: rs.GetAddresses(), - minToken: it.At().MinToken, - maxToken: it.At().MaxToken, + + bounds := mapTokenRangeToFingerprintRange(it.At().TokenRange) + servers = append(servers, addrsWithBounds{ + id: it.At().Instance.Id, + addrs: rs.GetAddresses(), + FingerprintBounds: bounds, }) } - if len(servers) > 0 && servers[len(servers)-1].maxToken < math.MaxUint32 { - // append the instance for the token range between the greates token and MaxUint32 - servers = append(servers, addrsWithTokenRange{ - id: servers[0].id, - addrs: servers[0].addrs, - minToken: servers[len(servers)-1].maxToken + 1, - maxToken: math.MaxUint32, + if len(servers) > 0 && servers[len(servers)-1].Max < math.MaxUint64 { + // append the instance for the range between the maxFp and MaxUint64 + // TODO(owen-d): support wrapping around keyspace for token ranges + servers = append(servers, addrsWithBounds{ + id: servers[0].id, + addrs: servers[0].addrs, + FingerprintBounds: v1.NewBounds( + servers[len(servers)-1].Max+1, + model.Fingerprint(math.MaxUint64), + ), }) } return servers, nil } -type instanceWithToken struct { - instance ring.InstanceDesc - token uint32 -} - -type addrsWithTokenRange struct { - id string - addrs []string - minToken, maxToken uint32 -} - -func (s addrsWithTokenRange) cmp(token uint32) v1.BoundsCheck { - if token < s.minToken { - return v1.Before - } else if token > s.maxToken { - return v1.After - } - return v1.Overlap +type addrsWithBounds struct { + v1.FingerprintBounds + id string + addrs []string } type instanceWithFingerprints struct { - instance addrsWithTokenRange + instance addrsWithBounds fingerprints []*logproto.GroupedChunkRefs } -func partitionFingerprintsByAddresses(fingerprints []*logproto.GroupedChunkRefs, addresses []addrsWithTokenRange) (result []instanceWithFingerprints) { +func partitionFingerprintsByAddresses(fingerprints []*logproto.GroupedChunkRefs, addresses []addrsWithBounds) (result []instanceWithFingerprints) { for _, instance := range addresses { - - min := sort.Search(len(fingerprints), func(i int) bool { - return instance.cmp(uint32(fingerprints[i].Fingerprint)) > v1.Before + min, _ := slices.BinarySearchFunc(fingerprints, instance.FingerprintBounds, func(g *logproto.GroupedChunkRefs, b v1.FingerprintBounds) int { + if g.Fingerprint < uint64(b.Min) { + return -1 + } else if g.Fingerprint > uint64(b.Min) { + return 1 + } + return 0 }) - max := sort.Search(len(fingerprints), func(i int) bool { - return instance.cmp(uint32(fingerprints[i].Fingerprint)) == v1.After + max, _ := slices.BinarySearchFunc(fingerprints, instance.FingerprintBounds, func(g *logproto.GroupedChunkRefs, b v1.FingerprintBounds) int { + if g.Fingerprint <= uint64(b.Max) { + return -1 + } else if g.Fingerprint > uint64(b.Max) { + return 1 + } + return 0 }) // fingerprint is out of boundaries @@ -401,7 +410,7 @@ func groupByInstance(boundedFingerprints []instanceWithFingerprints) []instanceW pos[cur.instance.id] = len(result) result = append(result, instanceWithFingerprints{ - instance: addrsWithTokenRange{ + instance: addrsWithBounds{ id: cur.instance.id, addrs: cur.instance.addrs, }, diff --git a/pkg/bloomgateway/client_test.go b/pkg/bloomgateway/client_test.go index b1716de8150ea..71ac0ec0639a8 100644 --- a/pkg/bloomgateway/client_test.go +++ b/pkg/bloomgateway/client_test.go @@ -2,6 +2,7 @@ package bloomgateway import ( "context" + "fmt" "math" "sort" "testing" @@ -16,6 +17,7 @@ import ( "github.com/grafana/loki/pkg/bloomutils" "github.com/grafana/loki/pkg/logproto" + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/validation" ) @@ -39,24 +41,20 @@ func TestBloomGatewayClient(t *testing.T) { } func TestBloomGatewayClient_PartitionFingerprintsByAddresses(t *testing.T) { + // Create 10 fingerprints [0, 2, 4, ... 18] + groups := make([]*logproto.GroupedChunkRefs, 0, 10) + for i := 0; i < 20; i += 2 { + groups = append(groups, &logproto.GroupedChunkRefs{Fingerprint: uint64(i)}) + } + // instance token ranges do not overlap t.Run("non-overlapping", func(t *testing.T) { - groups := []*logproto.GroupedChunkRefs{ - {Fingerprint: 0}, - {Fingerprint: 100}, - {Fingerprint: 101}, - {Fingerprint: 200}, - {Fingerprint: 201}, - {Fingerprint: 300}, - {Fingerprint: 301}, - {Fingerprint: 400}, - {Fingerprint: 401}, // out of bounds, will be dismissed - } - servers := []addrsWithTokenRange{ - {id: "instance-1", addrs: []string{"10.0.0.1"}, minToken: 0, maxToken: 100}, - {id: "instance-2", addrs: []string{"10.0.0.2"}, minToken: 101, maxToken: 200}, - {id: "instance-3", addrs: []string{"10.0.0.3"}, minToken: 201, maxToken: 300}, - {id: "instance-2", addrs: []string{"10.0.0.2"}, minToken: 301, maxToken: 400}, + + servers := []addrsWithBounds{ + {id: "instance-1", addrs: []string{"10.0.0.1"}, FingerprintBounds: v1.NewBounds(0, 4)}, + {id: "instance-2", addrs: []string{"10.0.0.2"}, FingerprintBounds: v1.NewBounds(5, 9)}, + {id: "instance-3", addrs: []string{"10.0.0.3"}, FingerprintBounds: v1.NewBounds(10, 14)}, + {id: "instance-2", addrs: []string{"10.0.0.2"}, FingerprintBounds: v1.NewBounds(15, 19)}, } // partition fingerprints @@ -66,28 +64,30 @@ func TestBloomGatewayClient_PartitionFingerprintsByAddresses(t *testing.T) { instance: servers[0], fingerprints: []*logproto.GroupedChunkRefs{ {Fingerprint: 0}, - {Fingerprint: 100}, + {Fingerprint: 2}, + {Fingerprint: 4}, }, }, { instance: servers[1], fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 101}, - {Fingerprint: 200}, + {Fingerprint: 6}, + {Fingerprint: 8}, }, }, { instance: servers[2], fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 201}, - {Fingerprint: 300}, + {Fingerprint: 10}, + {Fingerprint: 12}, + {Fingerprint: 14}, }, }, { instance: servers[3], fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 301}, - {Fingerprint: 400}, + {Fingerprint: 16}, + {Fingerprint: 18}, }, }, } @@ -99,26 +99,28 @@ func TestBloomGatewayClient_PartitionFingerprintsByAddresses(t *testing.T) { expected = []instanceWithFingerprints{ { - instance: addrsWithTokenRange{id: "instance-1", addrs: []string{"10.0.0.1"}}, + instance: addrsWithBounds{id: "instance-1", addrs: []string{"10.0.0.1"}}, fingerprints: []*logproto.GroupedChunkRefs{ {Fingerprint: 0}, - {Fingerprint: 100}, + {Fingerprint: 2}, + {Fingerprint: 4}, }, }, { - instance: addrsWithTokenRange{id: "instance-2", addrs: []string{"10.0.0.2"}}, + instance: addrsWithBounds{id: "instance-2", addrs: []string{"10.0.0.2"}}, fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 101}, - {Fingerprint: 200}, - {Fingerprint: 301}, - {Fingerprint: 400}, + {Fingerprint: 6}, + {Fingerprint: 8}, + {Fingerprint: 16}, + {Fingerprint: 18}, }, }, { - instance: addrsWithTokenRange{id: "instance-3", addrs: []string{"10.0.0.3"}}, + instance: addrsWithBounds{id: "instance-3", addrs: []string{"10.0.0.3"}}, fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 201}, - {Fingerprint: 300}, + {Fingerprint: 10}, + {Fingerprint: 12}, + {Fingerprint: 14}, }, }, } @@ -128,33 +130,45 @@ func TestBloomGatewayClient_PartitionFingerprintsByAddresses(t *testing.T) { // instance token ranges overlap t.Run("overlapping", func(t *testing.T) { - groups := []*logproto.GroupedChunkRefs{ - {Fingerprint: 50}, - {Fingerprint: 150}, - {Fingerprint: 250}, - {Fingerprint: 350}, - } - servers := []addrsWithTokenRange{ - {id: "instance-1", addrs: []string{"10.0.0.1"}, minToken: 0, maxToken: 200}, - {id: "instance-2", addrs: []string{"10.0.0.2"}, minToken: 100, maxToken: 300}, - {id: "instance-3", addrs: []string{"10.0.0.3"}, minToken: 200, maxToken: 400}, + servers := []addrsWithBounds{ + {id: "instance-1", addrs: []string{"10.0.0.1"}, FingerprintBounds: v1.NewBounds(0, 9)}, + {id: "instance-2", addrs: []string{"10.0.0.2"}, FingerprintBounds: v1.NewBounds(5, 14)}, + {id: "instance-3", addrs: []string{"10.0.0.3"}, FingerprintBounds: v1.NewBounds(10, 19)}, } // partition fingerprints expected := []instanceWithFingerprints{ - {instance: servers[0], fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 50}, - {Fingerprint: 150}, - }}, - {instance: servers[1], fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 150}, - {Fingerprint: 250}, - }}, - {instance: servers[2], fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 250}, - {Fingerprint: 350}, - }}, + { + instance: servers[0], + fingerprints: []*logproto.GroupedChunkRefs{ + {Fingerprint: 0}, + {Fingerprint: 2}, + {Fingerprint: 4}, + {Fingerprint: 6}, + {Fingerprint: 8}, + }, + }, + { + instance: servers[1], + fingerprints: []*logproto.GroupedChunkRefs{ + {Fingerprint: 6}, + {Fingerprint: 8}, + {Fingerprint: 10}, + {Fingerprint: 12}, + {Fingerprint: 14}, + }, + }, + { + instance: servers[2], + fingerprints: []*logproto.GroupedChunkRefs{ + {Fingerprint: 10}, + {Fingerprint: 12}, + {Fingerprint: 14}, + {Fingerprint: 16}, + {Fingerprint: 18}, + }, + }, } bounded := partitionFingerprintsByAddresses(groups, servers) @@ -162,34 +176,85 @@ func TestBloomGatewayClient_PartitionFingerprintsByAddresses(t *testing.T) { }) } +func BenchmarkPartitionFingerprintsByAddresses(b *testing.B) { + numFp := 100000 + fpStep := math.MaxUint64 / uint64(numFp) + + groups := make([]*logproto.GroupedChunkRefs, 0, numFp) + for i := uint64(0); i < math.MaxUint64-fpStep; i += fpStep { + groups = append(groups, &logproto.GroupedChunkRefs{Fingerprint: i}) + } + + numServers := 100 + tokenStep := math.MaxUint32 / uint32(numServers) + servers := make([]addrsWithBounds, 0, numServers) + for i := uint32(0); i < math.MaxUint32-tokenStep; i += tokenStep { + servers = append(servers, addrsWithBounds{ + id: fmt.Sprintf("instance-%x", i), + addrs: []string{fmt.Sprintf("%d", i)}, + FingerprintBounds: v1.NewBounds( + model.Fingerprint(i)<<32, + model.Fingerprint(i+tokenStep)<<32, + ), + }) + } + + b.ResetTimer() + b.ReportAllocs() + for i := 0; i < b.N; i++ { + _ = partitionFingerprintsByAddresses(groups, servers) + } +} + +func TestBloomGatewayClient_MapTokenRangeToFingerprintRange(t *testing.T) { + testCases := map[string]struct { + lshift int + inp bloomutils.Range[uint32] + exp v1.FingerprintBounds + }{ + "single token expands to multiple fingerprints": { + inp: bloomutils.NewTokenRange(0, 0), + exp: v1.NewBounds(0, 0xffffffff), + }, + "max value expands to max value of new range": { + inp: bloomutils.NewTokenRange((1 << 31), math.MaxUint32), + exp: v1.NewBounds((1 << 63), 0xffffffffffffffff), + }, + } + for desc, tc := range testCases { + t.Run(desc, func(t *testing.T) { + actual := mapTokenRangeToFingerprintRange(tc.inp) + require.Equal(t, tc.exp, actual) + }) + } +} + func TestBloomGatewayClient_ServerAddressesWithTokenRanges(t *testing.T) { testCases := map[string]struct { instances []ring.InstanceDesc - expected []addrsWithTokenRange + expected []addrsWithBounds }{ - "one token per instance": { + "one token per instance, no gaps between fingerprint ranges": { instances: []ring.InstanceDesc{ - {Id: "instance-1", Addr: "10.0.0.1", Tokens: []uint32{math.MaxUint32 / 6 * 1}}, - {Id: "instance-2", Addr: "10.0.0.2", Tokens: []uint32{math.MaxUint32 / 6 * 3}}, - {Id: "instance-3", Addr: "10.0.0.3", Tokens: []uint32{math.MaxUint32 / 6 * 5}}, + {Id: "instance-1", Addr: "10.0.0.1", Tokens: []uint32{(1 << 30) * 1}}, // 0x40000000 + {Id: "instance-2", Addr: "10.0.0.2", Tokens: []uint32{(1 << 30) * 2}}, // 0x80000000 + {Id: "instance-3", Addr: "10.0.0.3", Tokens: []uint32{(1 << 30) * 3}}, // 0xc0000000 }, - expected: []addrsWithTokenRange{ - {id: "instance-1", addrs: []string{"10.0.0.1"}, minToken: 0, maxToken: math.MaxUint32 / 6 * 1}, - {id: "instance-2", addrs: []string{"10.0.0.2"}, minToken: math.MaxUint32/6*1 + 1, maxToken: math.MaxUint32 / 6 * 3}, - {id: "instance-3", addrs: []string{"10.0.0.3"}, minToken: math.MaxUint32/6*3 + 1, maxToken: math.MaxUint32 / 6 * 5}, - {id: "instance-1", addrs: []string{"10.0.0.1"}, minToken: math.MaxUint32/6*5 + 1, maxToken: math.MaxUint32}, + expected: []addrsWithBounds{ + {id: "instance-1", addrs: []string{"10.0.0.1"}, FingerprintBounds: v1.NewBounds(0, 4611686022722355199)}, + {id: "instance-2", addrs: []string{"10.0.0.2"}, FingerprintBounds: v1.NewBounds(4611686022722355200, 9223372041149743103)}, + {id: "instance-3", addrs: []string{"10.0.0.3"}, FingerprintBounds: v1.NewBounds(9223372041149743104, 13835058059577131007)}, + {id: "instance-1", addrs: []string{"10.0.0.1"}, FingerprintBounds: v1.NewBounds(13835058059577131008, 18446744073709551615)}, }, }, - "MinUint32 and MaxUint32 are tokens in the ring": { + "MinUint32 and MaxUint32 are actual tokens in the ring": { instances: []ring.InstanceDesc{ - {Id: "instance-1", Addr: "10.0.0.1", Tokens: []uint32{0, math.MaxUint32 / 3 * 2}}, - {Id: "instance-2", Addr: "10.0.0.2", Tokens: []uint32{math.MaxUint32 / 3 * 1, math.MaxUint32}}, + {Id: "instance-1", Addr: "10.0.0.1", Tokens: []uint32{0}}, + {Id: "instance-2", Addr: "10.0.0.2", Tokens: []uint32{math.MaxUint32}}, }, - expected: []addrsWithTokenRange{ - {id: "instance-1", addrs: []string{"10.0.0.1"}, minToken: 0, maxToken: 0}, - {id: "instance-2", addrs: []string{"10.0.0.2"}, minToken: 1, maxToken: math.MaxUint32 / 3}, - {id: "instance-1", addrs: []string{"10.0.0.1"}, minToken: math.MaxUint32/3*1 + 1, maxToken: math.MaxUint32 / 3 * 2}, - {id: "instance-2", addrs: []string{"10.0.0.2"}, minToken: math.MaxUint32/3*2 + 1, maxToken: math.MaxUint32}, + expected: []addrsWithBounds{ + {id: "instance-1", addrs: []string{"10.0.0.1"}, FingerprintBounds: v1.NewBounds(0, (1<<32)-1)}, + {id: "instance-2", addrs: []string{"10.0.0.2"}, FingerprintBounds: v1.NewBounds((1 << 32), math.MaxUint64)}, }, }, } @@ -208,15 +273,27 @@ func TestBloomGatewayClient_ServerAddressesWithTokenRanges(t *testing.T) { func TestBloomGatewayClient_GroupFingerprintsByServer(t *testing.T) { instances := []ring.InstanceDesc{ - {Id: "instance-1", Addr: "10.0.0.1", Tokens: []uint32{2146405214, 1029997044, 678878693}}, - {Id: "instance-2", Addr: "10.0.0.2", Tokens: []uint32{296463531, 1697323986, 800258284}}, - {Id: "instance-3", Addr: "10.0.0.3", Tokens: []uint32{2014002871, 315617625, 1036168527}}, + {Id: "instance-1", Addr: "10.0.0.1", Tokens: []uint32{0x1fffffff, 0x7fffffff}}, + {Id: "instance-2", Addr: "10.0.0.2", Tokens: []uint32{0x3fffffff, 0x9fffffff}}, + {Id: "instance-3", Addr: "10.0.0.3", Tokens: []uint32{0x5fffffff, 0xbfffffff}}, } - it := bloomutils.NewInstanceSortMergeIterator(instances) - for it.Next() { - t.Log(it.At().MaxToken, it.At().Instance.Addr) - } + subRing := newMockRing(instances) + servers, err := serverAddressesWithTokenRanges(subRing, instances) + require.NoError(t, err) + + // for _, s := range servers { + // t.Log(s, v1.NewBounds(model.Fingerprint(s.fpRange.Min), model.Fingerprint(s.fpRange.Max))) + // } + /** + {instance-1 [10.0.0.1] { 0 536870911} { 0 2305843004918726656}} 0000000000000000-1fffffff00000000 + {instance-2 [10.0.0.2] { 536870912 1073741823} { 2305843009213693952 4611686014132420608}} 2000000000000000-3fffffff00000000 + {instance-3 [10.0.0.3] {1073741824 1610612735} { 4611686018427387904 6917529023346114560}} 4000000000000000-5fffffff00000000 + {instance-1 [10.0.0.1] {1610612736 2147483647} { 6917529027641081856 9223372032559808512}} 6000000000000000-7fffffff00000000 + {instance-2 [10.0.0.2] {2147483648 2684354559} { 9223372036854775808 11529215041773502464}} 8000000000000000-9fffffff00000000 + {instance-3 [10.0.0.3] {2684354560 3221225471} {11529215046068469760 13835058050987196416}} a000000000000000-bfffffff00000000 + {instance-1 [10.0.0.1] {3221225472 4294967295} {13835058055282163712 18446744073709551615}} c000000000000000-ffffffffffffffff + **/ testCases := []struct { name string @@ -231,18 +308,20 @@ func TestBloomGatewayClient_GroupFingerprintsByServer(t *testing.T) { { name: "fingerprints within a single token range are grouped", chunks: []*logproto.GroupedChunkRefs{ - {Fingerprint: 1000000000, Refs: []*logproto.ShortRef{{Checksum: 1}}}, - {Fingerprint: 1000000001, Refs: []*logproto.ShortRef{{Checksum: 2}}}, + {Fingerprint: 0x5000000000000001}, + {Fingerprint: 0x5000000000000010}, + {Fingerprint: 0x5000000000000100}, }, expected: []instanceWithFingerprints{ { - instance: addrsWithTokenRange{ - id: "instance-1", - addrs: []string{"10.0.0.1"}, + instance: addrsWithBounds{ + id: "instance-3", + addrs: []string{"10.0.0.3"}, }, fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 1000000000, Refs: []*logproto.ShortRef{{Checksum: 1}}}, - {Fingerprint: 1000000001, Refs: []*logproto.ShortRef{{Checksum: 2}}}, + {Fingerprint: 0x5000000000000001}, + {Fingerprint: 0x5000000000000010}, + {Fingerprint: 0x5000000000000100}, }, }, }, @@ -250,18 +329,20 @@ func TestBloomGatewayClient_GroupFingerprintsByServer(t *testing.T) { { name: "fingerprints within multiple token ranges of a single instance are grouped", chunks: []*logproto.GroupedChunkRefs{ - {Fingerprint: 1000000000, Refs: []*logproto.ShortRef{{Checksum: 1}}}, - {Fingerprint: 2100000000, Refs: []*logproto.ShortRef{{Checksum: 2}}}, + {Fingerprint: 0x1000000000000000}, + {Fingerprint: 0x7000000000000000}, + {Fingerprint: 0xd000000000000000}, }, expected: []instanceWithFingerprints{ { - instance: addrsWithTokenRange{ + instance: addrsWithBounds{ id: "instance-1", addrs: []string{"10.0.0.1"}, }, fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 1000000000, Refs: []*logproto.ShortRef{{Checksum: 1}}}, - {Fingerprint: 2100000000, Refs: []*logproto.ShortRef{{Checksum: 2}}}, + {Fingerprint: 0x1000000000000000}, + {Fingerprint: 0x7000000000000000}, + {Fingerprint: 0xd000000000000000}, }, }, }, @@ -269,55 +350,52 @@ func TestBloomGatewayClient_GroupFingerprintsByServer(t *testing.T) { { name: "fingerprints with token ranges of multiple instances are grouped", chunks: []*logproto.GroupedChunkRefs{ - // instance 1 - {Fingerprint: 1000000000, Refs: []*logproto.ShortRef{{Checksum: 1}}}, - // instance 1 - {Fingerprint: 2100000000, Refs: []*logproto.ShortRef{{Checksum: 2}}}, - // instance 2 - {Fingerprint: 290000000, Refs: []*logproto.ShortRef{{Checksum: 3}}}, - // instance 2 (fingerprint equals instance token) - {Fingerprint: 800258284, Refs: []*logproto.ShortRef{{Checksum: 4}}}, - // instance 2 (fingerprint greater than greatest token) - {Fingerprint: 2147483648, Refs: []*logproto.ShortRef{{Checksum: 5}}}, - // instance 3 - {Fingerprint: 1029997045, Refs: []*logproto.ShortRef{{Checksum: 6}}}, + {Fingerprint: 0x1000000000000000}, + {Fingerprint: 0x3000000000000000}, + {Fingerprint: 0x5000000000000000}, + {Fingerprint: 0x7000000000000000}, + {Fingerprint: 0x9000000000000000}, + {Fingerprint: 0xb000000000000000}, + {Fingerprint: 0xd000000000000000}, + {Fingerprint: 0xf000000000000000}, }, expected: []instanceWithFingerprints{ { - instance: addrsWithTokenRange{ - id: "instance-2", - addrs: []string{"10.0.0.2"}, + instance: addrsWithBounds{ + id: "instance-1", + addrs: []string{"10.0.0.1"}, }, fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 290000000, Refs: []*logproto.ShortRef{{Checksum: 3}}}, - {Fingerprint: 800258284, Refs: []*logproto.ShortRef{{Checksum: 4}}}, - {Fingerprint: 2147483648, Refs: []*logproto.ShortRef{{Checksum: 5}}}, + {Fingerprint: 0x1000000000000000}, + {Fingerprint: 0x7000000000000000}, + {Fingerprint: 0xd000000000000000}, + {Fingerprint: 0xf000000000000000}, }, }, { - instance: addrsWithTokenRange{ - id: "instance-1", - addrs: []string{"10.0.0.1"}, + instance: addrsWithBounds{ + id: "instance-2", + addrs: []string{"10.0.0.2"}, }, fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 1000000000, Refs: []*logproto.ShortRef{{Checksum: 1}}}, - {Fingerprint: 2100000000, Refs: []*logproto.ShortRef{{Checksum: 2}}}, + {Fingerprint: 0x3000000000000000}, + {Fingerprint: 0x9000000000000000}, }, }, { - instance: addrsWithTokenRange{ + instance: addrsWithBounds{ id: "instance-3", addrs: []string{"10.0.0.3"}, }, fingerprints: []*logproto.GroupedChunkRefs{ - {Fingerprint: 1029997045, Refs: []*logproto.ShortRef{{Checksum: 6}}}, + {Fingerprint: 0x5000000000000000}, + {Fingerprint: 0xb000000000000000}, }, }, }, }, } - subRing := newMockRing(instances) for _, tc := range testCases { tc := tc t.Run(tc.name, func(t *testing.T) { @@ -325,9 +403,6 @@ func TestBloomGatewayClient_GroupFingerprintsByServer(t *testing.T) { sort.Slice(tc.chunks, func(i, j int) bool { return tc.chunks[i].Fingerprint < tc.chunks[j].Fingerprint }) - - servers, err := serverAddressesWithTokenRanges(subRing, instances) - require.NoError(t, err) res := groupFingerprintsByServer(tc.chunks, servers) require.Equal(t, tc.expected, res) }) @@ -357,10 +432,10 @@ type mockRing struct { // Get implements ring.ReadRing. func (r *mockRing) Get(key uint32, _ ring.Operation, _ []ring.InstanceDesc, _ []string, _ []string) (ring.ReplicationSet, error) { idx, _ := sort.Find(len(r.ranges), func(i int) int { - if r.ranges[i].MaxToken < key { + if r.ranges[i].TokenRange.Max < key { return 1 } - if r.ranges[i].MaxToken > key { + if r.ranges[i].TokenRange.Max > key { return -1 } return 0 diff --git a/pkg/bloomgateway/processor.go b/pkg/bloomgateway/processor.go index 5eab7a858c74b..a7641bde0c960 100644 --- a/pkg/bloomgateway/processor.go +++ b/pkg/bloomgateway/processor.go @@ -3,7 +3,6 @@ package bloomgateway import ( "context" "math" - "sort" "time" "github.com/go-kit/log" @@ -13,11 +12,6 @@ import ( "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" ) -type tasksForBlock struct { - blockRef bloomshipper.BlockRef - tasks []Task -} - func newProcessor(id string, store bloomshipper.Store, logger log.Logger, metrics *workerMetrics) *processor { return &processor{ id: id, @@ -66,20 +60,20 @@ func (p *processor) processTasks(ctx context.Context, tenant string, day config. p.metrics.metasFetched.WithLabelValues(p.id).Observe(float64(len(metas))) blocksRefs := bloomshipper.BlocksForMetas(metas, interval, keyspaces) - return p.processBlocks(ctx, partition(tasks, blocksRefs)) + return p.processBlocks(ctx, partitionTasks(tasks, blocksRefs)) } -func (p *processor) processBlocks(ctx context.Context, data []tasksForBlock) error { +func (p *processor) processBlocks(ctx context.Context, data []blockWithTasks) error { refs := make([]bloomshipper.BlockRef, len(data)) for _, block := range data { - refs = append(refs, block.blockRef) + refs = append(refs, block.ref) } bqs, err := p.store.FetchBlocks(ctx, refs) if err != nil { return err } - p.metrics.metasFetched.WithLabelValues(p.id).Observe(float64(len(bqs))) + p.metrics.blocksFetched.WithLabelValues(p.id).Observe(float64(len(bqs))) blockIter := v1.NewSliceIter(bqs) @@ -87,7 +81,7 @@ outer: for blockIter.Next() { bq := blockIter.At() for i, block := range data { - if block.blockRef.Bounds.Equal(bq.Bounds) { + if block.ref.Bounds.Equal(bq.Bounds) { err := p.processBlock(ctx, bq.BlockQuerier, block.tasks) bq.Close() if err != nil { @@ -146,37 +140,3 @@ func group[K comparable, V any, S ~[]V](s S, f func(v V) K) map[K]S { } return m } - -func partition(tasks []Task, blocks []bloomshipper.BlockRef) []tasksForBlock { - result := make([]tasksForBlock, 0, len(blocks)) - - for _, block := range blocks { - bounded := tasksForBlock{ - blockRef: block, - } - - for _, task := range tasks { - refs := task.series - min := sort.Search(len(refs), func(i int) bool { - return block.Cmp(refs[i].Fingerprint) > v1.Before - }) - - max := sort.Search(len(refs), func(i int) bool { - return block.Cmp(refs[i].Fingerprint) == v1.After - }) - - // All fingerprints fall outside of the consumer's range - if min == len(refs) || max == 0 { - continue - } - - bounded.tasks = append(bounded.tasks, task.Copy(refs[min:max])) - } - - if len(bounded.tasks) > 0 { - result = append(result, bounded) - } - - } - return result -} diff --git a/pkg/bloomgateway/querier.go b/pkg/bloomgateway/querier.go index 4b2366e83f287..02608bfdf71c4 100644 --- a/pkg/bloomgateway/querier.go +++ b/pkg/bloomgateway/querier.go @@ -5,17 +5,56 @@ import ( "sort" "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/common/model" "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql/syntax" ) +type querierMetrics struct { + chunksTotal prometheus.Counter + chunksFiltered prometheus.Counter + seriesTotal prometheus.Counter + seriesFiltered prometheus.Counter +} + +func newQuerierMetrics(registerer prometheus.Registerer, namespace, subsystem string) *querierMetrics { + return &querierMetrics{ + chunksTotal: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "chunks_total", + Help: "Total amount of chunks pre filtering. Does not count chunks in failed requests.", + }), + chunksFiltered: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "chunks_filtered_total", + Help: "Total amount of chunks that have been filtered out. Does not count chunks in failed requests.", + }), + seriesTotal: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "series_total", + Help: "Total amount of series pre filtering. Does not count series in failed requests.", + }), + seriesFiltered: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "series_filtered_total", + Help: "Total amount of series that have been filtered out. Does not count series in failed requests.", + }), + } +} + // BloomQuerier is a store-level abstraction on top of Client // It is used by the index gateway to filter ChunkRefs based on given line fiter expression. type BloomQuerier struct { - c Client - logger log.Logger + c Client + logger log.Logger + metrics *querierMetrics } func NewQuerier(c Client, logger log.Logger) *BloomQuerier { @@ -37,6 +76,9 @@ func (bq *BloomQuerier) FilterChunkRefs(ctx context.Context, tenant string, from defer groupedChunksRefPool.Put(grouped) grouped = groupChunkRefs(chunkRefs, grouped) + preFilterChunks := len(chunkRefs) + preFilterSeries := len(grouped) + refs, err := bq.c.FilterChunks(ctx, tenant, from, through, grouped, filters...) if err != nil { return nil, err @@ -55,6 +97,15 @@ func (bq *BloomQuerier) FilterChunkRefs(ctx context.Context, tenant string, from }) } } + + postFilterChunks := len(result) + postFilterSeries := len(refs) + + bq.metrics.chunksTotal.Add(float64(preFilterChunks)) + bq.metrics.chunksFiltered.Add(float64(preFilterChunks - postFilterChunks)) + bq.metrics.seriesTotal.Add(float64(preFilterSeries)) + bq.metrics.seriesFiltered.Add(float64(preFilterSeries - postFilterSeries)) + return result, nil } diff --git a/pkg/bloomgateway/util.go b/pkg/bloomgateway/util.go index 3793076f7c385..3ab234aaa8ae0 100644 --- a/pkg/bloomgateway/util.go +++ b/pkg/bloomgateway/util.go @@ -83,15 +83,17 @@ func convertToChunkRefs(refs []*logproto.ShortRef) v1.ChunkRefs { return result } -type boundedTasks struct { - blockRef bloomshipper.BlockRef - tasks []Task +type blockWithTasks struct { + ref bloomshipper.BlockRef + tasks []Task } -func partitionFingerprintRange(tasks []Task, blocks []bloomshipper.BlockRef) (result []boundedTasks) { +func partitionTasks(tasks []Task, blocks []bloomshipper.BlockRef) []blockWithTasks { + result := make([]blockWithTasks, 0, len(blocks)) + for _, block := range blocks { - bounded := boundedTasks{ - blockRef: block, + bounded := blockWithTasks{ + ref: block, } for _, task := range tasks { diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index 5f4d254e8f045..e9776dfef78f5 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -73,7 +73,7 @@ func mkBlockRef(minFp, maxFp uint64) bloomshipper.BlockRef { } } -func TestPartitionFingerprintRange(t *testing.T) { +func TestPartitionTasks(t *testing.T) { t.Run("consecutive block ranges", func(t *testing.T) { bounds := []bloomshipper.BlockRef{ @@ -93,7 +93,7 @@ func TestPartitionFingerprintRange(t *testing.T) { tasks[i%nTasks].series = append(tasks[i%nTasks].series, &logproto.GroupedChunkRefs{Fingerprint: uint64(i)}) } - results := partitionFingerprintRange(tasks, bounds) + results := partitionTasks(tasks, bounds) require.Equal(t, 3, len(results)) // ensure we only return bounds in range actualFingerprints := make([]*logproto.GroupedChunkRefs, 0, nSeries) @@ -128,7 +128,7 @@ func TestPartitionFingerprintRange(t *testing.T) { task.series = append(task.series, &logproto.GroupedChunkRefs{Fingerprint: uint64(i)}) } - results := partitionFingerprintRange([]Task{task}, bounds) + results := partitionTasks([]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 @@ -311,7 +311,7 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, } ref := bloomshipper.Ref{ TenantID: tenant, - TableName: config.NewDayTime(truncateDay(from)).Table(), + TableName: config.NewDayTable(config.NewDayTime(truncateDay(from)), "").Addr(), Bounds: v1.NewBounds(fromFp, throughFp), StartTimestamp: from, EndTimestamp: through, @@ -323,8 +323,7 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, MetaRef: bloomshipper.MetaRef{ Ref: ref, }, - Tombstones: []bloomshipper.BlockRef{}, - Blocks: []bloomshipper.BlockRef{blockRef}, + Blocks: []bloomshipper.BlockRef{blockRef}, } block, data, _ := v1.MakeBlock(t, n, fromFp, throughFp, from, through) // Printing fingerprints and the log lines of its chunks comes handy for debugging... diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go index 5c57c0a2e4952..ec44081c1b30c 100644 --- a/pkg/bloomgateway/worker.go +++ b/pkg/bloomgateway/worker.go @@ -163,7 +163,7 @@ func (w *worker) running(_ context.Context) error { err = p.run(taskCtx, tasks) if err != nil { - w.metrics.processDuration.WithLabelValues(w.id, labelSuccess).Observe(time.Since(start).Seconds()) + w.metrics.processDuration.WithLabelValues(w.id, labelFailure).Observe(time.Since(start).Seconds()) w.metrics.tasksProcessed.WithLabelValues(w.id, labelFailure).Add(float64(len(tasks))) level.Error(w.logger).Log("msg", "failed to process tasks", "err", err) } else { diff --git a/pkg/bloomutils/iter.go b/pkg/bloomutils/iter.go deleted file mode 100644 index fdbe4a5e62587..0000000000000 --- a/pkg/bloomutils/iter.go +++ /dev/null @@ -1,37 +0,0 @@ -package bloomutils - -import ( - "io" - - v1 "github.com/grafana/loki/pkg/storage/bloom/v1" -) - -// sortMergeIterator implements v1.Iterator -type sortMergeIterator[T any, C comparable, R any] struct { - curr *R - heap *v1.HeapIterator[v1.IndexedValue[C]] - items []T - transform func(T, C, *R) *R - err error -} - -func (it *sortMergeIterator[T, C, R]) Next() bool { - ok := it.heap.Next() - if !ok { - it.err = io.EOF - return false - } - - group := it.heap.At() - it.curr = it.transform(it.items[group.Index()], group.Value(), it.curr) - - return true -} - -func (it *sortMergeIterator[T, C, R]) At() R { - return *it.curr -} - -func (it *sortMergeIterator[T, C, R]) Err() error { - return it.err -} diff --git a/pkg/bloomutils/ring.go b/pkg/bloomutils/ring.go index 08e62a13acb71..bc58bf09c8865 100644 --- a/pkg/bloomutils/ring.go +++ b/pkg/bloomutils/ring.go @@ -1,32 +1,65 @@ // This file contains a bunch of utility functions for bloom components. -// TODO: Find a better location for this package package bloomutils import ( + "fmt" "math" "sort" "github.com/grafana/dskit/ring" - "golang.org/x/exp/slices" + "github.com/prometheus/common/model" + "golang.org/x/exp/constraints" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" ) -type InstanceWithTokenRange struct { - Instance ring.InstanceDesc - MinToken, MaxToken uint32 +var ( + Uint32Range = Range[uint32]{Min: 0, Max: math.MaxUint32} + Uint64Range = Range[uint64]{Min: 0, Max: math.MaxUint64} +) + +type Range[T constraints.Unsigned] struct { + Min, Max T } -func (i InstanceWithTokenRange) Cmp(token uint32) v1.BoundsCheck { - if token < i.MinToken { +func (r Range[T]) String() string { + return fmt.Sprintf("%016x-%016x", r.Min, r.Max) +} + +func (r Range[T]) Less(other Range[T]) bool { + if r.Min != other.Min { + return r.Min < other.Min + } + return r.Max <= other.Max +} + +func (r Range[T]) Cmp(t T) v1.BoundsCheck { + if t < r.Min { return v1.Before - } else if token > i.MaxToken { + } else if t > r.Max { return v1.After } return v1.Overlap } +func NewRange[T constraints.Unsigned](min, max T) Range[T] { + return Range[T]{Min: min, Max: max} +} + +func NewTokenRange(min, max uint32) Range[uint32] { + return Range[uint32]{Min: min, Max: max} +} + +type InstanceWithTokenRange struct { + Instance ring.InstanceDesc + TokenRange Range[uint32] +} + +func (i InstanceWithTokenRange) Cmp(token uint32) v1.BoundsCheck { + return i.TokenRange.Cmp(token) +} + type InstancesWithTokenRange []InstanceWithTokenRange func (i InstancesWithTokenRange) Contains(token uint32) bool { @@ -38,109 +71,51 @@ func (i InstancesWithTokenRange) Contains(token uint32) bool { return false } -// GetInstanceTokenRange calculates the token range for a specific instance -// with given id based on the first token in the ring. -// This assumes that each instance in the ring is configured with only a single -// token. -func GetInstanceWithTokenRange(id string, instances []ring.InstanceDesc) InstancesWithTokenRange { - - // Sorting the tokens of the instances would not be necessary if there is - // only a single token per instances, however, since we only assume one - // token, but don't enforce one token, we keep the sorting. - for _, inst := range instances { - sort.Slice(inst.Tokens, func(i, j int) bool { - return inst.Tokens[i] < inst.Tokens[j] +// TODO(owen-d): use https://github.com/grafana/loki/pull/11975 after merge +func KeyspacesFromTokenRanges(tokenRanges ring.TokenRanges) []v1.FingerprintBounds { + keyspaces := make([]v1.FingerprintBounds, 0, len(tokenRanges)/2) + for i := 0; i < len(tokenRanges)-1; i += 2 { + keyspaces = append(keyspaces, v1.FingerprintBounds{ + Min: model.Fingerprint(tokenRanges[i]) << 32, + Max: model.Fingerprint(tokenRanges[i+1])<<32 | model.Fingerprint(math.MaxUint32), }) } - - // Sort instances - sort.Slice(instances, func(i, j int) bool { - return instances[i].Tokens[0] < instances[j].Tokens[0] - }) - - idx := slices.IndexFunc(instances, func(inst ring.InstanceDesc) bool { - return inst.Id == id - }) - - // instance with Id == id not found - if idx == -1 { - return InstancesWithTokenRange{} - } - - i := uint32(idx) - n := uint32(len(instances)) - step := math.MaxUint32 / n - - minToken := step * i - maxToken := step*i + step - 1 - if i == n-1 { - // extend the last token tange to MaxUint32 - maxToken = math.MaxUint32 - } - - return InstancesWithTokenRange{ - {MinToken: minToken, MaxToken: maxToken, Instance: instances[i]}, - } -} - -// GetInstancesWithTokenRanges calculates the token ranges for a specific -// instance with given id based on all tokens in the ring. -// If the instances in the ring are configured with a single token, such as the -// bloom compactor, use GetInstanceWithTokenRange() instead. -func GetInstancesWithTokenRanges(id string, instances []ring.InstanceDesc) InstancesWithTokenRange { - servers := make([]InstanceWithTokenRange, 0, len(instances)) - it := NewInstanceSortMergeIterator(instances) - var firstInst ring.InstanceDesc - var lastToken uint32 - for it.Next() { - if firstInst.Id == "" { - firstInst = it.At().Instance - } - if it.At().Instance.Id == id { - servers = append(servers, it.At()) - } - lastToken = it.At().MaxToken - } - // append token range from lastToken+1 to MaxUint32 - // only if the instance with the first token is the current one - if len(servers) > 0 && firstInst.Id == id { - servers = append(servers, InstanceWithTokenRange{ - MinToken: lastToken + 1, - MaxToken: math.MaxUint32, - Instance: servers[0].Instance, - }) - } - return servers + return keyspaces } // NewInstanceSortMergeIterator creates an iterator that yields instanceWithToken elements // where the token of the elements are sorted in ascending order. func NewInstanceSortMergeIterator(instances []ring.InstanceDesc) v1.Iterator[InstanceWithTokenRange] { - it := &sortMergeIterator[ring.InstanceDesc, uint32, InstanceWithTokenRange]{ - items: instances, - transform: func(item ring.InstanceDesc, val uint32, prev *InstanceWithTokenRange) *InstanceWithTokenRange { - var prevToken uint32 - if prev != nil { - prevToken = prev.MaxToken + 1 - } - return &InstanceWithTokenRange{Instance: item, MinToken: prevToken, MaxToken: val} - }, + tokenIters := make([]v1.PeekingIterator[v1.IndexedValue[uint32]], 0, len(instances)) + for i, inst := range instances { + sort.Slice(inst.Tokens, func(a, b int) bool { return inst.Tokens[a] < inst.Tokens[b] }) + itr := v1.NewIterWithIndex(v1.NewSliceIter[uint32](inst.Tokens), i) + tokenIters = append(tokenIters, v1.NewPeekingIter[v1.IndexedValue[uint32]](itr)) } - sequences := make([]v1.PeekingIterator[v1.IndexedValue[uint32]], 0, len(instances)) - for i := range instances { - sort.Slice(instances[i].Tokens, func(a, b int) bool { - return instances[i].Tokens[a] < instances[i].Tokens[b] - }) - iter := v1.NewIterWithIndex[uint32](v1.NewSliceIter(instances[i].Tokens), i) - sequences = append(sequences, v1.NewPeekingIter[v1.IndexedValue[uint32]](iter)) - } - it.heap = v1.NewHeapIterator( - func(i, j v1.IndexedValue[uint32]) bool { - return i.Value() < j.Value() + + heapIter := v1.NewHeapIterator[v1.IndexedValue[uint32]]( + func(iv1, iv2 v1.IndexedValue[uint32]) bool { + return iv1.Value() < iv2.Value() }, - sequences..., + tokenIters..., ) - it.err = nil - return it + prevToken := -1 + return v1.NewDedupingIter[v1.IndexedValue[uint32], InstanceWithTokenRange]( + func(iv v1.IndexedValue[uint32], iwtr InstanceWithTokenRange) bool { + return false + }, + func(iv v1.IndexedValue[uint32]) InstanceWithTokenRange { + minToken, maxToken := uint32(prevToken+1), iv.Value() + prevToken = int(maxToken) + return InstanceWithTokenRange{ + Instance: instances[iv.Index()], + TokenRange: NewTokenRange(minToken, maxToken), + } + }, + func(iv v1.IndexedValue[uint32], iwtr InstanceWithTokenRange) InstanceWithTokenRange { + panic("must not be called, because Eq() is always false") + }, + v1.NewPeekingIter(heapIter), + ) } diff --git a/pkg/bloomutils/ring_test.go b/pkg/bloomutils/ring_test.go index 30da072021edf..47ebb4766490f 100644 --- a/pkg/bloomutils/ring_test.go +++ b/pkg/bloomutils/ring_test.go @@ -1,25 +1,33 @@ package bloomutils import ( + "fmt" "math" "testing" "github.com/grafana/dskit/ring" "github.com/stretchr/testify/require" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" ) -func TestBloomGatewayClient_SortInstancesByToken(t *testing.T) { +func TestBloomGatewayClient_InstanceSortMergeIterator(t *testing.T) { + // | 0 1 2 3 4 5 6 7 8 9 | + // ---------+---------------------+ + // ID 1 | ***o ***o | + // ID 2 | ***o ***o | + // ID 3 | **o | input := []ring.InstanceDesc{ {Id: "1", Tokens: []uint32{5, 9}}, {Id: "2", Tokens: []uint32{3, 7}}, {Id: "3", Tokens: []uint32{1}}, } expected := []InstanceWithTokenRange{ - {Instance: input[2], MinToken: 0, MaxToken: 1}, - {Instance: input[1], MinToken: 2, MaxToken: 3}, - {Instance: input[0], MinToken: 4, MaxToken: 5}, - {Instance: input[1], MinToken: 6, MaxToken: 7}, - {Instance: input[0], MinToken: 8, MaxToken: 9}, + {Instance: input[2], TokenRange: NewTokenRange(0, 1)}, + {Instance: input[1], TokenRange: NewTokenRange(2, 3)}, + {Instance: input[0], TokenRange: NewTokenRange(4, 5)}, + {Instance: input[1], TokenRange: NewTokenRange(6, 7)}, + {Instance: input[0], TokenRange: NewTokenRange(8, 9)}, } var i int @@ -31,82 +39,38 @@ func TestBloomGatewayClient_SortInstancesByToken(t *testing.T) { } } -func TestBloomGatewayClient_GetInstancesWithTokenRanges(t *testing.T) { - t.Run("instance does not own first token in the ring", func(t *testing.T) { - input := []ring.InstanceDesc{ - {Id: "1", Tokens: []uint32{5, 9}}, - {Id: "2", Tokens: []uint32{3, 7}}, - {Id: "3", Tokens: []uint32{1}}, - } - expected := InstancesWithTokenRange{ - {Instance: input[1], MinToken: 2, MaxToken: 3}, - {Instance: input[1], MinToken: 6, MaxToken: 7}, - } - - result := GetInstancesWithTokenRanges("2", input) - require.Equal(t, expected, result) - }) - - t.Run("instance owns first token in the ring", func(t *testing.T) { - input := []ring.InstanceDesc{ - {Id: "1", Tokens: []uint32{5, 9}}, - {Id: "2", Tokens: []uint32{3, 7}}, - {Id: "3", Tokens: []uint32{1}}, - } - expected := InstancesWithTokenRange{ - {Instance: input[2], MinToken: 0, MaxToken: 1}, - {Instance: input[2], MinToken: 10, MaxToken: math.MaxUint32}, - } - - result := GetInstancesWithTokenRanges("3", input) - require.Equal(t, expected, result) - }) +func uint64Range(min, max uint64) Range[uint64] { + return Range[uint64]{min, max} } -func TestBloomGatewayClient_GetInstanceWithTokenRange(t *testing.T) { - for name, tc := range map[string]struct { - id string - input []ring.InstanceDesc - expected InstancesWithTokenRange +func TestKeyspacesFromTokenRanges(t *testing.T) { + for i, tc := range []struct { + tokenRanges ring.TokenRanges + exp []v1.FingerprintBounds }{ - "first instance includes 0 token": { - id: "3", - input: []ring.InstanceDesc{ - {Id: "1", Tokens: []uint32{3}}, - {Id: "2", Tokens: []uint32{5}}, - {Id: "3", Tokens: []uint32{1}}, - }, - expected: InstancesWithTokenRange{ - {Instance: ring.InstanceDesc{Id: "3", Tokens: []uint32{1}}, MinToken: 0, MaxToken: math.MaxUint32/3 - 1}, - }, - }, - "middle instance": { - id: "1", - input: []ring.InstanceDesc{ - {Id: "1", Tokens: []uint32{3}}, - {Id: "2", Tokens: []uint32{5}}, - {Id: "3", Tokens: []uint32{1}}, + { + tokenRanges: ring.TokenRanges{ + 0, math.MaxUint32 / 2, + math.MaxUint32/2 + 1, math.MaxUint32, }, - expected: InstancesWithTokenRange{ - {Instance: ring.InstanceDesc{Id: "1", Tokens: []uint32{3}}, MinToken: math.MaxUint32 / 3, MaxToken: math.MaxUint32/3*2 - 1}, + exp: []v1.FingerprintBounds{ + v1.NewBounds(0, math.MaxUint64/2), + v1.NewBounds(math.MaxUint64/2+1, math.MaxUint64), }, }, - "last instance includes MaxUint32 token": { - id: "2", - input: []ring.InstanceDesc{ - {Id: "1", Tokens: []uint32{3}}, - {Id: "2", Tokens: []uint32{5}}, - {Id: "3", Tokens: []uint32{1}}, + { + tokenRanges: ring.TokenRanges{ + 0, math.MaxUint8, + math.MaxUint16, math.MaxUint16 << 1, }, - expected: InstancesWithTokenRange{ - {Instance: ring.InstanceDesc{Id: "2", Tokens: []uint32{5}}, MinToken: math.MaxUint32 / 3 * 2, MaxToken: math.MaxUint32}, + exp: []v1.FingerprintBounds{ + v1.NewBounds(0, 0xff00000000|math.MaxUint32), + v1.NewBounds(math.MaxUint16<<32, math.MaxUint16<<33|math.MaxUint32), }, }, } { - tc := tc - t.Run(name, func(t *testing.T) { - result := GetInstanceWithTokenRange(tc.id, tc.input) - require.Equal(t, tc.expected, result) + t.Run(fmt.Sprint(i), func(t *testing.T) { + require.Equal(t, tc.exp, KeyspacesFromTokenRanges(tc.tokenRanges)) }) } } diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index a5229b0ca1498..f47148fa42b0d 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -52,7 +52,7 @@ import ( const ( ringKey = "distributor" - ringAutoForgetUnhealthyPeriods = 10 + ringAutoForgetUnhealthyPeriods = 2 ) var ( diff --git a/pkg/ingester/stream.go b/pkg/ingester/stream.go index 4c6aa4f9a122e..81ce436929251 100644 --- a/pkg/ingester/stream.go +++ b/pkg/ingester/stream.go @@ -288,30 +288,28 @@ func (s *stream) recordAndSendToTailers(record *wal.Record, entries []logproto.E hasTailers := len(s.tailers) != 0 s.tailerMtx.RUnlock() if hasTailers { - go func() { - stream := logproto.Stream{Labels: s.labelsString, Entries: entries} - - closedTailers := []uint32{} - - s.tailerMtx.RLock() - for _, tailer := range s.tailers { - if tailer.isClosed() { - closedTailers = append(closedTailers, tailer.getID()) - continue - } - tailer.send(stream, s.labels) + stream := logproto.Stream{Labels: s.labelsString, Entries: entries} + + closedTailers := []uint32{} + + s.tailerMtx.RLock() + for _, tailer := range s.tailers { + if tailer.isClosed() { + closedTailers = append(closedTailers, tailer.getID()) + continue } - s.tailerMtx.RUnlock() + tailer.send(stream, s.labels) + } + s.tailerMtx.RUnlock() - if len(closedTailers) != 0 { - s.tailerMtx.Lock() - defer s.tailerMtx.Unlock() + if len(closedTailers) != 0 { + s.tailerMtx.Lock() + defer s.tailerMtx.Unlock() - for _, closedTailerID := range closedTailers { - delete(s.tailers, closedTailerID) - } + for _, closedTailerID := range closedTailers { + delete(s.tailers, closedTailerID) } - }() + } } } diff --git a/pkg/ingester/tailer.go b/pkg/ingester/tailer.go index 3e9a8a64cfd88..25fdfdb740d7a 100644 --- a/pkg/ingester/tailer.go +++ b/pkg/ingester/tailer.go @@ -17,13 +17,21 @@ import ( util_log "github.com/grafana/loki/pkg/util/log" ) -const bufferSizeForTailResponse = 5 +const ( + bufferSizeForTailResponse = 5 + bufferSizeForTailStream = 100 +) type TailServer interface { Send(*logproto.TailResponse) error Context() context.Context } +type tailRequest struct { + stream logproto.Stream + lbs labels.Labels +} + type tailer struct { id uint32 orgID string @@ -31,6 +39,7 @@ type tailer struct { pipeline syntax.Pipeline pipelineMtx sync.Mutex + queue chan tailRequest sendChan chan *logproto.Stream // Signaling channel used to notify once the tailer gets closed @@ -59,6 +68,7 @@ func newTailer(orgID string, expr syntax.LogSelectorExpr, conn TailServer, maxDr orgID: orgID, matchers: matchers, sendChan: make(chan *logproto.Stream, bufferSizeForTailResponse), + queue: make(chan tailRequest, bufferSizeForTailStream), conn: conn, droppedStreams: make([]*logproto.DroppedStream, 0, maxDroppedStreams), maxDroppedStreams: maxDroppedStreams, @@ -73,6 +83,9 @@ func (t *tailer) loop() { var err error var ok bool + // Launch a go routine to receive streams sent with t.send + go t.receiveStreamsLoop() + for { select { case <-t.conn.Context().Done(): @@ -102,6 +115,37 @@ func (t *tailer) loop() { } } +func (t *tailer) receiveStreamsLoop() { + defer t.close() + for { + select { + case <-t.conn.Context().Done(): + return + case <-t.closeChan: + return + case req, ok := <-t.queue: + if !ok { + return + } + + streams := t.processStream(req.stream, req.lbs) + if len(streams) == 0 { + continue + } + + for _, s := range streams { + select { + case t.sendChan <- s: + default: + t.dropStream(*s) + } + } + } + } +} + +// send sends a stream to the tailer for processing and sending to the client. +// It will drop the stream if the tailer is blocked or the queue is full. func (t *tailer) send(stream logproto.Stream, lbs labels.Labels) { if t.isClosed() { return @@ -117,16 +161,16 @@ func (t *tailer) send(stream logproto.Stream, lbs labels.Labels) { return } - streams := t.processStream(stream, lbs) - if len(streams) == 0 { - return + // Send stream to queue for processing asynchronously + // If the queue is full, drop the stream + req := tailRequest{ + stream: stream, + lbs: lbs, } - for _, s := range streams { - select { - case t.sendChan <- s: - default: - t.dropStream(*s) - } + select { + case t.queue <- req: + default: + t.dropStream(stream) } } diff --git a/pkg/ingester/tailer_test.go b/pkg/ingester/tailer_test.go index 674dde3df8af0..11de0d4daf82c 100644 --- a/pkg/ingester/tailer_test.go +++ b/pkg/ingester/tailer_test.go @@ -2,6 +2,7 @@ package ingester import ( "context" + "fmt" "math/rand" "sync" "testing" @@ -15,6 +16,55 @@ import ( "github.com/grafana/loki/pkg/logql/syntax" ) +func TestTailer_RoundTrip(t *testing.T) { + server := &fakeTailServer{} + + lbs := makeRandomLabels() + expr, err := syntax.ParseLogSelector(lbs.String(), true) + require.NoError(t, err) + tail, err := newTailer("org-id", expr, server, 10) + require.NoError(t, err) + var wg sync.WaitGroup + wg.Add(1) + go func() { + tail.loop() + wg.Done() + }() + + const numStreams = 1000 + var entries []logproto.Entry + for i := 0; i < numStreams; i += 3 { + var iterEntries []logproto.Entry + for j := 0; j < 3; j++ { + iterEntries = append(iterEntries, logproto.Entry{Timestamp: time.Unix(0, int64(i+j)), Line: fmt.Sprintf("line %d", i+j)}) + } + entries = append(entries, iterEntries...) + + tail.send(logproto.Stream{ + Labels: lbs.String(), + Entries: iterEntries, + }, lbs) + + // sleep a bit to allow the tailer to process the stream without dropping + // This should take about 5 seconds to process all the streams + time.Sleep(5 * time.Millisecond) + } + + // Wait for the stream to be received by the server. + require.Eventually(t, func() bool { + return len(server.GetResponses()) > 0 + }, 30*time.Second, 1*time.Second, "stream was not received") + + var processedEntries []logproto.Entry + for _, response := range server.GetResponses() { + processedEntries = append(processedEntries, response.Stream.Entries...) + } + require.ElementsMatch(t, entries, processedEntries) + + tail.close() + wg.Wait() +} + func TestTailer_sendRaceConditionOnSendWhileClosing(t *testing.T) { runs := 100 diff --git a/pkg/logql/downstream.go b/pkg/logql/downstream.go index 33d945f11b923..6946c06e54a09 100644 --- a/pkg/logql/downstream.go +++ b/pkg/logql/downstream.go @@ -636,6 +636,10 @@ func NewResultStepEvaluator(res logqlmodel.Result, params Params) (StepEvaluator step = params.Step() ) + if res.Data == nil { + return nil, fmt.Errorf("data in the passed result is nil (res.Data), cannot be processed by stepevaluator") + } + switch data := res.Data.(type) { case promql.Vector: return NewVectorStepEvaluator(start, data), nil diff --git a/pkg/logql/downstream_test.go b/pkg/logql/downstream_test.go index ec5f3170468d0..46575c44d8edb 100644 --- a/pkg/logql/downstream_test.go +++ b/pkg/logql/downstream_test.go @@ -146,7 +146,7 @@ func TestMappingEquivalenceSketches(t *testing.T) { regular := NewEngine(opts, q, NoLimits, log.NewNopLogger()) sharded := NewDownstreamEngine(opts, MockDownstreamer{regular}, NoLimits, log.NewNopLogger()) - t.Run(tc.query, func(t *testing.T) { + t.Run(tc.query+"_range", func(t *testing.T) { params, err := NewLiteralParams( tc.query, start, @@ -178,6 +178,40 @@ func TestMappingEquivalenceSketches(t *testing.T) { relativeError(t, res.Data.(promql.Matrix), shardedRes.Data.(promql.Matrix), tc.realtiveError) }) + t.Run(tc.query+"_instant", func(t *testing.T) { + // for an instant query we set the start and end to the same timestamp + // plus set step and interval to 0 + params, err := NewLiteralParams( + tc.query, + time.Unix(0, int64(rounds+1)), + time.Unix(0, int64(rounds+1)), + 0, + 0, + logproto.FORWARD, + uint32(limit), + nil, + ) + require.NoError(t, err) + qry := regular.Query(params) + ctx := user.InjectOrgID(context.Background(), "fake") + + mapper := NewShardMapper(ConstantShards(shards), nilShardMetrics, []string{ShardQuantileOverTime}) + _, _, mapped, err := mapper.Parse(params.GetExpression()) + require.NoError(t, err) + + shardedQry := sharded.Query(ctx, ParamsWithExpressionOverride{ + Params: params, + ExpressionOverride: mapped, + }) + + res, err := qry.Exec(ctx) + require.NoError(t, err) + + shardedRes, err := shardedQry.Exec(ctx) + require.NoError(t, err) + + relativeErrorVector(t, res.Data.(promql.Vector), shardedRes.Data.(promql.Vector), tc.realtiveError) + }) } } @@ -546,6 +580,21 @@ func relativeError(t *testing.T, expected, actual promql.Matrix, alpha float64) } } +func relativeErrorVector(t *testing.T, expected, actual promql.Vector, alpha float64) { + require.Len(t, actual, len(expected)) + + e := make([]float64, len(expected)) + a := make([]float64, len(expected)) + for i := 0; i < len(expected); i++ { + require.Equal(t, expected[i].Metric, actual[i].Metric) + + e[i] = expected[i].F + a[i] = expected[i].F + } + require.InEpsilonSlice(t, e, a, alpha) + +} + func TestFormat_ShardedExpr(t *testing.T) { oldMax := syntax.MaxCharsPerLine syntax.MaxCharsPerLine = 20 diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index 8d951ad64c946..a9f3dabe14eed 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -363,7 +363,7 @@ func (q *query) evalSample(ctx context.Context, expr syntax.SampleExpr) (promql_ maxSeries := validation.SmallestPositiveIntPerTenant(tenantIDs, maxSeriesCapture) return q.JoinSampleVector(next, ts, vec, stepEvaluator, maxSeries) case ProbabilisticQuantileVector: - return JoinQuantileSketchVector(next, vec, stepEvaluator, q.params) + return MergeQuantileSketchVector(next, vec, stepEvaluator, q.params) default: return nil, fmt.Errorf("unsupported result type: %T", r) } diff --git a/pkg/logql/metrics.go b/pkg/logql/metrics.go index 40fbece82d87d..f9b9d1a8f952b 100644 --- a/pkg/logql/metrics.go +++ b/pkg/logql/metrics.go @@ -94,7 +94,8 @@ func RecordRangeAndInstantQueryMetrics( ) { var ( logger = fixLogger(ctx, log) - rt = string(GetRangeType(p)) + rangeType = GetRangeType(p) + rt = string(rangeType) latencyType = latencyTypeFast returnedLines = 0 ) @@ -103,6 +104,12 @@ func RecordRangeAndInstantQueryMetrics( level.Warn(logger).Log("msg", "error parsing query type", "err", err) } + resultCache := stats.Caches.Result + + if queryType == QueryTypeMetric && rangeType == InstantType { + resultCache = stats.Caches.InstantMetricResult + } + // Tag throughput metric by latency type based on a threshold. // Latency below the threshold is fast, above is slow. if stats.Summary.ExecTime > slowQueryThresholdSecond { @@ -135,9 +142,9 @@ func RecordRangeAndInstantQueryMetrics( "status", status, "limit", p.Limit(), "returned_lines", returnedLines, - "throughput", strings.Replace(humanize.Bytes(uint64(stats.Summary.BytesProcessedPerSecond)), " ", "", 1), - "total_bytes", strings.Replace(humanize.Bytes(uint64(stats.Summary.TotalBytesProcessed)), " ", "", 1), - "total_bytes_structured_metadata", strings.Replace(humanize.Bytes(uint64(stats.Summary.TotalStructuredMetadataBytesProcessed)), " ", "", 1), + "throughput", humanizeBytes(uint64(stats.Summary.BytesProcessedPerSecond)), + "total_bytes", humanizeBytes(uint64(stats.Summary.TotalBytesProcessed)), + "total_bytes_structured_metadata", humanizeBytes(uint64(stats.Summary.TotalStructuredMetadataBytesProcessed)), "lines_per_second", stats.Summary.LinesProcessedPerSecond, "total_lines", stats.Summary.TotalLinesProcessed, "post_filter_lines", stats.Summary.TotalPostFilterLines, @@ -162,10 +169,26 @@ func RecordRangeAndInstantQueryMetrics( "cache_volume_results_req", stats.Caches.VolumeResult.EntriesRequested, "cache_volume_results_hit", stats.Caches.VolumeResult.EntriesFound, "cache_volume_results_download_time", stats.Caches.VolumeResult.CacheDownloadTime(), - "cache_result_req", stats.Caches.Result.EntriesRequested, - "cache_result_hit", stats.Caches.Result.EntriesFound, - "cache_result_download_time", stats.Caches.Result.CacheDownloadTime(), - "cache_result_query_length_served", stats.Caches.Result.CacheQueryLengthServed(), + "cache_result_req", resultCache.EntriesRequested, + "cache_result_hit", resultCache.EntriesFound, + "cache_result_download_time", resultCache.CacheDownloadTime(), + "cache_result_query_length_served", resultCache.CacheQueryLengthServed(), + // The total of chunk reference fetched from index. + "ingester_chunk_refs", stats.Ingester.Store.GetTotalChunksRef(), + // Total number of chunks fetched. + "ingester_chunk_downloaded", stats.Ingester.Store.GetTotalChunksDownloaded(), + // Total of chunks matched by the query from ingesters. + "ingester_chunk_matches", stats.Ingester.GetTotalChunksMatched(), + // Total ingester reached for this query. + "ingester_requests", stats.Ingester.GetTotalReached(), + // Total bytes processed but was already in memory (found in the headchunk). Includes structured metadata bytes. + "ingester_chunk_head_bytes", humanizeBytes(uint64(stats.Ingester.Store.Chunk.GetHeadChunkBytes())), + // Total bytes of compressed chunks (blocks) processed. + "ingester_chunk_compressed_bytes", humanizeBytes(uint64(stats.Ingester.Store.Chunk.GetCompressedBytes())), + // Total bytes decompressed and processed from chunks. Includes structured metadata bytes. + "ingester_chunk_decompressed_bytes", humanizeBytes(uint64(stats.Ingester.Store.Chunk.GetDecompressedBytes())), + // Total lines post filtering. + "ingester_post_filter_lines", stats.Ingester.Store.Chunk.GetPostFilterLines(), }...) logValues = append(logValues, tagsToKeyValues(queryTags)...) @@ -193,6 +216,10 @@ func RecordRangeAndInstantQueryMetrics( recordUsageStats(queryType, stats) } +func humanizeBytes(val uint64) string { + return strings.Replace(humanize.Bytes(val), " ", "", 1) +} + func RecordLabelQueryMetrics( ctx context.Context, log log.Logger, diff --git a/pkg/logql/quantile_over_time_sketch.go b/pkg/logql/quantile_over_time_sketch.go index 507c72b208ab8..24a8a05d89ede 100644 --- a/pkg/logql/quantile_over_time_sketch.go +++ b/pkg/logql/quantile_over_time_sketch.go @@ -262,13 +262,17 @@ func (r *quantileSketchBatchRangeVectorIterator) agg(samples []promql.FPoint) sk return s } -// JoinQuantileSketchVector joins the results from stepEvaluator into a ProbabilisticQuantileMatrix. -func JoinQuantileSketchVector(next bool, r StepResult, stepEvaluator StepEvaluator, params Params) (promql_parser.Value, error) { +// MergeQuantileSketchVector joins the results from stepEvaluator into a ProbabilisticQuantileMatrix. +func MergeQuantileSketchVector(next bool, r StepResult, stepEvaluator StepEvaluator, params Params) (promql_parser.Value, error) { vec := r.QuantileSketchVec() if stepEvaluator.Error() != nil { return nil, stepEvaluator.Error() } + if GetRangeType(params) == InstantType { + return ProbabilisticQuantileMatrix{vec}, nil + } + stepCount := int(math.Ceil(float64(params.End().Sub(params.Start()).Nanoseconds()) / float64(params.Step().Nanoseconds()))) if stepCount <= 0 { stepCount = 1 diff --git a/pkg/logql/quantile_over_time_sketch_test.go b/pkg/logql/quantile_over_time_sketch_test.go index dc1ff31f509a4..488ebdec26f06 100644 --- a/pkg/logql/quantile_over_time_sketch_test.go +++ b/pkg/logql/quantile_over_time_sketch_test.go @@ -69,7 +69,7 @@ func TestJoinQuantileSketchVectorError(t *testing.T) { ev := errorStepEvaluator{ err: errors.New("could not evaluate"), } - _, err := JoinQuantileSketchVector(true, result, ev, LiteralParams{}) + _, err := MergeQuantileSketchVector(true, result, ev, LiteralParams{}) require.ErrorContains(t, err, "could not evaluate") } @@ -136,7 +136,7 @@ func BenchmarkJoinQuantileSketchVector(b *testing.B) { iter: iter, } _, _, r := ev.Next() - m, err := JoinQuantileSketchVector(true, r.QuantileSketchVec(), ev, params) + m, err := MergeQuantileSketchVector(true, r.QuantileSketchVec(), ev, params) require.NoError(b, err) m.(ProbabilisticQuantileMatrix).Release() } @@ -148,7 +148,9 @@ func BenchmarkQuantileBatchRangeVectorIteratorAt(b *testing.B) { }{ {numberSamples: 1}, {numberSamples: 1_000}, + {numberSamples: 10_000}, {numberSamples: 100_000}, + {numberSamples: 1_000_000}, } { b.Run(fmt.Sprintf("%d-samples", tc.numberSamples), func(b *testing.B) { r := rand.New(rand.NewSource(42)) diff --git a/pkg/logql/rangemapper.go b/pkg/logql/rangemapper.go index 975f63f4c9523..14cf76f1475a5 100644 --- a/pkg/logql/rangemapper.go +++ b/pkg/logql/rangemapper.go @@ -57,6 +57,20 @@ type RangeMapper struct { splitByInterval time.Duration metrics *MapperMetrics stats *MapperStats + + splitAlignTs time.Time +} + +// NewRangeMapperWithSplitAlign is similar to `NewRangeMapper` except it accepts additonal `splitAlign` argument and used to +// align the subqueries generated according to that. Look at `rangeSplitAlign` method for more information. +func NewRangeMapperWithSplitAlign(interval time.Duration, splitAlign time.Time, metrics *MapperMetrics, stats *MapperStats) (RangeMapper, error) { + rm, err := NewRangeMapper(interval, metrics, stats) + if err != nil { + return RangeMapper{}, err + } + rm.splitAlignTs = splitAlign + + return rm, nil } // NewRangeMapper creates a new RangeMapper instance with the given duration as @@ -327,6 +341,77 @@ func (m RangeMapper) getOriginalOffset(expr syntax.SampleExpr) (offset time.Dura // rangeInterval should be greater than m.splitByInterval, otherwise the resultant expression // will have an unnecessary aggregation operation func (m RangeMapper) mapConcatSampleExpr(expr syntax.SampleExpr, rangeInterval time.Duration, recorder *downstreamRecorder) syntax.SampleExpr { + if m.splitAlignTs.IsZero() { + return m.rangeSplit(expr, rangeInterval, recorder) + } + return m.rangeSplitAlign(expr, rangeInterval, recorder) +} + +// rangeSplitAlign try to split given `rangeInterval` into units of `m.splitByInterval` by making sure `rangeInterval` is aligned with `m.splitByInterval` for as much as the units as possible. +// Consider following example with real use case. +// Instant Query: `sum(rate({foo="bar"}[3h])` +// execTs: 12:34:00 +// splitBy: 1h +// Given above parameters, queries will be split into following +// 1. sum(rate({foo="bar"}[34m])) +// 2. sum(rate({foo="bar"}[1h] offset 34m)) +// 3. sum(rate({foo="bar"}[1h] offset 1h34m)) +// 4. sum(rate({foo="bar"}[26m] offset 2h34m)) +func (m RangeMapper) rangeSplitAlign( + expr syntax.SampleExpr, rangeInterval time.Duration, recorder *downstreamRecorder, +) syntax.SampleExpr { + if rangeInterval <= m.splitByInterval { + return expr + } + + originalOffset, err := m.getOriginalOffset(expr) + if err != nil { + return expr + } + + align := m.splitAlignTs.Sub(m.splitAlignTs.Truncate(m.splitByInterval)) // say, 12:34:00 - 12:00:00(truncated) = 34m + + if align == 0 { + return m.rangeSplit(expr, rangeInterval, recorder) // Don't have to align + } + + var ( + newRng = align + + // TODO(kavi): If the originalOffset is non-zero, there may be a edge case, where subqueries generated won't be aligned correctly. Handle this edge case in separate PR. + newOffset = originalOffset + downstreams *ConcatSampleExpr + pendingRangeInterval = rangeInterval + splits = 0 + ) + + // first subquery + downstreams = appendDownstream(downstreams, expr, newRng, newOffset) + splits++ + + newOffset += align // e.g: offset 34m + pendingRangeInterval -= newRng + newRng = m.splitByInterval // [1h] + + // Rest of the subqueries. + for pendingRangeInterval > 0 { + if pendingRangeInterval < m.splitByInterval { + newRng = pendingRangeInterval // last subquery + } + downstreams = appendDownstream(downstreams, expr, newRng, newOffset) + newOffset += m.splitByInterval + pendingRangeInterval -= newRng + splits++ + } + + // update stats and metrics + m.stats.AddSplitQueries(splits) + recorder.Add(splits, MetricsKey) + + return downstreams +} + +func (m RangeMapper) rangeSplit(expr syntax.SampleExpr, rangeInterval time.Duration, recorder *downstreamRecorder) syntax.SampleExpr { splitCount := int(math.Ceil(float64(rangeInterval) / float64(m.splitByInterval))) if splitCount <= 1 { return expr diff --git a/pkg/logql/rangemapper_test.go b/pkg/logql/rangemapper_test.go index 562ac0cd168e9..5e95486a8c8e2 100644 --- a/pkg/logql/rangemapper_test.go +++ b/pkg/logql/rangemapper_test.go @@ -93,6 +93,84 @@ func Test_SplitRangeInterval(t *testing.T) { } } +func Test_RangeMapperSplitAlign(t *testing.T) { + cases := []struct { + name string + expr string + queryTime time.Time + splityByInterval time.Duration + expected string + expectedSplits int + }{ + { + name: "query_time_aligned_with_split_by", + expr: `bytes_over_time({app="foo"}[3m])`, + expected: `sum without() ( + downstream> + ++ downstream> + ++ downstream> + )`, + queryTime: time.Unix(60, 0), // 1970 00:01:00 + splityByInterval: 1 * time.Minute, + expectedSplits: 3, + }, + { + name: "query_time_aligned_with_split_by_with_original_offset", + expr: `bytes_over_time({app="foo"}[3m] offset 20m10s)`, // NOTE: original query has offset, which should be considered in all the splits subquery + expected: `sum without() ( + downstream> + ++ downstream> + ++ downstream> + )`, + queryTime: time.Unix(60, 0), // 1970 00:01:00 + splityByInterval: 1 * time.Minute, + expectedSplits: 3, + }, + { + name: "query_time_not_aligned_with_split_by", + expr: `bytes_over_time({app="foo"}[3h])`, + expected: `sum without() ( + downstream> + ++ downstream> + ++ downstream> + ++ downstream> + )`, + queryTime: time.Date(0, 0, 0, 12, 54, 0, 0, time.UTC), // 1970 12:54:00 + splityByInterval: 1 * time.Hour, + expectedSplits: 4, + }, + { + name: "query_time_not_aligned_with_split_by_with_original_offset", + expr: `bytes_over_time({app="foo"}[3h] offset 1h2m20s)`, // NOTE: original query has offset, which should be considered in all the splits subquery + expected: `sum without() ( + downstream> + ++ downstream> + ++ downstream> + ++ downstream> + )`, + queryTime: time.Date(0, 0, 0, 12, 54, 0, 0, time.UTC), // 1970 12:54:00 + splityByInterval: 1 * time.Hour, + expectedSplits: 4, + }, + } + + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + mapperStats := NewMapperStats() + rvm, err := NewRangeMapperWithSplitAlign(tc.splityByInterval, tc.queryTime, nilShardMetrics, mapperStats) + require.NoError(t, err) + + noop, mappedExpr, err := rvm.Parse(syntax.MustParseExpr(tc.expr)) + require.NoError(t, err) + + require.Equal(t, removeWhiteSpace(tc.expected), removeWhiteSpace(mappedExpr.String())) + require.Equal(t, tc.expectedSplits, mapperStats.GetSplitQueries()) + require.False(t, noop) + + }) + } +} + func Test_SplitRangeVectorMapping(t *testing.T) { for _, tc := range []struct { expr string @@ -1675,7 +1753,7 @@ func Test_SplitRangeVectorMapping(t *testing.T) { // Non-splittable vector aggregators - should go deeper in the AST { `topk(2, count_over_time({app="foo"}[3m]))`, - `topk(2, + `topk(2, sum without () ( downstream> ++ downstream> @@ -1713,7 +1791,7 @@ func Test_SplitRangeVectorMapping(t *testing.T) { ++ downstream> ++ downstream> ) - ), + ), "x", "$1", "a", "(.*)" )`, 3, @@ -1727,7 +1805,7 @@ func Test_SplitRangeVectorMapping(t *testing.T) { ++ downstream> ++ downstream> ) - / 180), + / 180), "foo", "$1", "service", "(.*):.*" )`, 3, diff --git a/pkg/logql/sketch/quantile.go b/pkg/logql/sketch/quantile.go index 1fa20c38e5bcc..3b8b0f22fc8e0 100644 --- a/pkg/logql/sketch/quantile.go +++ b/pkg/logql/sketch/quantile.go @@ -47,7 +47,7 @@ const relativeAccuracy = 0.01 var ddsketchPool = sync.Pool{ New: func() any { m, _ := mapping.NewCubicallyInterpolatedMapping(relativeAccuracy) - return ddsketch.NewDDSketchFromStoreProvider(m, store.SparseStoreConstructor) + return ddsketch.NewDDSketch(m, store.NewCollapsingLowestDenseStore(2048), store.NewCollapsingLowestDenseStore(2048)) }, } diff --git a/pkg/logqlmodel/stats/context.go b/pkg/logqlmodel/stats/context.go index 4fbddc790b8b2..1271fa6d9c210 100644 --- a/pkg/logqlmodel/stats/context.go +++ b/pkg/logqlmodel/stats/context.go @@ -55,17 +55,18 @@ type Context struct { type CacheType string const ( - ChunkCache CacheType = "chunk" //nolint:staticcheck - IndexCache CacheType = "index" //nolint:staticcheck - ResultCache CacheType = "result" //nolint:staticcheck - StatsResultCache CacheType = "stats-result" //nolint:staticcheck - VolumeResultCache CacheType = "volume-result" //nolint:staticcheck - WriteDedupeCache CacheType = "write-dedupe" //nolint:staticcheck - SeriesResultCache CacheType = "series-result" //nolint:staticcheck - LabelResultCache CacheType = "label-result" //nolint:staticcheck - BloomFilterCache CacheType = "bloom-filter" //nolint:staticcheck - BloomBlocksCache CacheType = "bloom-blocks" //nolint:staticcheck - BloomMetasCache CacheType = "bloom-metas" //nolint:staticcheck + ChunkCache CacheType = "chunk" //nolint:staticcheck + IndexCache CacheType = "index" //nolint:staticcheck + ResultCache CacheType = "result" //nolint:staticcheck + StatsResultCache CacheType = "stats-result" //nolint:staticcheck + VolumeResultCache CacheType = "volume-result" //nolint:staticcheck + InstantMetricResultsCache CacheType = "instant-metric-result" // nolint:staticcheck + WriteDedupeCache CacheType = "write-dedupe" //nolint:staticcheck + SeriesResultCache CacheType = "series-result" //nolint:staticcheck + LabelResultCache CacheType = "label-result" //nolint:staticcheck + BloomFilterCache CacheType = "bloom-filter" //nolint:staticcheck + BloomBlocksCache CacheType = "bloom-blocks" //nolint:staticcheck + BloomMetasCache CacheType = "bloom-metas" //nolint:staticcheck ) // NewContext creates a new statistics context @@ -98,13 +99,14 @@ func (c *Context) Ingester() Ingester { // Caches returns the cache statistics accumulated so far. func (c *Context) Caches() Caches { return Caches{ - Chunk: c.caches.Chunk, - Index: c.caches.Index, - Result: c.caches.Result, - StatsResult: c.caches.StatsResult, - VolumeResult: c.caches.VolumeResult, - SeriesResult: c.caches.SeriesResult, - LabelResult: c.caches.LabelResult, + Chunk: c.caches.Chunk, + Index: c.caches.Index, + Result: c.caches.Result, + StatsResult: c.caches.StatsResult, + VolumeResult: c.caches.VolumeResult, + SeriesResult: c.caches.SeriesResult, + LabelResult: c.caches.LabelResult, + InstantMetricResult: c.caches.InstantMetricResult, } } @@ -197,6 +199,10 @@ func (s *Store) Merge(m Store) { } } +func (s *Store) ChunksDownloadDuration() time.Duration { + return time.Duration(s.GetChunksDownloadTime()) +} + func (s *Summary) Merge(m Summary) { s.Splits += m.Splits s.Shards += m.Shards @@ -222,6 +228,7 @@ func (c *Caches) Merge(m Caches) { c.VolumeResult.Merge(m.VolumeResult) c.SeriesResult.Merge(m.SeriesResult) c.LabelResult.Merge(m.LabelResult) + c.InstantMetricResult.Merge(m.InstantMetricResult) } func (c *Cache) Merge(m Cache) { @@ -470,6 +477,8 @@ func (c *Context) getCacheStatsByType(t CacheType) *Cache { stats = &c.caches.SeriesResult case LabelResultCache: stats = &c.caches.LabelResult + case InstantMetricResultsCache: + stats = &c.caches.InstantMetricResult default: return nil } @@ -571,6 +580,12 @@ func (c Caches) Log(log log.Logger) { "Cache.Result.EntriesStored", c.Result.EntriesStored, "Cache.Result.BytesSent", humanize.Bytes(uint64(c.Result.BytesSent)), "Cache.Result.BytesReceived", humanize.Bytes(uint64(c.Result.BytesReceived)), - "Cache.Result.DownloadTime", c.Result.CacheDownloadTime(), + "Cache.InstantMetricResult.Requests", c.InstantMetricResult.Requests, + "Cache.InstantMetricResult.EntriesRequested", c.InstantMetricResult.EntriesRequested, + "Cache.InstantMetricResult.EntriesFound", c.InstantMetricResult.EntriesFound, + "Cache.InstantMetricResult.EntriesStored", c.InstantMetricResult.EntriesStored, + "Cache.InstantMetricResult.BytesSent", humanize.Bytes(uint64(c.InstantMetricResult.BytesSent)), + "Cache.InstantMetricResult.BytesReceived", humanize.Bytes(uint64(c.InstantMetricResult.BytesReceived)), + "Cache.InstantMetricResult.DownloadTime", c.InstantMetricResult.CacheDownloadTime(), ) } diff --git a/pkg/logqlmodel/stats/stats.pb.go b/pkg/logqlmodel/stats/stats.pb.go index 75be704020c97..65f8f0f642381 100644 --- a/pkg/logqlmodel/stats/stats.pb.go +++ b/pkg/logqlmodel/stats/stats.pb.go @@ -95,13 +95,14 @@ func (m *Result) GetCaches() Caches { } type Caches struct { - Chunk Cache `protobuf:"bytes,1,opt,name=chunk,proto3" json:"chunk"` - Index Cache `protobuf:"bytes,2,opt,name=index,proto3" json:"index"` - Result Cache `protobuf:"bytes,3,opt,name=result,proto3" json:"result"` - StatsResult Cache `protobuf:"bytes,4,opt,name=statsResult,proto3" json:"statsResult"` - VolumeResult Cache `protobuf:"bytes,5,opt,name=volumeResult,proto3" json:"volumeResult"` - SeriesResult Cache `protobuf:"bytes,6,opt,name=seriesResult,proto3" json:"seriesResult"` - LabelResult Cache `protobuf:"bytes,7,opt,name=labelResult,proto3" json:"labelResult"` + Chunk Cache `protobuf:"bytes,1,opt,name=chunk,proto3" json:"chunk"` + Index Cache `protobuf:"bytes,2,opt,name=index,proto3" json:"index"` + Result Cache `protobuf:"bytes,3,opt,name=result,proto3" json:"result"` + StatsResult Cache `protobuf:"bytes,4,opt,name=statsResult,proto3" json:"statsResult"` + VolumeResult Cache `protobuf:"bytes,5,opt,name=volumeResult,proto3" json:"volumeResult"` + SeriesResult Cache `protobuf:"bytes,6,opt,name=seriesResult,proto3" json:"seriesResult"` + LabelResult Cache `protobuf:"bytes,7,opt,name=labelResult,proto3" json:"labelResult"` + InstantMetricResult Cache `protobuf:"bytes,8,opt,name=instantMetricResult,proto3" json:"instantMetricResult"` } func (m *Caches) Reset() { *m = Caches{} } @@ -185,6 +186,13 @@ func (m *Caches) GetLabelResult() Cache { return Cache{} } +func (m *Caches) GetInstantMetricResult() Cache { + if m != nil { + return m.InstantMetricResult + } + return Cache{} +} + // Summary is the summary of a query statistics. type Summary struct { // Total bytes processed per second. @@ -773,83 +781,85 @@ func init() { func init() { proto.RegisterFile("pkg/logqlmodel/stats/stats.proto", fileDescriptor_6cdfe5d2aea33ebb) } var fileDescriptor_6cdfe5d2aea33ebb = []byte{ - // 1215 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x57, 0x4d, 0x6f, 0xe3, 0x54, - 0x17, 0x8e, 0x27, 0xaf, 0x93, 0xce, 0xed, 0xe7, 0xdc, 0x76, 0xde, 0xc9, 0x80, 0x64, 0x97, 0xc0, - 0x88, 0x22, 0x50, 0x23, 0x3e, 0x24, 0x04, 0x62, 0x24, 0xe4, 0x0e, 0x95, 0x2a, 0x75, 0x44, 0x39, - 0x81, 0x0d, 0x3b, 0xc7, 0xbe, 0x4d, 0xa2, 0x3a, 0x76, 0x6a, 0x5f, 0x97, 0xe9, 0x0a, 0x7e, 0x02, - 0x3f, 0x83, 0x0d, 0x2b, 0x56, 0x48, 0x88, 0x0d, 0x9b, 0x59, 0x76, 0x39, 0x2b, 0x8b, 0xa6, 0x1b, - 0xe4, 0xd5, 0x48, 0xfc, 0x01, 0x74, 0xcf, 0xbd, 0xf1, 0x57, 0x9c, 0x99, 0x6e, 0xe2, 0x7b, 0x9e, - 0xf3, 0x3c, 0xe7, 0x7e, 0x9e, 0x73, 0x6f, 0xc8, 0xee, 0xf4, 0x6c, 0xd8, 0xf3, 0x82, 0xe1, 0xb9, - 0x37, 0x09, 0x5c, 0xe6, 0xf5, 0x22, 0x6e, 0xf3, 0x48, 0xfe, 0xee, 0x4f, 0xc3, 0x80, 0x07, 0x54, - 0x47, 0xe3, 0x8d, 0x9d, 0x61, 0x30, 0x0c, 0x10, 0xe9, 0x89, 0x96, 0x74, 0x76, 0xff, 0xd5, 0x48, - 0x0b, 0x58, 0x14, 0x7b, 0x9c, 0x7e, 0x46, 0xda, 0x51, 0x3c, 0x99, 0xd8, 0xe1, 0x65, 0x47, 0xdb, - 0xd5, 0xf6, 0x56, 0x3f, 0xda, 0xd8, 0x97, 0x61, 0xfa, 0x12, 0xb5, 0x36, 0x9f, 0x27, 0x66, 0x23, - 0x4d, 0xcc, 0x39, 0x0d, 0xe6, 0x0d, 0x21, 0x3d, 0x8f, 0x59, 0x38, 0x66, 0x61, 0xe7, 0x4e, 0x49, - 0xfa, 0x8d, 0x44, 0x73, 0xa9, 0xa2, 0xc1, 0xbc, 0x41, 0x1f, 0x93, 0x95, 0xb1, 0x3f, 0x64, 0x11, - 0x67, 0x61, 0xa7, 0x89, 0xda, 0x4d, 0xa5, 0x3d, 0x52, 0xb0, 0xb5, 0xa5, 0xc4, 0x19, 0x11, 0xb2, - 0x16, 0xfd, 0x84, 0xb4, 0x1c, 0xdb, 0x19, 0xb1, 0xa8, 0xf3, 0x3f, 0x14, 0xaf, 0x2b, 0xf1, 0x01, - 0x82, 0xd6, 0xba, 0x92, 0xea, 0x48, 0x02, 0xc5, 0xed, 0xfe, 0xd9, 0x24, 0x2d, 0xc9, 0xa0, 0x1f, - 0x12, 0xdd, 0x19, 0xc5, 0xfe, 0x99, 0x9a, 0xf3, 0x5a, 0x51, 0x5f, 0x90, 0x0b, 0x0a, 0xc8, 0x8f, - 0x90, 0x8c, 0x7d, 0x97, 0x3d, 0x53, 0x73, 0x5d, 0x22, 0x41, 0x0a, 0xc8, 0x8f, 0x18, 0x66, 0x88, - 0xab, 0xac, 0xe6, 0x58, 0xd6, 0x6c, 0x28, 0x8d, 0xe2, 0x80, 0xfa, 0xd2, 0x03, 0xb2, 0x8a, 0x34, - 0xb9, 0x41, 0x6a, 0x86, 0x65, 0xe9, 0xb6, 0x92, 0x16, 0x89, 0x50, 0x34, 0xe8, 0x21, 0x59, 0xbb, - 0x08, 0xbc, 0x78, 0xc2, 0x54, 0x14, 0xbd, 0x26, 0xca, 0x8e, 0x8a, 0x52, 0x62, 0x42, 0xc9, 0x12, - 0x71, 0x22, 0xb1, 0x65, 0xf3, 0xd1, 0xb4, 0x5e, 0x15, 0xa7, 0xc8, 0x84, 0x92, 0x25, 0x26, 0xe5, - 0xd9, 0x03, 0xe6, 0xa9, 0x30, 0xed, 0x57, 0x4d, 0xaa, 0x40, 0x84, 0xa2, 0xd1, 0xfd, 0xbd, 0x45, - 0xda, 0xea, 0x58, 0xd2, 0xef, 0xc8, 0x83, 0xc1, 0x25, 0x67, 0xd1, 0x49, 0x18, 0x38, 0x2c, 0x8a, - 0x98, 0x7b, 0xc2, 0xc2, 0x3e, 0x73, 0x02, 0xdf, 0xc5, 0x3d, 0x6d, 0x5a, 0x6f, 0xa6, 0x89, 0xb9, - 0x8c, 0x02, 0xcb, 0x1c, 0x22, 0xac, 0x37, 0xf6, 0x6b, 0xc3, 0xde, 0xc9, 0xc3, 0x2e, 0xa1, 0xc0, - 0x32, 0x07, 0x3d, 0x22, 0xdb, 0x3c, 0xe0, 0xb6, 0x67, 0x95, 0xba, 0xc5, 0x63, 0xd1, 0xb4, 0x1e, - 0xa4, 0x89, 0x59, 0xe7, 0x86, 0x3a, 0x30, 0x0b, 0x75, 0x5c, 0xea, 0x0a, 0x8f, 0x49, 0x31, 0x54, - 0xd9, 0x0d, 0x75, 0x20, 0xdd, 0x23, 0x2b, 0xec, 0x19, 0x73, 0xbe, 0x1d, 0x4f, 0x18, 0x1e, 0x10, - 0xcd, 0x5a, 0x13, 0x09, 0x37, 0xc7, 0x20, 0x6b, 0xd1, 0xf7, 0xc9, 0xdd, 0xf3, 0x98, 0xc5, 0x0c, - 0xa9, 0x2d, 0xa4, 0xae, 0xa7, 0x89, 0x99, 0x83, 0x90, 0x37, 0xe9, 0x3e, 0x21, 0x51, 0x3c, 0x90, - 0xa9, 0x1e, 0xe1, 0x56, 0x37, 0xad, 0x8d, 0x34, 0x31, 0x0b, 0x28, 0x14, 0xda, 0xf4, 0x98, 0xec, - 0xe0, 0xe8, 0xbe, 0xf2, 0xb9, 0x3c, 0x31, 0x3c, 0x0e, 0x7d, 0xe6, 0x76, 0x56, 0x50, 0xd9, 0x49, - 0x13, 0xb3, 0xd6, 0x0f, 0xb5, 0x28, 0xed, 0x92, 0x56, 0x34, 0xf5, 0xc6, 0x3c, 0xea, 0xdc, 0x45, - 0x3d, 0x11, 0x29, 0x26, 0x11, 0x50, 0x5f, 0xe4, 0x8c, 0xec, 0xd0, 0x8d, 0x3a, 0xa4, 0xc0, 0x41, - 0x04, 0xd4, 0x37, 0x1b, 0xd5, 0x49, 0x10, 0xf1, 0xc3, 0xb1, 0xc7, 0x59, 0x88, 0xab, 0xd7, 0x59, - 0xad, 0x8c, 0xaa, 0xe2, 0x87, 0x5a, 0x94, 0xfe, 0x48, 0x1e, 0x21, 0xde, 0xe7, 0x61, 0xec, 0xf0, - 0x38, 0x64, 0xee, 0x53, 0xc6, 0x6d, 0xd7, 0xe6, 0x76, 0xe5, 0x48, 0xac, 0x61, 0xf8, 0xf7, 0xd2, - 0xc4, 0xbc, 0x9d, 0x00, 0x6e, 0x47, 0xeb, 0x7e, 0x41, 0xda, 0xaa, 0x2c, 0x8b, 0x4a, 0x16, 0xf1, - 0x20, 0x64, 0x95, 0xe2, 0xd7, 0x17, 0x58, 0x5e, 0xc9, 0x90, 0x02, 0xf2, 0xd3, 0xfd, 0xf5, 0x0e, - 0x59, 0x39, 0xca, 0xab, 0xef, 0x1a, 0xf6, 0x09, 0x4c, 0xe4, 0xad, 0xcc, 0x37, 0xdd, 0xda, 0x12, - 0x15, 0xa0, 0x88, 0x43, 0xc9, 0xa2, 0x87, 0x84, 0xa2, 0x7d, 0x20, 0xaa, 0x69, 0xf4, 0xd4, 0xe6, - 0xa8, 0x95, 0x49, 0xf5, 0xff, 0x34, 0x31, 0x6b, 0xbc, 0x50, 0x83, 0x65, 0xbd, 0x5b, 0x68, 0x47, - 0x2a, 0x87, 0xf2, 0xde, 0x15, 0x0e, 0x25, 0x8b, 0x7e, 0x4e, 0x36, 0xf2, 0x0c, 0xe8, 0x33, 0x9f, - 0xab, 0x84, 0xa1, 0x69, 0x62, 0x56, 0x3c, 0x50, 0xb1, 0xf3, 0xf5, 0xd2, 0x6f, 0xbd, 0x5e, 0x7f, - 0x34, 0x89, 0x8e, 0xfe, 0xac, 0x63, 0x39, 0x09, 0x60, 0xa7, 0xaa, 0x3c, 0xe5, 0x1d, 0x67, 0x1e, - 0xa8, 0xd8, 0xf4, 0x6b, 0x72, 0xbf, 0x80, 0x3c, 0x09, 0x7e, 0xf0, 0xbd, 0xc0, 0x76, 0xb3, 0x55, - 0x7b, 0x98, 0x26, 0x66, 0x3d, 0x01, 0xea, 0x61, 0xb1, 0x07, 0x4e, 0x09, 0xc3, 0x7c, 0x6e, 0xe6, - 0x7b, 0xb0, 0xe8, 0x85, 0x1a, 0x8c, 0x3a, 0xe4, 0xa1, 0x48, 0xde, 0x4b, 0x60, 0xa7, 0x2c, 0x64, - 0xbe, 0xc3, 0xdc, 0xfc, 0xfc, 0x75, 0xd6, 0x77, 0xb5, 0xbd, 0x15, 0xeb, 0x51, 0x9a, 0x98, 0x6f, - 0x2d, 0x25, 0xcd, 0x0f, 0x29, 0x2c, 0x8f, 0x93, 0xdf, 0xd1, 0x95, 0x1b, 0x50, 0x60, 0x4b, 0xee, - 0xe8, 0xf9, 0xfc, 0x80, 0x9d, 0x46, 0x87, 0x8c, 0x3b, 0xa3, 0xac, 0xb4, 0x15, 0xe7, 0x57, 0xf2, - 0x42, 0x0d, 0xd6, 0xfd, 0x4d, 0x27, 0x3a, 0xf6, 0x23, 0xb6, 0x6f, 0xc4, 0x6c, 0x57, 0x76, 0x2a, - 0x32, 0xaa, 0x78, 0x6e, 0xca, 0x1e, 0xa8, 0xd8, 0x25, 0xad, 0xac, 0x1d, 0x7a, 0x8d, 0x56, 0x56, - 0x8d, 0x8a, 0x4d, 0x0f, 0xc8, 0x3d, 0x97, 0x39, 0xc1, 0x64, 0x1a, 0x62, 0xfa, 0xca, 0xae, 0x5b, - 0x28, 0xbf, 0x9f, 0x26, 0xe6, 0xa2, 0x13, 0x16, 0xa1, 0x6a, 0x10, 0x39, 0x86, 0x76, 0x7d, 0x10, - 0x39, 0x8c, 0x45, 0x88, 0x3e, 0x26, 0x9b, 0xd5, 0x71, 0xc8, 0xc2, 0xbc, 0x9d, 0x26, 0x66, 0xd5, - 0x05, 0x55, 0x40, 0xc8, 0xf1, 0x2c, 0x3e, 0x89, 0xa7, 0xde, 0xd8, 0xb1, 0x85, 0xfc, 0x6e, 0x2e, - 0xaf, 0xb8, 0xa0, 0x0a, 0x08, 0xf9, 0xb4, 0x52, 0x80, 0x49, 0x2e, 0xaf, 0xb8, 0xa0, 0x0a, 0xd0, - 0x29, 0xd9, 0xcd, 0x16, 0x76, 0x49, 0x89, 0x54, 0x05, 0xfd, 0x9d, 0x34, 0x31, 0x5f, 0xcb, 0x85, - 0xd7, 0x32, 0xe8, 0x25, 0x79, 0xbb, 0xb8, 0x86, 0xcb, 0x3a, 0x95, 0x65, 0xfe, 0xdd, 0x34, 0x31, - 0x6f, 0x43, 0x87, 0xdb, 0x90, 0xba, 0x7f, 0x35, 0x89, 0x8e, 0x4f, 0x29, 0x51, 0x23, 0x99, 0xbc, - 0x16, 0x0f, 0x83, 0xd8, 0x2f, 0x55, 0xe8, 0x22, 0x0e, 0x25, 0x8b, 0x7e, 0x49, 0xb6, 0xd8, 0xfc, - 0x32, 0x3d, 0x8f, 0x45, 0xad, 0x97, 0x95, 0x46, 0xb7, 0x76, 0xd2, 0xc4, 0x5c, 0xf0, 0xc1, 0x02, - 0x42, 0x3f, 0x25, 0xeb, 0x0a, 0xc3, 0xe2, 0x27, 0x1f, 0x38, 0xba, 0x75, 0x2f, 0x4d, 0xcc, 0xb2, - 0x03, 0xca, 0xa6, 0x10, 0xe2, 0x8b, 0x0c, 0x98, 0xc3, 0xc6, 0x17, 0xd9, 0x73, 0x06, 0x85, 0x25, - 0x07, 0x94, 0x4d, 0xf1, 0x30, 0x41, 0x00, 0x4b, 0xba, 0x4c, 0x2f, 0x7c, 0x98, 0x64, 0x20, 0xe4, - 0x4d, 0xf1, 0xde, 0x09, 0xe5, 0x58, 0x65, 0x2e, 0xe9, 0xf2, 0xbd, 0x33, 0xc7, 0x20, 0x6b, 0x89, - 0x05, 0x74, 0x8b, 0x25, 0xb2, 0x9d, 0x5f, 0x32, 0x45, 0x1c, 0x4a, 0x96, 0xc8, 0x37, 0x2c, 0x67, - 0xc7, 0xcc, 0x1f, 0xf2, 0x51, 0x9f, 0x85, 0x17, 0xd9, 0x2b, 0x06, 0xf3, 0x6d, 0xc1, 0x09, 0x8b, - 0x90, 0x35, 0xb8, 0xba, 0x36, 0x1a, 0x2f, 0xae, 0x8d, 0xc6, 0xcb, 0x6b, 0x43, 0xfb, 0x69, 0x66, - 0x68, 0xbf, 0xcc, 0x0c, 0xed, 0xf9, 0xcc, 0xd0, 0xae, 0x66, 0x86, 0xf6, 0xf7, 0xcc, 0xd0, 0xfe, - 0x99, 0x19, 0x8d, 0x97, 0x33, 0x43, 0xfb, 0xf9, 0xc6, 0x68, 0x5c, 0xdd, 0x18, 0x8d, 0x17, 0x37, - 0x46, 0xe3, 0xfb, 0x0f, 0x86, 0x63, 0x3e, 0x8a, 0x07, 0xfb, 0x4e, 0x30, 0xe9, 0x0d, 0x43, 0xfb, - 0xd4, 0xf6, 0xed, 0x9e, 0x17, 0x9c, 0x8d, 0x7b, 0x75, 0x7f, 0x14, 0x07, 0x2d, 0xfc, 0x1b, 0xf8, - 0xf1, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xa8, 0xe8, 0xef, 0xe7, 0x47, 0x0e, 0x00, 0x00, + // 1241 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x57, 0x4b, 0x6f, 0xe3, 0x54, + 0x14, 0x8e, 0x27, 0xe3, 0xa4, 0xbd, 0x7d, 0xce, 0x6d, 0x87, 0xc9, 0x30, 0x92, 0x5d, 0x02, 0x23, + 0x8a, 0x40, 0x8d, 0x78, 0x48, 0x08, 0xc4, 0x48, 0xc8, 0x1d, 0x2a, 0x55, 0x6a, 0x45, 0x39, 0x81, + 0x0d, 0xac, 0x1c, 0xfb, 0x36, 0xb1, 0xea, 0xd8, 0xa9, 0x7d, 0x5d, 0xa6, 0x2b, 0xf8, 0x09, 0xec, + 0xf9, 0x03, 0x6c, 0x58, 0xb1, 0x42, 0x62, 0xc7, 0x66, 0x96, 0x5d, 0xce, 0xca, 0xa2, 0xe9, 0x06, + 0x79, 0x35, 0x12, 0x7f, 0x00, 0xdd, 0x47, 0x6c, 0x5f, 0xc7, 0x99, 0xe9, 0x26, 0xbe, 0xe7, 0x3b, + 0xdf, 0x77, 0xee, 0xc3, 0xe7, 0x1c, 0xdf, 0xa0, 0x9d, 0xc9, 0xd9, 0xb0, 0xe7, 0x87, 0xc3, 0x73, + 0x7f, 0x1c, 0xba, 0xc4, 0xef, 0xc5, 0xd4, 0xa6, 0xb1, 0xf8, 0xdd, 0x9b, 0x44, 0x21, 0x0d, 0xb1, + 0xce, 0x8d, 0x37, 0xb7, 0x87, 0xe1, 0x30, 0xe4, 0x48, 0x8f, 0x8d, 0x84, 0xb3, 0xfb, 0x9f, 0x86, + 0x5a, 0x40, 0xe2, 0xc4, 0xa7, 0xf8, 0x33, 0xd4, 0x8e, 0x93, 0xf1, 0xd8, 0x8e, 0x2e, 0x3b, 0xda, + 0x8e, 0xb6, 0xbb, 0xf2, 0xd1, 0xfa, 0x9e, 0x08, 0xd3, 0x17, 0xa8, 0xb5, 0xf1, 0x3c, 0x35, 0x1b, + 0x59, 0x6a, 0xce, 0x68, 0x30, 0x1b, 0x30, 0xe9, 0x79, 0x42, 0x22, 0x8f, 0x44, 0x9d, 0x3b, 0x8a, + 0xf4, 0x1b, 0x81, 0x16, 0x52, 0x49, 0x83, 0xd9, 0x00, 0x3f, 0x41, 0x4b, 0x5e, 0x30, 0x24, 0x31, + 0x25, 0x51, 0xa7, 0xc9, 0xb5, 0x1b, 0x52, 0x7b, 0x28, 0x61, 0x6b, 0x53, 0x8a, 0x73, 0x22, 0xe4, + 0x23, 0xfc, 0x09, 0x6a, 0x39, 0xb6, 0x33, 0x22, 0x71, 0xe7, 0x2e, 0x17, 0xaf, 0x49, 0xf1, 0x3e, + 0x07, 0xad, 0x35, 0x29, 0xd5, 0x39, 0x09, 0x24, 0xb7, 0xfb, 0xeb, 0x5d, 0xd4, 0x12, 0x0c, 0xfc, + 0x21, 0xd2, 0x9d, 0x51, 0x12, 0x9c, 0xc9, 0x3d, 0xaf, 0x96, 0xf5, 0x25, 0x39, 0xa3, 0x80, 0x78, + 0x30, 0x89, 0x17, 0xb8, 0xe4, 0x99, 0xdc, 0xeb, 0x02, 0x09, 0xa7, 0x80, 0x78, 0xb0, 0x65, 0x46, + 0xfc, 0x94, 0xe5, 0x1e, 0x55, 0xcd, 0xba, 0xd4, 0x48, 0x0e, 0xc8, 0x27, 0xde, 0x47, 0x2b, 0x9c, + 0x26, 0x5e, 0x90, 0xdc, 0xa1, 0x2a, 0xdd, 0x92, 0xd2, 0x32, 0x11, 0xca, 0x06, 0x3e, 0x40, 0xab, + 0x17, 0xa1, 0x9f, 0x8c, 0x89, 0x8c, 0xa2, 0xd7, 0x44, 0xd9, 0x96, 0x51, 0x14, 0x26, 0x28, 0x16, + 0x8b, 0x13, 0xb3, 0x57, 0x36, 0x5b, 0x4d, 0xeb, 0x55, 0x71, 0xca, 0x4c, 0x50, 0x2c, 0xb6, 0x29, + 0xdf, 0x1e, 0x10, 0x5f, 0x86, 0x69, 0xbf, 0x6a, 0x53, 0x25, 0x22, 0x94, 0x0d, 0xfc, 0x03, 0xda, + 0xf2, 0x82, 0x98, 0xda, 0x01, 0x3d, 0x26, 0x34, 0xf2, 0x1c, 0x19, 0x6c, 0xa9, 0x26, 0xd8, 0x23, + 0x19, 0xac, 0x4e, 0x00, 0x75, 0x60, 0xf7, 0xcf, 0x16, 0x6a, 0xcb, 0x9c, 0xc7, 0xdf, 0xa1, 0x07, + 0x83, 0x4b, 0x4a, 0xe2, 0x93, 0x28, 0x74, 0x48, 0x1c, 0x13, 0xf7, 0x84, 0x44, 0x7d, 0xe2, 0x84, + 0x81, 0xcb, 0x13, 0xa6, 0x69, 0x3d, 0xca, 0x52, 0x73, 0x11, 0x05, 0x16, 0x39, 0x58, 0x58, 0xdf, + 0x0b, 0x6a, 0xc3, 0xde, 0x29, 0xc2, 0x2e, 0xa0, 0xc0, 0x22, 0x07, 0x3e, 0x44, 0x5b, 0x34, 0xa4, + 0xb6, 0x6f, 0x29, 0xd3, 0xf2, 0x9c, 0x6b, 0x5a, 0x0f, 0xd8, 0x21, 0xd4, 0xb8, 0xa1, 0x0e, 0xcc, + 0x43, 0x1d, 0x29, 0x53, 0xf1, 0x1c, 0x2c, 0x87, 0x52, 0xdd, 0x50, 0x07, 0xe2, 0x5d, 0xb4, 0x44, + 0x9e, 0x11, 0xe7, 0x5b, 0x6f, 0x4c, 0x78, 0xf6, 0x69, 0xd6, 0x2a, 0xab, 0xe6, 0x19, 0x06, 0xf9, + 0x08, 0xbf, 0x8f, 0x96, 0xcf, 0x13, 0x92, 0x10, 0x4e, 0x6d, 0x71, 0xea, 0x5a, 0x96, 0x9a, 0x05, + 0x08, 0xc5, 0x10, 0xef, 0x21, 0x14, 0x27, 0x03, 0xd1, 0x47, 0x62, 0x9e, 0x47, 0x4d, 0x6b, 0x3d, + 0x4b, 0xcd, 0x12, 0x0a, 0xa5, 0x31, 0x3e, 0x42, 0xdb, 0x7c, 0x75, 0x5f, 0x05, 0x54, 0xa4, 0x23, + 0x4d, 0xa2, 0x80, 0xb8, 0x3c, 0x69, 0x9a, 0x56, 0x27, 0x4b, 0xcd, 0x5a, 0x3f, 0xd4, 0xa2, 0xb8, + 0x8b, 0x5a, 0xf1, 0xc4, 0xf7, 0x68, 0xdc, 0x59, 0xe6, 0x7a, 0xc4, 0xea, 0x57, 0x20, 0x20, 0x9f, + 0x9c, 0x33, 0xb2, 0x23, 0x37, 0xee, 0xa0, 0x12, 0x87, 0x23, 0x20, 0x9f, 0xf9, 0xaa, 0x4e, 0xc2, + 0x98, 0x1e, 0x78, 0x3e, 0x25, 0x11, 0x3f, 0xbd, 0xce, 0x4a, 0x65, 0x55, 0x15, 0x3f, 0xd4, 0xa2, + 0xf8, 0x27, 0xf4, 0x98, 0xe3, 0x7d, 0x1a, 0x25, 0x0e, 0x4d, 0x22, 0xe2, 0x1e, 0x13, 0x6a, 0xbb, + 0x36, 0xb5, 0x2b, 0x29, 0xb1, 0xca, 0xc3, 0xbf, 0x97, 0xa5, 0xe6, 0xed, 0x04, 0x70, 0x3b, 0x5a, + 0xf7, 0x0b, 0xd4, 0x96, 0x3d, 0x9f, 0xb5, 0xc9, 0x98, 0x86, 0x11, 0xa9, 0x74, 0xd6, 0x3e, 0xc3, + 0x8a, 0x36, 0xc9, 0x29, 0x20, 0x1e, 0xdd, 0xdf, 0xef, 0xa0, 0xa5, 0xc3, 0xa2, 0xb5, 0xaf, 0xf2, + 0x39, 0x81, 0xb0, 0x3a, 0x16, 0xf5, 0xa6, 0x5b, 0x9b, 0xac, 0xbd, 0x94, 0x71, 0x50, 0x2c, 0x7c, + 0x80, 0x30, 0xb7, 0xf7, 0x59, 0xab, 0x8e, 0x8f, 0x6d, 0xca, 0xb5, 0xa2, 0xa8, 0xde, 0xc8, 0x52, + 0xb3, 0xc6, 0x0b, 0x35, 0x58, 0x3e, 0xbb, 0xc5, 0xed, 0x58, 0xd6, 0x50, 0x31, 0xbb, 0xc4, 0x41, + 0xb1, 0xf0, 0xe7, 0x68, 0xbd, 0xa8, 0x80, 0x3e, 0x09, 0xa8, 0x2c, 0x18, 0x9c, 0xa5, 0x66, 0xc5, + 0x03, 0x15, 0xbb, 0x38, 0x2f, 0xfd, 0xd6, 0xe7, 0xf5, 0x57, 0x13, 0xe9, 0xdc, 0x9f, 0x4f, 0x2c, + 0x36, 0x01, 0xe4, 0x54, 0xb6, 0xa7, 0x62, 0xe2, 0xdc, 0x03, 0x15, 0x1b, 0x7f, 0x8d, 0xee, 0x97, + 0x90, 0xa7, 0xe1, 0x8f, 0x81, 0x1f, 0xda, 0x6e, 0x7e, 0x6a, 0x0f, 0xb3, 0xd4, 0xac, 0x27, 0x40, + 0x3d, 0xcc, 0xde, 0x81, 0xa3, 0x60, 0xbc, 0x9e, 0x9b, 0xc5, 0x3b, 0x98, 0xf7, 0x42, 0x0d, 0x86, + 0x1d, 0xf4, 0x90, 0x15, 0xef, 0x25, 0x90, 0x53, 0x12, 0x91, 0xc0, 0x21, 0x6e, 0x91, 0x7f, 0x9d, + 0xb5, 0x1d, 0x6d, 0x77, 0xc9, 0x7a, 0x9c, 0xa5, 0xe6, 0x5b, 0x0b, 0x49, 0xb3, 0x24, 0x85, 0xc5, + 0x71, 0x8a, 0x0b, 0x40, 0xe5, 0xf3, 0xca, 0xb0, 0x05, 0x17, 0x80, 0xd9, 0xfe, 0x80, 0x9c, 0xc6, + 0x07, 0x84, 0x3a, 0xa3, 0xbc, 0xb5, 0x95, 0xf7, 0xa7, 0x78, 0xa1, 0x06, 0xeb, 0xfe, 0xa1, 0x23, + 0x9d, 0xcf, 0xc3, 0x5e, 0xdf, 0x88, 0xd8, 0xae, 0x98, 0x94, 0x55, 0x54, 0x39, 0x6f, 0x54, 0x0f, + 0x54, 0x6c, 0x45, 0x2b, 0x7a, 0x87, 0x5e, 0xa3, 0x15, 0x5d, 0xa3, 0x62, 0xe3, 0x7d, 0x74, 0xcf, + 0x25, 0x4e, 0x38, 0x9e, 0x44, 0xbc, 0x7c, 0xc5, 0xd4, 0x2d, 0x2e, 0xbf, 0x9f, 0xa5, 0xe6, 0xbc, + 0x13, 0xe6, 0xa1, 0x6a, 0x10, 0xb1, 0x86, 0x76, 0x7d, 0x10, 0xb1, 0x8c, 0x79, 0x08, 0x3f, 0x41, + 0x1b, 0xd5, 0x75, 0x88, 0xc6, 0xbc, 0x95, 0xa5, 0x66, 0xd5, 0x05, 0x55, 0x80, 0xc9, 0x79, 0x2e, + 0x3e, 0x4d, 0x26, 0xbe, 0xe7, 0xd8, 0x4c, 0xbe, 0x5c, 0xc8, 0x2b, 0x2e, 0xa8, 0x02, 0x4c, 0x3e, + 0xa9, 0x34, 0x60, 0x54, 0xc8, 0x2b, 0x2e, 0xa8, 0x02, 0x78, 0x82, 0x76, 0xf2, 0x83, 0x5d, 0xd0, + 0x22, 0x65, 0x43, 0x7f, 0x27, 0x4b, 0xcd, 0xd7, 0x72, 0xe1, 0xb5, 0x0c, 0x7c, 0x89, 0xde, 0x2e, + 0x9f, 0xe1, 0xa2, 0x49, 0x45, 0x9b, 0x7f, 0x37, 0x4b, 0xcd, 0xdb, 0xd0, 0xe1, 0x36, 0xa4, 0xee, + 0xdf, 0x4d, 0xa4, 0xf3, 0xab, 0x15, 0xeb, 0x91, 0x44, 0x7c, 0x16, 0x0f, 0xc2, 0x24, 0x50, 0x3a, + 0x74, 0x19, 0x07, 0xc5, 0xc2, 0x5f, 0xa2, 0x4d, 0x32, 0xfb, 0x98, 0x9e, 0x27, 0xac, 0xd7, 0x8b, + 0x4e, 0xa3, 0x5b, 0xdb, 0x59, 0x6a, 0xce, 0xf9, 0x60, 0x0e, 0xc1, 0x9f, 0xa2, 0x35, 0x89, 0xf1, + 0xe6, 0x27, 0x2e, 0x38, 0xba, 0x75, 0x2f, 0x4b, 0x4d, 0xd5, 0x01, 0xaa, 0xc9, 0x84, 0xfc, 0x46, + 0x06, 0xc4, 0x21, 0xde, 0x45, 0x7e, 0x9d, 0xe1, 0x42, 0xc5, 0x01, 0xaa, 0xc9, 0x2e, 0x26, 0x1c, + 0xe0, 0x2d, 0x5d, 0x94, 0x17, 0xbf, 0x98, 0xe4, 0x20, 0x14, 0x43, 0x76, 0xdf, 0x89, 0xc4, 0x5a, + 0x45, 0x2d, 0xe9, 0xe2, 0xbe, 0x33, 0xc3, 0x20, 0x1f, 0xb1, 0x03, 0x74, 0xcb, 0x2d, 0xb2, 0x5d, + 0x7c, 0x64, 0xca, 0x38, 0x28, 0x16, 0xab, 0x37, 0xde, 0xce, 0x8e, 0x48, 0x30, 0xa4, 0xa3, 0x3e, + 0x89, 0x2e, 0xf2, 0x5b, 0x0c, 0xaf, 0xb7, 0x39, 0x27, 0xcc, 0x43, 0xd6, 0xe0, 0xea, 0xda, 0x68, + 0xbc, 0xb8, 0x36, 0x1a, 0x2f, 0xaf, 0x0d, 0xed, 0xe7, 0xa9, 0xa1, 0xfd, 0x36, 0x35, 0xb4, 0xe7, + 0x53, 0x43, 0xbb, 0x9a, 0x1a, 0xda, 0x3f, 0x53, 0x43, 0xfb, 0x77, 0x6a, 0x34, 0x5e, 0x4e, 0x0d, + 0xed, 0x97, 0x1b, 0xa3, 0x71, 0x75, 0x63, 0x34, 0x5e, 0xdc, 0x18, 0x8d, 0xef, 0x3f, 0x18, 0x7a, + 0x74, 0x94, 0x0c, 0xf6, 0x9c, 0x70, 0xdc, 0x1b, 0x46, 0xf6, 0xa9, 0x1d, 0xd8, 0x3d, 0x3f, 0x3c, + 0xf3, 0x7a, 0x75, 0xff, 0x42, 0x07, 0x2d, 0xfe, 0x1f, 0xf3, 0xe3, 0xff, 0x03, 0x00, 0x00, 0xff, + 0xff, 0x38, 0x60, 0xd8, 0x7d, 0xa4, 0x0e, 0x00, 0x00, } func (this *Result) Equal(that interface{}) bool { @@ -925,6 +935,9 @@ func (this *Caches) Equal(that interface{}) bool { if !this.LabelResult.Equal(&that1.LabelResult) { return false } + if !this.InstantMetricResult.Equal(&that1.InstantMetricResult) { + return false + } return true } func (this *Summary) Equal(that interface{}) bool { @@ -1193,7 +1206,7 @@ func (this *Caches) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 11) + s := make([]string, 0, 12) s = append(s, "&stats.Caches{") s = append(s, "Chunk: "+strings.Replace(this.Chunk.GoString(), `&`, ``, 1)+",\n") s = append(s, "Index: "+strings.Replace(this.Index.GoString(), `&`, ``, 1)+",\n") @@ -1202,6 +1215,7 @@ func (this *Caches) GoString() string { s = append(s, "VolumeResult: "+strings.Replace(this.VolumeResult.GoString(), `&`, ``, 1)+",\n") s = append(s, "SeriesResult: "+strings.Replace(this.SeriesResult.GoString(), `&`, ``, 1)+",\n") s = append(s, "LabelResult: "+strings.Replace(this.LabelResult.GoString(), `&`, ``, 1)+",\n") + s = append(s, "InstantMetricResult: "+strings.Replace(this.InstantMetricResult.GoString(), `&`, ``, 1)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -1391,6 +1405,16 @@ func (m *Caches) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + { + size, err := m.InstantMetricResult.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintStats(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 { size, err := m.LabelResult.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -1877,6 +1901,8 @@ func (m *Caches) Size() (n int) { n += 1 + l + sovStats(uint64(l)) l = m.LabelResult.Size() n += 1 + l + sovStats(uint64(l)) + l = m.InstantMetricResult.Size() + n += 1 + l + sovStats(uint64(l)) return n } @@ -2085,6 +2111,7 @@ func (this *Caches) String() string { `VolumeResult:` + strings.Replace(strings.Replace(this.VolumeResult.String(), "Cache", "Cache", 1), `&`, ``, 1) + `,`, `SeriesResult:` + strings.Replace(strings.Replace(this.SeriesResult.String(), "Cache", "Cache", 1), `&`, ``, 1) + `,`, `LabelResult:` + strings.Replace(strings.Replace(this.LabelResult.String(), "Cache", "Cache", 1), `&`, ``, 1) + `,`, + `InstantMetricResult:` + strings.Replace(strings.Replace(this.InstantMetricResult.String(), "Cache", "Cache", 1), `&`, ``, 1) + `,`, `}`, }, "") return s @@ -2637,6 +2664,39 @@ func (m *Caches) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InstantMetricResult", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStats + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthStats + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.InstantMetricResult.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipStats(dAtA[iNdEx:]) diff --git a/pkg/logqlmodel/stats/stats.proto b/pkg/logqlmodel/stats/stats.proto index 8db5b474a7906..d36b8e557d984 100644 --- a/pkg/logqlmodel/stats/stats.proto +++ b/pkg/logqlmodel/stats/stats.proto @@ -57,6 +57,10 @@ message Caches { (gogoproto.nullable) = false, (gogoproto.jsontag) = "labelResult" ]; + Cache instantMetricResult = 8 [ + (gogoproto.nullable) = false, + (gogoproto.jsontag) = "instantMetricResult" + ]; } // Summary is the summary of a query statistics. diff --git a/pkg/loki/config_wrapper.go b/pkg/loki/config_wrapper.go index 9817c04afdc5e..1914c8ab3edfc 100644 --- a/pkg/loki/config_wrapper.go +++ b/pkg/loki/config_wrapper.go @@ -646,6 +646,13 @@ func applyEmbeddedCacheConfig(r *ConfigWrapper) { r.QueryRange.LabelsCacheConfig.CacheConfig = r.QueryRange.ResultsCacheConfig.CacheConfig r.QueryRange.LabelsCacheConfig.CacheConfig.Prefix = prefix } + + instantMetricCacheConfig := r.QueryRange.InstantMetricCacheConfig.CacheConfig + if !cache.IsCacheConfigured(instantMetricCacheConfig) { + prefix := instantMetricCacheConfig.Prefix + r.QueryRange.InstantMetricCacheConfig.CacheConfig = r.QueryRange.ResultsCacheConfig.CacheConfig + r.QueryRange.InstantMetricCacheConfig.CacheConfig.Prefix = prefix + } } func applyIngesterFinalSleep(cfg *ConfigWrapper) { diff --git a/pkg/loki/config_wrapper_test.go b/pkg/loki/config_wrapper_test.go index 866079b71f60f..3b1237dad4d1d 100644 --- a/pkg/loki/config_wrapper_test.go +++ b/pkg/loki/config_wrapper_test.go @@ -1055,6 +1055,49 @@ query_range: }) }) + t.Run("for the instant-metric results cache config", func(t *testing.T) { + t.Run("no embedded cache enabled by default if Redis is set", func(t *testing.T) { + configFileString := `--- +query_range: + instant_metric_results_cache: + cache: + redis: + endpoint: endpoint.redis.org` + + config, _, _ := configWrapperFromYAML(t, configFileString, nil) + assert.EqualValues(t, "endpoint.redis.org", config.QueryRange.InstantMetricCacheConfig.CacheConfig.Redis.Endpoint) + assert.EqualValues(t, "frontend.instant-metric-results-cache.", config.QueryRange.InstantMetricCacheConfig.CacheConfig.Prefix) + assert.False(t, config.QueryRange.InstantMetricCacheConfig.CacheConfig.EmbeddedCache.Enabled) + }) + + t.Run("no embedded cache enabled by default if Memcache is set", func(t *testing.T) { + configFileString := `--- +query_range: + instant_metric_results_cache: + cache: + memcached_client: + host: memcached.host.org` + + config, _, _ := configWrapperFromYAML(t, configFileString, nil) + assert.EqualValues(t, "memcached.host.org", config.QueryRange.InstantMetricCacheConfig.CacheConfig.MemcacheClient.Host) + assert.EqualValues(t, "frontend.instant-metric-results-cache.", config.QueryRange.InstantMetricCacheConfig.CacheConfig.Prefix) + assert.False(t, config.QueryRange.InstantMetricCacheConfig.CacheConfig.EmbeddedCache.Enabled) + }) + + t.Run("embedded cache is enabled by default if no other cache is set", func(t *testing.T) { + config, _, _ := configWrapperFromYAML(t, minimalConfig, nil) + assert.True(t, config.QueryRange.InstantMetricCacheConfig.CacheConfig.EmbeddedCache.Enabled) + assert.EqualValues(t, "frontend.instant-metric-results-cache.", config.QueryRange.InstantMetricCacheConfig.CacheConfig.Prefix) + }) + + t.Run("gets results cache config if not configured directly", func(t *testing.T) { + config, _, _ := configWrapperFromYAML(t, defaultResulsCacheString, nil) + assert.EqualValues(t, "memcached.host.org", config.QueryRange.InstantMetricCacheConfig.CacheConfig.MemcacheClient.Host) + assert.EqualValues(t, "frontend.instant-metric-results-cache.", config.QueryRange.InstantMetricCacheConfig.CacheConfig.Prefix) + assert.False(t, config.QueryRange.InstantMetricCacheConfig.CacheConfig.EmbeddedCache.Enabled) + }) + }) + t.Run("for the labels results cache config", func(t *testing.T) { t.Run("no embedded cache enabled by default if Redis is set", func(t *testing.T) { configFileString := `--- diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 111d313956881..57c6e96a2b3d6 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -75,6 +75,7 @@ import ( util_log "github.com/grafana/loki/pkg/util/log" "github.com/grafana/loki/pkg/util/querylimits" lokiring "github.com/grafana/loki/pkg/util/ring" + util_ring "github.com/grafana/loki/pkg/util/ring" serverutil "github.com/grafana/loki/pkg/util/server" "github.com/grafana/loki/pkg/validation" ) @@ -1418,7 +1419,7 @@ func (t *Loki) initBloomCompactor() (services.Service, error) { logger := log.With(util_log.Logger, "component", "bloom-compactor") - shuffleSharding := bloomcompactor.NewShuffleShardingStrategy(t.bloomCompactorRingManager.Ring, t.bloomCompactorRingManager.RingLifecycler, t.Overrides) + shuffleSharding := util_ring.NewTenantShuffleSharding(t.bloomCompactorRingManager.Ring, t.bloomCompactorRingManager.RingLifecycler, t.Overrides.BloomCompactorShardSize) compactor, err := bloomcompactor.New( t.Cfg.BloomCompactor, @@ -1445,7 +1446,9 @@ func (t *Loki) initBloomCompactorRing() (services.Service, error) { // is LegacyMode needed? // legacyReadMode := t.Cfg.LegacyReadTarget && t.isModuleActive(Read) - rm, err := lokiring.NewRingManager(bloomCompactorRingKey, lokiring.ServerMode, t.Cfg.BloomCompactor.Ring, 1, 1, util_log.Logger, prometheus.DefaultRegisterer) + // TODO(owen-d): configurable num tokens, just use lifecycler config? + numTokens := 10 + rm, err := lokiring.NewRingManager(bloomCompactorRingKey, lokiring.ServerMode, t.Cfg.BloomCompactor.Ring, 1, numTokens, util_log.Logger, prometheus.DefaultRegisterer) if err != nil { return nil, gerrors.Wrap(err, "error initializing bloom-compactor ring manager") diff --git a/pkg/querier/queryrange/codec_test.go b/pkg/querier/queryrange/codec_test.go index 976665df95b99..52e3cc8551b7f 100644 --- a/pkg/querier/queryrange/codec_test.go +++ b/pkg/querier/queryrange/codec_test.go @@ -427,10 +427,12 @@ func Test_codec_DecodeResponse(t *testing.T) { func Test_codec_DecodeProtobufResponseParity(t *testing.T) { // test fixtures from pkg/util/marshal_test var queryTests = []struct { + name string actual parser.Value expected string }{ { + "basic", logqlmodel.Streams{ logproto.Stream{ Entries: []logproto.Entry{ @@ -462,6 +464,7 @@ func Test_codec_DecodeProtobufResponseParity(t *testing.T) { }, // vector test { + "vector", promql.Vector{ { T: 1568404331324, @@ -524,6 +527,7 @@ func Test_codec_DecodeProtobufResponseParity(t *testing.T) { }, // matrix test { + "matrix", promql.Matrix{ { Floats: []promql.FPoint{ @@ -607,50 +611,53 @@ func Test_codec_DecodeProtobufResponseParity(t *testing.T) { } codec := RequestProtobufCodec{} for i, queryTest := range queryTests { - params := url.Values{ - "query": []string{`{app="foo"}`}, - } - u := &url.URL{ - Path: "/loki/api/v1/query_range", - RawQuery: params.Encode(), - } - httpReq := &http.Request{ - Method: "GET", - RequestURI: u.String(), - URL: u, - } - req, err := codec.DecodeRequest(context.TODO(), httpReq, nil) - require.NoError(t, err) + i := i + t.Run(queryTest.name, func(t *testing.T) { + params := url.Values{ + "query": []string{`{app="foo"}`}, + } + u := &url.URL{ + Path: "/loki/api/v1/query_range", + RawQuery: params.Encode(), + } + httpReq := &http.Request{ + Method: "GET", + RequestURI: u.String(), + URL: u, + } + req, err := codec.DecodeRequest(context.TODO(), httpReq, nil) + require.NoError(t, err) - // parser.Value -> queryrange.QueryResponse - var b bytes.Buffer - result := logqlmodel.Result{ - Data: queryTest.actual, - Statistics: statsResult, - } - err = WriteQueryResponseProtobuf(&logql.LiteralParams{}, result, &b) - require.NoError(t, err) + // parser.Value -> queryrange.QueryResponse + var b bytes.Buffer + result := logqlmodel.Result{ + Data: queryTest.actual, + Statistics: statsResult, + } + err = WriteQueryResponseProtobuf(&logql.LiteralParams{}, result, &b) + require.NoError(t, err) - // queryrange.QueryResponse -> queryrangebase.Response - querierResp := &http.Response{ - StatusCode: 200, - Body: io.NopCloser(&b), - Header: http.Header{ - "Content-Type": []string{ProtobufType}, - }, - } - resp, err := codec.DecodeResponse(context.TODO(), querierResp, req) - require.NoError(t, err) + // queryrange.QueryResponse -> queryrangebase.Response + querierResp := &http.Response{ + StatusCode: 200, + Body: io.NopCloser(&b), + Header: http.Header{ + "Content-Type": []string{ProtobufType}, + }, + } + resp, err := codec.DecodeResponse(context.TODO(), querierResp, req) + require.NoError(t, err) - // queryrange.Response -> JSON - ctx := user.InjectOrgID(context.Background(), "1") - httpResp, err := codec.EncodeResponse(ctx, httpReq, resp) - require.NoError(t, err) + // queryrange.Response -> JSON + ctx := user.InjectOrgID(context.Background(), "1") + httpResp, err := codec.EncodeResponse(ctx, httpReq, resp) + require.NoError(t, err) - body, _ := io.ReadAll(httpResp.Body) - require.JSONEqf(t, queryTest.expected, string(body), "Protobuf Decode Query Test %d failed", i) + body, err := io.ReadAll(httpResp.Body) + require.NoError(t, err) + require.JSONEqf(t, queryTest.expected, string(body), "Protobuf Decode Query Test %d failed", i) + }) } - } func Test_codec_EncodeRequest(t *testing.T) { @@ -1645,6 +1652,16 @@ var ( "downloadTime": 0, "queryLengthServed": 0 }, + "instantMetricResult": { + "entriesFound": 0, + "entriesRequested": 0, + "entriesStored": 0, + "bytesReceived": 0, + "bytesSent": 0, + "requests": 0, + "downloadTime": 0, + "queryLengthServed": 0 + }, "result": { "entriesFound": 0, "entriesRequested": 0, @@ -2027,13 +2044,14 @@ var ( }, Caches: stats.Caches{ - Chunk: stats.Cache{}, - Index: stats.Cache{}, - StatsResult: stats.Cache{}, - VolumeResult: stats.Cache{}, - SeriesResult: stats.Cache{}, - LabelResult: stats.Cache{}, - Result: stats.Cache{}, + Chunk: stats.Cache{}, + Index: stats.Cache{}, + StatsResult: stats.Cache{}, + VolumeResult: stats.Cache{}, + SeriesResult: stats.Cache{}, + LabelResult: stats.Cache{}, + Result: stats.Cache{}, + InstantMetricResult: stats.Cache{}, }, } ) diff --git a/pkg/querier/queryrange/downstreamer.go b/pkg/querier/queryrange/downstreamer.go index 31f8997ed767e..4db8034291f64 100644 --- a/pkg/querier/queryrange/downstreamer.go +++ b/pkg/querier/queryrange/downstreamer.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "reflect" + "time" "github.com/go-kit/log/level" "github.com/grafana/dskit/concurrency" @@ -14,6 +15,7 @@ import ( "github.com/prometheus/prometheus/promql/parser" "github.com/grafana/loki/pkg/logql" + "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/logqlmodel" "github.com/grafana/loki/pkg/querier/plan" "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" @@ -27,6 +29,8 @@ const ( type DownstreamHandler struct { limits Limits next queryrangebase.Handler + + splitAlign bool } func ParamsToLokiRequest(params logql.Params) queryrangebase.Request { @@ -86,6 +90,7 @@ func (h DownstreamHandler) Downstreamer(ctx context.Context) logql.Downstreamer parallelism: p, locks: locks, handler: h.next, + splitAlign: h.splitAlign, } } @@ -94,16 +99,50 @@ type instance struct { parallelism int locks chan struct{} handler queryrangebase.Handler + + splitAlign bool +} + +// withoutOffset returns the given query string with offsets removed and timestamp adjusted accordingly. If no offset is present in original query, it will be returned as is. +func withoutOffset(query logql.DownstreamQuery) (string, time.Time, time.Time) { + expr := query.Params.GetExpression() + + var ( + newStart = query.Params.Start() + newEnd = query.Params.End() + ) + expr.Walk(func(e syntax.Expr) { + switch rng := e.(type) { + case *syntax.RangeAggregationExpr: + off := rng.Left.Offset + + if off != 0 { + rng.Left.Offset = 0 // remove offset + + // adjust start and end time + newEnd = newEnd.Add(-off) + newStart = newStart.Add(-off) + + } + } + }) + return expr.String(), newStart, newEnd } func (in instance) Downstream(ctx context.Context, queries []logql.DownstreamQuery, acc logql.Accumulator) ([]logqlmodel.Result, error) { return in.For(ctx, queries, acc, func(qry logql.DownstreamQuery) (logqlmodel.Result, error) { - req := ParamsToLokiRequest(qry.Params).WithQuery(qry.Params.GetExpression().String()) + var req queryrangebase.Request + if in.splitAlign { + qs, newStart, newEnd := withoutOffset(qry) + req = ParamsToLokiRequest(qry.Params).WithQuery(qs).WithStartEnd(newStart, newEnd) + } else { + req = ParamsToLokiRequest(qry.Params).WithQuery(qry.Params.GetExpression().String()) + } sp, ctx := opentracing.StartSpanFromContext(ctx, "DownstreamHandler.instance") defer sp.Finish() logger := spanlogger.FromContext(ctx) defer logger.Finish() - level.Debug(logger).Log("shards", fmt.Sprintf("%+v", qry.Params.Shards()), "query", req.GetQuery(), "step", req.GetStep(), "handler", reflect.TypeOf(in.handler)) + level.Debug(logger).Log("shards", fmt.Sprintf("%+v", qry.Params.Shards()), "query", req.GetQuery(), "step", req.GetStep(), "handler", reflect.TypeOf(in.handler), "engine", "downstream") res, err := in.handler.Do(ctx, req) if err != nil { diff --git a/pkg/querier/queryrange/downstreamer_test.go b/pkg/querier/queryrange/downstreamer_test.go index a23f2a381b007..cadfceeee20e3 100644 --- a/pkg/querier/queryrange/downstreamer_test.go +++ b/pkg/querier/queryrange/downstreamer_test.go @@ -3,6 +3,7 @@ package queryrange import ( "context" "errors" + "fmt" "strconv" "strings" "sync" @@ -12,6 +13,7 @@ import ( "github.com/grafana/dskit/user" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/promql" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "go.uber.org/atomic" @@ -325,71 +327,142 @@ func TestInstanceFor(t *testing.T) { } func TestInstanceDownstream(t *testing.T) { - params, err := logql.NewLiteralParams( - `{foo="bar"}`, - time.Now(), - time.Now(), - 0, - 0, - logproto.BACKWARD, - 1000, - nil, - ) - require.NoError(t, err) - expr, err := syntax.ParseExpr(`{foo="bar"}`) - require.NoError(t, err) - - expectedResp := func() *LokiResponse { - return &LokiResponse{ - Data: LokiData{ - Result: []logproto.Stream{{ - Labels: `{foo="bar"}`, - Entries: []logproto.Entry{ - {Timestamp: time.Unix(0, 0), Line: "foo"}, - }, - }}, + t.Run("Downstream simple query", func(t *testing.T) { + ts := time.Unix(1, 0) + + params, err := logql.NewLiteralParams( + `{foo="bar"}`, + ts, + ts, + 0, + 0, + logproto.BACKWARD, + 1000, + nil, + ) + require.NoError(t, err) + expr, err := syntax.ParseExpr(`{foo="bar"}`) + require.NoError(t, err) + + expectedResp := func() *LokiResponse { + return &LokiResponse{ + Data: LokiData{ + Result: []logproto.Stream{{ + Labels: `{foo="bar"}`, + Entries: []logproto.Entry{ + {Timestamp: time.Unix(0, 0), Line: "foo"}, + }, + }}, + }, + Statistics: stats.Result{ + Summary: stats.Summary{QueueTime: 1, ExecTime: 2}, + }, + } + } + + queries := []logql.DownstreamQuery{ + { + Params: logql.ParamsWithShardsOverride{ + Params: logql.ParamsWithExpressionOverride{Params: params, ExpressionOverride: expr}, + ShardsOverride: logql.Shards{{Shard: 0, Of: 2}}.Encode(), + }, }, - Statistics: stats.Result{ - Summary: stats.Summary{QueueTime: 1, ExecTime: 2}, + } + + var got queryrangebase.Request + var want queryrangebase.Request + handler := queryrangebase.HandlerFunc( + func(_ context.Context, req queryrangebase.Request) (queryrangebase.Response, error) { + // for some reason these seemingly can't be checked in their own goroutines, + // so we assign them to scoped variables for later comparison. + got = req + want = ParamsToLokiRequest(queries[0].Params).WithQuery(expr.String()) + + return expectedResp(), nil }, + ) + + expected, err := ResponseToResult(expectedResp()) + require.Nil(t, err) + + results, err := DownstreamHandler{ + limits: fakeLimits{}, + next: handler, + }.Downstreamer(context.Background()).Downstream(context.Background(), queries, logql.NewBufferedAccumulator(len(queries))) + + fmt.Println("want", want.GetEnd(), want.GetStart(), "got", got.GetEnd(), got.GetStart()) + require.Equal(t, want, got) + require.Nil(t, err) + require.Equal(t, 1, len(results)) + require.Equal(t, expected.Data, results[0].Data) + }) + + t.Run("Downstream with offset removed", func(t *testing.T) { + ts := time.Unix(1, 0) + + params, err := logql.NewLiteralParams( + `sum(rate({foo="bar"}[2h] offset 1h))`, + ts, + ts, + 0, + 0, + logproto.BACKWARD, + 1000, + nil, + ) + require.NoError(t, err) + + expectedResp := func() *LokiResponse { + return &LokiResponse{ + Data: LokiData{ + Result: []logproto.Stream{{ + Labels: `{foo="bar"}`, + Entries: []logproto.Entry{ + {Timestamp: time.Unix(0, 0), Line: "foo"}, + }, + }}, + }, + Statistics: stats.Result{ + Summary: stats.Summary{QueueTime: 1, ExecTime: 2}, + }, + } } - } - queries := []logql.DownstreamQuery{ - { - Params: logql.ParamsWithShardsOverride{ - Params: logql.ParamsWithExpressionOverride{Params: params, ExpressionOverride: expr}, - ShardsOverride: logql.Shards{{Shard: 0, Of: 2}}.Encode(), + queries := []logql.DownstreamQuery{ + { + Params: params, }, - }, - } + } - var got queryrangebase.Request - var want queryrangebase.Request - handler := queryrangebase.HandlerFunc( - func(_ context.Context, req queryrangebase.Request) (queryrangebase.Response, error) { - // for some reason these seemingly can't be checked in their own goroutines, - // so we assign them to scoped variables for later comparison. - got = req - want = ParamsToLokiRequest(queries[0].Params).WithQuery(expr.String()) + var got queryrangebase.Request + var want queryrangebase.Request + handler := queryrangebase.HandlerFunc( + func(_ context.Context, req queryrangebase.Request) (queryrangebase.Response, error) { + // for some reason these seemingly can't be checked in their own goroutines, + // so we assign them to scoped variables for later comparison. + got = req + want = ParamsToLokiRequest(params).WithQuery(`sum(rate({foo="bar"}[2h]))`).WithStartEnd(ts.Add(-1*time.Hour), ts.Add(-1*time.Hour)) // without offset and start, end adjusted for instant query - return expectedResp(), nil - }, - ) + return expectedResp(), nil + }, + ) - expected, err := ResponseToResult(expectedResp()) - require.Nil(t, err) + expected, err := ResponseToResult(expectedResp()) + require.NoError(t, err) - results, err := DownstreamHandler{ - limits: fakeLimits{}, - next: handler, - }.Downstreamer(context.Background()).Downstream(context.Background(), queries, logql.NewBufferedAccumulator(len(queries))) + results, err := DownstreamHandler{ + limits: fakeLimits{}, + next: handler, + splitAlign: true, + }.Downstreamer(context.Background()).Downstream(context.Background(), queries, logql.NewBufferedAccumulator(len(queries))) - require.Equal(t, want, got) + assert.Equal(t, want, got) - require.Nil(t, err) - require.Equal(t, 1, len(results)) - require.Equal(t, expected.Data, results[0].Data) + require.Nil(t, err) + require.Equal(t, 1, len(results)) + require.Equal(t, expected.Data, results[0].Data) + + }) } func TestCancelWhileWaitingResponse(t *testing.T) { diff --git a/pkg/querier/queryrange/instant_metric_cache.go b/pkg/querier/queryrange/instant_metric_cache.go new file mode 100644 index 0000000000000..ef1083e6cd229 --- /dev/null +++ b/pkg/querier/queryrange/instant_metric_cache.go @@ -0,0 +1,85 @@ +package queryrange + +import ( + "context" + "flag" + "fmt" + "time" + + "github.com/go-kit/log" + + "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" + "github.com/grafana/loki/pkg/storage/chunk/cache" + "github.com/grafana/loki/pkg/storage/chunk/cache/resultscache" +) + +type InstantMetricSplitter struct { + Limits + transformer UserIDTransformer +} + +// GenerateCacheKey generates a cache key based on the userID, Request and interval. +func (i InstantMetricSplitter) GenerateCacheKey(ctx context.Context, userID string, r resultscache.Request) string { + split := i.InstantMetricQuerySplitDuration(userID) + + var currentInterval int64 + if denominator := int64(split / time.Millisecond); denominator > 0 { + currentInterval = r.GetStart().UnixMilli() / denominator + } + + if i.transformer != nil { + userID = i.transformer(ctx, userID) + } + + // include both the currentInterval and the split duration in key to ensure + // a cache key can't be reused when an interval changes + return fmt.Sprintf("instant-metric:%s:%s:%d:%d", userID, r.GetQuery(), currentInterval, split) +} + +type InstantMetricCacheConfig struct { + queryrangebase.ResultsCacheConfig `yaml:",inline"` +} + +// RegisterFlags registers flags. +func (cfg *InstantMetricCacheConfig) RegisterFlags(f *flag.FlagSet) { + cfg.RegisterFlagsWithPrefix(f, "frontend.instant-metric-results-cache.") +} + +func (cfg *InstantMetricCacheConfig) Validate() error { + return cfg.ResultsCacheConfig.Validate() +} + +type instantMetricExtractor struct{} + +func NewInstantMetricCacheMiddleware( + log log.Logger, + limits Limits, + merger queryrangebase.Merger, + c cache.Cache, + cacheGenNumberLoader queryrangebase.CacheGenNumberLoader, + shouldCache queryrangebase.ShouldCacheFn, + parallelismForReq queryrangebase.ParallelismForReqFn, + retentionEnabled bool, + transformer UserIDTransformer, + metrics *queryrangebase.ResultsCacheMetrics, +) (queryrangebase.Middleware, error) { + return queryrangebase.NewResultsCacheMiddleware( + log, + c, + InstantMetricSplitter{limits, transformer}, + limits, + merger, + PrometheusExtractor{}, + cacheGenNumberLoader, + func(ctx context.Context, r queryrangebase.Request) bool { + if shouldCache != nil && !shouldCache(ctx, r) { + return false + } + return true + }, + parallelismForReq, + retentionEnabled, + false, + metrics, + ) +} diff --git a/pkg/querier/queryrange/limits.go b/pkg/querier/queryrange/limits.go index 2d14531909695..ab7818460738f 100644 --- a/pkg/querier/queryrange/limits.go +++ b/pkg/querier/queryrange/limits.go @@ -68,6 +68,15 @@ func (l limits) QuerySplitDuration(user string) time.Duration { return *l.splitDuration } +func (l limits) InstantMetricQuerySplitDuration(user string) time.Duration { + // NOTE: It returns `splitDuration` for both instant and range queries. + // no need to have separate limits for now. + if l.splitDuration == nil { + return l.Limits.QuerySplitDuration(user) + } + return *l.splitDuration +} + func (l limits) TSDBMaxQueryParallelism(ctx context.Context, user string) int { if l.maxQueryParallelism == nil { return l.Limits.TSDBMaxQueryParallelism(ctx, user) diff --git a/pkg/querier/queryrange/limits/definitions.go b/pkg/querier/queryrange/limits/definitions.go index 3e78b34420760..9e1232b750797 100644 --- a/pkg/querier/queryrange/limits/definitions.go +++ b/pkg/querier/queryrange/limits/definitions.go @@ -14,6 +14,7 @@ type Limits interface { queryrangebase.Limits logql.Limits QuerySplitDuration(string) time.Duration + InstantMetricQuerySplitDuration(string) time.Duration MetadataQuerySplitDuration(string) time.Duration RecentMetadataQuerySplitDuration(string) time.Duration RecentMetadataQueryWindow(string) time.Duration diff --git a/pkg/querier/queryrange/prometheus_test.go b/pkg/querier/queryrange/prometheus_test.go index a8e09b378bb2c..4ec798b534a73 100644 --- a/pkg/querier/queryrange/prometheus_test.go +++ b/pkg/querier/queryrange/prometheus_test.go @@ -118,6 +118,16 @@ var emptyStats = `"stats": { "downloadTime": 0, "queryLengthServed": 0 }, + "instantMetricResult": { + "entriesFound": 0, + "entriesRequested": 0, + "entriesStored": 0, + "bytesReceived": 0, + "bytesSent": 0, + "requests": 0, + "downloadTime": 0, + "queryLengthServed": 0 + }, "result": { "entriesFound": 0, "entriesRequested": 0, diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index 10246f4d8277e..5532eab989c1e 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -44,16 +44,19 @@ const ( // Config is the configuration for the queryrange tripperware type Config struct { - base.Config `yaml:",inline"` - Transformer UserIDTransformer `yaml:"-"` - CacheIndexStatsResults bool `yaml:"cache_index_stats_results"` - StatsCacheConfig IndexStatsCacheConfig `yaml:"index_stats_results_cache" doc:"description=If a cache config is not specified and cache_index_stats_results is true, the config for the results cache is used."` - CacheVolumeResults bool `yaml:"cache_volume_results"` - VolumeCacheConfig VolumeCacheConfig `yaml:"volume_results_cache" doc:"description=If a cache config is not specified and cache_volume_results is true, the config for the results cache is used."` - CacheSeriesResults bool `yaml:"cache_series_results"` - SeriesCacheConfig SeriesCacheConfig `yaml:"series_results_cache" doc:"description=If series_results_cache is not configured and cache_series_results is true, the config for the results cache is used."` - CacheLabelResults bool `yaml:"cache_label_results"` - LabelsCacheConfig LabelsCacheConfig `yaml:"label_results_cache" doc:"description=If label_results_cache is not configured and cache_label_results is true, the config for the results cache is used."` + base.Config `yaml:",inline"` + Transformer UserIDTransformer `yaml:"-"` + CacheIndexStatsResults bool `yaml:"cache_index_stats_results"` + StatsCacheConfig IndexStatsCacheConfig `yaml:"index_stats_results_cache" doc:"description=If a cache config is not specified and cache_index_stats_results is true, the config for the results cache is used."` + CacheVolumeResults bool `yaml:"cache_volume_results"` + VolumeCacheConfig VolumeCacheConfig `yaml:"volume_results_cache" doc:"description=If a cache config is not specified and cache_volume_results is true, the config for the results cache is used."` + CacheInstantMetricResults bool `yaml:"cache_instant_metric_results"` + InstantMetricCacheConfig InstantMetricCacheConfig `yaml:"instant_metric_results_cache" doc:"description=If a cache config is not specified and cache_instant_metric_results is true, the config for the results cache is used."` + InstantMetricQuerySplitAlign bool `yaml:"instant_metric_query_split_align" doc:"description=Whether to align the splits of instant metric query with splitByInterval and query's exec time. Useful when instant_metric_cache is enabled"` + CacheSeriesResults bool `yaml:"cache_series_results"` + SeriesCacheConfig SeriesCacheConfig `yaml:"series_results_cache" doc:"description=If series_results_cache is not configured and cache_series_results is true, the config for the results cache is used."` + CacheLabelResults bool `yaml:"cache_label_results"` + LabelsCacheConfig LabelsCacheConfig `yaml:"label_results_cache" doc:"description=If label_results_cache is not configured and cache_label_results is true, the config for the results cache is used."` } // RegisterFlags adds the flags required to configure this flag set. @@ -63,6 +66,9 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { cfg.StatsCacheConfig.RegisterFlags(f) f.BoolVar(&cfg.CacheVolumeResults, "querier.cache-volume-results", false, "Cache volume query results.") cfg.VolumeCacheConfig.RegisterFlags(f) + f.BoolVar(&cfg.CacheInstantMetricResults, "querier.cache-instant-metric-results", false, "Cache instant metric query results.") + cfg.InstantMetricCacheConfig.RegisterFlags(f) + f.BoolVar(&cfg.InstantMetricQuerySplitAlign, "querier.instant-metric-query-split-align", false, "Align the instant metric splits with splityByInterval and query's exec time.") f.BoolVar(&cfg.CacheSeriesResults, "querier.cache-series-results", false, "Cache series query results.") cfg.SeriesCacheConfig.RegisterFlags(f) f.BoolVar(&cfg.CacheLabelResults, "querier.cache-label-results", false, "Cache label query results.") @@ -132,12 +138,13 @@ func NewMiddleware( metrics := NewMetrics(registerer, metricsNamespace) var ( - resultsCache cache.Cache - statsCache cache.Cache - volumeCache cache.Cache - seriesCache cache.Cache - labelsCache cache.Cache - err error + resultsCache cache.Cache + statsCache cache.Cache + volumeCache cache.Cache + instantMetricCache cache.Cache + seriesCache cache.Cache + labelsCache cache.Cache + err error ) if cfg.CacheResults { @@ -161,6 +168,13 @@ func NewMiddleware( } } + if cfg.CacheInstantMetricResults { + instantMetricCache, err = newResultsCacheFromConfig(cfg.InstantMetricCacheConfig.ResultsCacheConfig, registerer, log, stats.InstantMetricResultsCache) + if err != nil { + return nil, nil, err + } + } + if cfg.CacheSeriesResults { seriesCache, err = newResultsCacheFromConfig(cfg.SeriesCacheConfig.ResultsCacheConfig, registerer, log, stats.SeriesResultCache) if err != nil { @@ -211,7 +225,7 @@ func NewMiddleware( return nil, nil, err } - instantMetricTripperware, err := NewInstantMetricTripperware(cfg, engineOpts, log, limits, schema, metrics, indexStatsTripperware, metricsNamespace) + instantMetricTripperware, err := NewInstantMetricTripperware(cfg, engineOpts, log, limits, schema, metrics, codec, instantMetricCache, cacheGenNumLoader, retentionEnabled, indexStatsTripperware, metricsNamespace) if err != nil { return nil, nil, err } @@ -761,7 +775,51 @@ func NewMetricTripperware(cfg Config, engineOpts logql.EngineOpts, log log.Logge } // NewInstantMetricTripperware creates a new frontend tripperware responsible for handling metric queries -func NewInstantMetricTripperware(cfg Config, engineOpts logql.EngineOpts, log log.Logger, limits Limits, schema config.SchemaConfig, metrics *Metrics, indexStatsTripperware base.Middleware, metricsNamespace string) (base.Middleware, error) { +func NewInstantMetricTripperware( + cfg Config, + engineOpts logql.EngineOpts, + log log.Logger, + limits Limits, + schema config.SchemaConfig, + metrics *Metrics, + merger base.Merger, + c cache.Cache, + cacheGenNumLoader base.CacheGenNumberLoader, + retentionEnabled bool, + indexStatsTripperware base.Middleware, + metricsNamespace string, +) (base.Middleware, error) { + var cacheMiddleware base.Middleware + if cfg.CacheInstantMetricResults { + var err error + cacheMiddleware, err = NewInstantMetricCacheMiddleware( + log, + limits, + merger, + c, + cacheGenNumLoader, + func(_ context.Context, r base.Request) bool { + return !r.GetCachingOptions().Disabled + }, + func(ctx context.Context, tenantIDs []string, r base.Request) int { + return MinWeightedParallelism( + ctx, + tenantIDs, + schema.Configs, + limits, + model.Time(r.GetStart().UnixMilli()), + model.Time(r.GetEnd().UnixMilli()), + ) + }, + retentionEnabled, + cfg.Transformer, + metrics.ResultsCacheMetrics, + ) + if err != nil { + return nil, err + } + } + return base.MiddlewareFunc(func(next base.Handler) base.Handler { statsHandler := indexStatsTripperware.Wrap(next) @@ -769,11 +827,19 @@ func NewInstantMetricTripperware(cfg Config, engineOpts logql.EngineOpts, log lo StatsCollectorMiddleware(), NewLimitsMiddleware(limits), NewQuerySizeLimiterMiddleware(schema.Configs, engineOpts, log, limits, statsHandler), + NewSplitByRangeMiddleware(log, engineOpts, limits, cfg.InstantMetricQuerySplitAlign, metrics.MiddlewareMapperMetrics.rangeMapper), + } + + if cfg.CacheInstantMetricResults { + queryRangeMiddleware = append( + queryRangeMiddleware, + base.InstrumentMiddleware("instant_metric_results_cache", metrics.InstrumentMiddlewareMetrics), + cacheMiddleware, + ) } if cfg.ShardedQueries { queryRangeMiddleware = append(queryRangeMiddleware, - NewSplitByRangeMiddleware(log, engineOpts, limits, metrics.MiddlewareMapperMetrics.rangeMapper), NewQueryShardMiddleware( log, schema.Configs, diff --git a/pkg/querier/queryrange/roundtrip_test.go b/pkg/querier/queryrange/roundtrip_test.go index 7d74b0dd615c8..206822a50f6e8 100644 --- a/pkg/querier/queryrange/roundtrip_test.go +++ b/pkg/querier/queryrange/roundtrip_test.go @@ -1247,6 +1247,7 @@ type fakeLimits struct { metadataSplitDuration map[string]time.Duration recentMetadataSplitDuration map[string]time.Duration recentMetadataQueryWindow map[string]time.Duration + instantMetricSplitDuration map[string]time.Duration ingesterSplitDuration map[string]time.Duration minShardingLookback time.Duration queryTimeout time.Duration @@ -1266,6 +1267,13 @@ func (f fakeLimits) QuerySplitDuration(key string) time.Duration { return f.splitDuration[key] } +func (f fakeLimits) InstantMetricQuerySplitDuration(key string) time.Duration { + if f.instantMetricSplitDuration == nil { + return 0 + } + return f.instantMetricSplitDuration[key] +} + func (f fakeLimits) MetadataQuerySplitDuration(key string) time.Duration { if f.metadataSplitDuration == nil { return 0 diff --git a/pkg/querier/queryrange/split_by_range.go b/pkg/querier/queryrange/split_by_range.go index 6845846d4deaa..16076cd948596 100644 --- a/pkg/querier/queryrange/split_by_range.go +++ b/pkg/querier/queryrange/split_by_range.go @@ -26,20 +26,25 @@ type splitByRange struct { limits Limits ng *logql.DownstreamEngine metrics *logql.MapperMetrics + + // Whether to align rangeInterval align to splitByInterval in the subqueries. + splitAlign bool } // NewSplitByRangeMiddleware creates a new Middleware that splits log requests by the range interval. -func NewSplitByRangeMiddleware(logger log.Logger, engineOpts logql.EngineOpts, limits Limits, metrics *logql.MapperMetrics) queryrangebase.Middleware { +func NewSplitByRangeMiddleware(logger log.Logger, engineOpts logql.EngineOpts, limits Limits, splitAlign bool, metrics *logql.MapperMetrics) queryrangebase.Middleware { return queryrangebase.MiddlewareFunc(func(next queryrangebase.Handler) queryrangebase.Handler { return &splitByRange{ logger: log.With(logger, "middleware", "InstantQuery.splitByRangeVector"), next: next, limits: limits, ng: logql.NewDownstreamEngine(engineOpts, DownstreamHandler{ - limits: limits, - next: next, + limits: limits, + next: next, + splitAlign: splitAlign, }, limits, logger), - metrics: metrics, + metrics: metrics, + splitAlign: splitAlign, } }) } @@ -57,14 +62,26 @@ func (s *splitByRange) Do(ctx context.Context, request queryrangebase.Request) ( return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) } - interval := validation.SmallestPositiveNonZeroDurationPerTenant(tenants, s.limits.QuerySplitDuration) + interval := validation.SmallestPositiveNonZeroDurationPerTenant(tenants, s.limits.InstantMetricQuerySplitDuration) // if no interval configured, continue to the next middleware if interval == 0 { return s.next.Do(ctx, request) } mapperStats := logql.NewMapperStats() - mapper, err := logql.NewRangeMapper(interval, s.metrics, mapperStats) + + ir, ok := request.(*LokiInstantRequest) + if !ok { + return nil, fmt.Errorf("expected *LokiInstantRequest, got %T", request) + } + + var mapper logql.RangeMapper + + if s.splitAlign { + mapper, err = logql.NewRangeMapperWithSplitAlign(interval, ir.TimeTs, s.metrics, mapperStats) + } else { + mapper, err = logql.NewRangeMapper(interval, s.metrics, mapperStats) + } if err != nil { return nil, err } @@ -85,10 +102,6 @@ func (s *splitByRange) Do(ctx context.Context, request queryrangebase.Request) ( queryStatsCtx := stats.FromContext(ctx) queryStatsCtx.AddSplitQueries(int64(mapperStats.GetSplitQueries())) - if _, ok := request.(*LokiInstantRequest); !ok { - return nil, fmt.Errorf("expected *LokiInstantRequest, got %T", request) - } - query := s.ng.Query(ctx, logql.ParamsWithExpressionOverride{Params: params, ExpressionOverride: parsed}) res, err := query.Exec(ctx) diff --git a/pkg/querier/queryrange/split_by_range_test.go b/pkg/querier/queryrange/split_by_range_test.go index b1687611abc1d..af66c10a2f08a 100644 --- a/pkg/querier/queryrange/split_by_range_test.go +++ b/pkg/querier/queryrange/split_by_range_test.go @@ -8,6 +8,7 @@ import ( "github.com/go-kit/log" "github.com/grafana/dskit/user" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/loghttp" @@ -17,14 +18,291 @@ import ( "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" ) +func Test_RangeVectorSplitAlign(t *testing.T) { + var ( + twelve34 = time.Date(1970, 1, 1, 12, 34, 0, 0, time.UTC) // 1970 12:34:00 UTC + twelve = time.Date(1970, 1, 1, 12, 00, 0, 0, time.UTC) // 1970 12:00:00 UTC + eleven = twelve.Add(-1 * time.Hour) // 1970 11:00:00 UTC + ten = eleven.Add(-1 * time.Hour) // 1970 10:00:00 UTC + ) + + for _, tc := range []struct { + name string + in queryrangebase.Request + subQueries []queryrangebase.RequestResponse + expected queryrangebase.Response + splitByInterval time.Duration + }{ + { + name: "sum_splitBy_aligned_with_query_time", + splitByInterval: 1 * time.Minute, + in: &LokiInstantRequest{ + Query: `sum(bytes_over_time({app="foo"}[3m]))`, + TimeTs: time.Unix(180, 0), + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum(bytes_over_time({app="foo"}[3m]))`), + }, + }, + subQueries: []queryrangebase.RequestResponse{ + subQueryRequestResponseWithQueryTime(`sum(bytes_over_time({app="foo"}[1m]))`, 1, time.Unix(60, 0)), + subQueryRequestResponseWithQueryTime(`sum(bytes_over_time({app="foo"}[1m]))`, 2, time.Unix(120, 0)), + subQueryRequestResponseWithQueryTime(`sum(bytes_over_time({app="foo"}[1m]))`, 3, time.Unix(180, 0)), + }, + expected: expectedMergedResponseWithTime(1+2+3, time.Unix(180, 0)), // original `TimeTs` of the query. + }, + { + name: "sum_splitBy_not_aligned_query_time", + splitByInterval: 1 * time.Hour, + in: &LokiInstantRequest{ + Query: `sum(bytes_over_time({app="foo"}[3h]))`, + TimeTs: twelve34, + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum(bytes_over_time({app="foo"}[3h]))`), + }, + }, + subQueries: []queryrangebase.RequestResponse{ + subQueryRequestResponseWithQueryTime(`sum(bytes_over_time({app="foo"}[34m]))`, 1, twelve34), + subQueryRequestResponseWithQueryTime(`sum(bytes_over_time({app="foo"}[1h]))`, 2, twelve), + subQueryRequestResponseWithQueryTime(`sum(bytes_over_time({app="foo"}[1h]))`, 3, eleven), + subQueryRequestResponseWithQueryTime(`sum(bytes_over_time({app="foo"}[26m]))`, 4, ten), + }, + expected: expectedMergedResponseWithTime(1+2+3+4, twelve34), // original `TimeTs` of the query. + }, + { + name: "sum_aggregation_splitBy_aligned_with_query_time", + splitByInterval: 1 * time.Minute, + in: &LokiInstantRequest{ + Query: `sum by (bar) (bytes_over_time({app="foo"}[3m]))`, + TimeTs: time.Unix(180, 0), + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum by (bar) (bytes_over_time({app="foo"}[3m]))`), + }, + }, + subQueries: []queryrangebase.RequestResponse{ + subQueryRequestResponseWithQueryTime(`sum by (bar)(bytes_over_time({app="foo"}[1m]))`, 10, time.Unix(60, 0)), + subQueryRequestResponseWithQueryTime(`sum by (bar)(bytes_over_time({app="foo"}[1m]))`, 20, time.Unix(120, 0)), + subQueryRequestResponseWithQueryTime(`sum by (bar)(bytes_over_time({app="foo"}[1m]))`, 30, time.Unix(180, 0)), + }, + expected: expectedMergedResponseWithTime(10+20+30, time.Unix(180, 0)), + }, + { + name: "sum_aggregation_splitBy_not_aligned_with_query_time", + splitByInterval: 1 * time.Hour, + in: &LokiInstantRequest{ + Query: `sum by (bar) (bytes_over_time({app="foo"}[3h]))`, + TimeTs: twelve34, + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum by (bar) (bytes_over_time({app="foo"}[3h]))`), + }, + }, + subQueries: []queryrangebase.RequestResponse{ + subQueryRequestResponseWithQueryTime(`sum by (bar)(bytes_over_time({app="foo"}[34m]))`, 10, twelve34), // 12:34:00 + subQueryRequestResponseWithQueryTime(`sum by (bar)(bytes_over_time({app="foo"}[1h]))`, 20, twelve), // 12:00:00 aligned + subQueryRequestResponseWithQueryTime(`sum by (bar)(bytes_over_time({app="foo"}[1h]))`, 30, eleven), // 11:00:00 aligned + subQueryRequestResponseWithQueryTime(`sum by (bar)(bytes_over_time({app="foo"}[26m]))`, 40, ten), // 10:00:00 + }, + expected: expectedMergedResponseWithTime(10+20+30+40, twelve34), + }, + { + name: "count_over_time_aligned_with_query_time", + splitByInterval: 1 * time.Minute, + in: &LokiInstantRequest{ + Query: `sum(count_over_time({app="foo"}[3m]))`, + TimeTs: time.Unix(180, 0), + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum(count_over_time({app="foo"}[3m]))`), + }, + }, + subQueries: []queryrangebase.RequestResponse{ + subQueryRequestResponseWithQueryTime(`sum(count_over_time({app="foo"}[1m]))`, 1, time.Unix(60, 0)), + subQueryRequestResponseWithQueryTime(`sum(count_over_time({app="foo"}[1m]))`, 1, time.Unix(120, 0)), + subQueryRequestResponseWithQueryTime(`sum(count_over_time({app="foo"}[1m]))`, 1, time.Unix(180, 0)), + }, + expected: expectedMergedResponseWithTime(1+1+1, time.Unix(180, 0)), + }, + { + name: "count_over_time_not_aligned_with_query_time", + splitByInterval: 1 * time.Hour, + in: &LokiInstantRequest{ + Query: `sum(count_over_time({app="foo"}[3h]))`, + TimeTs: twelve34, + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum(count_over_time({app="foo"}[3h]))`), + }, + }, + subQueries: []queryrangebase.RequestResponse{ + subQueryRequestResponseWithQueryTime(`sum(count_over_time({app="foo"}[34m]))`, 1, twelve34), + subQueryRequestResponseWithQueryTime(`sum(count_over_time({app="foo"}[1h]))`, 1, twelve), + subQueryRequestResponseWithQueryTime(`sum(count_over_time({app="foo"}[1h]))`, 1, eleven), + subQueryRequestResponseWithQueryTime(`sum(count_over_time({app="foo"}[26m]))`, 1, ten), + }, + expected: expectedMergedResponseWithTime(1+1+1+1, twelve34), + }, + { + name: "sum_agg_count_over_time_align_with_query_time", + splitByInterval: 1 * time.Minute, + in: &LokiInstantRequest{ + Query: `sum by (bar) (count_over_time({app="foo"}[3m]))`, + TimeTs: time.Unix(180, 0), + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum by (bar) (count_over_time({app="foo"}[3m]))`), + }, + }, + subQueries: []queryrangebase.RequestResponse{ + subQueryRequestResponseWithQueryTime(`sum by (bar)(count_over_time({app="foo"}[1m]))`, 0, time.Unix(60, 0)), + subQueryRequestResponseWithQueryTime(`sum by (bar)(count_over_time({app="foo"}[1m]))`, 0, time.Unix(120, 0)), + subQueryRequestResponseWithQueryTime(`sum by (bar)(count_over_time({app="foo"}[1m]))`, 0, time.Unix(180, 0)), + }, + expected: expectedMergedResponseWithTime(0+0+0, time.Unix(180, 0)), + }, + { + name: "sum_agg_count_over_time_not_align_with_query_time", + splitByInterval: 1 * time.Hour, + in: &LokiInstantRequest{ + Query: `sum by (bar) (count_over_time({app="foo"}[3h]))`, + TimeTs: twelve34, + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum by (bar) (count_over_time({app="foo"}[3h]))`), + }, + }, + subQueries: []queryrangebase.RequestResponse{ + subQueryRequestResponseWithQueryTime(`sum by (bar)(count_over_time({app="foo"}[34m]))`, 0, twelve34), + subQueryRequestResponseWithQueryTime(`sum by (bar)(count_over_time({app="foo"}[1h]))`, 0, twelve), + subQueryRequestResponseWithQueryTime(`sum by (bar)(count_over_time({app="foo"}[1h]))`, 0, eleven), + subQueryRequestResponseWithQueryTime(`sum by (bar)(count_over_time({app="foo"}[26m]))`, 0, ten), + }, + expected: expectedMergedResponseWithTime(0+0+0+0, twelve34), + }, + { + name: "sum_over_time_aligned_with_query_time", + splitByInterval: 1 * time.Minute, + in: &LokiInstantRequest{ + Query: `sum(sum_over_time({app="foo"} | unwrap bar [3m]))`, + TimeTs: time.Unix(180, 0), + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum(sum_over_time({app="foo"} | unwrap bar [3m]))`), + }, + }, + subQueries: []queryrangebase.RequestResponse{ + subQueryRequestResponseWithQueryTime(`sum(sum_over_time({app="foo"} | unwrap bar[1m]))`, 1, time.Unix(60, 0)), + subQueryRequestResponseWithQueryTime(`sum(sum_over_time({app="foo"} | unwrap bar[1m]))`, 2, time.Unix(120, 0)), + subQueryRequestResponseWithQueryTime(`sum(sum_over_time({app="foo"} | unwrap bar[1m]))`, 3, time.Unix(180, 0)), + }, + expected: expectedMergedResponseWithTime(1+2+3, time.Unix(180, 0)), + }, + { + name: "sum_over_time_not_aligned_with_query_time", + splitByInterval: 1 * time.Hour, + in: &LokiInstantRequest{ + Query: `sum(sum_over_time({app="foo"} | unwrap bar [3h]))`, + TimeTs: twelve34, + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum(sum_over_time({app="foo"} | unwrap bar [3h]))`), + }, + }, + subQueries: []queryrangebase.RequestResponse{ + subQueryRequestResponseWithQueryTime(`sum(sum_over_time({app="foo"} | unwrap bar[34m]))`, 1, twelve34), + subQueryRequestResponseWithQueryTime(`sum(sum_over_time({app="foo"} | unwrap bar[1h]))`, 2, twelve), + subQueryRequestResponseWithQueryTime(`sum(sum_over_time({app="foo"} | unwrap bar[1h]))`, 3, eleven), + subQueryRequestResponseWithQueryTime(`sum(sum_over_time({app="foo"} | unwrap bar[26m]))`, 4, ten), + }, + expected: expectedMergedResponseWithTime(1+2+3+4, twelve34), + }, + { + name: "sum_agg_sum_over_time_aligned_with_query_time", + splitByInterval: 1 * time.Minute, + in: &LokiInstantRequest{ + Query: `sum by (bar) (sum_over_time({app="foo"} | unwrap bar [3m]))`, + TimeTs: time.Unix(180, 0), + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum by (bar) (sum_over_time({app="foo"} | unwrap bar [3m]))`), + }, + }, + subQueries: []queryrangebase.RequestResponse{ + subQueryRequestResponseWithQueryTime(`sum by (bar)(sum_over_time({app="foo"} | unwrap bar[1m]))`, 1, time.Unix(60, 0)), + subQueryRequestResponseWithQueryTime(`sum by (bar)(sum_over_time({app="foo"} | unwrap bar[1m]))`, 2, time.Unix(120, 0)), + subQueryRequestResponseWithQueryTime(`sum by (bar)(sum_over_time({app="foo"} | unwrap bar[1m]))`, 3, time.Unix(180, 0)), + }, + expected: expectedMergedResponseWithTime(1+2+3, time.Unix(180, 0)), + }, + { + name: "sum_agg_sum_over_time_not_aligned_with_query_time", + splitByInterval: 1 * time.Hour, + in: &LokiInstantRequest{ + Query: `sum by (bar) (sum_over_time({app="foo"} | unwrap bar [3h]))`, + TimeTs: twelve34, + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum by (bar) (sum_over_time({app="foo"} | unwrap bar [3h]))`), + }, + }, + subQueries: []queryrangebase.RequestResponse{ + subQueryRequestResponseWithQueryTime(`sum by (bar)(sum_over_time({app="foo"} | unwrap bar[34m]))`, 1, twelve34), + subQueryRequestResponseWithQueryTime(`sum by (bar)(sum_over_time({app="foo"} | unwrap bar[1h]))`, 2, twelve), + subQueryRequestResponseWithQueryTime(`sum by (bar)(sum_over_time({app="foo"} | unwrap bar[1h]))`, 3, eleven), + subQueryRequestResponseWithQueryTime(`sum by (bar)(sum_over_time({app="foo"} | unwrap bar[26m]))`, 4, ten), + }, + expected: expectedMergedResponseWithTime(1+2+3+4, twelve34), + }, + } { + tc := tc + t.Run(tc.name, func(t *testing.T) { + srm := NewSplitByRangeMiddleware(log.NewNopLogger(), testEngineOpts, fakeLimits{ + maxSeries: 10000, + queryTimeout: time.Second, + instantMetricSplitDuration: map[string]time.Duration{ + "tenant": tc.splitByInterval, + }, + }, true, nilShardingMetrics) // enable splitAlign + + ctx := user.InjectOrgID(context.TODO(), "tenant") + + byTimeTs := make(map[int64]queryrangebase.RequestResponse) + for _, v := range tc.subQueries { + key := v.Request.(*LokiInstantRequest).TimeTs.UnixNano() + byTimeTs[key] = v + } + + resp, err := srm.Wrap(queryrangebase.HandlerFunc( + func(ctx context.Context, req queryrangebase.Request) (queryrangebase.Response, error) { + // req should match with one of the subqueries. + ts := req.(*LokiInstantRequest).TimeTs + subq, ok := byTimeTs[ts.UnixNano()] + if !ok { // every req **should** match with one of the subqueries + return nil, fmt.Errorf("subquery request '%s-%s' not found", req.GetQuery(), ts) + } + + // Assert subquery request + assert.Equal(t, subq.Request.GetQuery(), req.GetQuery()) + assert.Equal(t, subq.Request, req) + return subq.Response, nil + + })).Do(ctx, tc.in) + require.NoError(t, err) + assert.Equal(t, tc.expected, resp.(*LokiPromResponse).Response) + }) + } +} + func Test_RangeVectorSplit(t *testing.T) { srm := NewSplitByRangeMiddleware(log.NewNopLogger(), testEngineOpts, fakeLimits{ maxSeries: 10000, queryTimeout: time.Second, - splitDuration: map[string]time.Duration{ + instantMetricSplitDuration: map[string]time.Duration{ "tenant": time.Minute, }, - }, nilShardingMetrics) + }, false, nilShardingMetrics) ctx := user.InjectOrgID(context.TODO(), "tenant") @@ -151,6 +429,39 @@ func Test_RangeVectorSplit(t *testing.T) { } } +// subQueryRequestResponse returns a RequestResponse containing the expected subQuery instant request +// and a response containing a sample value returned from the following wrapper +func subQueryRequestResponseWithQueryTime(expectedSubQuery string, sampleValue float64, exec time.Time) queryrangebase.RequestResponse { + return queryrangebase.RequestResponse{ + Request: &LokiInstantRequest{ + Query: expectedSubQuery, + TimeTs: exec, + Path: "/loki/api/v1/query", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(expectedSubQuery), + }, + }, + Response: &LokiPromResponse{ + Response: &queryrangebase.PrometheusResponse{ + Status: loghttp.QueryStatusSuccess, + Data: queryrangebase.PrometheusData{ + ResultType: loghttp.ResultTypeVector, + Result: []queryrangebase.SampleStream{ + { + Labels: []logproto.LabelAdapter{ + {Name: "app", Value: "foo"}, + }, + Samples: []logproto.LegacySample{ + {TimestampMs: 1000, Value: sampleValue}, + }, + }, + }, + }, + }, + }, + } +} + // subQueryRequestResponse returns a RequestResponse containing the expected subQuery instant request // and a response containing a sample value returned from the following wrapper func subQueryRequestResponse(expectedSubQuery string, sampleValue float64) queryrangebase.RequestResponse { @@ -202,3 +513,20 @@ func expectedMergedResponse(expectedSampleValue float64) *queryrangebase.Prometh }, } } + +func expectedMergedResponseWithTime(expectedSampleValue float64, exec time.Time) *queryrangebase.PrometheusResponse { + return &queryrangebase.PrometheusResponse{ + Status: loghttp.QueryStatusSuccess, + Data: queryrangebase.PrometheusData{ + ResultType: loghttp.ResultTypeVector, + Result: []queryrangebase.SampleStream{ + { + Labels: []logproto.LabelAdapter{}, + Samples: []logproto.LegacySample{ + {TimestampMs: exec.UnixMilli(), Value: expectedSampleValue}, + }, + }, + }, + }, + } +} diff --git a/pkg/storage/bloom/v1/bounds.go b/pkg/storage/bloom/v1/bounds.go index 507f6e45aefc2..8f3edd914209d 100644 --- a/pkg/storage/bloom/v1/bounds.go +++ b/pkg/storage/bloom/v1/bounds.go @@ -125,10 +125,10 @@ func (b FingerprintBounds) Intersection(target FingerprintBounds) *FingerprintBo // Union returns the union of the two bounds func (b FingerprintBounds) Union(target FingerprintBounds) (res []FingerprintBounds) { if !b.Overlaps(target) { - if b.Cmp(target.Min) == Before { - return []FingerprintBounds{target, b} + if b.Less(target) { + return []FingerprintBounds{b, target} } - return []FingerprintBounds{b, target} + return []FingerprintBounds{target, b} } return []FingerprintBounds{ @@ -145,7 +145,7 @@ func (b FingerprintBounds) Unless(target FingerprintBounds) (res []FingerprintBo return []FingerprintBounds{b} } - if b == target { + if b.Within(target) { return nil } diff --git a/pkg/storage/bloom/v1/bounds_test.go b/pkg/storage/bloom/v1/bounds_test.go index 629eac61a2af7..e8362a2b283ff 100644 --- a/pkg/storage/bloom/v1/bounds_test.go +++ b/pkg/storage/bloom/v1/bounds_test.go @@ -98,7 +98,7 @@ func Test_FingerprintBounds_Union(t *testing.T) { }, NewBounds(5, 15).Union(target)) } -func Test_FingerprintBounds_Xor(t *testing.T) { +func Test_FingerprintBounds_Unless(t *testing.T) { t.Parallel() target := NewBounds(10, 20) assert.Equal(t, []FingerprintBounds{ diff --git a/pkg/storage/bloom/v1/builder.go b/pkg/storage/bloom/v1/builder.go index 8821816958b6f..b094b847f2ef5 100644 --- a/pkg/storage/bloom/v1/builder.go +++ b/pkg/storage/bloom/v1/builder.go @@ -15,7 +15,7 @@ import ( ) var ( - DefaultBlockOptions = NewBlockOptions(4, 0, 50<<20) // 50MB + DefaultBlockOptions = NewBlockOptions(4, 1, 50<<20) // 50MB ) type BlockOptions struct { @@ -521,11 +521,12 @@ func (b *IndexBuilder) Close() (uint32, error) { // from a list of blocks and a store of series. type MergeBuilder struct { // existing blocks - blocks []PeekingIterator[*SeriesWithBloom] + blocks Iterator[*SeriesWithBloom] // store store Iterator[*Series] // Add chunks to a bloom populate func(*Series, *Bloom) error + metrics *Metrics } // NewMergeBuilder is a specific builder which does the following: @@ -533,14 +534,16 @@ type MergeBuilder struct { // i) When two blocks have the same series, it will prefer the one with the most chunks already indexed // 2. iterates through the store, adding chunks to the relevant blooms via the `populate` argument func NewMergeBuilder( - blocks []PeekingIterator[*SeriesWithBloom], + blocks Iterator[*SeriesWithBloom], store Iterator[*Series], populate func(*Series, *Bloom) error, + metrics *Metrics, ) *MergeBuilder { return &MergeBuilder{ blocks: blocks, store: store, populate: populate, + metrics: metrics, } } @@ -549,24 +552,7 @@ func (mb *MergeBuilder) Build(builder *BlockBuilder) (uint32, error) { nextInBlocks *SeriesWithBloom ) - // Turn the list of blocks into a single iterator that returns the next series - mergedBlocks := NewPeekingIter[*SeriesWithBloom](NewHeapIterForSeriesWithBloom(mb.blocks...)) - // two overlapping blocks can conceivably have the same series, so we need to dedupe, - // preferring the one with the most chunks already indexed since we'll have - // to add fewer chunks to the bloom - deduped := NewDedupingIter[*SeriesWithBloom]( - func(a, b *SeriesWithBloom) bool { - return a.Series.Fingerprint == b.Series.Fingerprint - }, - Identity[*SeriesWithBloom], - func(a, b *SeriesWithBloom) *SeriesWithBloom { - if len(a.Series.Chunks) > len(b.Series.Chunks) { - return a - } - return b - }, - mergedBlocks, - ) + deduped := mb.blocks for mb.store.Next() { nextInStore := mb.store.At() @@ -585,6 +571,8 @@ func (mb *MergeBuilder) Build(builder *BlockBuilder) (uint32, error) { nextInBlocks = deduped.At() } + var chunksIndexed, chunksCopied int + cur := nextInBlocks chunksToAdd := nextInStore.Chunks // The next series from the store doesn't exist in the blocks, so we add it @@ -600,8 +588,11 @@ func (mb *MergeBuilder) Build(builder *BlockBuilder) (uint32, error) { } else { // if the series already exists in the block, we only need to add the new chunks chunksToAdd = nextInStore.Chunks.Unless(nextInBlocks.Series.Chunks) + chunksCopied = len(nextInStore.Chunks) - len(chunksToAdd) } + chunksIndexed = len(chunksToAdd) + if len(chunksToAdd) > 0 { if err := mb.populate( &Series{ @@ -614,6 +605,9 @@ func (mb *MergeBuilder) Build(builder *BlockBuilder) (uint32, error) { } } + mb.metrics.chunksIndexed.WithLabelValues(chunkIndexedTypeIterated).Add(float64(chunksIndexed)) + mb.metrics.chunksIndexed.WithLabelValues(chunkIndexedTypeCopied).Add(float64(chunksCopied)) + blockFull, err := builder.AddSeries(*cur) if err != nil { return 0, errors.Wrap(err, "adding series to block") @@ -623,6 +617,10 @@ func (mb *MergeBuilder) Build(builder *BlockBuilder) (uint32, error) { } } + if err := mb.store.Err(); err != nil { + return 0, errors.Wrap(err, "iterating store") + } + checksum, err := builder.Close() if err != nil { return 0, errors.Wrap(err, "closing block") diff --git a/pkg/storage/bloom/v1/builder_test.go b/pkg/storage/bloom/v1/builder_test.go index ac38cdeec427a..0013ad8744579 100644 --- a/pkg/storage/bloom/v1/builder_test.go +++ b/pkg/storage/bloom/v1/builder_test.go @@ -150,6 +150,23 @@ func TestBlockBuilderRoundTrip(t *testing.T) { } } +func dedupedBlocks(blocks []PeekingIterator[*SeriesWithBloom]) Iterator[*SeriesWithBloom] { + orderedBlocks := NewHeapIterForSeriesWithBloom(blocks...) + return NewDedupingIter[*SeriesWithBloom]( + func(a *SeriesWithBloom, b *SeriesWithBloom) bool { + return a.Series.Fingerprint == b.Series.Fingerprint + }, + Identity[*SeriesWithBloom], + func(a *SeriesWithBloom, b *SeriesWithBloom) *SeriesWithBloom { + if len(a.Series.Chunks) > len(b.Series.Chunks) { + return a + } + return b + }, + NewPeekingIter[*SeriesWithBloom](orderedBlocks), + ) +} + func TestMergeBuilder(t *testing.T) { t.Parallel() @@ -209,7 +226,7 @@ func TestMergeBuilder(t *testing.T) { ) // Ensure that the merge builder combines all the blocks correctly - mergeBuilder := NewMergeBuilder(blocks, storeItr, pop) + mergeBuilder := NewMergeBuilder(dedupedBlocks(blocks), storeItr, pop, NewMetrics(nil)) indexBuf := bytes.NewBuffer(nil) bloomsBuf := bytes.NewBuffer(nil) writer := NewMemoryBlockWriter(indexBuf, bloomsBuf) @@ -377,12 +394,13 @@ func TestMergeBuilder_Roundtrip(t *testing.T) { writer := NewMemoryBlockWriter(indexBuf, bloomBuf) reader := NewByteReader(indexBuf, bloomBuf) mb := NewMergeBuilder( - blocks, + dedupedBlocks(blocks), dedupedStore, func(s *Series, b *Bloom) error { // We're not actually indexing new data in this test return nil }, + NewMetrics(nil), ) builder, err := NewBlockBuilder(DefaultBlockOptions, writer) require.Nil(t, err) diff --git a/pkg/storage/bloom/v1/fuse_test.go b/pkg/storage/bloom/v1/fuse_test.go index 2ab0a51bced2c..1e35895794c22 100644 --- a/pkg/storage/bloom/v1/fuse_test.go +++ b/pkg/storage/bloom/v1/fuse_test.go @@ -13,7 +13,6 @@ import ( ) func TestFusedQuerier(t *testing.T) { - t.Parallel() // references for linking in memory reader+writer indexBuf := bytes.NewBuffer(nil) bloomsBuf := bytes.NewBuffer(nil) diff --git a/pkg/storage/bloom/v1/index.go b/pkg/storage/bloom/v1/index.go index e3a14dc5453ea..58d43b8cd0aca 100644 --- a/pkg/storage/bloom/v1/index.go +++ b/pkg/storage/bloom/v1/index.go @@ -234,8 +234,8 @@ func aggregateHeaders(xs []SeriesHeader) SeriesHeader { Bounds: NewBounds(fromFp, throughFP), } - for _, x := range xs { - if x.FromTs < res.FromTs { + for i, x := range xs { + if i == 0 || x.FromTs < res.FromTs { res.FromTs = x.FromTs } if x.ThroughTs > res.ThroughTs { diff --git a/pkg/storage/bloom/v1/metrics.go b/pkg/storage/bloom/v1/metrics.go index aa604c29f1573..f5568a9d76596 100644 --- a/pkg/storage/bloom/v1/metrics.go +++ b/pkg/storage/bloom/v1/metrics.go @@ -10,12 +10,16 @@ type Metrics struct { 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 + chunksIndexed *prometheus.CounterVec } +const chunkIndexedTypeIterated = "iterated" +const chunkIndexedTypeCopied = "copied" + func NewMetrics(r prometheus.Registerer) *Metrics { return &Metrics{ sbfCreationTime: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Name: "bloom_creation_time", + Name: "bloom_creation_time_total", Help: "Time spent creating scalable bloom filters", }), bloomSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ @@ -33,5 +37,9 @@ func NewMetrics(r prometheus.Registerer) *Metrics { Help: "Estimated number of elements in the bloom filter", Buckets: prometheus.ExponentialBucketsRange(1, 33554432, 10), }), + chunksIndexed: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Name: "bloom_chunks_indexed_total", + Help: "Number of chunks indexed in bloom filters, partitioned by type. Type can be iterated or copied, where iterated indicates the chunk data was fetched and ngrams for it's contents generated whereas copied indicates the chunk already existed in another source block and was copied to the new block", + }, []string{"type"}), } } diff --git a/pkg/storage/bloom/v1/util.go b/pkg/storage/bloom/v1/util.go index 3b9e0631b715d..14c2ea2a03f84 100644 --- a/pkg/storage/bloom/v1/util.go +++ b/pkg/storage/bloom/v1/util.go @@ -276,3 +276,61 @@ func NewPeekCloseIter[T any](itr CloseableIterator[T]) *PeekCloseIter[T] { func (it *PeekCloseIter[T]) Close() error { return it.close() } + +type ResettableIterator[T any] interface { + Reset() error + Iterator[T] +} + +type CloseableResettableIterator[T any] interface { + CloseableIterator[T] + ResettableIterator[T] +} + +type Predicate[T any] func(T) bool + +func NewFilterIter[T any](it Iterator[T], p Predicate[T]) *FilterIter[T] { + return &FilterIter[T]{ + Iterator: it, + match: p, + } +} + +type FilterIter[T any] struct { + Iterator[T] + match Predicate[T] +} + +func (i *FilterIter[T]) Next() bool { + hasNext := i.Iterator.Next() + for hasNext && !i.match(i.Iterator.At()) { + hasNext = i.Iterator.Next() + } + return hasNext +} + +type CounterIterator[T any] interface { + Iterator[T] + Count() int +} + +type CounterIter[T any] struct { + Iterator[T] // the underlying iterator + count int +} + +func NewCounterIter[T any](itr Iterator[T]) *CounterIter[T] { + return &CounterIter[T]{Iterator: itr} +} + +func (it *CounterIter[T]) Next() bool { + if it.Iterator.Next() { + it.count++ + return true + } + return false +} + +func (it *CounterIter[T]) Count() int { + return it.count +} diff --git a/pkg/storage/bloom/v1/util_test.go b/pkg/storage/bloom/v1/util_test.go index ad89a226ec7f7..afafa4d05a870 100644 --- a/pkg/storage/bloom/v1/util_test.go +++ b/pkg/storage/bloom/v1/util_test.go @@ -26,3 +26,29 @@ func TestPeekingIterator(t *testing.T) { require.False(t, itr.Next()) } + +func TestCounterIter(t *testing.T) { + t.Parallel() + + data := []int{1, 2, 3, 4, 5} + itr := NewCounterIter[int](NewSliceIter[int](data)) + peekItr := NewPeekingIter[int](itr) + + // Consume the outer iter and use peek + for { + if _, ok := peekItr.Peek(); !ok { + break + } + if !peekItr.Next() { + break + } + } + // Both iterators should be exhausted + require.False(t, itr.Next()) + require.Nil(t, itr.Err()) + require.False(t, peekItr.Next()) + require.Nil(t, peekItr.Err()) + + // Assert that the count is correct and peeking hasn't jeopardized the count + require.Equal(t, len(data), itr.Count()) +} diff --git a/pkg/storage/config/schema_config.go b/pkg/storage/config/schema_config.go index 30b9de98b14ba..968ca87e609b7 100644 --- a/pkg/storage/config/schema_config.go +++ b/pkg/storage/config/schema_config.go @@ -200,10 +200,6 @@ func (cfg *PeriodConfig) GetIndexTableNumberRange(schemaEndDate DayTime) TableRa } } -func (cfg *PeriodConfig) GetFullTableName(t model.Time) string { - return NewDayTime(t).TableWithPrefix(cfg) -} - func NewDayTime(d model.Time) DayTime { return DayTime{d} } @@ -237,17 +233,6 @@ func (d DayTime) String() string { return d.Time.Time().UTC().Format("2006-01-02") } -func (d DayTime) Table() string { - return fmt.Sprintf("%d", - d.ModelTime().Time().UnixNano()/int64(ObjectStorageIndexRequiredPeriod)) -} - -func (d DayTime) TableWithPrefix(cfg *PeriodConfig) string { - return fmt.Sprintf("%s%d", - cfg.IndexTables.Prefix, - d.ModelTime().Time().UnixNano()/int64(ObjectStorageIndexRequiredPeriod)) -} - func (d DayTime) Inc() DayTime { return DayTime{d.Add(ObjectStorageIndexRequiredPeriod)} } @@ -272,6 +257,30 @@ func (d DayTime) Bounds() (model.Time, model.Time) { return d.Time, d.Inc().Time } +type DayTable struct { + DayTime + Prefix string +} + +func (d DayTable) String() string { + return d.Addr() +} + +func NewDayTable(d DayTime, prefix string) DayTable { + return DayTable{ + DayTime: d, + Prefix: prefix, + } +} + +// Addr returns the prefix (if any) and the unix day offset as a string, which is used +// as the address for the index table in storage. +func (d DayTable) Addr() string { + return fmt.Sprintf("%s%d", + d.Prefix, + d.ModelTime().Time().UnixNano()/int64(ObjectStorageIndexRequiredPeriod)) +} + // SchemaConfig contains the config for our chunk index schemas type SchemaConfig struct { Configs []PeriodConfig `yaml:"configs"` diff --git a/pkg/storage/stores/shipper/bloomshipper/cache.go b/pkg/storage/stores/shipper/bloomshipper/cache.go index f5ceb930017f7..d249cb68ce569 100644 --- a/pkg/storage/stores/shipper/bloomshipper/cache.go +++ b/pkg/storage/stores/shipper/bloomshipper/cache.go @@ -29,6 +29,13 @@ func (c *CloseableBlockQuerier) Close() error { return nil } +func (c *CloseableBlockQuerier) SeriesIter() (v1.PeekingIterator[*v1.SeriesWithBloom], error) { + if err := c.Reset(); err != nil { + return nil, err + } + return v1.NewPeekingIter[*v1.SeriesWithBloom](c.BlockQuerier), nil +} + func NewBlocksCache(cfg cache.EmbeddedCacheConfig, reg prometheus.Registerer, logger log.Logger) *cache.EmbeddedCache[string, BlockDirectory] { return cache.NewTypedEmbeddedCache[string, BlockDirectory]( "bloom-blocks-cache", diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index 1dbfac579c5aa..240f2b5166588 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -88,45 +88,62 @@ type Meta struct { // The specific TSDB files used to generate the block. Sources []tsdb.SingleTenantTSDBIdentifier - // Old blocks which can be deleted in the future. These should be from previous compaction rounds. - Tombstones []BlockRef - // A list of blocks that were generated Blocks []BlockRef } -// TODO(owen-d): use this to update internal ref's checksum. -func (m Meta) Checksum() (uint32, error) { +func MetaRefFrom( + tenant, + table string, + bounds v1.FingerprintBounds, + sources []tsdb.SingleTenantTSDBIdentifier, + blocks []BlockRef, +) (MetaRef, error) { + h := v1.Crc32HashPool.Get() defer v1.Crc32HashPool.Put(h) - err := m.Bounds.Hash(h) + err := bounds.Hash(h) if err != nil { - return 0, errors.Wrap(err, "writing OwnershipRange") + return MetaRef{}, errors.Wrap(err, "writing OwnershipRange") } - for _, tombstone := range m.Tombstones { - err = tombstone.Hash(h) + for _, source := range sources { + err = source.Hash(h) if err != nil { - return 0, errors.Wrap(err, "writing Tombstones") + return MetaRef{}, errors.Wrap(err, "writing Sources") } } - for _, source := range m.Sources { - err = source.Hash(h) - if err != nil { - return 0, errors.Wrap(err, "writing Sources") + var ( + start, end model.Time + ) + + for i, block := range blocks { + if i == 0 || block.StartTimestamp.Before(start) { + start = block.StartTimestamp + } + + if block.EndTimestamp.After(end) { + end = block.EndTimestamp } - } - for _, block := range m.Blocks { err = block.Hash(h) if err != nil { - return 0, errors.Wrap(err, "writing Blocks") + return MetaRef{}, errors.Wrap(err, "writing Blocks") } } - return h.Sum32(), nil + return MetaRef{ + Ref: Ref{ + TenantID: tenant, + TableName: table, + Bounds: bounds, + StartTimestamp: start, + EndTimestamp: end, + Checksum: h.Sum32(), + }, + }, nil } @@ -200,6 +217,7 @@ type BlockClient interface { type Client interface { MetaClient BlockClient + IsObjectNotFoundErr(err error) bool Stop() } @@ -224,6 +242,10 @@ func NewBloomClient(cfg bloomStoreConfig, client client.ObjectClient, logger log }, nil } +func (b *BloomClient) IsObjectNotFoundErr(err error) bool { + return b.client.IsObjectNotFoundErr(err) +} + func (b *BloomClient) PutMeta(ctx context.Context, meta Meta) error { data, err := json.Marshal(meta) if err != nil { @@ -300,6 +322,7 @@ func (b *BloomClient) DeleteBlocks(ctx context.Context, references []BlockRef) e ref := references[idx] key := b.Block(ref).Addr() err := b.client.DeleteObject(ctx, key) + if err != nil { return fmt.Errorf("error deleting block file: %w", err) } diff --git a/pkg/storage/stores/shipper/bloomshipper/client_test.go b/pkg/storage/stores/shipper/bloomshipper/client_test.go index 02d80429d7f11..e5bbe3b5b1bf5 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/client_test.go @@ -63,8 +63,7 @@ func putMeta(c *BloomClient, tenant string, start model.Time, minFp, maxFp model // EndTimestamp: start.Add(12 * time.Hour), }, }, - Blocks: []BlockRef{}, - Tombstones: []BlockRef{}, + Blocks: []BlockRef{}, } raw, _ := json.Marshal(meta) return meta, c.client.PutObject(context.Background(), c.Meta(meta.MetaRef).Addr(), bytes.NewReader(raw)) @@ -129,8 +128,7 @@ func TestBloomClient_PutMeta(t *testing.T) { // EndTimestamp: start.Add(12 * time.Hour), }, }, - Blocks: []BlockRef{}, - Tombstones: []BlockRef{}, + Blocks: []BlockRef{}, } err := c.PutMeta(ctx, meta) diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go index fd1e6157a1cae..962bebb9956fd 100644 --- a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go @@ -34,8 +34,7 @@ func makeMetas(t *testing.T, schemaCfg config.SchemaConfig, ts model.Time, keysp EndTimestamp: ts, }, }, - Tombstones: []BlockRef{}, - Blocks: []BlockRef{}, + Blocks: []BlockRef{}, } } return metas diff --git a/pkg/storage/stores/shipper/bloomshipper/resolver.go b/pkg/storage/stores/shipper/bloomshipper/resolver.go index 40a59cee42dbc..7d224b9f01392 100644 --- a/pkg/storage/stores/shipper/bloomshipper/resolver.go +++ b/pkg/storage/stores/shipper/bloomshipper/resolver.go @@ -14,6 +14,9 @@ const ( BloomPrefix = "bloom" MetasPrefix = "metas" BlocksPrefix = "blocks" + + extTarGz = ".tar.gz" + extJSON = ".json" ) // KeyResolver is an interface for resolving keys to locations. @@ -36,7 +39,7 @@ func (defaultKeyResolver) Meta(ref MetaRef) Location { fmt.Sprintf("%v", ref.TableName), ref.TenantID, MetasPrefix, - fmt.Sprintf("%v-%v", ref.Bounds, ref.Checksum), + fmt.Sprintf("%v-%x%s", ref.Bounds, ref.Checksum, extJSON), } } @@ -50,7 +53,8 @@ func (defaultKeyResolver) ParseMetaKey(loc Location) (MetaRef, error) { if err != nil { return MetaRef{}, fmt.Errorf("failed to parse bounds of meta key %s : %w", loc, err) } - checksum, err := strconv.ParseUint(fnParts[2], 16, 64) + withoutExt := strings.TrimSuffix(fnParts[2], extJSON) + checksum, err := strconv.ParseUint(withoutExt, 16, 64) if err != nil { return MetaRef{}, fmt.Errorf("failed to parse checksum of meta key %s : %w", loc, err) } @@ -77,7 +81,7 @@ func (defaultKeyResolver) Block(ref BlockRef) Location { ref.TenantID, BlocksPrefix, ref.Bounds.String(), - fmt.Sprintf("%d-%d-%x", ref.StartTimestamp, ref.EndTimestamp, ref.Checksum), + fmt.Sprintf("%d-%d-%x%s", ref.StartTimestamp, ref.EndTimestamp, ref.Checksum, extTarGz), } } diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index dee8f2464bcbb..3267886ac063e 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -55,30 +55,15 @@ func (s *Shipper) Stop() { } // BlocksForMetas returns all the blocks from all the metas listed that are within the requested bounds -// and not tombstoned in any of the metas -func BlocksForMetas(metas []Meta, interval Interval, keyspaces []v1.FingerprintBounds) []BlockRef { - blocks := make(map[BlockRef]bool) // block -> isTombstoned - +func BlocksForMetas(metas []Meta, interval Interval, keyspaces []v1.FingerprintBounds) (refs []BlockRef) { for _, meta := range metas { - for _, tombstone := range meta.Tombstones { - blocks[tombstone] = true - } for _, block := range meta.Blocks { - tombstoned, ok := blocks[block] - if ok && tombstoned { - // skip tombstoned blocks - continue + if !isOutsideRange(block, interval, keyspaces) { + refs = append(refs, block) } - blocks[block] = false } } - refs := make([]BlockRef, 0, len(blocks)) - for ref, tombstoned := range blocks { - if !tombstoned && !isOutsideRange(ref, interval, keyspaces) { - refs = append(refs, ref) - } - } sort.Slice(refs, func(i, j int) bool { return refs[i].Bounds.Less(refs[j].Bounds) }) diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go index a3a9442ddfb03..e03d72c26ba37 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go @@ -14,49 +14,6 @@ import ( ) func TestBloomShipper_findBlocks(t *testing.T) { - t.Run("expected block that are specified in tombstones to be filtered out", func(t *testing.T) { - metas := []Meta{ - { - Blocks: []BlockRef{ - //this blockRef is marked as deleted in the next meta - createMatchingBlockRef(1), - createMatchingBlockRef(2), - }, - }, - { - Blocks: []BlockRef{ - //this blockRef is marked as deleted in the next meta - createMatchingBlockRef(3), - createMatchingBlockRef(4), - }, - }, - { - Tombstones: []BlockRef{ - createMatchingBlockRef(1), - createMatchingBlockRef(3), - }, - Blocks: []BlockRef{ - createMatchingBlockRef(5), - }, - }, - } - - ts := model.Now() - - interval := NewInterval( - ts.Add(-2*time.Hour), - ts.Add(-1*time.Hour), - ) - blocks := BlocksForMetas(metas, interval, []v1.FingerprintBounds{{Min: 100, Max: 200}}) - - expectedBlockRefs := []BlockRef{ - createMatchingBlockRef(2), - createMatchingBlockRef(4), - createMatchingBlockRef(5), - } - require.ElementsMatch(t, expectedBlockRefs, blocks) - }) - tests := map[string]struct { minFingerprint uint64 maxFingerprint uint64 diff --git a/pkg/storage/stores/shipper/bloomshipper/store_test.go b/pkg/storage/stores/shipper/bloomshipper/store_test.go index 56179cde97c36..c99aa46df4bf3 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/store_test.go @@ -83,8 +83,7 @@ func createMetaInStorage(store *BloomStore, tenant string, start model.Time, min // EndTimestamp: start.Add(12 * time.Hour), }, }, - Blocks: []BlockRef{}, - Tombstones: []BlockRef{}, + Blocks: []BlockRef{}, } err := store.storeDo(start, func(s *bloomStoreEntry) error { raw, _ := json.Marshal(meta) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go index 451688d040e36..943127f3e6b67 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go @@ -128,7 +128,7 @@ func ParseSingleTenantTSDBPath(p string) (id SingleTenantTSDBIdentifier, ok bool return } - checksum, err := strconv.ParseInt(elems[4], 16, 32) + checksum, err := strconv.ParseUint(elems[4], 16, 32) if err != nil { return } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier_test.go index 7fcd56f89b0eb..b21e8352b7a84 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier_test.go @@ -1,6 +1,8 @@ package tsdb import ( + "fmt" + "math" "testing" "time" @@ -9,11 +11,10 @@ import ( func TestParseSingleTenantTSDBPath(t *testing.T) { for _, tc := range []struct { - desc string - input string - id SingleTenantTSDBIdentifier - parent string - ok bool + desc string + input string + id SingleTenantTSDBIdentifier + ok bool }{ { desc: "simple_works", @@ -24,8 +25,18 @@ func TestParseSingleTenantTSDBPath(t *testing.T) { Through: 10, Checksum: 255, }, - parent: "parent", - ok: true, + ok: true, + }, + { + desc: "uint32_max_checksum_works", + input: fmt.Sprintf("1-compactor-1-10-%x.tsdb", math.MaxUint32), + id: SingleTenantTSDBIdentifier{ + TS: time.Unix(1, 0), + From: 1, + Through: 10, + Checksum: math.MaxUint32, + }, + ok: true, }, { desc: "wrong uploader name", @@ -45,8 +56,8 @@ func TestParseSingleTenantTSDBPath(t *testing.T) { } { t.Run(tc.desc, func(t *testing.T) { id, ok := ParseSingleTenantTSDBPath(tc.input) - require.Equal(t, tc.id, id) require.Equal(t, tc.ok, ok) + require.Equal(t, tc.id, id) }) } } diff --git a/pkg/tracing/config.go b/pkg/tracing/config.go index f9faefa6a7303..a0877a9ce836f 100644 --- a/pkg/tracing/config.go +++ b/pkg/tracing/config.go @@ -11,6 +11,7 @@ type Config struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.BoolVar(&cfg.Enabled, "tracing.enabled", true, "Set to false to disable tracing.") + f.BoolVar(&cfg.ProfilingEnabled, "tracing.profiling-enabled", true, "Set to true to enable profiling integration.") } func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { diff --git a/pkg/util/marshal/legacy/marshal_test.go b/pkg/util/marshal/legacy/marshal_test.go index 6e07d84615928..a3dca73ac299f 100644 --- a/pkg/util/marshal/legacy/marshal_test.go +++ b/pkg/util/marshal/legacy/marshal_test.go @@ -161,6 +161,16 @@ var queryTests = []struct { "downloadTime": 0, "queryLengthServed": 0 }, + "instantMetricResult": { + "entriesFound": 0, + "entriesRequested": 0, + "entriesStored": 0, + "bytesReceived": 0, + "bytesSent": 0, + "requests": 0, + "downloadTime": 0, + "queryLengthServed": 0 + }, "result": { "entriesFound": 0, "entriesRequested": 0, @@ -180,7 +190,7 @@ var queryTests = []struct { "shards": 0, "splits": 0, "subqueries": 0, - "totalBytesProcessed": 0, + "totalBytesProcessed": 0, "totalEntriesReturned": 0, "totalLinesProcessed": 0, "totalStructuredMetadataBytesProcessed": 0, diff --git a/pkg/util/marshal/marshal_test.go b/pkg/util/marshal/marshal_test.go index d5336298c37c8..ce7a49f97e76c 100644 --- a/pkg/util/marshal/marshal_test.go +++ b/pkg/util/marshal/marshal_test.go @@ -129,6 +129,16 @@ const emptyStats = `{ "downloadTime": 0, "queryLengthServed": 0 }, + "instantMetricResult": { + "entriesFound": 0, + "entriesRequested": 0, + "entriesStored": 0, + "bytesReceived": 0, + "bytesSent": 0, + "requests": 0, + "downloadTime": 0, + "queryLengthServed": 0 + }, "result": { "entriesFound": 0, "entriesRequested": 0, @@ -208,13 +218,13 @@ var queryTestWithEncodingFlags = []struct { [ "123456789012346", "super line with labels", { "structuredMetadata": { "foo": "a", - "bar": "b" - } + "bar": "b" + } }], [ "123456789012347", "super line with labels msg=text", { "structuredMetadata": { "foo": "a", - "bar": "b" + "bar": "b" }, "parsed": { "msg": "text" @@ -549,13 +559,13 @@ var tailTestWithEncodingFlags = []struct { [ "123456789012346", "super line with labels", { "structuredMetadata": { "foo": "a", - "bar": "b" - } + "bar": "b" + } }], [ "123456789012347", "super line with labels msg=text", { "structuredMetadata": { "foo": "a", - "bar": "b" + "bar": "b" }, "parsed": { "msg": "text" diff --git a/pkg/util/ring/sharding.go b/pkg/util/ring/sharding.go index 45a53cf40cfe7..460e22d7f4c45 100644 --- a/pkg/util/ring/sharding.go +++ b/pkg/util/ring/sharding.go @@ -2,12 +2,10 @@ package ring import ( "github.com/grafana/dskit/ring" - "github.com/prometheus/common/model" ) type TenantSharding interface { - GetTenantSubRing(tenantID string) ring.ReadRing - OwnsTenant(tenantID string) bool + OwnsTenant(tenantID string) (tenantRing ring.ReadRing, owned bool) } type TenantShuffleSharding struct { @@ -28,60 +26,19 @@ func NewTenantShuffleSharding( } } -func (s *TenantShuffleSharding) GetTenantSubRing(tenantID string) ring.ReadRing { - shardSize := s.shardSizeForTenant(tenantID) - +func (s *TenantShuffleSharding) OwnsTenant(tenantID string) (ring.ReadRing, bool) { // A shard size of 0 means shuffle sharding is disabled for this specific user, + shardSize := s.shardSizeForTenant(tenantID) if shardSize <= 0 { - return s.r + return s.r, true } - return s.r.ShuffleShard(tenantID, shardSize) -} - -func (s *TenantShuffleSharding) OwnsTenant(tenantID string) bool { - subRing := s.GetTenantSubRing(tenantID) - return subRing.HasInstance(s.ringLifeCycler.GetInstanceID()) -} - -type FingerprintSharding interface { - OwnsFingerprint(fp model.Fingerprint) (bool, error) -} - -// FingerprintShuffleSharding is not thread-safe. -type FingerprintShuffleSharding struct { - r ring.ReadRing - ringLifeCycler *ring.BasicLifecycler - ringOp ring.Operation - - // Buffers for ring.Get() calls. - bufDescs []ring.InstanceDesc - bufHosts, bufZones []string -} - -func NewFingerprintShuffleSharding( - r ring.ReadRing, - ringLifeCycler *ring.BasicLifecycler, - ringOp ring.Operation, -) *FingerprintShuffleSharding { - s := FingerprintShuffleSharding{ - r: r, - ringLifeCycler: ringLifeCycler, - ringOp: ringOp, + subRing := s.r.ShuffleShard(tenantID, shardSize) + if subRing.HasInstance(s.ringLifeCycler.GetInstanceID()) { + return subRing, true } - s.bufDescs, s.bufHosts, s.bufZones = ring.MakeBuffersForGet() - - return &s -} - -func (s *FingerprintShuffleSharding) OwnsFingerprint(fp uint64) (bool, error) { - rs, err := s.r.Get(uint32(fp), s.ringOp, s.bufDescs, s.bufHosts, s.bufZones) - if err != nil { - return false, err - } - - return rs.Includes(s.ringLifeCycler.GetInstanceAddr()), nil + return nil, false } // NoopStrategy is an implementation of the ShardingStrategy that does not @@ -89,16 +46,6 @@ func (s *FingerprintShuffleSharding) OwnsFingerprint(fp uint64) (bool, error) { type NoopStrategy struct{} // OwnsTenant implements TenantShuffleSharding. -func (s *NoopStrategy) OwnsTenant(_ string) bool { - return false -} - -// GetTenantSubRing implements TenantShuffleSharding. -func (s *NoopStrategy) GetTenantSubRing(_ string) ring.ReadRing { - return nil -} - -// OwnsFingerprint implements FingerprintSharding. -func (s *NoopStrategy) OwnsFingerprint(_ uint64) (bool, error) { - return false, nil +func (s *NoopStrategy) OwnsTenant(_ string) (ring.ReadRing, bool) { + return nil, false } diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index 13885c0fcb52d..ab845380f9682 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -111,6 +111,7 @@ type Limits struct { MetadataQuerySplitDuration model.Duration `yaml:"split_metadata_queries_by_interval" json:"split_metadata_queries_by_interval"` RecentMetadataQuerySplitDuration model.Duration `yaml:"split_recent_metadata_queries_by_interval" json:"split_recent_metadata_queries_by_interval"` RecentMetadataQueryWindow model.Duration `yaml:"recent_metadata_query_window" json:"recent_metadata_query_window"` + InstantMetricQuerySplitDuration model.Duration `yaml:"split_instant_metric_queries_by_interval" json:"split_instant_metric_queries_by_interval"` IngesterQuerySplitDuration model.Duration `yaml:"split_ingester_queries_by_interval" json:"split_ingester_queries_by_interval"` MinShardingLookback model.Duration `yaml:"min_sharding_lookback" json:"min_sharding_lookback"` MaxQueryBytesRead flagext.ByteSize `yaml:"max_query_bytes_read" json:"max_query_bytes_read"` @@ -307,6 +308,8 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { _ = l.QuerySplitDuration.Set("1h") f.Var(&l.QuerySplitDuration, "querier.split-queries-by-interval", "Split queries by a time interval and execute in parallel. The value 0 disables splitting by time. This also determines how cache keys are chosen when result caching is enabled.") + _ = l.InstantMetricQuerySplitDuration.Set("1h") + f.Var(&l.InstantMetricQuerySplitDuration, "querier.split-instant-metric-queries-by-interval", "Split instant metric queries by a time interval and execute in parallel. The value 0 disables splitting instant metric queries by time. This also determines how cache keys are chosen when instant metric query result caching is enabled.") _ = l.MetadataQuerySplitDuration.Set("24h") f.Var(&l.MetadataQuerySplitDuration, "querier.split-metadata-queries-by-interval", "Split metadata queries by a time interval and execute in parallel. The value 0 disables splitting metadata queries by time. This also determines how cache keys are chosen when label/series result caching is enabled.") @@ -334,12 +337,17 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { f.BoolVar(&l.BloomCompactorEnabled, "bloom-compactor.enable-compaction", false, "Whether to compact chunks into bloom filters.") f.IntVar(&l.BloomCompactorChunksBatchSize, "bloom-compactor.chunks-batch-size", 100, "The batch size of the chunks the bloom-compactor downloads at once.") f.IntVar(&l.BloomNGramLength, "bloom-compactor.ngram-length", 4, "Length of the n-grams created when computing blooms from log lines.") - f.IntVar(&l.BloomNGramSkip, "bloom-compactor.ngram-skip", 0, "Skip factor for the n-grams created when computing blooms from log lines.") + f.IntVar(&l.BloomNGramSkip, "bloom-compactor.ngram-skip", 1, "Skip factor for the n-grams created when computing blooms from log lines.") f.Float64Var(&l.BloomFalsePositiveRate, "bloom-compactor.false-positive-rate", 0.01, "Scalable Bloom Filter desired false-positive rate.") f.IntVar(&l.BloomGatewayBlocksDownloadingParallelism, "bloom-gateway.blocks-downloading-parallelism", 50, "Maximum number of blocks will be downloaded in parallel by the Bloom Gateway.") f.DurationVar(&l.BloomGatewayCacheKeyInterval, "bloom-gateway.cache-key-interval", 15*time.Minute, "Interval for computing the cache key in the Bloom Gateway.") _ = l.BloomCompactorMaxBlockSize.Set(defaultBloomCompactorMaxBlockSize) - f.Var(&l.BloomCompactorMaxBlockSize, "bloom-compactor.max-block-size", "The maximum bloom block size. A value of 0 sets an unlimited size. Default is 200MB. The actual block size might exceed this limit since blooms will be added to blocks until the block exceeds the maximum block size.") + f.Var(&l.BloomCompactorMaxBlockSize, "bloom-compactor.max-block-size", + fmt.Sprintf( + "The maximum bloom block size. A value of 0 sets an unlimited size. Default is %s. The actual block size might exceed this limit since blooms will be added to blocks until the block exceeds the maximum block size.", + defaultBloomCompactorMaxBlockSize, + ), + ) l.ShardStreams = &shardstreams.Config{} l.ShardStreams.RegisterFlagsWithPrefix("shard-streams", f) @@ -596,6 +604,11 @@ func (o *Overrides) QuerySplitDuration(userID string) time.Duration { return time.Duration(o.getOverridesForUser(userID).QuerySplitDuration) } +// InstantMetricQuerySplitDuration returns the tenant specific instant metric queries splitby interval applied in the query frontend. +func (o *Overrides) InstantMetricQuerySplitDuration(userID string) time.Duration { + return time.Duration(o.getOverridesForUser(userID).InstantMetricQuerySplitDuration) +} + // MetadataQuerySplitDuration returns the tenant specific metadata splitby interval applied in the query frontend. func (o *Overrides) MetadataQuerySplitDuration(userID string) time.Duration { return time.Duration(o.getOverridesForUser(userID).MetadataQuerySplitDuration) diff --git a/production/helm/loki/CHANGELOG.md b/production/helm/loki/CHANGELOG.md index 47d8f6333e4e9..59487c984d6e3 100644 --- a/production/helm/loki/CHANGELOG.md +++ b/production/helm/loki/CHANGELOG.md @@ -13,6 +13,13 @@ 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.43.2 + +- [BUGFIX] Added `alibabacloud` to `isUsingObjectStorage` check. + +## 5.43.1 + +- [BUGFIX] Fix `toPorts` fields in the `ciliumnetworkpolicy` template ## 5.43.0 diff --git a/production/helm/loki/Chart.yaml b/production/helm/loki/Chart.yaml index ffa62c88d5cd3..c20abdba25d80 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.4 -version: 5.43.0 +version: 5.43.2 home: https://grafana.github.io/helm-charts sources: - https://github.com/grafana/loki diff --git a/production/helm/loki/README.md b/production/helm/loki/README.md index 5db87e6d801e0..86a13201c2ebd 100644 --- a/production/helm/loki/README.md +++ b/production/helm/loki/README.md @@ -1,6 +1,6 @@ # loki -![Version: 5.43.0](https://img.shields.io/badge/Version-5.43.0-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 2.9.4](https://img.shields.io/badge/AppVersion-2.9.4-informational?style=flat-square) +![Version: 5.43.2](https://img.shields.io/badge/Version-5.43.2-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 2.9.4](https://img.shields.io/badge/AppVersion-2.9.4-informational?style=flat-square) Helm chart for Grafana Loki in simple, scalable mode diff --git a/production/helm/loki/templates/_helpers.tpl b/production/helm/loki/templates/_helpers.tpl index 9dd70123189e9..502c7650010b6 100644 --- a/production/helm/loki/templates/_helpers.tpl +++ b/production/helm/loki/templates/_helpers.tpl @@ -597,7 +597,7 @@ Create the service endpoint including port for MinIO. {{/* Determine if deployment is using object storage */}} {{- define "loki.isUsingObjectStorage" -}} -{{- or (eq .Values.loki.storage.type "gcs") (eq .Values.loki.storage.type "s3") (eq .Values.loki.storage.type "azure") (eq .Values.loki.storage.type "swift") -}} +{{- or (eq .Values.loki.storage.type "gcs") (eq .Values.loki.storage.type "s3") (eq .Values.loki.storage.type "azure") (eq .Values.loki.storage.type "swift") (eq .Values.loki.storage.type "alibabacloud") -}} {{- end -}} {{/* Configure the correct name for the memberlist service */}} diff --git a/production/helm/loki/templates/ciliumnetworkpolicy.yaml b/production/helm/loki/templates/ciliumnetworkpolicy.yaml index ddcef3b61a8ec..1eb1644dd850e 100644 --- a/production/helm/loki/templates/ciliumnetworkpolicy.yaml +++ b/production/helm/loki/templates/ciliumnetworkpolicy.yaml @@ -60,8 +60,9 @@ spec: {{- include "loki.selectorLabels" . | nindent 6 }} ingress: - toPorts: - - port: http - protocol: TCP + - ports: + - port: http + protocol: TCP {{- if .Values.networkPolicy.ingress.namespaceSelector }} fromEndpoints: - matchLabels: @@ -85,8 +86,9 @@ spec: {{- include "loki.selectorLabels" . | nindent 6 }} ingress: - toPorts: - - port: http-metrics - protocol: TCP + - ports: + - port: http-metrics + protocol: TCP {{- if .Values.networkPolicy.metrics.cidrs }} {{- range $cidr := .Values.networkPolicy.metrics.cidrs }} toCIDR: @@ -116,8 +118,9 @@ spec: {{- include "loki.backendSelectorLabels" . | nindent 6 }} egress: - toPorts: - - port: {{ .Values.networkPolicy.alertmanager.port }} - protocol: TCP + - ports: + - port: "{{ .Values.networkPolicy.alertmanager.port }}" + protocol: TCP {{- if .Values.networkPolicy.alertmanager.namespaceSelector }} toEndpoints: - matchLabels: @@ -142,10 +145,11 @@ spec: {{- include "loki.selectorLabels" . | nindent 6 }} egress: - toPorts: - {{- range $port := .Values.networkPolicy.externalStorage.ports }} - - port: {{ $port }} - protocol: TCP - {{- end }} + - ports: + {{- range $port := .Values.networkPolicy.externalStorage.ports }} + - port: "{{ $port }}" + protocol: TCP + {{- end }}à {{- if .Values.networkPolicy.externalStorage.cidrs }} {{- range $cidr := .Values.networkPolicy.externalStorage.cidrs }} toCIDR: @@ -171,8 +175,9 @@ spec: {{- include "loki.selectorLabels" . | nindent 6 }} egress: - toPorts: - - port: {{ .Values.networkPolicy.discovery.port }} - protocol: TCP + - ports: + - port: "{{ .Values.networkPolicy.discovery.port }}" + protocol: TCP {{- if .Values.networkPolicy.discovery.namespaceSelector }} toEndpoints: - matchLabels: diff --git a/production/ksonnet/loki/bloom-compactor.libsonnet b/production/ksonnet/loki/bloom-compactor.libsonnet deleted file mode 100644 index d8c5e862fa106..0000000000000 --- a/production/ksonnet/loki/bloom-compactor.libsonnet +++ /dev/null @@ -1,125 +0,0 @@ -{ - local k = import 'ksonnet-util/kausal.libsonnet', - local container = k.core.v1.container, - local containerPort = k.core.v1.containerPort, - local pvc = k.core.v1.persistentVolumeClaim, - local service = k.core.v1.service, - local statefulSet = k.apps.v1.statefulSet, - local volume = k.core.v1.volume, - local volumeMount = k.core.v1.volumeMount, - - local name = 'bloom-compactor', - - _config+:: { - bloom_compactor+: { - // number of replicas - replicas: if $._config.use_bloom_filters then 3 else 0, - // PVC config - pvc_size: if $._config.use_bloom_filters then error 'bloom_compactor.pvc_size needs to be defined' else '', - pvc_class: if $._config.use_bloom_filters then error 'bloom_compactor.pvc_class needs to be defined' else '', - }, - loki+: - if $._config.use_bloom_filters - then - { - bloom_compactor: { - enabled: true, - working_directory: '/data/blooms', - compaction_interval: '15m', - max_compaction_parallelism: 1, - }, - } - else {}, - }, - - local cfg = self._config.bloom_compactor, - - local volumeName = name + '-data', - local volumeMounts = [volumeMount.new(volumeName, '/data')], - - bloom_compactor_args:: - if $._config.use_bloom_filters - then - $._config.commonArgs { - target: 'bloom-compactor', - } - else {}, - - bloom_compactor_ports:: [ - containerPort.new(name='http-metrics', port=$._config.http_listen_port), - containerPort.new(name='grpc', port=9095), - ], - - bloom_compactor_data_pvc:: - if $._config.use_bloom_filters - then - pvc.new(volumeName) - // set disk size - + pvc.mixin.spec.resources.withRequests({ storage: $._config.bloom_compactor.pvc_size }) - // mount the volume as read-write by a single node - + pvc.mixin.spec.withAccessModes(['ReadWriteOnce']) - // set persistent volume storage class - + pvc.mixin.spec.withStorageClassName($._config.bloom_compactor.pvc_class) - else {}, - - - bloom_compactor_container:: - if $._config.use_bloom_filters - then - container.new(name, $._images.bloom_compactor) - // add default ports - + container.withPorts($.bloom_compactor_ports) - // add target specific CLI arguments - + container.withArgsMixin(k.util.mapToFlags($.bloom_compactor_args)) - // mount the data pvc at given mountpoint - + container.withVolumeMountsMixin(volumeMounts) - // add globale environment variables - + container.withEnvMixin($._config.commonEnvs) - // add HTTP readiness probe - + container.mixin.readinessProbe.httpGet.withPath('/ready') - + container.mixin.readinessProbe.httpGet.withPort($._config.http_listen_port) - + container.mixin.readinessProbe.withTimeoutSeconds(1) - // define container resource requests - + k.util.resourcesRequests('2', '4Gi') - // define container resource limits - + k.util.resourcesLimits(null, '8Gi') - else {}, - - bloom_compactor_statefulset: - if $._config.use_bloom_filters - then - statefulSet.new(name, cfg.replicas, [$.bloom_compactor_container], $.bloom_compactor_data_pvc) - // add clusterIP service - + statefulSet.mixin.spec.withServiceName(name) - // perform rolling update when statefulset configuration changes - + statefulSet.mixin.spec.updateStrategy.withType('RollingUpdate') - // launch or terminate pods in parallel, *does not* affect upgrades - + statefulSet.mixin.spec.withPodManagementPolicy('Parallel') - // 10001 is the user/group ID assigned to Loki in the Dockerfile - + statefulSet.mixin.spec.template.spec.securityContext.withRunAsUser(10001) - + statefulSet.mixin.spec.template.spec.securityContext.withRunAsGroup(10001) - + statefulSet.mixin.spec.template.spec.securityContext.withFsGroup(10001) - // ensure statefulset is updated when loki config changes - + $.config_hash_mixin - // ensure no other workloads are scheduled - + k.util.antiAffinity - // mount the loki config.yaml - + k.util.configVolumeMount('loki', '/etc/loki/config') - // mount the runtime overrides.yaml - + k.util.configVolumeMount('overrides', '/etc/loki/overrides') - else {}, - - bloom_compactor_service: - if $._config.use_bloom_filters - then - k.util.serviceFor($.bloom_compactor_statefulset, $._config.service_ignored_labels) - else {}, - - bloom_compactor_headless_service: - if $._config.use_bloom_filters - then - k.util.serviceFor($.bloom_compactor_statefulset, $._config.service_ignored_labels) - + service.mixin.metadata.withName(name + '-headless') - + service.mixin.spec.withClusterIp('None') - else {}, -} diff --git a/production/ksonnet/loki/bloom-gateway.libsonnet b/production/ksonnet/loki/bloom-gateway.libsonnet deleted file mode 100644 index 18e50c7e0d91b..0000000000000 --- a/production/ksonnet/loki/bloom-gateway.libsonnet +++ /dev/null @@ -1,170 +0,0 @@ -{ - local k = import 'ksonnet-util/kausal.libsonnet', - local container = k.core.v1.container, - local containerPort = k.core.v1.containerPort, - local pvc = k.core.v1.persistentVolumeClaim, - local service = k.core.v1.service, - local statefulSet = k.apps.v1.statefulSet, - local volume = k.core.v1.volume, - local volumeMount = k.core.v1.volumeMount, - - local name = 'bloom-gateway', - - _config+:: { - bloom_gateway+: { - // number of replicas - replicas: if $._config.use_bloom_filters then 3 else 0, - // if true, the host needs to have local SSD disks mounted, otherwise PVCs are used - use_local_ssd: false, - // PVC config - pvc_size: if !self.use_local_ssd then error 'bloom_gateway.pvc_size needs to be defined when using PVC' else '', - pvc_class: if !self.use_local_ssd then error 'bloom_gateway.pvc_class needs to be defined when using PVC' else '', - // local SSD config - hostpath: if self.use_local_ssd then error 'bloom_gateway.hostpath needs to be defined when using local SSDs' else '', - node_selector: if self.use_local_ssd then error 'bloom_gateway.node_selector needs to be defined when using local SSDs' else {}, - tolerations: if self.use_local_ssd then error 'bloom_gateway.tolerations needs to be defined when using local SSDs' else [], - }, - loki+: - if $._config.use_bloom_filters - then - { - bloom_gateway+: { - enabled: true, - worker_concurrency: 8, - ring: { - replication_factor: 3, - }, - client: { - cache_results: false, - }, - }, - storage_config+: { - bloom_shipper+: { - working_directory: '/data/blooms', - blocks_downloading_queue: { - workers_count: 10, - }, - blocks_cache: { - enabled: true, - max_size_mb: error 'set bloom_shipper.blocks_cache.max_size_mb to ~80% of available disk size', - ttl: '24h', - }, - }, - }, - } - else {}, - }, - - local cfg = self._config.bloom_gateway, - - local volumeName = name + '-data', - - local volumes = - if cfg.use_local_ssd - then [volume.fromHostPath(volumeName, cfg.hostpath)] - else [], - - local volumeMounts = [ - volumeMount.new(volumeName, '/data'), - ], - - bloom_gateway_args:: - if $._config.use_bloom_filters - then - $._config.commonArgs { - target: 'bloom-gateway', - } - else {}, - - bloom_gateway_ports:: [ - containerPort.new(name='http-metrics', port=$._config.http_listen_port), - containerPort.new(name='grpc', port=9095), - ], - - bloom_gateway_data_pvc:: - if $._config.use_bloom_filters && !cfg.use_local_ssd - then - pvc.new(volumeName) - // set disk size - + pvc.mixin.spec.resources.withRequests({ storage: $._config.bloom_gateway.pvc_size }) - // mount the volume as read-write by a single node - + pvc.mixin.spec.withAccessModes(['ReadWriteOnce']) - // set persistent volume storage class - + pvc.mixin.spec.withStorageClassName($._config.bloom_compactor.pvc_class) - else - null, - - bloom_gateway_container:: - if $._config.use_bloom_filters - then - container.new(name, $._images.bloom_gateway) - // add default ports - + container.withPorts($.bloom_gateway_ports) - // add target specific CLI arguments - + container.withArgsMixin(k.util.mapToFlags($.bloom_gateway_args)) - // mount local SSD or PVC - + container.withVolumeMountsMixin(volumeMounts) - // add globale environment variables - + container.withEnvMixin($._config.commonEnvs) - // add HTTP readiness probe - + container.mixin.readinessProbe.httpGet.withPath('/ready') - + container.mixin.readinessProbe.httpGet.withPort($._config.http_listen_port) - + container.mixin.readinessProbe.withTimeoutSeconds(1) - // define container resource requests - + k.util.resourcesRequests('2', '4Gi') - // define container resource limits - + k.util.resourcesLimits(null, '8Gi') - else {}, - - bloom_gateway_statefulset: - if $._config.use_bloom_filters - then - statefulSet.new(name, cfg.replicas, [$.bloom_gateway_container]) - // add clusterIP service - + statefulSet.mixin.spec.withServiceName(name) - // perform rolling update when statefulset configuration changes - + statefulSet.mixin.spec.updateStrategy.withType('RollingUpdate') - // launch or terminate pods in parallel, *does not* affect upgrades - + statefulSet.mixin.spec.withPodManagementPolicy('Parallel') - // 10001 is the user/group ID assigned to Loki in the Dockerfile - + statefulSet.mixin.spec.template.spec.securityContext.withRunAsUser(10001) - + statefulSet.mixin.spec.template.spec.securityContext.withRunAsGroup(10001) - + statefulSet.mixin.spec.template.spec.securityContext.withFsGroup(10001) - // ensure statefulset is updated when loki config changes - + $.config_hash_mixin - // ensure no other workloads are scheduled - + k.util.antiAffinity - // mount the loki config.yaml - + k.util.configVolumeMount('loki', '/etc/loki/config') - // mount the runtime overrides.yaml - + k.util.configVolumeMount('overrides', '/etc/loki/overrides') - // configuration specific to SSD/PVC usage - + ( - if cfg.use_local_ssd - then - // ensure the pod is scheduled on a node with local SSDs if needed - statefulSet.mixin.spec.template.spec.withNodeSelector(cfg.node_selector) - // tolerate the local-ssd taint - + statefulSet.mixin.spec.template.spec.withTolerationsMixin(cfg.tolerations) - // mount the local SSDs - + statefulSet.mixin.spec.template.spec.withVolumesMixin(volumes) - else - // create persistent volume claim - statefulSet.mixin.spec.withVolumeClaimTemplates([$.bloom_gateway_data_pvc]) - ) - else {}, - - bloom_gateway_service: - if $._config.use_bloom_filters - then - k.util.serviceFor($.bloom_gateway_statefulset, $._config.service_ignored_labels) - else {}, - - bloom_gateway_headless_service: - if $._config.use_bloom_filters - then - k.util.serviceFor($.bloom_gateway_statefulset, $._config.service_ignored_labels) - + service.mixin.metadata.withName(name + '-headless') - + service.mixin.spec.withClusterIp('None') - else {}, -} diff --git a/production/ksonnet/loki/bloomfilters.libsonnet b/production/ksonnet/loki/bloomfilters.libsonnet deleted file mode 100644 index 78231a808e1a0..0000000000000 --- a/production/ksonnet/loki/bloomfilters.libsonnet +++ /dev/null @@ -1,8 +0,0 @@ -{ - _config+:: { - // globally enable/disable bloom gateway and bloom compactor - use_bloom_filters: false, - }, -} -+ (import 'bloom-compactor.libsonnet') -+ (import 'bloom-gateway.libsonnet') diff --git a/production/ksonnet/loki/loki.libsonnet b/production/ksonnet/loki/loki.libsonnet index 871a68025e990..ad0489a69cd3f 100644 --- a/production/ksonnet/loki/loki.libsonnet +++ b/production/ksonnet/loki/loki.libsonnet @@ -26,9 +26,6 @@ // BoltDB and TSDB Shipper support. Anything that modifies the compactor must be imported after this. (import 'shipper.libsonnet') + -// Accelerated search using bloom filters -(import 'bloomfilters.libsonnet') + - (import 'table-manager.libsonnet') + // Multi-zone ingester related config diff --git a/production/ksonnet/loki/shipper.libsonnet b/production/ksonnet/loki/shipper.libsonnet index 18f38f3ab89e8..5fe8320af8ae4 100644 --- a/production/ksonnet/loki/shipper.libsonnet +++ b/production/ksonnet/loki/shipper.libsonnet @@ -62,6 +62,7 @@ container.mixin.readinessProbe.httpGet.withPort($._config.http_listen_port) + container.mixin.readinessProbe.withTimeoutSeconds(1) + k.util.resourcesRequests('4', '2Gi') + + k.util.resourcesLimits(null, '4Gi') + container.withEnvMixin($._config.commonEnvs) else {}, diff --git a/vendor/modules.txt b/vendor/modules.txt index e7cbf47887d61..2992d1e44075e 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -400,7 +400,7 @@ github.com/aws/smithy-go/rand github.com/aws/smithy-go/time github.com/aws/smithy-go/transport/http github.com/aws/smithy-go/transport/http/internal/io -# github.com/axiomhq/hyperloglog v0.0.0-20230201085229-3ddf4bad03dc +# github.com/axiomhq/hyperloglog v0.0.0-20240124082744-24bca3a5b39b ## explicit; go 1.12 github.com/axiomhq/hyperloglog # github.com/baidubce/bce-sdk-go v0.9.141