Like Prometheus, but for logs.
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
loki/integration/loki_micro_services_test.go

1107 lines
33 KiB

package integration
import (
"context"
Flag categorize labels on streams response (#10419) We recently introduced support for ingesting and querying structured metadata in Loki. This adds a new dimension to Loki's labels since now we arguably have three categories of labels: _stream_, _structured metadata_, and _parsed_ labels. Depending on the origin of the labels, they should be used in LogQL expressions differently to achieve optimal performance. _stream_ labels should be added to stream matchers, _structured metadata_ labels should be used in a filter expression before any parsing expression, and _parsed_ labels should be placed after the parser expression extracting them. The Grafana UI has a hard time dealing with this same problem. Before https://github.com/grafana/grafana/pull/73955, the filtering functionality in Grafana was broken since it was not able to distinguish between _stream_ and _structured metadata_ labels. Also, as soon as a parser expression was added to the query, filters added by Grafana would be appended to the end of the query regardless of the label category. The PR above implements a workaround for this problem but needs a better API on Loki's end to mitigate all corner cases. Loki currently returns the following JSON for log queries: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "info", "traceID": "68810cf0c94bfcca" }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n" ], ... }, { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "debug", "traceID": "a7116cj54c4bjz8s" }, "values": [ [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n" ], ... }, ... ``` As can be seen, there is no way we can distinguish the category of each label. This PR introduces a new flag `X-Loki-Response-Encoding-Flags: categorize-labels` that makes Loki return categorized labels as follows: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n", { "structuredMetadata": { "traceID": "68810cf0c94bfcca" }, "parsed": { "level": "info" } } ], [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n", { "structuredMetadata": { "traceID": "a7116cj54c4bjz8s" }, "parsed": { "level": "debug" } } ], ... }, ... ``` Note that this PR only supports log queries, not metric queries. From a UX perspective, being able to categorize labels in metric queries doesn't have any benefit yet. Having said that, supporting this for metric queries would require some minor refactoring on top of what has been implemented here. If we decide to do that, I think we should do it on a separate PR to avoid making this PR even larger. I also decided to leave out support for Tail queries to avoid making this PR even larger. Once this one gets merged, we can work to support tailing. --- **Note to reviewers** This PR is huge since we need to forward categorized all over the codebase (from parsing logs all the way to marshaling), fortunately, many of the changes come from updating tests and refactoring iterators. Tested out in a dev cell with query `'{stream="stdout"} | label_format new="text"`. - Without the new flag: ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED { "data": { "result": [ { "stream": { "new": "text", "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n" ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n" ], ... ``` - With the new flag ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED X-Loki-Response-Encoding-Flags:categorize-labels { "data": { "encodingFlags": [ "categorize-labels" ], "result": [ { "stream": { "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n", { "parsed": { "new": "text" } } ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n", { "parsed": { "new": "text" } } ], ... ```
2 years ago
"encoding/json"
"strings"
"sync"
"testing"
"time"
dto "github.com/prometheus/client_model/go"
"github.com/prometheus/common/expfmt"
Flag categorize labels on streams response (#10419) We recently introduced support for ingesting and querying structured metadata in Loki. This adds a new dimension to Loki's labels since now we arguably have three categories of labels: _stream_, _structured metadata_, and _parsed_ labels. Depending on the origin of the labels, they should be used in LogQL expressions differently to achieve optimal performance. _stream_ labels should be added to stream matchers, _structured metadata_ labels should be used in a filter expression before any parsing expression, and _parsed_ labels should be placed after the parser expression extracting them. The Grafana UI has a hard time dealing with this same problem. Before https://github.com/grafana/grafana/pull/73955, the filtering functionality in Grafana was broken since it was not able to distinguish between _stream_ and _structured metadata_ labels. Also, as soon as a parser expression was added to the query, filters added by Grafana would be appended to the end of the query regardless of the label category. The PR above implements a workaround for this problem but needs a better API on Loki's end to mitigate all corner cases. Loki currently returns the following JSON for log queries: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "info", "traceID": "68810cf0c94bfcca" }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n" ], ... }, { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "debug", "traceID": "a7116cj54c4bjz8s" }, "values": [ [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n" ], ... }, ... ``` As can be seen, there is no way we can distinguish the category of each label. This PR introduces a new flag `X-Loki-Response-Encoding-Flags: categorize-labels` that makes Loki return categorized labels as follows: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n", { "structuredMetadata": { "traceID": "68810cf0c94bfcca" }, "parsed": { "level": "info" } } ], [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n", { "structuredMetadata": { "traceID": "a7116cj54c4bjz8s" }, "parsed": { "level": "debug" } } ], ... }, ... ``` Note that this PR only supports log queries, not metric queries. From a UX perspective, being able to categorize labels in metric queries doesn't have any benefit yet. Having said that, supporting this for metric queries would require some minor refactoring on top of what has been implemented here. If we decide to do that, I think we should do it on a separate PR to avoid making this PR even larger. I also decided to leave out support for Tail queries to avoid making this PR even larger. Once this one gets merged, we can work to support tailing. --- **Note to reviewers** This PR is huge since we need to forward categorized all over the codebase (from parsing logs all the way to marshaling), fortunately, many of the changes come from updating tests and refactoring iterators. Tested out in a dev cell with query `'{stream="stdout"} | label_format new="text"`. - Without the new flag: ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED { "data": { "result": [ { "stream": { "new": "text", "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n" ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n" ], ... ``` - With the new flag ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED X-Loki-Response-Encoding-Flags:categorize-labels { "data": { "encodingFlags": [ "categorize-labels" ], "result": [ { "stream": { "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n", { "parsed": { "new": "text" } } ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n", { "parsed": { "new": "text" } } ], ... ```
2 years ago
"github.com/prometheus/prometheus/model/labels"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"google.golang.org/protobuf/proto"
"github.com/grafana/loki/integration/client"
"github.com/grafana/loki/integration/cluster"
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
"github.com/grafana/loki/pkg/storage"
Flag categorize labels on streams response (#10419) We recently introduced support for ingesting and querying structured metadata in Loki. This adds a new dimension to Loki's labels since now we arguably have three categories of labels: _stream_, _structured metadata_, and _parsed_ labels. Depending on the origin of the labels, they should be used in LogQL expressions differently to achieve optimal performance. _stream_ labels should be added to stream matchers, _structured metadata_ labels should be used in a filter expression before any parsing expression, and _parsed_ labels should be placed after the parser expression extracting them. The Grafana UI has a hard time dealing with this same problem. Before https://github.com/grafana/grafana/pull/73955, the filtering functionality in Grafana was broken since it was not able to distinguish between _stream_ and _structured metadata_ labels. Also, as soon as a parser expression was added to the query, filters added by Grafana would be appended to the end of the query regardless of the label category. The PR above implements a workaround for this problem but needs a better API on Loki's end to mitigate all corner cases. Loki currently returns the following JSON for log queries: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "info", "traceID": "68810cf0c94bfcca" }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n" ], ... }, { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "debug", "traceID": "a7116cj54c4bjz8s" }, "values": [ [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n" ], ... }, ... ``` As can be seen, there is no way we can distinguish the category of each label. This PR introduces a new flag `X-Loki-Response-Encoding-Flags: categorize-labels` that makes Loki return categorized labels as follows: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n", { "structuredMetadata": { "traceID": "68810cf0c94bfcca" }, "parsed": { "level": "info" } } ], [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n", { "structuredMetadata": { "traceID": "a7116cj54c4bjz8s" }, "parsed": { "level": "debug" } } ], ... }, ... ``` Note that this PR only supports log queries, not metric queries. From a UX perspective, being able to categorize labels in metric queries doesn't have any benefit yet. Having said that, supporting this for metric queries would require some minor refactoring on top of what has been implemented here. If we decide to do that, I think we should do it on a separate PR to avoid making this PR even larger. I also decided to leave out support for Tail queries to avoid making this PR even larger. Once this one gets merged, we can work to support tailing. --- **Note to reviewers** This PR is huge since we need to forward categorized all over the codebase (from parsing logs all the way to marshaling), fortunately, many of the changes come from updating tests and refactoring iterators. Tested out in a dev cell with query `'{stream="stdout"} | label_format new="text"`. - Without the new flag: ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED { "data": { "result": [ { "stream": { "new": "text", "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n" ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n" ], ... ``` - With the new flag ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED X-Loki-Response-Encoding-Flags:categorize-labels { "data": { "encodingFlags": [ "categorize-labels" ], "result": [ { "stream": { "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n", { "parsed": { "new": "text" } } ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n", { "parsed": { "new": "text" } } ], ... ```
2 years ago
"github.com/grafana/loki/pkg/util/httpreq"
"github.com/grafana/loki/pkg/util/querylimits"
)
func TestMicroServicesIngestQuery(t *testing.T) {
clu := cluster.New(nil)
defer func() {
assert.NoError(t, clu.Cleanup())
}()
// run initially the compactor, indexgateway, and distributor.
var (
tCompactor = clu.AddComponent(
"compactor",
"-target=compactor",
config: remove flags & YAML settings deprecated in 2.9.x (#11151) **What this PR does / why we need it**: - remove `-ruler.evaluation-delay-duration` - remove `boltdb.shipper.compactor.deletion-mode` - remove compactor flags with prefix`-boltdb.shipper.compactor.*` - remove `validation.enforce-metric-name` - remove `log.use-buffered`, `log.use-sync` **Special notes for your reviewer**: `log.use-buffered`, `log.use-sync` are not mentioned in the upgrade guide since they were hidden fields **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [X] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [X] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
"-compactor.compaction-interval=1s",
"-compactor.retention-delete-delay=1s",
// By default, a minute is added to the delete request start time. This compensates for that.
config: remove flags & YAML settings deprecated in 2.9.x (#11151) **What this PR does / why we need it**: - remove `-ruler.evaluation-delay-duration` - remove `boltdb.shipper.compactor.deletion-mode` - remove compactor flags with prefix`-boltdb.shipper.compactor.*` - remove `validation.enforce-metric-name` - remove `log.use-buffered`, `log.use-sync` **Special notes for your reviewer**: `log.use-buffered`, `log.use-sync` are not mentioned in the upgrade guide since they were hidden fields **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [X] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [X] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
"-compactor.delete-request-cancel-period=-60s",
"-compactor.deletion-mode=filter-and-delete",
)
tIndexGateway = clu.AddComponent(
"index-gateway",
"-target=index-gateway",
)
tDistributor = clu.AddComponent(
"distributor",
"-target=distributor",
)
)
require.NoError(t, clu.Run())
// then, run only the ingester and query scheduler.
var (
tIngester = clu.AddComponent(
"ingester",
"-target=ingester",
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
)
tQueryScheduler = clu.AddComponent(
"query-scheduler",
"-target=query-scheduler",
"-query-scheduler.use-scheduler-ring=false",
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
)
)
require.NoError(t, clu.Run())
// the run querier.
var (
tQuerier = clu.AddComponent(
"querier",
"-target=querier",
"-querier.scheduler-address="+tQueryScheduler.GRPCURL(),
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
)
)
require.NoError(t, clu.Run())
// finally, run the query-frontend.
var (
tQueryFrontend = clu.AddComponent(
"query-frontend",
"-target=query-frontend",
"-frontend.scheduler-address="+tQueryScheduler.GRPCURL(),
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
"-querier.per-request-limits-enabled=true",
"-frontend.encoding=protobuf",
"-querier.shard-aggregations=quantile_over_time",
"-frontend.tail-proxy-url="+tQuerier.HTTPURL(),
)
)
require.NoError(t, clu.Run())
tenantID := randStringRunes()
now := time.Now()
cliDistributor := client.New(tenantID, "", tDistributor.HTTPURL())
cliDistributor.Now = now
cliIngester := client.New(tenantID, "", tIngester.HTTPURL())
cliIngester.Now = now
cliQueryFrontend := client.New(tenantID, "", tQueryFrontend.HTTPURL())
cliQueryFrontend.Now = now
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
t.Run("ingest-logs", func(t *testing.T) {
// ingest some log lines
ingestion: native otlp ingestion support (#10727) **What this PR does / why we need it**: Add support for natively supporting logs ingestion in OTLP format. `/otlp/v1/logs` is the new endpoint where users can push logs in OTLP format. It accepts logs serialized in JSON or proto format. Since OTEL format is very different than what Loki storage model, here is how data in OTEL format will be mapped to Loki data model: * Index labels: The Resource Attributes map quite well to Index labels in Loki since both usually identify the source of the logs. The problem however is that Resource attributes in OTLP can have an unbounded number of values while Loki has a default limit of having up to 30 labels. Since Index labels in Loki can largely drive the kind of querying experience the users are going to have, we have chosen select attributes which would be picked as Index Labels. The ones that are not picked up as Index labels would be stored as Structured Metadata with each log entry. * Timestamp: LogRecord.TimeUnixNano * LogLine: LogRecord.Body holds the body of the log. However, since Loki only supports Log body in string format, we will stringify non-string values using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). * Structured Metadata: Anything which can’t be stored in Index labels and LogLine. Here is a non-exhaustive list of what will be stored in Structured Metadata to give a sense of what it will hold: * Resource Attributes not stored as Index labels is replicated and stored with each log entry. * Everything under InstrumentationScope is replicated and stored with each log entry. * Everything under LogRecord except LogRecord.Body, LogRecord.TimeUnixNano and sometimes LogRecord.ObservedTimestamp. *NOTES*: * Since Loki does not support `.` or any other special characters other than `_` in label names, we replace all non-supported characters with `_`. * Since Loki only supports string in values of Index Labels and Structured Metadata, all the complex types are converted as follows: * Map would be flattened into label keys using `_` as separator, same as how we do it in [json parser in LogQL](https://grafana.com/docs/loki/latest/query/log_queries/#json). * Everything else is stringified using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353) **Special notes for your reviewer**: I will open follow-up PRs for: * Documentation * Make blessed attributes list configurable per tenant. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label
2 years ago
require.NoError(t, cliDistributor.PushLogLine("lineA", now.Add(-45*time.Minute), nil, map[string]string{"job": "fake"}))
require.NoError(t, cliDistributor.PushLogLine("lineB", now.Add(-45*time.Minute), nil, map[string]string{"job": "fake"}))
ingestion: native otlp ingestion support (#10727) **What this PR does / why we need it**: Add support for natively supporting logs ingestion in OTLP format. `/otlp/v1/logs` is the new endpoint where users can push logs in OTLP format. It accepts logs serialized in JSON or proto format. Since OTEL format is very different than what Loki storage model, here is how data in OTEL format will be mapped to Loki data model: * Index labels: The Resource Attributes map quite well to Index labels in Loki since both usually identify the source of the logs. The problem however is that Resource attributes in OTLP can have an unbounded number of values while Loki has a default limit of having up to 30 labels. Since Index labels in Loki can largely drive the kind of querying experience the users are going to have, we have chosen select attributes which would be picked as Index Labels. The ones that are not picked up as Index labels would be stored as Structured Metadata with each log entry. * Timestamp: LogRecord.TimeUnixNano * LogLine: LogRecord.Body holds the body of the log. However, since Loki only supports Log body in string format, we will stringify non-string values using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). * Structured Metadata: Anything which can’t be stored in Index labels and LogLine. Here is a non-exhaustive list of what will be stored in Structured Metadata to give a sense of what it will hold: * Resource Attributes not stored as Index labels is replicated and stored with each log entry. * Everything under InstrumentationScope is replicated and stored with each log entry. * Everything under LogRecord except LogRecord.Body, LogRecord.TimeUnixNano and sometimes LogRecord.ObservedTimestamp. *NOTES*: * Since Loki does not support `.` or any other special characters other than `_` in label names, we replace all non-supported characters with `_`. * Since Loki only supports string in values of Index Labels and Structured Metadata, all the complex types are converted as follows: * Map would be flattened into label keys using `_` as separator, same as how we do it in [json parser in LogQL](https://grafana.com/docs/loki/latest/query/log_queries/#json). * Everything else is stringified using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353) **Special notes for your reviewer**: I will open follow-up PRs for: * Documentation * Make blessed attributes list configurable per tenant. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label
2 years ago
require.NoError(t, cliDistributor.PushLogLine("lineC", now, nil, map[string]string{"job": "fake"}))
require.NoError(t, cliDistributor.PushLogLine("lineD", now, nil, map[string]string{"job": "fake"}))
})
t.Run("query", func(t *testing.T) {
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
assert.Equal(t, "streams", resp.Data.ResultType)
var lines []string
for _, stream := range resp.Data.Stream {
for _, val := range stream.Values {
lines = append(lines, val[1])
}
}
assert.ElementsMatch(t, []string{"lineA", "lineB", "lineC", "lineD"}, lines)
})
t.Run("label-names", func(t *testing.T) {
resp, err := cliQueryFrontend.LabelNames(context.Background())
require.NoError(t, err)
assert.ElementsMatch(t, []string{"job"}, resp)
})
t.Run("label-values", func(t *testing.T) {
resp, err := cliQueryFrontend.LabelValues(context.Background(), "job")
require.NoError(t, err)
assert.ElementsMatch(t, []string{"fake"}, resp)
})
t.Run("series", func(t *testing.T) {
resp, err := cliQueryFrontend.Series(context.Background(), `{job="fake"}`)
require.NoError(t, err)
assert.ElementsMatch(t, []map[string]string{{"job": "fake"}}, resp)
})
t.Run("series error", func(t *testing.T) {
_, err := cliQueryFrontend.Series(context.Background(), `{job="fake"}|= "search"`)
require.ErrorContains(t, err, "status code 400: only label matchers are supported")
})
t.Run("stats error", func(t *testing.T) {
_, err := cliQueryFrontend.Stats(context.Background(), `{job="fake"}|= "search"`)
require.ErrorContains(t, err, "status code 400: only label matchers are supported")
})
t.Run("per-request-limits", func(t *testing.T) {
queryLimitsPolicy := client.InjectHeadersOption(map[string][]string{querylimits.HTTPHeaderQueryLimitsKey: {`{"maxQueryLength": "1m"}`}})
cliQueryFrontendLimited := client.New(tenantID, "", tQueryFrontend.HTTPURL(), queryLimitsPolicy)
cliQueryFrontendLimited.Now = now
_, err := cliQueryFrontendLimited.LabelNames(context.Background())
require.ErrorContains(t, err, "the query time range exceeds the limit (query length")
})
t.Run("tail", func(t *testing.T) {
ctx, cancelFunc := context.WithCancel(context.Background())
defer cancelFunc()
out := make(chan client.TailResult)
wc, err := cliQueryFrontend.Tail(ctx, `{job="fake"}`, out)
require.NoError(t, err)
defer wc.Close()
var lines []string
mu := sync.Mutex{}
done := make(chan struct{})
go func() {
for resp := range out {
require.NoError(t, resp.Err)
for _, stream := range resp.Response.Streams {
for _, e := range stream.Entries {
mu.Lock()
lines = append(lines, e.Line)
mu.Unlock()
}
}
}
done <- struct{}{}
}()
assert.Eventually(
t,
func() bool {
mu.Lock()
defer mu.Unlock()
return len(lines) == 4
},
10*time.Second,
100*time.Millisecond,
)
wc.Close()
cancelFunc()
<-done
assert.ElementsMatch(t, []string{"lineA", "lineB", "lineC", "lineD"}, lines)
})
}
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
func TestMicroServicesIngestQueryWithSchemaChange(t *testing.T) {
// init the cluster with a single tsdb period. Uses prefix index_tsdb_
clu := cluster.New(nil, cluster.SchemaWithTSDB)
defer func() {
assert.NoError(t, clu.Cleanup())
}()
// initially, run only compactor and distributor.
var (
tCompactor = clu.AddComponent(
"compactor",
"-target=compactor",
config: remove flags & YAML settings deprecated in 2.9.x (#11151) **What this PR does / why we need it**: - remove `-ruler.evaluation-delay-duration` - remove `boltdb.shipper.compactor.deletion-mode` - remove compactor flags with prefix`-boltdb.shipper.compactor.*` - remove `validation.enforce-metric-name` - remove `log.use-buffered`, `log.use-sync` **Special notes for your reviewer**: `log.use-buffered`, `log.use-sync` are not mentioned in the upgrade guide since they were hidden fields **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [X] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [X] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
"-compactor.compaction-interval=1s",
)
tDistributor = clu.AddComponent(
"distributor",
"-target=distributor",
)
)
require.NoError(t, clu.Run())
// then, run only ingester and query-scheduler.
var (
tIngester = clu.AddComponent(
"ingester",
"-target=ingester",
"-ingester.flush-on-shutdown=true",
)
tQueryScheduler = clu.AddComponent(
"query-scheduler",
"-target=query-scheduler",
"-query-scheduler.use-scheduler-ring=false",
)
)
require.NoError(t, clu.Run())
// finally, run the query-frontend and querier.
var (
tQueryFrontend = clu.AddComponent(
"query-frontend",
"-target=query-frontend",
"-frontend.scheduler-address="+tQueryScheduler.GRPCURL(),
"-frontend.default-validity=0s",
"-common.compactor-address="+tCompactor.HTTPURL(),
)
tQuerier = clu.AddComponent(
"querier",
"-target=querier",
"-querier.scheduler-address="+tQueryScheduler.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
)
)
require.NoError(t, clu.Run())
tenantID := randStringRunes()
now := time.Now()
cliDistributor := client.New(tenantID, "", tDistributor.HTTPURL())
cliDistributor.Now = now
cliIngester := client.New(tenantID, "", tIngester.HTTPURL())
cliIngester.Now = now
cliQueryFrontend := client.New(tenantID, "", tQueryFrontend.HTTPURL())
cliQueryFrontend.Now = now
t.Run("ingest-logs", func(t *testing.T) {
ingestion: native otlp ingestion support (#10727) **What this PR does / why we need it**: Add support for natively supporting logs ingestion in OTLP format. `/otlp/v1/logs` is the new endpoint where users can push logs in OTLP format. It accepts logs serialized in JSON or proto format. Since OTEL format is very different than what Loki storage model, here is how data in OTEL format will be mapped to Loki data model: * Index labels: The Resource Attributes map quite well to Index labels in Loki since both usually identify the source of the logs. The problem however is that Resource attributes in OTLP can have an unbounded number of values while Loki has a default limit of having up to 30 labels. Since Index labels in Loki can largely drive the kind of querying experience the users are going to have, we have chosen select attributes which would be picked as Index Labels. The ones that are not picked up as Index labels would be stored as Structured Metadata with each log entry. * Timestamp: LogRecord.TimeUnixNano * LogLine: LogRecord.Body holds the body of the log. However, since Loki only supports Log body in string format, we will stringify non-string values using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). * Structured Metadata: Anything which can’t be stored in Index labels and LogLine. Here is a non-exhaustive list of what will be stored in Structured Metadata to give a sense of what it will hold: * Resource Attributes not stored as Index labels is replicated and stored with each log entry. * Everything under InstrumentationScope is replicated and stored with each log entry. * Everything under LogRecord except LogRecord.Body, LogRecord.TimeUnixNano and sometimes LogRecord.ObservedTimestamp. *NOTES*: * Since Loki does not support `.` or any other special characters other than `_` in label names, we replace all non-supported characters with `_`. * Since Loki only supports string in values of Index Labels and Structured Metadata, all the complex types are converted as follows: * Map would be flattened into label keys using `_` as separator, same as how we do it in [json parser in LogQL](https://grafana.com/docs/loki/latest/query/log_queries/#json). * Everything else is stringified using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353) **Special notes for your reviewer**: I will open follow-up PRs for: * Documentation * Make blessed attributes list configurable per tenant. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label
2 years ago
require.NoError(t, cliDistributor.PushLogLine("lineA", time.Now().Add(-72*time.Hour), nil, map[string]string{"job": "fake"}))
require.NoError(t, cliDistributor.PushLogLine("lineB", time.Now().Add(-48*time.Hour), nil, map[string]string{"job": "fake"}))
})
t.Run("query-lookback-default", func(t *testing.T) {
// queries ingesters with the default lookback period (3h)
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
assert.Equal(t, "streams", resp.Data.ResultType)
var lines []string
for _, stream := range resp.Data.Stream {
for _, val := range stream.Values {
lines = append(lines, val[1])
}
}
assert.ElementsMatch(t, []string{}, lines)
})
t.Run("query-lookback-7d", func(t *testing.T) {
tQuerier.AddFlags("-querier.query-ingesters-within=168h")
require.NoError(t, tQuerier.Restart())
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
assert.Equal(t, "streams", resp.Data.ResultType)
var lines []string
for _, stream := range resp.Data.Stream {
for _, val := range stream.Values {
lines = append(lines, val[1])
}
}
assert.ElementsMatch(t, []string{"lineA", "lineB"}, lines)
tQuerier.AddFlags("-querier.query-ingesters-within=3h")
require.NoError(t, tQuerier.Restart())
})
t.Run("flush-logs-and-restart-ingester-querier", func(t *testing.T) {
// restart ingester which should flush the chunks and index
require.NoError(t, tIngester.Restart())
// restart querier and index shipper to sync the index
tQuerier.AddFlags("-querier.query-store-only=true")
require.NoError(t, tQuerier.Restart())
})
// Query lines
t.Run("query again to verify logs being served from storage", func(t *testing.T) {
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
assert.Equal(t, "streams", resp.Data.ResultType)
var lines []string
for _, stream := range resp.Data.Stream {
for _, val := range stream.Values {
lines = append(lines, val[1])
}
}
assert.ElementsMatch(t, []string{"lineA", "lineB"}, lines)
tQuerier.AddFlags("-querier.query-store-only=false")
require.NoError(t, tQuerier.Restart())
})
// Add new tsdb period with a different index prefix(index_)
clu.ResetSchemaConfig()
cluster.SchemaWithTSDBAndTSDB(clu)
// restart to load the new schema
require.NoError(t, tIngester.Restart())
require.NoError(t, tQuerier.Restart())
t.Run("ingest-logs-new-period", func(t *testing.T) {
// ingest logs to the new period
ingestion: native otlp ingestion support (#10727) **What this PR does / why we need it**: Add support for natively supporting logs ingestion in OTLP format. `/otlp/v1/logs` is the new endpoint where users can push logs in OTLP format. It accepts logs serialized in JSON or proto format. Since OTEL format is very different than what Loki storage model, here is how data in OTEL format will be mapped to Loki data model: * Index labels: The Resource Attributes map quite well to Index labels in Loki since both usually identify the source of the logs. The problem however is that Resource attributes in OTLP can have an unbounded number of values while Loki has a default limit of having up to 30 labels. Since Index labels in Loki can largely drive the kind of querying experience the users are going to have, we have chosen select attributes which would be picked as Index Labels. The ones that are not picked up as Index labels would be stored as Structured Metadata with each log entry. * Timestamp: LogRecord.TimeUnixNano * LogLine: LogRecord.Body holds the body of the log. However, since Loki only supports Log body in string format, we will stringify non-string values using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). * Structured Metadata: Anything which can’t be stored in Index labels and LogLine. Here is a non-exhaustive list of what will be stored in Structured Metadata to give a sense of what it will hold: * Resource Attributes not stored as Index labels is replicated and stored with each log entry. * Everything under InstrumentationScope is replicated and stored with each log entry. * Everything under LogRecord except LogRecord.Body, LogRecord.TimeUnixNano and sometimes LogRecord.ObservedTimestamp. *NOTES*: * Since Loki does not support `.` or any other special characters other than `_` in label names, we replace all non-supported characters with `_`. * Since Loki only supports string in values of Index Labels and Structured Metadata, all the complex types are converted as follows: * Map would be flattened into label keys using `_` as separator, same as how we do it in [json parser in LogQL](https://grafana.com/docs/loki/latest/query/log_queries/#json). * Everything else is stringified using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353) **Special notes for your reviewer**: I will open follow-up PRs for: * Documentation * Make blessed attributes list configurable per tenant. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label
2 years ago
require.NoError(t, cliDistributor.PushLogLine("lineC", now, nil, map[string]string{"job": "fake"}))
require.NoError(t, cliDistributor.PushLogLine("lineD", now, nil, map[string]string{"job": "fake"}))
})
t.Run("query-both-periods-with-default-lookback", func(t *testing.T) {
// queries with the default lookback period (3h)
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
assert.Equal(t, "streams", resp.Data.ResultType)
var lines []string
for _, stream := range resp.Data.Stream {
for _, val := range stream.Values {
lines = append(lines, val[1])
}
}
assert.ElementsMatch(t, []string{"lineA", "lineB", "lineC", "lineD"}, lines)
})
t.Run("flush-logs-and-restart-ingester-querier", func(t *testing.T) {
// restart ingester which should flush the chunks and index
require.NoError(t, tIngester.Restart())
// restart querier and index shipper to sync the index
tQuerier.AddFlags("-querier.query-store-only=true")
require.NoError(t, tQuerier.Restart())
})
// Query lines
t.Run("query both periods to verify logs being served from storage", func(t *testing.T) {
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
assert.Equal(t, "streams", resp.Data.ResultType)
var lines []string
for _, stream := range resp.Data.Stream {
for _, val := range stream.Values {
lines = append(lines, val[1])
}
}
assert.ElementsMatch(t, []string{"lineA", "lineB", "lineC", "lineD"}, lines)
})
}
func TestMicroServicesIngestQueryOverMultipleBucketSingleProvider(t *testing.T) {
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
for name, opt := range map[string]func(c *cluster.Cluster){
"boltdb-index": cluster.SchemaWithBoltDBAndBoltDB,
"tsdb-index": cluster.SchemaWithTSDBAndTSDB,
"boltdb-and-tsdb": cluster.SchemaWithBoltDBAndTSDB,
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
} {
t.Run(name, func(t *testing.T) {
clu := cluster.New(nil, opt)
defer func() {
assert.NoError(t, clu.Cleanup())
}()
// initially, run only compactor and distributor.
var (
tCompactor = clu.AddComponent(
"compactor",
"-target=compactor",
config: remove flags & YAML settings deprecated in 2.9.x (#11151) **What this PR does / why we need it**: - remove `-ruler.evaluation-delay-duration` - remove `boltdb.shipper.compactor.deletion-mode` - remove compactor flags with prefix`-boltdb.shipper.compactor.*` - remove `validation.enforce-metric-name` - remove `log.use-buffered`, `log.use-sync` **Special notes for your reviewer**: `log.use-buffered`, `log.use-sync` are not mentioned in the upgrade guide since they were hidden fields **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [X] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [X] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
"-compactor.compaction-interval=1s",
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
)
tDistributor = clu.AddComponent(
"distributor",
"-target=distributor",
)
)
require.NoError(t, clu.Run())
// then, run only ingester and query-scheduler.
var (
tIngester = clu.AddComponent(
"ingester",
"-target=ingester",
"-ingester.flush-on-shutdown=true",
)
tQueryScheduler = clu.AddComponent(
"query-scheduler",
"-target=query-scheduler",
"-query-scheduler.use-scheduler-ring=false",
)
)
require.NoError(t, clu.Run())
// finally, run the query-frontend and querier.
var (
tQueryFrontend = clu.AddComponent(
"query-frontend",
"-target=query-frontend",
"-frontend.scheduler-address="+tQueryScheduler.GRPCURL(),
"-frontend.default-validity=0s",
"-common.compactor-address="+tCompactor.HTTPURL(),
)
tQuerier = clu.AddComponent(
"querier",
"-target=querier",
"-querier.scheduler-address="+tQueryScheduler.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
)
)
require.NoError(t, clu.Run())
tenantID := randStringRunes()
now := time.Now()
cliDistributor := client.New(tenantID, "", tDistributor.HTTPURL())
cliDistributor.Now = now
cliIngester := client.New(tenantID, "", tIngester.HTTPURL())
cliIngester.Now = now
cliQueryFrontend := client.New(tenantID, "", tQueryFrontend.HTTPURL())
cliQueryFrontend.Now = now
t.Run("ingest-logs", func(t *testing.T) {
ingestion: native otlp ingestion support (#10727) **What this PR does / why we need it**: Add support for natively supporting logs ingestion in OTLP format. `/otlp/v1/logs` is the new endpoint where users can push logs in OTLP format. It accepts logs serialized in JSON or proto format. Since OTEL format is very different than what Loki storage model, here is how data in OTEL format will be mapped to Loki data model: * Index labels: The Resource Attributes map quite well to Index labels in Loki since both usually identify the source of the logs. The problem however is that Resource attributes in OTLP can have an unbounded number of values while Loki has a default limit of having up to 30 labels. Since Index labels in Loki can largely drive the kind of querying experience the users are going to have, we have chosen select attributes which would be picked as Index Labels. The ones that are not picked up as Index labels would be stored as Structured Metadata with each log entry. * Timestamp: LogRecord.TimeUnixNano * LogLine: LogRecord.Body holds the body of the log. However, since Loki only supports Log body in string format, we will stringify non-string values using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). * Structured Metadata: Anything which can’t be stored in Index labels and LogLine. Here is a non-exhaustive list of what will be stored in Structured Metadata to give a sense of what it will hold: * Resource Attributes not stored as Index labels is replicated and stored with each log entry. * Everything under InstrumentationScope is replicated and stored with each log entry. * Everything under LogRecord except LogRecord.Body, LogRecord.TimeUnixNano and sometimes LogRecord.ObservedTimestamp. *NOTES*: * Since Loki does not support `.` or any other special characters other than `_` in label names, we replace all non-supported characters with `_`. * Since Loki only supports string in values of Index Labels and Structured Metadata, all the complex types are converted as follows: * Map would be flattened into label keys using `_` as separator, same as how we do it in [json parser in LogQL](https://grafana.com/docs/loki/latest/query/log_queries/#json). * Everything else is stringified using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353) **Special notes for your reviewer**: I will open follow-up PRs for: * Documentation * Make blessed attributes list configurable per tenant. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label
2 years ago
require.NoError(t, cliDistributor.PushLogLine("lineA", time.Now().Add(-48*time.Hour), map[string]string{"traceID": "123"}, map[string]string{"job": "fake"}))
require.NoError(t, cliDistributor.PushLogLine("lineB", time.Now().Add(-36*time.Hour), map[string]string{"traceID": "456"}, map[string]string{"job": "fake"}))
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
// ingest logs to the current period
ingestion: native otlp ingestion support (#10727) **What this PR does / why we need it**: Add support for natively supporting logs ingestion in OTLP format. `/otlp/v1/logs` is the new endpoint where users can push logs in OTLP format. It accepts logs serialized in JSON or proto format. Since OTEL format is very different than what Loki storage model, here is how data in OTEL format will be mapped to Loki data model: * Index labels: The Resource Attributes map quite well to Index labels in Loki since both usually identify the source of the logs. The problem however is that Resource attributes in OTLP can have an unbounded number of values while Loki has a default limit of having up to 30 labels. Since Index labels in Loki can largely drive the kind of querying experience the users are going to have, we have chosen select attributes which would be picked as Index Labels. The ones that are not picked up as Index labels would be stored as Structured Metadata with each log entry. * Timestamp: LogRecord.TimeUnixNano * LogLine: LogRecord.Body holds the body of the log. However, since Loki only supports Log body in string format, we will stringify non-string values using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). * Structured Metadata: Anything which can’t be stored in Index labels and LogLine. Here is a non-exhaustive list of what will be stored in Structured Metadata to give a sense of what it will hold: * Resource Attributes not stored as Index labels is replicated and stored with each log entry. * Everything under InstrumentationScope is replicated and stored with each log entry. * Everything under LogRecord except LogRecord.Body, LogRecord.TimeUnixNano and sometimes LogRecord.ObservedTimestamp. *NOTES*: * Since Loki does not support `.` or any other special characters other than `_` in label names, we replace all non-supported characters with `_`. * Since Loki only supports string in values of Index Labels and Structured Metadata, all the complex types are converted as follows: * Map would be flattened into label keys using `_` as separator, same as how we do it in [json parser in LogQL](https://grafana.com/docs/loki/latest/query/log_queries/#json). * Everything else is stringified using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353) **Special notes for your reviewer**: I will open follow-up PRs for: * Documentation * Make blessed attributes list configurable per tenant. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label
2 years ago
require.NoError(t, cliDistributor.PushLogLine("lineC", now, map[string]string{"traceID": "789"}, map[string]string{"job": "fake"}))
require.NoError(t, cliDistributor.PushLogLine("lineD", now, map[string]string{"traceID": "123"}, map[string]string{"job": "fake"}))
Add metadata to push payload (#9694) **What this PR does / why we need it**: We are adding support for attaching labels to each log line. This is one of the series of the PRs broken up to make it easier to review changes. This PR updates the push payload to send labels with each log entry optionally. The log labels are supposed to be in the same format as the stream labels. Just to put it out, here is how it would look for proto and json push payload with same data: **proto(`endpoint`: `(/loki/api/v1/push|/api/prom/push)`, `Content-Type`: `application/x-protobuf`)**(payload built using [push.Stream](https://github.com/grafana/loki/blob/4cd1246b8830ccc241fa4afff85d208dc6ae2129/pkg/push/types.go#L12)): ``` push.Stream{ Entries: []logproto.Entry{ { Timestamp: time.Unix(0, 1688515200000000000), Line: "log line", Labels: `{foo="bar"}`, }, }, Labels: `{app="test"}`, } ``` **v1(`endpoint`: `/loki/api/v1/push`, `Content-Type`: `application/json`)**: ```json { "streams": [{ "stream": { "app": "test" }, "values": [ ["1688515200000000000", "log line", { "foo": "bar" }] ] }] } ``` **legacy-json(`/api/prom/push`, `Content-Type`: `application/json`)**: ```json { "streams": [{ "labels": "{app=\"test\"}", "entries": [{ "ts": "2023-07-05T00:00:00.000000000Z", "line": "log line", "labels": "{foo=\"bar\"}" }] }] } ``` **Which issue(s) this PR fixes**: **Special notes for your reviewer**: We may need to add more thoughtful tests. **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) --------- Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
2 years ago
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
})
t.Run("query-lookback-default", func(t *testing.T) {
// queries ingesters with the default lookback period (3h)
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
assert.Equal(t, "streams", resp.Data.ResultType)
var lines []string
for _, stream := range resp.Data.Stream {
for _, val := range stream.Values {
lines = append(lines, val[1])
}
}
assert.ElementsMatch(t, []string{"lineC", "lineD"}, lines)
})
t.Run("flush-logs-and-restart-ingester-querier", func(t *testing.T) {
// restart ingester which should flush the chunks and index
require.NoError(t, tIngester.Restart())
// restart querier and index shipper to sync the index
storage.ResetBoltDBIndexClientsWithShipper()
tQuerier.AddFlags("-querier.query-store-only=true")
require.NoError(t, tQuerier.Restart())
})
// Query lines
t.Run("query again to verify logs being served from storage", func(t *testing.T) {
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
assert.Equal(t, "streams", resp.Data.ResultType)
var lines []string
for _, stream := range resp.Data.Stream {
for _, val := range stream.Values {
lines = append(lines, val[1])
}
}
assert.ElementsMatch(t, []string{"lineA", "lineB", "lineC", "lineD"}, lines)
})
})
}
}
query-scheduler: fix query distribution in SSD mode (#9471) **What this PR does / why we need it**: When we run the `query-scheduler` in `ring` mode, `queriers` and `query-frontend` discover the available `query-scheduler` instances using the ring. However, we have a problem when `query-schedulers` are not running in the same process as queriers and query-frontend since [we try to get the ring client interface from the scheduler instance](https://github.com/grafana/loki/blob/abd6131bba18db7f3575241c5e6dc4eed879fbc0/pkg/loki/modules.go#L358). This causes queries not to be spread across all the available queriers when running in SSD mode because [we point querier workers to query frontend when there is no ring client and scheduler address configured](https://github.com/grafana/loki/blob/b05f4fced305800b32641ae84e3bed5f1794fa7d/pkg/querier/worker_service.go#L115). I have fixed this issue by adding a new hidden target to initialize the ring client in `reader`/`member` mode based on which service is initializing it. `reader` mode will be used by `queriers` and `query-frontend` for discovering `query-scheduler` instances from the ring. `member` mode will be used by `query-schedulers` for registering themselves in the ring. I have also made a couple of changes not directly related to the issue but it fixes some problems: * [reset metric registry for each integration test](https://github.com/grafana/loki/commit/18c4fe59078b649ad6a788a48765b101d0b97618) - Previously we were reusing the same registry for all the tests and just [ignored the attempts to register same metrics](https://github.com/grafana/loki/blob/01f0ded7fcb57e3a7b26ffc1e8e3abf04a403825/integration/cluster/cluster.go#L113). This causes the registry to have metrics registered only from the first test so any updates from subsequent tests won't reflect in the metrics. metrics was the only reliable way for me to verify that `query-schedulers` were connected to `queriers` and `query-frontend` when running in ring mode in the integration test that I added to test my changes. This should also help with other tests where earlier it was hard to reliably check the metrics. * [load config from cli as well before applying dynamic config](https://github.com/grafana/loki/commit/f9e2448fc7e718db107165cd908054c806b84337) - Previously we were applying dynamic config considering just the config from config file. This results in unexpected config changes, for example, [this config change](https://github.com/grafana/loki/blob/4148dd2c51cb827ec3889298508b95ec7731e7fd/integration/loki_micro_services_test.go#L66) was getting ignored and [dynamic config tuning was unexpectedly turning on ring mode](https://github.com/grafana/loki/blob/52cd0a39b8266564352c61ab9b845ab597008770/pkg/loki/config_wrapper.go#L94) in the config. It is better to do any config tuning based on both file and cli args configs. **Which issue(s) this PR fixes**: Fixes #9195
2 years ago
func TestSchedulerRing(t *testing.T) {
clu := cluster.New(nil)
defer func() {
assert.NoError(t, clu.Cleanup())
}()
// run initially the compactor, indexgateway, and distributor.
var (
tCompactor = clu.AddComponent(
"compactor",
"-target=compactor",
config: remove flags & YAML settings deprecated in 2.9.x (#11151) **What this PR does / why we need it**: - remove `-ruler.evaluation-delay-duration` - remove `boltdb.shipper.compactor.deletion-mode` - remove compactor flags with prefix`-boltdb.shipper.compactor.*` - remove `validation.enforce-metric-name` - remove `log.use-buffered`, `log.use-sync` **Special notes for your reviewer**: `log.use-buffered`, `log.use-sync` are not mentioned in the upgrade guide since they were hidden fields **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [X] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [X] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
"-compactor.compaction-interval=1s",
"-compactor.retention-delete-delay=1s",
query-scheduler: fix query distribution in SSD mode (#9471) **What this PR does / why we need it**: When we run the `query-scheduler` in `ring` mode, `queriers` and `query-frontend` discover the available `query-scheduler` instances using the ring. However, we have a problem when `query-schedulers` are not running in the same process as queriers and query-frontend since [we try to get the ring client interface from the scheduler instance](https://github.com/grafana/loki/blob/abd6131bba18db7f3575241c5e6dc4eed879fbc0/pkg/loki/modules.go#L358). This causes queries not to be spread across all the available queriers when running in SSD mode because [we point querier workers to query frontend when there is no ring client and scheduler address configured](https://github.com/grafana/loki/blob/b05f4fced305800b32641ae84e3bed5f1794fa7d/pkg/querier/worker_service.go#L115). I have fixed this issue by adding a new hidden target to initialize the ring client in `reader`/`member` mode based on which service is initializing it. `reader` mode will be used by `queriers` and `query-frontend` for discovering `query-scheduler` instances from the ring. `member` mode will be used by `query-schedulers` for registering themselves in the ring. I have also made a couple of changes not directly related to the issue but it fixes some problems: * [reset metric registry for each integration test](https://github.com/grafana/loki/commit/18c4fe59078b649ad6a788a48765b101d0b97618) - Previously we were reusing the same registry for all the tests and just [ignored the attempts to register same metrics](https://github.com/grafana/loki/blob/01f0ded7fcb57e3a7b26ffc1e8e3abf04a403825/integration/cluster/cluster.go#L113). This causes the registry to have metrics registered only from the first test so any updates from subsequent tests won't reflect in the metrics. metrics was the only reliable way for me to verify that `query-schedulers` were connected to `queriers` and `query-frontend` when running in ring mode in the integration test that I added to test my changes. This should also help with other tests where earlier it was hard to reliably check the metrics. * [load config from cli as well before applying dynamic config](https://github.com/grafana/loki/commit/f9e2448fc7e718db107165cd908054c806b84337) - Previously we were applying dynamic config considering just the config from config file. This results in unexpected config changes, for example, [this config change](https://github.com/grafana/loki/blob/4148dd2c51cb827ec3889298508b95ec7731e7fd/integration/loki_micro_services_test.go#L66) was getting ignored and [dynamic config tuning was unexpectedly turning on ring mode](https://github.com/grafana/loki/blob/52cd0a39b8266564352c61ab9b845ab597008770/pkg/loki/config_wrapper.go#L94) in the config. It is better to do any config tuning based on both file and cli args configs. **Which issue(s) this PR fixes**: Fixes #9195
2 years ago
// By default, a minute is added to the delete request start time. This compensates for that.
config: remove flags & YAML settings deprecated in 2.9.x (#11151) **What this PR does / why we need it**: - remove `-ruler.evaluation-delay-duration` - remove `boltdb.shipper.compactor.deletion-mode` - remove compactor flags with prefix`-boltdb.shipper.compactor.*` - remove `validation.enforce-metric-name` - remove `log.use-buffered`, `log.use-sync` **Special notes for your reviewer**: `log.use-buffered`, `log.use-sync` are not mentioned in the upgrade guide since they were hidden fields **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [X] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [X] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
"-compactor.delete-request-cancel-period=-60s",
query-scheduler: fix query distribution in SSD mode (#9471) **What this PR does / why we need it**: When we run the `query-scheduler` in `ring` mode, `queriers` and `query-frontend` discover the available `query-scheduler` instances using the ring. However, we have a problem when `query-schedulers` are not running in the same process as queriers and query-frontend since [we try to get the ring client interface from the scheduler instance](https://github.com/grafana/loki/blob/abd6131bba18db7f3575241c5e6dc4eed879fbc0/pkg/loki/modules.go#L358). This causes queries not to be spread across all the available queriers when running in SSD mode because [we point querier workers to query frontend when there is no ring client and scheduler address configured](https://github.com/grafana/loki/blob/b05f4fced305800b32641ae84e3bed5f1794fa7d/pkg/querier/worker_service.go#L115). I have fixed this issue by adding a new hidden target to initialize the ring client in `reader`/`member` mode based on which service is initializing it. `reader` mode will be used by `queriers` and `query-frontend` for discovering `query-scheduler` instances from the ring. `member` mode will be used by `query-schedulers` for registering themselves in the ring. I have also made a couple of changes not directly related to the issue but it fixes some problems: * [reset metric registry for each integration test](https://github.com/grafana/loki/commit/18c4fe59078b649ad6a788a48765b101d0b97618) - Previously we were reusing the same registry for all the tests and just [ignored the attempts to register same metrics](https://github.com/grafana/loki/blob/01f0ded7fcb57e3a7b26ffc1e8e3abf04a403825/integration/cluster/cluster.go#L113). This causes the registry to have metrics registered only from the first test so any updates from subsequent tests won't reflect in the metrics. metrics was the only reliable way for me to verify that `query-schedulers` were connected to `queriers` and `query-frontend` when running in ring mode in the integration test that I added to test my changes. This should also help with other tests where earlier it was hard to reliably check the metrics. * [load config from cli as well before applying dynamic config](https://github.com/grafana/loki/commit/f9e2448fc7e718db107165cd908054c806b84337) - Previously we were applying dynamic config considering just the config from config file. This results in unexpected config changes, for example, [this config change](https://github.com/grafana/loki/blob/4148dd2c51cb827ec3889298508b95ec7731e7fd/integration/loki_micro_services_test.go#L66) was getting ignored and [dynamic config tuning was unexpectedly turning on ring mode](https://github.com/grafana/loki/blob/52cd0a39b8266564352c61ab9b845ab597008770/pkg/loki/config_wrapper.go#L94) in the config. It is better to do any config tuning based on both file and cli args configs. **Which issue(s) this PR fixes**: Fixes #9195
2 years ago
"-compactor.deletion-mode=filter-and-delete",
)
tIndexGateway = clu.AddComponent(
"index-gateway",
"-target=index-gateway",
)
tDistributor = clu.AddComponent(
"distributor",
"-target=distributor",
)
)
require.NoError(t, clu.Run())
// then, run only the ingester and query scheduler.
var (
tIngester = clu.AddComponent(
"ingester",
"-target=ingester",
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
)
tQueryScheduler = clu.AddComponent(
"query-scheduler",
"-target=query-scheduler",
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
"-query-scheduler.use-scheduler-ring=true",
)
)
require.NoError(t, clu.Run())
// finally, run the query-frontend and querier.
var (
tQueryFrontend = clu.AddComponent(
"query-frontend",
"-target=query-frontend",
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
"-querier.per-request-limits-enabled=true",
"-query-scheduler.use-scheduler-ring=true",
"-frontend.scheduler-worker-concurrency=5",
)
_ = clu.AddComponent(
"querier",
"-target=querier",
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
"-query-scheduler.use-scheduler-ring=true",
"-querier.max-concurrent=4",
)
)
require.NoError(t, clu.Run())
tenantID := randStringRunes()
now := time.Now()
cliDistributor := client.New(tenantID, "", tDistributor.HTTPURL())
cliDistributor.Now = now
cliIngester := client.New(tenantID, "", tIngester.HTTPURL())
cliIngester.Now = now
cliQueryFrontend := client.New(tenantID, "", tQueryFrontend.HTTPURL())
cliQueryFrontend.Now = now
cliQueryScheduler := client.New(tenantID, "", tQueryScheduler.HTTPURL())
cliQueryScheduler.Now = now
t.Run("verify-scheduler-connections", func(t *testing.T) {
require.Eventually(t, func() bool {
// Check metrics to see if query scheduler is connected with query-frontend
metrics, err := cliQueryScheduler.Metrics()
require.NoError(t, err)
Change default of metrics.namespace (#11110) **What this PR does / why we need it**: Change the default for a subset of metrics. After this all Loki metrics should start with "loki_". This changes the following metrics: cortex_distributor_ingester_clients cortex_dns_failures_total cortex_dns_lookups_total cortex_dns_provider_results cortex_frontend_query_range_duration_seconds_bucket cortex_frontend_query_range_duration_seconds_count cortex_frontend_query_range_duration_seconds_sum cortex_ingester_flush_queue_length cortex_kv_request_duration_seconds_bucket cortex_kv_request_duration_seconds_count cortex_kv_request_duration_seconds_sum cortex_member_consul_heartbeats_total cortex_prometheus_last_evaluation_samples cortex_prometheus_notifications_alertmanagers_discovered cortex_prometheus_notifications_dropped_total cortex_prometheus_notifications_errors_total cortex_prometheus_notifications_latency_seconds cortex_prometheus_notifications_latency_seconds_count cortex_prometheus_notifications_latency_seconds_sum cortex_prometheus_notifications_queue_capacity cortex_prometheus_notifications_queue_length cortex_prometheus_notifications_sent_total cortex_prometheus_rule_evaluation_duration_seconds cortex_prometheus_rule_evaluation_duration_seconds_count cortex_prometheus_rule_evaluation_duration_seconds_sum cortex_prometheus_rule_evaluation_failures_total cortex_prometheus_rule_evaluations_total cortex_prometheus_rule_group_duration_seconds cortex_prometheus_rule_group_duration_seconds_count cortex_prometheus_rule_group_duration_seconds_sum cortex_prometheus_rule_group_interval_seconds cortex_prometheus_rule_group_iterations_missed_total cortex_prometheus_rule_group_iterations_total cortex_prometheus_rule_group_last_duration_seconds cortex_prometheus_rule_group_last_evaluation_timestamp_seconds cortex_prometheus_rule_group_rules cortex_query_frontend_connected_schedulers cortex_query_frontend_queries_in_progress cortex_query_frontend_retries_bucket cortex_query_frontend_retries_count cortex_query_frontend_retries_sum cortex_query_scheduler_connected_frontend_clients cortex_query_scheduler_connected_querier_clients cortex_query_scheduler_inflight_requests cortex_query_scheduler_inflight_requests_count cortex_query_scheduler_inflight_requests_sum cortex_query_scheduler_queue_duration_seconds_bucket cortex_query_scheduler_queue_duration_seconds_count cortex_query_scheduler_queue_duration_seconds_sum cortex_query_scheduler_queue_length cortex_query_scheduler_running cortex_quota_cgroup_cpu_max cortex_quota_cgroup_cpu_period cortex_quota_cpu_count cortex_quota_gomaxprocs cortex_ring_member_heartbeats_total cortex_ring_member_tokens_owned cortex_ring_member_tokens_to_own cortex_ring_members cortex_ring_oldest_member_timestamp cortex_ring_tokens_total cortex_ruler_client_request_duration_seconds_bucket cortex_ruler_client_request_duration_seconds_count cortex_ruler_client_request_duration_seconds_sum cortex_ruler_clients cortex_ruler_config_last_reload_successful cortex_ruler_config_last_reload_successful_seconds cortex_ruler_config_updates_total cortex_ruler_managers_total cortex_ruler_ring_check_errors_total cortex_ruler_sync_rules_total **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [X] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Michel Hollands <michel.hollands@gmail.com> Co-authored-by: Ashwanth <iamashwanth@gmail.com>
2 years ago
return getMetricValue(t, "loki_query_scheduler_connected_frontend_clients", metrics) == 5
query-scheduler: fix query distribution in SSD mode (#9471) **What this PR does / why we need it**: When we run the `query-scheduler` in `ring` mode, `queriers` and `query-frontend` discover the available `query-scheduler` instances using the ring. However, we have a problem when `query-schedulers` are not running in the same process as queriers and query-frontend since [we try to get the ring client interface from the scheduler instance](https://github.com/grafana/loki/blob/abd6131bba18db7f3575241c5e6dc4eed879fbc0/pkg/loki/modules.go#L358). This causes queries not to be spread across all the available queriers when running in SSD mode because [we point querier workers to query frontend when there is no ring client and scheduler address configured](https://github.com/grafana/loki/blob/b05f4fced305800b32641ae84e3bed5f1794fa7d/pkg/querier/worker_service.go#L115). I have fixed this issue by adding a new hidden target to initialize the ring client in `reader`/`member` mode based on which service is initializing it. `reader` mode will be used by `queriers` and `query-frontend` for discovering `query-scheduler` instances from the ring. `member` mode will be used by `query-schedulers` for registering themselves in the ring. I have also made a couple of changes not directly related to the issue but it fixes some problems: * [reset metric registry for each integration test](https://github.com/grafana/loki/commit/18c4fe59078b649ad6a788a48765b101d0b97618) - Previously we were reusing the same registry for all the tests and just [ignored the attempts to register same metrics](https://github.com/grafana/loki/blob/01f0ded7fcb57e3a7b26ffc1e8e3abf04a403825/integration/cluster/cluster.go#L113). This causes the registry to have metrics registered only from the first test so any updates from subsequent tests won't reflect in the metrics. metrics was the only reliable way for me to verify that `query-schedulers` were connected to `queriers` and `query-frontend` when running in ring mode in the integration test that I added to test my changes. This should also help with other tests where earlier it was hard to reliably check the metrics. * [load config from cli as well before applying dynamic config](https://github.com/grafana/loki/commit/f9e2448fc7e718db107165cd908054c806b84337) - Previously we were applying dynamic config considering just the config from config file. This results in unexpected config changes, for example, [this config change](https://github.com/grafana/loki/blob/4148dd2c51cb827ec3889298508b95ec7731e7fd/integration/loki_micro_services_test.go#L66) was getting ignored and [dynamic config tuning was unexpectedly turning on ring mode](https://github.com/grafana/loki/blob/52cd0a39b8266564352c61ab9b845ab597008770/pkg/loki/config_wrapper.go#L94) in the config. It is better to do any config tuning based on both file and cli args configs. **Which issue(s) this PR fixes**: Fixes #9195
2 years ago
}, 5*time.Second, 500*time.Millisecond)
require.Eventually(t, func() bool {
// Check metrics to see if query scheduler is connected with query-frontend
metrics, err := cliQueryScheduler.Metrics()
require.NoError(t, err)
Change default of metrics.namespace (#11110) **What this PR does / why we need it**: Change the default for a subset of metrics. After this all Loki metrics should start with "loki_". This changes the following metrics: cortex_distributor_ingester_clients cortex_dns_failures_total cortex_dns_lookups_total cortex_dns_provider_results cortex_frontend_query_range_duration_seconds_bucket cortex_frontend_query_range_duration_seconds_count cortex_frontend_query_range_duration_seconds_sum cortex_ingester_flush_queue_length cortex_kv_request_duration_seconds_bucket cortex_kv_request_duration_seconds_count cortex_kv_request_duration_seconds_sum cortex_member_consul_heartbeats_total cortex_prometheus_last_evaluation_samples cortex_prometheus_notifications_alertmanagers_discovered cortex_prometheus_notifications_dropped_total cortex_prometheus_notifications_errors_total cortex_prometheus_notifications_latency_seconds cortex_prometheus_notifications_latency_seconds_count cortex_prometheus_notifications_latency_seconds_sum cortex_prometheus_notifications_queue_capacity cortex_prometheus_notifications_queue_length cortex_prometheus_notifications_sent_total cortex_prometheus_rule_evaluation_duration_seconds cortex_prometheus_rule_evaluation_duration_seconds_count cortex_prometheus_rule_evaluation_duration_seconds_sum cortex_prometheus_rule_evaluation_failures_total cortex_prometheus_rule_evaluations_total cortex_prometheus_rule_group_duration_seconds cortex_prometheus_rule_group_duration_seconds_count cortex_prometheus_rule_group_duration_seconds_sum cortex_prometheus_rule_group_interval_seconds cortex_prometheus_rule_group_iterations_missed_total cortex_prometheus_rule_group_iterations_total cortex_prometheus_rule_group_last_duration_seconds cortex_prometheus_rule_group_last_evaluation_timestamp_seconds cortex_prometheus_rule_group_rules cortex_query_frontend_connected_schedulers cortex_query_frontend_queries_in_progress cortex_query_frontend_retries_bucket cortex_query_frontend_retries_count cortex_query_frontend_retries_sum cortex_query_scheduler_connected_frontend_clients cortex_query_scheduler_connected_querier_clients cortex_query_scheduler_inflight_requests cortex_query_scheduler_inflight_requests_count cortex_query_scheduler_inflight_requests_sum cortex_query_scheduler_queue_duration_seconds_bucket cortex_query_scheduler_queue_duration_seconds_count cortex_query_scheduler_queue_duration_seconds_sum cortex_query_scheduler_queue_length cortex_query_scheduler_running cortex_quota_cgroup_cpu_max cortex_quota_cgroup_cpu_period cortex_quota_cpu_count cortex_quota_gomaxprocs cortex_ring_member_heartbeats_total cortex_ring_member_tokens_owned cortex_ring_member_tokens_to_own cortex_ring_members cortex_ring_oldest_member_timestamp cortex_ring_tokens_total cortex_ruler_client_request_duration_seconds_bucket cortex_ruler_client_request_duration_seconds_count cortex_ruler_client_request_duration_seconds_sum cortex_ruler_clients cortex_ruler_config_last_reload_successful cortex_ruler_config_last_reload_successful_seconds cortex_ruler_config_updates_total cortex_ruler_managers_total cortex_ruler_ring_check_errors_total cortex_ruler_sync_rules_total **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [X] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Michel Hollands <michel.hollands@gmail.com> Co-authored-by: Ashwanth <iamashwanth@gmail.com>
2 years ago
return getMetricValue(t, "loki_query_scheduler_connected_querier_clients", metrics) == 4
query-scheduler: fix query distribution in SSD mode (#9471) **What this PR does / why we need it**: When we run the `query-scheduler` in `ring` mode, `queriers` and `query-frontend` discover the available `query-scheduler` instances using the ring. However, we have a problem when `query-schedulers` are not running in the same process as queriers and query-frontend since [we try to get the ring client interface from the scheduler instance](https://github.com/grafana/loki/blob/abd6131bba18db7f3575241c5e6dc4eed879fbc0/pkg/loki/modules.go#L358). This causes queries not to be spread across all the available queriers when running in SSD mode because [we point querier workers to query frontend when there is no ring client and scheduler address configured](https://github.com/grafana/loki/blob/b05f4fced305800b32641ae84e3bed5f1794fa7d/pkg/querier/worker_service.go#L115). I have fixed this issue by adding a new hidden target to initialize the ring client in `reader`/`member` mode based on which service is initializing it. `reader` mode will be used by `queriers` and `query-frontend` for discovering `query-scheduler` instances from the ring. `member` mode will be used by `query-schedulers` for registering themselves in the ring. I have also made a couple of changes not directly related to the issue but it fixes some problems: * [reset metric registry for each integration test](https://github.com/grafana/loki/commit/18c4fe59078b649ad6a788a48765b101d0b97618) - Previously we were reusing the same registry for all the tests and just [ignored the attempts to register same metrics](https://github.com/grafana/loki/blob/01f0ded7fcb57e3a7b26ffc1e8e3abf04a403825/integration/cluster/cluster.go#L113). This causes the registry to have metrics registered only from the first test so any updates from subsequent tests won't reflect in the metrics. metrics was the only reliable way for me to verify that `query-schedulers` were connected to `queriers` and `query-frontend` when running in ring mode in the integration test that I added to test my changes. This should also help with other tests where earlier it was hard to reliably check the metrics. * [load config from cli as well before applying dynamic config](https://github.com/grafana/loki/commit/f9e2448fc7e718db107165cd908054c806b84337) - Previously we were applying dynamic config considering just the config from config file. This results in unexpected config changes, for example, [this config change](https://github.com/grafana/loki/blob/4148dd2c51cb827ec3889298508b95ec7731e7fd/integration/loki_micro_services_test.go#L66) was getting ignored and [dynamic config tuning was unexpectedly turning on ring mode](https://github.com/grafana/loki/blob/52cd0a39b8266564352c61ab9b845ab597008770/pkg/loki/config_wrapper.go#L94) in the config. It is better to do any config tuning based on both file and cli args configs. **Which issue(s) this PR fixes**: Fixes #9195
2 years ago
}, 5*time.Second, 500*time.Millisecond)
})
t.Run("ingest-logs", func(t *testing.T) {
// ingest some log lines
ingestion: native otlp ingestion support (#10727) **What this PR does / why we need it**: Add support for natively supporting logs ingestion in OTLP format. `/otlp/v1/logs` is the new endpoint where users can push logs in OTLP format. It accepts logs serialized in JSON or proto format. Since OTEL format is very different than what Loki storage model, here is how data in OTEL format will be mapped to Loki data model: * Index labels: The Resource Attributes map quite well to Index labels in Loki since both usually identify the source of the logs. The problem however is that Resource attributes in OTLP can have an unbounded number of values while Loki has a default limit of having up to 30 labels. Since Index labels in Loki can largely drive the kind of querying experience the users are going to have, we have chosen select attributes which would be picked as Index Labels. The ones that are not picked up as Index labels would be stored as Structured Metadata with each log entry. * Timestamp: LogRecord.TimeUnixNano * LogLine: LogRecord.Body holds the body of the log. However, since Loki only supports Log body in string format, we will stringify non-string values using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). * Structured Metadata: Anything which can’t be stored in Index labels and LogLine. Here is a non-exhaustive list of what will be stored in Structured Metadata to give a sense of what it will hold: * Resource Attributes not stored as Index labels is replicated and stored with each log entry. * Everything under InstrumentationScope is replicated and stored with each log entry. * Everything under LogRecord except LogRecord.Body, LogRecord.TimeUnixNano and sometimes LogRecord.ObservedTimestamp. *NOTES*: * Since Loki does not support `.` or any other special characters other than `_` in label names, we replace all non-supported characters with `_`. * Since Loki only supports string in values of Index Labels and Structured Metadata, all the complex types are converted as follows: * Map would be flattened into label keys using `_` as separator, same as how we do it in [json parser in LogQL](https://grafana.com/docs/loki/latest/query/log_queries/#json). * Everything else is stringified using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353) **Special notes for your reviewer**: I will open follow-up PRs for: * Documentation * Make blessed attributes list configurable per tenant. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label
2 years ago
require.NoError(t, cliDistributor.PushLogLine("lineA", now.Add(-45*time.Minute), nil, map[string]string{"job": "fake"}))
require.NoError(t, cliDistributor.PushLogLine("lineB", now.Add(-45*time.Minute), nil, map[string]string{"job": "fake"}))
query-scheduler: fix query distribution in SSD mode (#9471) **What this PR does / why we need it**: When we run the `query-scheduler` in `ring` mode, `queriers` and `query-frontend` discover the available `query-scheduler` instances using the ring. However, we have a problem when `query-schedulers` are not running in the same process as queriers and query-frontend since [we try to get the ring client interface from the scheduler instance](https://github.com/grafana/loki/blob/abd6131bba18db7f3575241c5e6dc4eed879fbc0/pkg/loki/modules.go#L358). This causes queries not to be spread across all the available queriers when running in SSD mode because [we point querier workers to query frontend when there is no ring client and scheduler address configured](https://github.com/grafana/loki/blob/b05f4fced305800b32641ae84e3bed5f1794fa7d/pkg/querier/worker_service.go#L115). I have fixed this issue by adding a new hidden target to initialize the ring client in `reader`/`member` mode based on which service is initializing it. `reader` mode will be used by `queriers` and `query-frontend` for discovering `query-scheduler` instances from the ring. `member` mode will be used by `query-schedulers` for registering themselves in the ring. I have also made a couple of changes not directly related to the issue but it fixes some problems: * [reset metric registry for each integration test](https://github.com/grafana/loki/commit/18c4fe59078b649ad6a788a48765b101d0b97618) - Previously we were reusing the same registry for all the tests and just [ignored the attempts to register same metrics](https://github.com/grafana/loki/blob/01f0ded7fcb57e3a7b26ffc1e8e3abf04a403825/integration/cluster/cluster.go#L113). This causes the registry to have metrics registered only from the first test so any updates from subsequent tests won't reflect in the metrics. metrics was the only reliable way for me to verify that `query-schedulers` were connected to `queriers` and `query-frontend` when running in ring mode in the integration test that I added to test my changes. This should also help with other tests where earlier it was hard to reliably check the metrics. * [load config from cli as well before applying dynamic config](https://github.com/grafana/loki/commit/f9e2448fc7e718db107165cd908054c806b84337) - Previously we were applying dynamic config considering just the config from config file. This results in unexpected config changes, for example, [this config change](https://github.com/grafana/loki/blob/4148dd2c51cb827ec3889298508b95ec7731e7fd/integration/loki_micro_services_test.go#L66) was getting ignored and [dynamic config tuning was unexpectedly turning on ring mode](https://github.com/grafana/loki/blob/52cd0a39b8266564352c61ab9b845ab597008770/pkg/loki/config_wrapper.go#L94) in the config. It is better to do any config tuning based on both file and cli args configs. **Which issue(s) this PR fixes**: Fixes #9195
2 years ago
ingestion: native otlp ingestion support (#10727) **What this PR does / why we need it**: Add support for natively supporting logs ingestion in OTLP format. `/otlp/v1/logs` is the new endpoint where users can push logs in OTLP format. It accepts logs serialized in JSON or proto format. Since OTEL format is very different than what Loki storage model, here is how data in OTEL format will be mapped to Loki data model: * Index labels: The Resource Attributes map quite well to Index labels in Loki since both usually identify the source of the logs. The problem however is that Resource attributes in OTLP can have an unbounded number of values while Loki has a default limit of having up to 30 labels. Since Index labels in Loki can largely drive the kind of querying experience the users are going to have, we have chosen select attributes which would be picked as Index Labels. The ones that are not picked up as Index labels would be stored as Structured Metadata with each log entry. * Timestamp: LogRecord.TimeUnixNano * LogLine: LogRecord.Body holds the body of the log. However, since Loki only supports Log body in string format, we will stringify non-string values using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). * Structured Metadata: Anything which can’t be stored in Index labels and LogLine. Here is a non-exhaustive list of what will be stored in Structured Metadata to give a sense of what it will hold: * Resource Attributes not stored as Index labels is replicated and stored with each log entry. * Everything under InstrumentationScope is replicated and stored with each log entry. * Everything under LogRecord except LogRecord.Body, LogRecord.TimeUnixNano and sometimes LogRecord.ObservedTimestamp. *NOTES*: * Since Loki does not support `.` or any other special characters other than `_` in label names, we replace all non-supported characters with `_`. * Since Loki only supports string in values of Index Labels and Structured Metadata, all the complex types are converted as follows: * Map would be flattened into label keys using `_` as separator, same as how we do it in [json parser in LogQL](https://grafana.com/docs/loki/latest/query/log_queries/#json). * Everything else is stringified using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353) **Special notes for your reviewer**: I will open follow-up PRs for: * Documentation * Make blessed attributes list configurable per tenant. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label
2 years ago
require.NoError(t, cliDistributor.PushLogLine("lineC", now, nil, map[string]string{"job": "fake"}))
require.NoError(t, cliDistributor.PushLogLine("lineD", now, nil, map[string]string{"job": "fake"}))
query-scheduler: fix query distribution in SSD mode (#9471) **What this PR does / why we need it**: When we run the `query-scheduler` in `ring` mode, `queriers` and `query-frontend` discover the available `query-scheduler` instances using the ring. However, we have a problem when `query-schedulers` are not running in the same process as queriers and query-frontend since [we try to get the ring client interface from the scheduler instance](https://github.com/grafana/loki/blob/abd6131bba18db7f3575241c5e6dc4eed879fbc0/pkg/loki/modules.go#L358). This causes queries not to be spread across all the available queriers when running in SSD mode because [we point querier workers to query frontend when there is no ring client and scheduler address configured](https://github.com/grafana/loki/blob/b05f4fced305800b32641ae84e3bed5f1794fa7d/pkg/querier/worker_service.go#L115). I have fixed this issue by adding a new hidden target to initialize the ring client in `reader`/`member` mode based on which service is initializing it. `reader` mode will be used by `queriers` and `query-frontend` for discovering `query-scheduler` instances from the ring. `member` mode will be used by `query-schedulers` for registering themselves in the ring. I have also made a couple of changes not directly related to the issue but it fixes some problems: * [reset metric registry for each integration test](https://github.com/grafana/loki/commit/18c4fe59078b649ad6a788a48765b101d0b97618) - Previously we were reusing the same registry for all the tests and just [ignored the attempts to register same metrics](https://github.com/grafana/loki/blob/01f0ded7fcb57e3a7b26ffc1e8e3abf04a403825/integration/cluster/cluster.go#L113). This causes the registry to have metrics registered only from the first test so any updates from subsequent tests won't reflect in the metrics. metrics was the only reliable way for me to verify that `query-schedulers` were connected to `queriers` and `query-frontend` when running in ring mode in the integration test that I added to test my changes. This should also help with other tests where earlier it was hard to reliably check the metrics. * [load config from cli as well before applying dynamic config](https://github.com/grafana/loki/commit/f9e2448fc7e718db107165cd908054c806b84337) - Previously we were applying dynamic config considering just the config from config file. This results in unexpected config changes, for example, [this config change](https://github.com/grafana/loki/blob/4148dd2c51cb827ec3889298508b95ec7731e7fd/integration/loki_micro_services_test.go#L66) was getting ignored and [dynamic config tuning was unexpectedly turning on ring mode](https://github.com/grafana/loki/blob/52cd0a39b8266564352c61ab9b845ab597008770/pkg/loki/config_wrapper.go#L94) in the config. It is better to do any config tuning based on both file and cli args configs. **Which issue(s) this PR fixes**: Fixes #9195
2 years ago
})
t.Run("query", func(t *testing.T) {
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
assert.Equal(t, "streams", resp.Data.ResultType)
var lines []string
for _, stream := range resp.Data.Stream {
for _, val := range stream.Values {
lines = append(lines, val[1])
}
}
assert.ElementsMatch(t, []string{"lineA", "lineB", "lineC", "lineD"}, lines)
})
}
ingestion: native otlp ingestion support (#10727) **What this PR does / why we need it**: Add support for natively supporting logs ingestion in OTLP format. `/otlp/v1/logs` is the new endpoint where users can push logs in OTLP format. It accepts logs serialized in JSON or proto format. Since OTEL format is very different than what Loki storage model, here is how data in OTEL format will be mapped to Loki data model: * Index labels: The Resource Attributes map quite well to Index labels in Loki since both usually identify the source of the logs. The problem however is that Resource attributes in OTLP can have an unbounded number of values while Loki has a default limit of having up to 30 labels. Since Index labels in Loki can largely drive the kind of querying experience the users are going to have, we have chosen select attributes which would be picked as Index Labels. The ones that are not picked up as Index labels would be stored as Structured Metadata with each log entry. * Timestamp: LogRecord.TimeUnixNano * LogLine: LogRecord.Body holds the body of the log. However, since Loki only supports Log body in string format, we will stringify non-string values using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). * Structured Metadata: Anything which can’t be stored in Index labels and LogLine. Here is a non-exhaustive list of what will be stored in Structured Metadata to give a sense of what it will hold: * Resource Attributes not stored as Index labels is replicated and stored with each log entry. * Everything under InstrumentationScope is replicated and stored with each log entry. * Everything under LogRecord except LogRecord.Body, LogRecord.TimeUnixNano and sometimes LogRecord.ObservedTimestamp. *NOTES*: * Since Loki does not support `.` or any other special characters other than `_` in label names, we replace all non-supported characters with `_`. * Since Loki only supports string in values of Index Labels and Structured Metadata, all the complex types are converted as follows: * Map would be flattened into label keys using `_` as separator, same as how we do it in [json parser in LogQL](https://grafana.com/docs/loki/latest/query/log_queries/#json). * Everything else is stringified using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353) **Special notes for your reviewer**: I will open follow-up PRs for: * Documentation * Make blessed attributes list configurable per tenant. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label
2 years ago
func TestOTLPLogsIngestQuery(t *testing.T) {
clu := cluster.New(nil, func(c *cluster.Cluster) {
c.SetSchemaVer("v13")
})
defer func() {
assert.NoError(t, clu.Cleanup())
}()
// run initially the compactor, indexgateway, and distributor.
var (
tCompactor = clu.AddComponent(
"compactor",
"-target=compactor",
config: remove flags & YAML settings deprecated in 2.9.x (#11151) **What this PR does / why we need it**: - remove `-ruler.evaluation-delay-duration` - remove `boltdb.shipper.compactor.deletion-mode` - remove compactor flags with prefix`-boltdb.shipper.compactor.*` - remove `validation.enforce-metric-name` - remove `log.use-buffered`, `log.use-sync` **Special notes for your reviewer**: `log.use-buffered`, `log.use-sync` are not mentioned in the upgrade guide since they were hidden fields **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [X] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [X] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
"-compactor.compaction-interval=1s",
"-compactor.retention-delete-delay=1s",
ingestion: native otlp ingestion support (#10727) **What this PR does / why we need it**: Add support for natively supporting logs ingestion in OTLP format. `/otlp/v1/logs` is the new endpoint where users can push logs in OTLP format. It accepts logs serialized in JSON or proto format. Since OTEL format is very different than what Loki storage model, here is how data in OTEL format will be mapped to Loki data model: * Index labels: The Resource Attributes map quite well to Index labels in Loki since both usually identify the source of the logs. The problem however is that Resource attributes in OTLP can have an unbounded number of values while Loki has a default limit of having up to 30 labels. Since Index labels in Loki can largely drive the kind of querying experience the users are going to have, we have chosen select attributes which would be picked as Index Labels. The ones that are not picked up as Index labels would be stored as Structured Metadata with each log entry. * Timestamp: LogRecord.TimeUnixNano * LogLine: LogRecord.Body holds the body of the log. However, since Loki only supports Log body in string format, we will stringify non-string values using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). * Structured Metadata: Anything which can’t be stored in Index labels and LogLine. Here is a non-exhaustive list of what will be stored in Structured Metadata to give a sense of what it will hold: * Resource Attributes not stored as Index labels is replicated and stored with each log entry. * Everything under InstrumentationScope is replicated and stored with each log entry. * Everything under LogRecord except LogRecord.Body, LogRecord.TimeUnixNano and sometimes LogRecord.ObservedTimestamp. *NOTES*: * Since Loki does not support `.` or any other special characters other than `_` in label names, we replace all non-supported characters with `_`. * Since Loki only supports string in values of Index Labels and Structured Metadata, all the complex types are converted as follows: * Map would be flattened into label keys using `_` as separator, same as how we do it in [json parser in LogQL](https://grafana.com/docs/loki/latest/query/log_queries/#json). * Everything else is stringified using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353) **Special notes for your reviewer**: I will open follow-up PRs for: * Documentation * Make blessed attributes list configurable per tenant. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label
2 years ago
// By default, a minute is added to the delete request start time. This compensates for that.
config: remove flags & YAML settings deprecated in 2.9.x (#11151) **What this PR does / why we need it**: - remove `-ruler.evaluation-delay-duration` - remove `boltdb.shipper.compactor.deletion-mode` - remove compactor flags with prefix`-boltdb.shipper.compactor.*` - remove `validation.enforce-metric-name` - remove `log.use-buffered`, `log.use-sync` **Special notes for your reviewer**: `log.use-buffered`, `log.use-sync` are not mentioned in the upgrade guide since they were hidden fields **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [X] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [X] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
"-compactor.delete-request-cancel-period=-60s",
ingestion: native otlp ingestion support (#10727) **What this PR does / why we need it**: Add support for natively supporting logs ingestion in OTLP format. `/otlp/v1/logs` is the new endpoint where users can push logs in OTLP format. It accepts logs serialized in JSON or proto format. Since OTEL format is very different than what Loki storage model, here is how data in OTEL format will be mapped to Loki data model: * Index labels: The Resource Attributes map quite well to Index labels in Loki since both usually identify the source of the logs. The problem however is that Resource attributes in OTLP can have an unbounded number of values while Loki has a default limit of having up to 30 labels. Since Index labels in Loki can largely drive the kind of querying experience the users are going to have, we have chosen select attributes which would be picked as Index Labels. The ones that are not picked up as Index labels would be stored as Structured Metadata with each log entry. * Timestamp: LogRecord.TimeUnixNano * LogLine: LogRecord.Body holds the body of the log. However, since Loki only supports Log body in string format, we will stringify non-string values using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). * Structured Metadata: Anything which can’t be stored in Index labels and LogLine. Here is a non-exhaustive list of what will be stored in Structured Metadata to give a sense of what it will hold: * Resource Attributes not stored as Index labels is replicated and stored with each log entry. * Everything under InstrumentationScope is replicated and stored with each log entry. * Everything under LogRecord except LogRecord.Body, LogRecord.TimeUnixNano and sometimes LogRecord.ObservedTimestamp. *NOTES*: * Since Loki does not support `.` or any other special characters other than `_` in label names, we replace all non-supported characters with `_`. * Since Loki only supports string in values of Index Labels and Structured Metadata, all the complex types are converted as follows: * Map would be flattened into label keys using `_` as separator, same as how we do it in [json parser in LogQL](https://grafana.com/docs/loki/latest/query/log_queries/#json). * Everything else is stringified using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353) **Special notes for your reviewer**: I will open follow-up PRs for: * Documentation * Make blessed attributes list configurable per tenant. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label
2 years ago
"-compactor.deletion-mode=filter-and-delete",
)
tIndexGateway = clu.AddComponent(
"index-gateway",
"-target=index-gateway",
)
tDistributor = clu.AddComponent(
"distributor",
"-target=distributor",
)
)
require.NoError(t, clu.Run())
// then, run only the ingester and query scheduler.
var (
tIngester = clu.AddComponent(
"ingester",
"-target=ingester",
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
)
tQueryScheduler = clu.AddComponent(
"query-scheduler",
"-target=query-scheduler",
"-query-scheduler.use-scheduler-ring=false",
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
)
)
require.NoError(t, clu.Run())
// finally, run the query-frontend and querier.
var (
tQueryFrontend = clu.AddComponent(
"query-frontend",
"-target=query-frontend",
"-frontend.scheduler-address="+tQueryScheduler.GRPCURL(),
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
"-querier.per-request-limits-enabled=true",
"-frontend.encoding=protobuf",
ingestion: native otlp ingestion support (#10727) **What this PR does / why we need it**: Add support for natively supporting logs ingestion in OTLP format. `/otlp/v1/logs` is the new endpoint where users can push logs in OTLP format. It accepts logs serialized in JSON or proto format. Since OTEL format is very different than what Loki storage model, here is how data in OTEL format will be mapped to Loki data model: * Index labels: The Resource Attributes map quite well to Index labels in Loki since both usually identify the source of the logs. The problem however is that Resource attributes in OTLP can have an unbounded number of values while Loki has a default limit of having up to 30 labels. Since Index labels in Loki can largely drive the kind of querying experience the users are going to have, we have chosen select attributes which would be picked as Index Labels. The ones that are not picked up as Index labels would be stored as Structured Metadata with each log entry. * Timestamp: LogRecord.TimeUnixNano * LogLine: LogRecord.Body holds the body of the log. However, since Loki only supports Log body in string format, we will stringify non-string values using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). * Structured Metadata: Anything which can’t be stored in Index labels and LogLine. Here is a non-exhaustive list of what will be stored in Structured Metadata to give a sense of what it will hold: * Resource Attributes not stored as Index labels is replicated and stored with each log entry. * Everything under InstrumentationScope is replicated and stored with each log entry. * Everything under LogRecord except LogRecord.Body, LogRecord.TimeUnixNano and sometimes LogRecord.ObservedTimestamp. *NOTES*: * Since Loki does not support `.` or any other special characters other than `_` in label names, we replace all non-supported characters with `_`. * Since Loki only supports string in values of Index Labels and Structured Metadata, all the complex types are converted as follows: * Map would be flattened into label keys using `_` as separator, same as how we do it in [json parser in LogQL](https://grafana.com/docs/loki/latest/query/log_queries/#json). * Everything else is stringified using [AsString method from OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353) **Special notes for your reviewer**: I will open follow-up PRs for: * Documentation * Make blessed attributes list configurable per tenant. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label
2 years ago
)
_ = clu.AddComponent(
"querier",
"-target=querier",
"-querier.scheduler-address="+tQueryScheduler.GRPCURL(),
"-boltdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
)
)
require.NoError(t, clu.Run())
tenantID := randStringRunes()
now := time.Now()
cliDistributor := client.New(tenantID, "", tDistributor.HTTPURL())
cliDistributor.Now = now
cliIngester := client.New(tenantID, "", tIngester.HTTPURL())
cliIngester.Now = now
cliQueryFrontend := client.New(tenantID, "", tQueryFrontend.HTTPURL())
cliQueryFrontend.Now = now
t.Run("ingest-logs", func(t *testing.T) {
// ingest some log lines
require.NoError(t, cliDistributor.PushOTLPLogLine("lineA", now.Add(-45*time.Minute), map[string]any{"trace_id": 1, "user_id": "2"}))
require.NoError(t, cliDistributor.PushOTLPLogLine("lineB", now.Add(-45*time.Minute), nil))
require.NoError(t, cliDistributor.PushOTLPLogLine("lineC", now, map[string]any{"order.ids": []any{5, 6}}))
require.NoError(t, cliDistributor.PushOTLPLogLine("lineD", now, nil))
})
t.Run("query", func(t *testing.T) {
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{service_name="varlog"}`)
require.NoError(t, err)
assert.Equal(t, "streams", resp.Data.ResultType)
numLinesReceived := 0
for i, stream := range resp.Data.Stream {
switch i {
case 0:
require.Len(t, stream.Values, 2)
require.Equal(t, "lineD", stream.Values[0][1])
require.Equal(t, "lineB", stream.Values[1][1])
require.Equal(t, map[string]string{
"service_name": "varlog",
}, stream.Stream)
numLinesReceived += 2
case 1:
require.Len(t, stream.Values, 1)
require.Equal(t, "lineA", stream.Values[0][1])
require.Equal(t, map[string]string{
"service_name": "varlog",
"trace_id": "1",
"user_id": "2",
}, stream.Stream)
numLinesReceived++
case 2:
require.Len(t, stream.Values, 1)
require.Equal(t, "lineC", stream.Values[0][1])
require.Equal(t, map[string]string{
"service_name": "varlog",
"order_ids": "[5,6]",
}, stream.Stream)
numLinesReceived++
default:
t.Errorf("unexpected case %d", i)
}
}
require.Equal(t, 4, numLinesReceived)
})
}
Flag categorize labels on streams response (#10419) We recently introduced support for ingesting and querying structured metadata in Loki. This adds a new dimension to Loki's labels since now we arguably have three categories of labels: _stream_, _structured metadata_, and _parsed_ labels. Depending on the origin of the labels, they should be used in LogQL expressions differently to achieve optimal performance. _stream_ labels should be added to stream matchers, _structured metadata_ labels should be used in a filter expression before any parsing expression, and _parsed_ labels should be placed after the parser expression extracting them. The Grafana UI has a hard time dealing with this same problem. Before https://github.com/grafana/grafana/pull/73955, the filtering functionality in Grafana was broken since it was not able to distinguish between _stream_ and _structured metadata_ labels. Also, as soon as a parser expression was added to the query, filters added by Grafana would be appended to the end of the query regardless of the label category. The PR above implements a workaround for this problem but needs a better API on Loki's end to mitigate all corner cases. Loki currently returns the following JSON for log queries: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "info", "traceID": "68810cf0c94bfcca" }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n" ], ... }, { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "debug", "traceID": "a7116cj54c4bjz8s" }, "values": [ [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n" ], ... }, ... ``` As can be seen, there is no way we can distinguish the category of each label. This PR introduces a new flag `X-Loki-Response-Encoding-Flags: categorize-labels` that makes Loki return categorized labels as follows: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n", { "structuredMetadata": { "traceID": "68810cf0c94bfcca" }, "parsed": { "level": "info" } } ], [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n", { "structuredMetadata": { "traceID": "a7116cj54c4bjz8s" }, "parsed": { "level": "debug" } } ], ... }, ... ``` Note that this PR only supports log queries, not metric queries. From a UX perspective, being able to categorize labels in metric queries doesn't have any benefit yet. Having said that, supporting this for metric queries would require some minor refactoring on top of what has been implemented here. If we decide to do that, I think we should do it on a separate PR to avoid making this PR even larger. I also decided to leave out support for Tail queries to avoid making this PR even larger. Once this one gets merged, we can work to support tailing. --- **Note to reviewers** This PR is huge since we need to forward categorized all over the codebase (from parsing logs all the way to marshaling), fortunately, many of the changes come from updating tests and refactoring iterators. Tested out in a dev cell with query `'{stream="stdout"} | label_format new="text"`. - Without the new flag: ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED { "data": { "result": [ { "stream": { "new": "text", "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n" ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n" ], ... ``` - With the new flag ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED X-Loki-Response-Encoding-Flags:categorize-labels { "data": { "encodingFlags": [ "categorize-labels" ], "result": [ { "stream": { "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n", { "parsed": { "new": "text" } } ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n", { "parsed": { "new": "text" } } ], ... ```
2 years ago
func TestCategorizedLabels(t *testing.T) {
clu := cluster.New(nil, cluster.SchemaWithTSDB, func(c *cluster.Cluster) {
c.SetSchemaVer("v13")
})
defer func() {
assert.NoError(t, clu.Cleanup())
}()
var (
tDistributor = clu.AddComponent(
"distributor",
"-target=distributor",
)
tIndexGateway = clu.AddComponent(
"index-gateway",
"-target=index-gateway",
"-store.index-cache-read.embedded-cache.enabled=true",
)
)
require.NoError(t, clu.Run())
var (
tIngester = clu.AddComponent(
"ingester",
"-target=ingester",
"-ingester.flush-on-shutdown=true",
"-ingester.wal-enabled=false",
"-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
)
tQueryScheduler = clu.AddComponent(
"query-scheduler",
"-target=query-scheduler",
"-query-scheduler.use-scheduler-ring=false",
"-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
)
tCompactor = clu.AddComponent(
"compactor",
"-target=compactor",
config: remove flags & YAML settings deprecated in 2.9.x (#11151) **What this PR does / why we need it**: - remove `-ruler.evaluation-delay-duration` - remove `boltdb.shipper.compactor.deletion-mode` - remove compactor flags with prefix`-boltdb.shipper.compactor.*` - remove `validation.enforce-metric-name` - remove `log.use-buffered`, `log.use-sync` **Special notes for your reviewer**: `log.use-buffered`, `log.use-sync` are not mentioned in the upgrade guide since they were hidden fields **Checklist** - [X] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [X] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [X] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com>
2 years ago
"-compactor.compaction-interval=1s",
Flag categorize labels on streams response (#10419) We recently introduced support for ingesting and querying structured metadata in Loki. This adds a new dimension to Loki's labels since now we arguably have three categories of labels: _stream_, _structured metadata_, and _parsed_ labels. Depending on the origin of the labels, they should be used in LogQL expressions differently to achieve optimal performance. _stream_ labels should be added to stream matchers, _structured metadata_ labels should be used in a filter expression before any parsing expression, and _parsed_ labels should be placed after the parser expression extracting them. The Grafana UI has a hard time dealing with this same problem. Before https://github.com/grafana/grafana/pull/73955, the filtering functionality in Grafana was broken since it was not able to distinguish between _stream_ and _structured metadata_ labels. Also, as soon as a parser expression was added to the query, filters added by Grafana would be appended to the end of the query regardless of the label category. The PR above implements a workaround for this problem but needs a better API on Loki's end to mitigate all corner cases. Loki currently returns the following JSON for log queries: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "info", "traceID": "68810cf0c94bfcca" }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n" ], ... }, { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "debug", "traceID": "a7116cj54c4bjz8s" }, "values": [ [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n" ], ... }, ... ``` As can be seen, there is no way we can distinguish the category of each label. This PR introduces a new flag `X-Loki-Response-Encoding-Flags: categorize-labels` that makes Loki return categorized labels as follows: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n", { "structuredMetadata": { "traceID": "68810cf0c94bfcca" }, "parsed": { "level": "info" } } ], [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n", { "structuredMetadata": { "traceID": "a7116cj54c4bjz8s" }, "parsed": { "level": "debug" } } ], ... }, ... ``` Note that this PR only supports log queries, not metric queries. From a UX perspective, being able to categorize labels in metric queries doesn't have any benefit yet. Having said that, supporting this for metric queries would require some minor refactoring on top of what has been implemented here. If we decide to do that, I think we should do it on a separate PR to avoid making this PR even larger. I also decided to leave out support for Tail queries to avoid making this PR even larger. Once this one gets merged, we can work to support tailing. --- **Note to reviewers** This PR is huge since we need to forward categorized all over the codebase (from parsing logs all the way to marshaling), fortunately, many of the changes come from updating tests and refactoring iterators. Tested out in a dev cell with query `'{stream="stdout"} | label_format new="text"`. - Without the new flag: ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED { "data": { "result": [ { "stream": { "new": "text", "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n" ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n" ], ... ``` - With the new flag ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED X-Loki-Response-Encoding-Flags:categorize-labels { "data": { "encodingFlags": [ "categorize-labels" ], "result": [ { "stream": { "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n", { "parsed": { "new": "text" } } ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n", { "parsed": { "new": "text" } } ], ... ```
2 years ago
"-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
)
)
require.NoError(t, clu.Run())
// finally, run the query-frontend and querier.
var (
tQueryFrontend = clu.AddComponent(
"query-frontend",
"-target=query-frontend",
"-frontend.scheduler-address="+tQueryScheduler.GRPCURL(),
"-frontend.default-validity=0s",
"-common.compactor-address="+tCompactor.HTTPURL(),
"-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
)
_ = clu.AddComponent(
"querier",
"-target=querier",
"-querier.scheduler-address="+tQueryScheduler.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
"-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(),
)
)
require.NoError(t, clu.Run())
tenantID := randStringRunes()
now := time.Now()
cliDistributor := client.New(tenantID, "", tDistributor.HTTPURL())
cliDistributor.Now = now
cliIngester := client.New(tenantID, "", tIngester.HTTPURL())
cliIngester.Now = now
cliQueryFrontend := client.New(tenantID, "", tQueryFrontend.HTTPURL())
cliQueryFrontend.Now = now
cliIndexGateway := client.New(tenantID, "", tIndexGateway.HTTPURL())
cliIndexGateway.Now = now
now = time.Now()
require.NoError(t, cliDistributor.PushLogLine("lineA", now.Add(-1*time.Second), nil, map[string]string{"job": "fake"}))
require.NoError(t, cliDistributor.PushLogLine("lineB", now.Add(-2*time.Second), map[string]string{"traceID": "123", "user": "a"}, map[string]string{"job": "fake"}))
require.NoError(t, tIngester.Restart())
require.NoError(t, cliDistributor.PushLogLine("lineC msg=foo", now.Add(-3*time.Second), map[string]string{"traceID": "456", "user": "b"}, map[string]string{"job": "fake"}))
require.NoError(t, cliDistributor.PushLogLine("lineD msg=foo text=bar", now.Add(-4*time.Second), map[string]string{"traceID": "789", "user": "c"}, map[string]string{"job": "fake"}))
type expectedStream struct {
Stream map[string]string
Lines []string
CategorizedLabels []map[string]map[string]string
}
for _, tc := range []struct {
name string
query string
encodingFlags []string
expectedStreams []expectedStream
}{
{
name: "no header - no parser ",
query: `{job="fake"}`,
expectedStreams: []expectedStream{
{
Stream: labels.FromStrings("job", "fake").Map(),
Lines: []string{"lineA"},
},
{
Stream: map[string]string{
"job": "fake",
"traceID": "123",
"user": "a",
},
Lines: []string{"lineB"},
},
{
Stream: map[string]string{
"job": "fake",
"traceID": "456",
"user": "b",
},
Lines: []string{"lineC msg=foo"},
},
{
Stream: map[string]string{
"job": "fake",
"traceID": "789",
"user": "c",
},
Lines: []string{"lineD msg=foo text=bar"},
},
},
},
{
name: "no header - with parser",
query: `{job="fake"} | logfmt`,
expectedStreams: []expectedStream{
{
Stream: map[string]string{
"job": "fake",
},
Lines: []string{"lineA"},
},
{
Stream: map[string]string{
"job": "fake",
"traceID": "123",
"user": "a",
},
Lines: []string{"lineB"},
},
{
Stream: map[string]string{
"job": "fake",
"traceID": "456",
"user": "b",
"msg": "foo",
},
Lines: []string{"lineC msg=foo"},
},
{
Stream: map[string]string{
"job": "fake",
"traceID": "789",
"user": "c",
"msg": "foo",
"text": "bar",
},
Lines: []string{"lineD msg=foo text=bar"},
},
},
},
{
name: "with header - no parser ",
query: `{job="fake"}`,
encodingFlags: []string{
string(httpreq.FlagCategorizeLabels),
},
expectedStreams: []expectedStream{
{
Stream: map[string]string{
"job": "fake",
},
Lines: []string{"lineA", "lineB", "lineC msg=foo", "lineD msg=foo text=bar"},
CategorizedLabels: []map[string]map[string]string{
{},
Flag categorize labels on streams response (#10419) We recently introduced support for ingesting and querying structured metadata in Loki. This adds a new dimension to Loki's labels since now we arguably have three categories of labels: _stream_, _structured metadata_, and _parsed_ labels. Depending on the origin of the labels, they should be used in LogQL expressions differently to achieve optimal performance. _stream_ labels should be added to stream matchers, _structured metadata_ labels should be used in a filter expression before any parsing expression, and _parsed_ labels should be placed after the parser expression extracting them. The Grafana UI has a hard time dealing with this same problem. Before https://github.com/grafana/grafana/pull/73955, the filtering functionality in Grafana was broken since it was not able to distinguish between _stream_ and _structured metadata_ labels. Also, as soon as a parser expression was added to the query, filters added by Grafana would be appended to the end of the query regardless of the label category. The PR above implements a workaround for this problem but needs a better API on Loki's end to mitigate all corner cases. Loki currently returns the following JSON for log queries: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "info", "traceID": "68810cf0c94bfcca" }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n" ], ... }, { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "debug", "traceID": "a7116cj54c4bjz8s" }, "values": [ [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n" ], ... }, ... ``` As can be seen, there is no way we can distinguish the category of each label. This PR introduces a new flag `X-Loki-Response-Encoding-Flags: categorize-labels` that makes Loki return categorized labels as follows: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n", { "structuredMetadata": { "traceID": "68810cf0c94bfcca" }, "parsed": { "level": "info" } } ], [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n", { "structuredMetadata": { "traceID": "a7116cj54c4bjz8s" }, "parsed": { "level": "debug" } } ], ... }, ... ``` Note that this PR only supports log queries, not metric queries. From a UX perspective, being able to categorize labels in metric queries doesn't have any benefit yet. Having said that, supporting this for metric queries would require some minor refactoring on top of what has been implemented here. If we decide to do that, I think we should do it on a separate PR to avoid making this PR even larger. I also decided to leave out support for Tail queries to avoid making this PR even larger. Once this one gets merged, we can work to support tailing. --- **Note to reviewers** This PR is huge since we need to forward categorized all over the codebase (from parsing logs all the way to marshaling), fortunately, many of the changes come from updating tests and refactoring iterators. Tested out in a dev cell with query `'{stream="stdout"} | label_format new="text"`. - Without the new flag: ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED { "data": { "result": [ { "stream": { "new": "text", "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n" ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n" ], ... ``` - With the new flag ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED X-Loki-Response-Encoding-Flags:categorize-labels { "data": { "encodingFlags": [ "categorize-labels" ], "result": [ { "stream": { "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n", { "parsed": { "new": "text" } } ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n", { "parsed": { "new": "text" } } ], ... ```
2 years ago
{
"structuredMetadata": {
"traceID": "123",
"user": "a",
},
},
{
"structuredMetadata": {
"traceID": "456",
"user": "b",
},
},
{
"structuredMetadata": {
"traceID": "789",
"user": "c",
},
},
},
},
},
},
{
name: "with header - with parser",
query: `{job="fake"} | logfmt`,
encodingFlags: []string{
string(httpreq.FlagCategorizeLabels),
},
expectedStreams: []expectedStream{
{
Stream: map[string]string{
"job": "fake",
},
Lines: []string{"lineA", "lineB", "lineC msg=foo", "lineD msg=foo text=bar"},
CategorizedLabels: []map[string]map[string]string{
{},
Flag categorize labels on streams response (#10419) We recently introduced support for ingesting and querying structured metadata in Loki. This adds a new dimension to Loki's labels since now we arguably have three categories of labels: _stream_, _structured metadata_, and _parsed_ labels. Depending on the origin of the labels, they should be used in LogQL expressions differently to achieve optimal performance. _stream_ labels should be added to stream matchers, _structured metadata_ labels should be used in a filter expression before any parsing expression, and _parsed_ labels should be placed after the parser expression extracting them. The Grafana UI has a hard time dealing with this same problem. Before https://github.com/grafana/grafana/pull/73955, the filtering functionality in Grafana was broken since it was not able to distinguish between _stream_ and _structured metadata_ labels. Also, as soon as a parser expression was added to the query, filters added by Grafana would be appended to the end of the query regardless of the label category. The PR above implements a workaround for this problem but needs a better API on Loki's end to mitigate all corner cases. Loki currently returns the following JSON for log queries: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "info", "traceID": "68810cf0c94bfcca" }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n" ], ... }, { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", "level": "debug", "traceID": "a7116cj54c4bjz8s" }, "values": [ [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n" ], ... }, ... ``` As can be seen, there is no way we can distinguish the category of each label. This PR introduces a new flag `X-Loki-Response-Encoding-Flags: categorize-labels` that makes Loki return categorized labels as follows: ```json ... { "stream": { "cluster": "us-central", "container": "query-frontend", "namespace": "loki", }, "values": [ [ "1693996529000222496", "1693996529000222496 aaaaaaaaa.....\n", { "structuredMetadata": { "traceID": "68810cf0c94bfcca" }, "parsed": { "level": "info" } } ], [ "1693996529000222497", "1693996529000222497 bbbbbbbbb.....\n", { "structuredMetadata": { "traceID": "a7116cj54c4bjz8s" }, "parsed": { "level": "debug" } } ], ... }, ... ``` Note that this PR only supports log queries, not metric queries. From a UX perspective, being able to categorize labels in metric queries doesn't have any benefit yet. Having said that, supporting this for metric queries would require some minor refactoring on top of what has been implemented here. If we decide to do that, I think we should do it on a separate PR to avoid making this PR even larger. I also decided to leave out support for Tail queries to avoid making this PR even larger. Once this one gets merged, we can work to support tailing. --- **Note to reviewers** This PR is huge since we need to forward categorized all over the codebase (from parsing logs all the way to marshaling), fortunately, many of the changes come from updating tests and refactoring iterators. Tested out in a dev cell with query `'{stream="stdout"} | label_format new="text"`. - Without the new flag: ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED { "data": { "result": [ { "stream": { "new": "text", "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n" ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n" ], ... ``` - With the new flag ``` $ http http://127.0.0.1:3100/loki/api/v1/query_range\?direction\=BACKWARD\&end\=1693996529322486000\&limit\=30\&query\=%7Bstream%3D%22stdout%22%7D+%7C+label_format+new%3D%22text%22\&start\=1693992929322486000 X-Scope-Orgid:REDACTED X-Loki-Response-Encoding-Flags:categorize-labels { "data": { "encodingFlags": [ "categorize-labels" ], "result": [ { "stream": { "pod": "loki-canary-986bd6f4b-xqmb7", "stream": "stdout" }, "values": [ [ "1693996529000222496", "1693996529000222496 pppppppppppp...\n", { "parsed": { "new": "text" } } ], [ "1693996528499160852", "1693996528499160852 pppppppppppp...\n", { "parsed": { "new": "text" } } ], ... ```
2 years ago
{
"structuredMetadata": {
"traceID": "123",
"user": "a",
},
},
{
"structuredMetadata": {
"traceID": "456",
"user": "b",
},
"parsed": {
"msg": "foo",
},
},
{
"structuredMetadata": {
"traceID": "789",
"user": "c",
},
"parsed": {
"msg": "foo",
"text": "bar",
},
},
},
},
},
},
} {
t.Run(tc.name, func(t *testing.T) {
// Add header with encoding flags and expect them to be returned in the response.
var headers []client.Header
var expectedEncodingFlags []string
if len(tc.encodingFlags) > 0 {
headers = append(headers, client.Header{Name: httpreq.LokiEncodingFlagsHeader, Value: strings.Join(tc.encodingFlags, httpreq.EncodeFlagsDelimiter)})
expectedEncodingFlags = tc.encodingFlags
}
resp, err := cliQueryFrontend.RunQuery(context.Background(), tc.query, headers...)
require.NoError(t, err)
assert.Equal(t, "streams", resp.Data.ResultType)
var streams []expectedStream
for _, stream := range resp.Data.Stream {
var lines []string
var categorizedLabels []map[string]map[string]string
for _, val := range stream.Values {
lines = append(lines, val[1])
var catLabels map[string]map[string]string
if len(val) >= 3 && val[2] != "" {
err = json.Unmarshal([]byte(val[2]), &catLabels)
require.NoError(t, err)
categorizedLabels = append(categorizedLabels, catLabels)
}
}
streams = append(streams, expectedStream{
Stream: stream.Stream,
Lines: lines,
CategorizedLabels: categorizedLabels,
})
}
assert.ElementsMatch(t, tc.expectedStreams, streams)
assert.ElementsMatch(t, expectedEncodingFlags, resp.Data.EncodingFlags)
})
}
}
func getValueFromMF(mf *dto.MetricFamily, lbs []*dto.LabelPair) float64 {
for _, m := range mf.Metric {
if !assert.ObjectsAreEqualValues(lbs, m.GetLabel()) {
continue
}
return m.Counter.GetValue()
}
return 0
}
func assertCacheState(t *testing.T, metrics string, e *expectedCacheState) {
var parser expfmt.TextParser
mfs, err := parser.TextToMetricFamilies(strings.NewReader(metrics))
require.NoError(t, err)
lbs := []*dto.LabelPair{
{
Name: proto.String("cache"),
Value: proto.String(e.cacheName),
},
}
embedded cache: update metric namespace and remove duplicate metrics (#10693) **What this PR does / why we need it**: Removes the following embedded cache metrics since `loki_cache_request_duration_seconds` (which instruments requests made to all the caches) [already covers them](https://github.com/grafana/loki/blob/6a07988491124c91d4acb1cbbf2ee2a4d7dc8900/pkg/storage/chunk/cache/cache.go#L110): - `querier_cache_added_total` - `querier_cache_gets_total` - `querier_cache_misses_total` Updates the namespace and subsystem for the following metrics - `querier_cache_added_new_total` is renamed to `loki_embeddedcache_added_new_total` - `querier_cache_evicted_total` is renamed to `loki_embeddedcache_evicted_total` - `querier_cache_entries` is renamed to `loki_embeddedcache_entries` - `querier_cache_memory_bytes` is renamed to `loki_embeddedcache_memory_bytes` Removes already deprecated metric `querier_cache_stale_gets_total` **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213)
2 years ago
mf, found := mfs["loki_embeddedcache_added_new_total"]
require.True(t, found)
require.Equal(t, e.added, getValueFromMF(mf, lbs))
embedded cache: update metric namespace and remove duplicate metrics (#10693) **What this PR does / why we need it**: Removes the following embedded cache metrics since `loki_cache_request_duration_seconds` (which instruments requests made to all the caches) [already covers them](https://github.com/grafana/loki/blob/6a07988491124c91d4acb1cbbf2ee2a4d7dc8900/pkg/storage/chunk/cache/cache.go#L110): - `querier_cache_added_total` - `querier_cache_gets_total` - `querier_cache_misses_total` Updates the namespace and subsystem for the following metrics - `querier_cache_added_new_total` is renamed to `loki_embeddedcache_added_new_total` - `querier_cache_evicted_total` is renamed to `loki_embeddedcache_evicted_total` - `querier_cache_entries` is renamed to `loki_embeddedcache_entries` - `querier_cache_memory_bytes` is renamed to `loki_embeddedcache_memory_bytes` Removes already deprecated metric `querier_cache_stale_gets_total` **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213)
2 years ago
lbs = []*dto.LabelPair{
{
Name: proto.String("name"),
Value: proto.String(e.cacheName),
},
}
config: loki better defaults (#10793) **What this PR does / why we need it**: Updates some of the configuration defaults to provide a better experience for users out of the box. I tried to add notes for non-trivial changes explaining the reason. | configuration | new default | old default | notes | | ------------------------------------------------------ | ----------- | ----------- | -------- | `compactor.delete-max-interval` | 24h | 0 | splits the delete requests into intervals no longer than `delete_max_interval` | | `distributor.max-line-size` | 256KB | 0 | - | | `ingester.sync-period` | 1h | 0 | ensures that the chunk cuts for a given stream are synchronized across the ingesters in the replication set. Helps with deduplicating chunks. | | `ingester.sync-min-utilization` | 0.1 | 0 | ^ | | `frontend.max-querier-bytes-read` | 150GB | 0 | - | | `frontend.max-cache-freshness` | 10m | 1m | avoid caching results for data that is still in churn | | `frontend.max-stats-cache-freshness` | 10m | 0 | avoid caching results for data that is still in churn | | `memcached.batchsize` | 256 | 1024 | - | | `memcached.parallelism` | 10 | 100 | - | | `querier.tsdb-max-query-parallelism` | 128 | 512 | avoid over parallelising queries for smaller installations | | `querier.split-queries-by-interval` | 1h | 30m | avoid over parallelising queries for smaller installations | | `querier.compress-http-responses` | true | false | compress response if the request accepts gzip encoding | | `query-scheduler.max-outstanding-requests-per-tenant` | 32000 | 100 | current default is too small, opening a big dashboard could hit this limit | | `validation.max-label-names-per-series` | 15 | 30 | reducing this to avoid blowing up the series count and index size. Also enforces users to think about what labels are being added. | **Special notes for your reviewer**: regarding `TestQueryTSDB_WithCachedPostings`: removed the assert on get calls as this is hard to get right (splits, querySize mw making stats calls) validating additions to the cache and misses is more important here I think. **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213)
2 years ago
gets, found := mfs["loki_cache_fetched_keys"]
require.True(t, found)
config: loki better defaults (#10793) **What this PR does / why we need it**: Updates some of the configuration defaults to provide a better experience for users out of the box. I tried to add notes for non-trivial changes explaining the reason. | configuration | new default | old default | notes | | ------------------------------------------------------ | ----------- | ----------- | -------- | `compactor.delete-max-interval` | 24h | 0 | splits the delete requests into intervals no longer than `delete_max_interval` | | `distributor.max-line-size` | 256KB | 0 | - | | `ingester.sync-period` | 1h | 0 | ensures that the chunk cuts for a given stream are synchronized across the ingesters in the replication set. Helps with deduplicating chunks. | | `ingester.sync-min-utilization` | 0.1 | 0 | ^ | | `frontend.max-querier-bytes-read` | 150GB | 0 | - | | `frontend.max-cache-freshness` | 10m | 1m | avoid caching results for data that is still in churn | | `frontend.max-stats-cache-freshness` | 10m | 0 | avoid caching results for data that is still in churn | | `memcached.batchsize` | 256 | 1024 | - | | `memcached.parallelism` | 10 | 100 | - | | `querier.tsdb-max-query-parallelism` | 128 | 512 | avoid over parallelising queries for smaller installations | | `querier.split-queries-by-interval` | 1h | 30m | avoid over parallelising queries for smaller installations | | `querier.compress-http-responses` | true | false | compress response if the request accepts gzip encoding | | `query-scheduler.max-outstanding-requests-per-tenant` | 32000 | 100 | current default is too small, opening a big dashboard could hit this limit | | `validation.max-label-names-per-series` | 15 | 30 | reducing this to avoid blowing up the series count and index size. Also enforces users to think about what labels are being added. | **Special notes for your reviewer**: regarding `TestQueryTSDB_WithCachedPostings`: removed the assert on get calls as this is hard to get right (splits, querySize mw making stats calls) validating additions to the cache and misses is more important here I think. **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213)
2 years ago
hits, found := mfs["loki_cache_hits"]
require.True(t, found)
config: loki better defaults (#10793) **What this PR does / why we need it**: Updates some of the configuration defaults to provide a better experience for users out of the box. I tried to add notes for non-trivial changes explaining the reason. | configuration | new default | old default | notes | | ------------------------------------------------------ | ----------- | ----------- | -------- | `compactor.delete-max-interval` | 24h | 0 | splits the delete requests into intervals no longer than `delete_max_interval` | | `distributor.max-line-size` | 256KB | 0 | - | | `ingester.sync-period` | 1h | 0 | ensures that the chunk cuts for a given stream are synchronized across the ingesters in the replication set. Helps with deduplicating chunks. | | `ingester.sync-min-utilization` | 0.1 | 0 | ^ | | `frontend.max-querier-bytes-read` | 150GB | 0 | - | | `frontend.max-cache-freshness` | 10m | 1m | avoid caching results for data that is still in churn | | `frontend.max-stats-cache-freshness` | 10m | 0 | avoid caching results for data that is still in churn | | `memcached.batchsize` | 256 | 1024 | - | | `memcached.parallelism` | 10 | 100 | - | | `querier.tsdb-max-query-parallelism` | 128 | 512 | avoid over parallelising queries for smaller installations | | `querier.split-queries-by-interval` | 1h | 30m | avoid over parallelising queries for smaller installations | | `querier.compress-http-responses` | true | false | compress response if the request accepts gzip encoding | | `query-scheduler.max-outstanding-requests-per-tenant` | 32000 | 100 | current default is too small, opening a big dashboard could hit this limit | | `validation.max-label-names-per-series` | 15 | 30 | reducing this to avoid blowing up the series count and index size. Also enforces users to think about what labels are being added. | **Special notes for your reviewer**: regarding `TestQueryTSDB_WithCachedPostings`: removed the assert on get calls as this is hard to get right (splits, querySize mw making stats calls) validating additions to the cache and misses is more important here I think. **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213)
2 years ago
require.Equal(t, e.misses, getValueFromMF(gets, lbs)-getValueFromMF(hits, lbs))
}
type expectedCacheState struct {
cacheName string
misses float64
added float64
}