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/pkg/ingester/instance.go

1046 lines
29 KiB

package ingester
import (
"context"
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
"fmt"
"math"
"net/http"
"os"
"sync"
"syscall"
"time"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/httpgrpc"
"github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
Chunking (#10) * Add checkenc without serialisation for now. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Simplify interface and add serialisatio` Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Move away from \xFF magic to something simple Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Add serialisation and Deserialisation Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Modify interface to be closer to logish interface. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Fix race b/w append and iteration. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Make iterators honour bounds Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Remove locks as safety is assured externally Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Add checksums Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Add code quotes around block design. * Split headBlock into it's own type. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Simplify encoding and decoding. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Expose flags. Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Use the already existing EntryIterator interface Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Use existing Chunk interface. Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Review feedback Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Integrate the compressed chunk and add metrics Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com>
8 years ago
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/tsdb/chunks"
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
tsdb_record "github.com/prometheus/prometheus/tsdb/record"
"go.uber.org/atomic"
"github.com/grafana/loki/pkg/analytics"
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
"github.com/grafana/loki/pkg/chunkenc"
"github.com/grafana/loki/pkg/distributor/writefailures"
Shard ingester queries. (#3852) * Shard ingester queries. This is still experimental but already yield from 2x to 6x faster for short period queries. I'm still playing with it but I want to share how I do it early. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notice of the code origin. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Align shards from ingester and storage. Utimately we should have a storage that relies on fingerprint, but that's harder to change. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove comment Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes delete index func Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test reverting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug causing non deterministic hash. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Fixes build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * got linted :( Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
5 years ago
"github.com/grafana/loki/pkg/ingester/index"
"github.com/grafana/loki/pkg/ingester/wal"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/querier/astmapper"
"github.com/grafana/loki/pkg/runtime"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/grafana/loki/pkg/util"
"github.com/grafana/loki/pkg/util/deletion"
util_log "github.com/grafana/loki/pkg/util/log"
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
mathutil "github.com/grafana/loki/pkg/util/math"
"github.com/grafana/loki/pkg/validation"
)
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
const (
// ShardLbName is the internal label to be used by Loki when dividing a stream into smaller pieces.
// Possible values are only increasing integers starting from 0.
ShardLbName = "__stream_shard__"
ShardLbPlaceholder = "__placeholder__"
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
queryBatchSize = 128
queryBatchSampleSize = 512
)
Chunking (#10) * Add checkenc without serialisation for now. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Simplify interface and add serialisatio` Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Move away from \xFF magic to something simple Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Add serialisation and Deserialisation Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Modify interface to be closer to logish interface. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Fix race b/w append and iteration. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Make iterators honour bounds Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Remove locks as safety is assured externally Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Add checksums Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Add code quotes around block design. * Split headBlock into it's own type. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Simplify encoding and decoding. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Expose flags. Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Use the already existing EntryIterator interface Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Use existing Chunk interface. Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Review feedback Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Integrate the compressed chunk and add metrics Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com>
8 years ago
var (
memoryStreams = promauto.NewGaugeVec(prometheus.GaugeOpts{
Namespace: "loki",
Name: "ingester_memory_streams",
Help: "The total number of streams in memory per tenant.",
}, []string{"tenant"})
memoryStreamsLabelsBytes = promauto.NewGauge(prometheus.GaugeOpts{
Namespace: "loki",
Name: "ingester_memory_streams_labels_bytes",
Help: "Total bytes of labels of the streams in memory.",
})
streamsCreatedTotal = promauto.NewCounterVec(prometheus.CounterOpts{
Namespace: "loki",
Chunking (#10) * Add checkenc without serialisation for now. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Simplify interface and add serialisatio` Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Move away from \xFF magic to something simple Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Add serialisation and Deserialisation Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Modify interface to be closer to logish interface. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Fix race b/w append and iteration. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Make iterators honour bounds Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Remove locks as safety is assured externally Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Add checksums Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Add code quotes around block design. * Split headBlock into it's own type. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Simplify encoding and decoding. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Expose flags. Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Use the already existing EntryIterator interface Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Use existing Chunk interface. Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Review feedback Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Integrate the compressed chunk and add metrics Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com>
8 years ago
Name: "ingester_streams_created_total",
Help: "The total number of streams created per tenant.",
}, []string{"tenant"})
streamsRemovedTotal = promauto.NewCounterVec(prometheus.CounterOpts{
Namespace: "loki",
Chunking (#10) * Add checkenc without serialisation for now. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Simplify interface and add serialisatio` Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Move away from \xFF magic to something simple Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Add serialisation and Deserialisation Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Modify interface to be closer to logish interface. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Fix race b/w append and iteration. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Make iterators honour bounds Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Remove locks as safety is assured externally Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Add checksums Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Add code quotes around block design. * Split headBlock into it's own type. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Simplify encoding and decoding. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Expose flags. Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Use the already existing EntryIterator interface Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Use existing Chunk interface. Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Review feedback Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Integrate the compressed chunk and add metrics Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com>
8 years ago
Name: "ingester_streams_removed_total",
Help: "The total number of streams removed per tenant.",
}, []string{"tenant"})
streamsCountStats = analytics.NewInt("ingester_streams_count")
Chunking (#10) * Add checkenc without serialisation for now. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Simplify interface and add serialisatio` Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Move away from \xFF magic to something simple Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Add serialisation and Deserialisation Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Modify interface to be closer to logish interface. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Fix race b/w append and iteration. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Make iterators honour bounds Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Remove locks as safety is assured externally Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Add checksums Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Add code quotes around block design. * Split headBlock into it's own type. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Simplify encoding and decoding. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Expose flags. Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Use the already existing EntryIterator interface Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Use existing Chunk interface. Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Review feedback Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Integrate the compressed chunk and add metrics Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com>
8 years ago
)
type instance struct {
cfg *Config
buf []byte // buffer used to compute fps.
streams *streamsMap
index *index.Multi
mapper *fpMapper // using of mapper no longer needs mutex because reading from streams is lock-free
Chunking (#10) * Add checkenc without serialisation for now. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Simplify interface and add serialisatio` Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Move away from \xFF magic to something simple Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Add serialisation and Deserialisation Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Modify interface to be closer to logish interface. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Fix race b/w append and iteration. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Make iterators honour bounds Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Remove locks as safety is assured externally Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * chunkenc: Add checksums Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Add code quotes around block design. * Split headBlock into it's own type. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Simplify encoding and decoding. Signed-off-by: Goutham Veeramachaneni <cs14btech11014@iith.ac.in> * Expose flags. Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Use the already existing EntryIterator interface Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Use existing Chunk interface. Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Review feedback Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com> * Integrate the compressed chunk and add metrics Signed-off-by: Goutham Veeramachaneni <gouthamve@gmail.com>
8 years ago
instanceID string
streamsCreatedTotal prometheus.Counter
streamsRemovedTotal prometheus.Counter
Adds configurable compression algorithms for chunks (#1411) * Adds L4Z encoding. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Adds encoding benchmarks Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Adds snappy encoding. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Adds chunk size test Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Adds snappy v2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve benchmarks Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove chunkenc Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update lz4 to latest master version. Signed-off-by: Peter Štibraný <peter.stibrany@grafana.com> * Use temporary buffer in serialise method to avoid allocations when doing string -> byte conversion. It also makes code little more readable. We pool those buffers for reuse. Signed-off-by: Peter Štibraný <peter.stibrany@grafana.com> * Added gzip -1 for comparison. Signed-off-by: Peter Štibraný <peter.stibrany@grafana.com> * Initialize reader and buffered reader lazily. This helps with reader/buffered reader reuse. Signed-off-by: Peter Štibraný <peter.stibrany@grafana.com> * Don't keep entries, extracted generateData function (mostly to get more understandable profile) Signed-off-by: Peter Štibraný <peter.stibrany@grafana.com> * Improve test and benchmark to cover all encodings. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Adds support for a new chunk format with encoding info. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Ingesters now support encoding config. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add support for no compression. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add docs Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove default Gzip for ByteChunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Removes none, snappyv2 and gzip-1 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Move log test lines to testdata and add supported encoding stringer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * got linted Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>
6 years ago
tailers map[uint32]*tailer
tailerMtx sync.RWMutex
limiter *Limiter
configs *runtime.TenantConfigs
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
wal WAL
// Denotes whether the ingester should flush on shutdown.
// Currently only used by the WAL to signal when the disk is full.
flushOnShutdownSwitch *OnceSwitch
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
metrics *ingesterMetrics
chunkFilter chunk.RequestChunkFilterer
streamRateCalculator *StreamRateCalculator
writeFailures *writefailures.Manager
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
schemaconfig *config.SchemaConfig
}
func newInstance(
cfg *Config,
periodConfigs []config.PeriodConfig,
instanceID string,
limiter *Limiter,
configs *runtime.TenantConfigs,
wal WAL,
metrics *ingesterMetrics,
flushOnShutdownSwitch *OnceSwitch,
chunkFilter chunk.RequestChunkFilterer,
streamRateCalculator *StreamRateCalculator,
writeFailures *writefailures.Manager,
) (*instance, error) {
invertedIndex, err := index.NewMultiInvertedIndex(periodConfigs, uint32(cfg.IndexShards))
if err != nil {
return nil, err
}
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
c := config.SchemaConfig{Configs: periodConfigs}
i := &instance{
cfg: cfg,
streams: newStreamsMap(),
buf: make([]byte, 0, 1024),
index: invertedIndex,
instanceID: instanceID,
streamsCreatedTotal: streamsCreatedTotal.WithLabelValues(instanceID),
streamsRemovedTotal: streamsRemovedTotal.WithLabelValues(instanceID),
Adds configurable compression algorithms for chunks (#1411) * Adds L4Z encoding. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Adds encoding benchmarks Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Adds snappy encoding. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Adds chunk size test Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Adds snappy v2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Improve benchmarks Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove chunkenc Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update lz4 to latest master version. Signed-off-by: Peter Štibraný <peter.stibrany@grafana.com> * Use temporary buffer in serialise method to avoid allocations when doing string -> byte conversion. It also makes code little more readable. We pool those buffers for reuse. Signed-off-by: Peter Štibraný <peter.stibrany@grafana.com> * Added gzip -1 for comparison. Signed-off-by: Peter Štibraný <peter.stibrany@grafana.com> * Initialize reader and buffered reader lazily. This helps with reader/buffered reader reuse. Signed-off-by: Peter Štibraný <peter.stibrany@grafana.com> * Don't keep entries, extracted generateData function (mostly to get more understandable profile) Signed-off-by: Peter Štibraný <peter.stibrany@grafana.com> * Improve test and benchmark to cover all encodings. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Adds support for a new chunk format with encoding info. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Ingesters now support encoding config. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add support for no compression. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add docs Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove default Gzip for ByteChunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Removes none, snappyv2 and gzip-1 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Move log test lines to testdata and add supported encoding stringer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * got linted Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>
6 years ago
tailers: map[uint32]*tailer{},
limiter: limiter,
configs: configs,
wal: wal,
metrics: metrics,
flushOnShutdownSwitch: flushOnShutdownSwitch,
chunkFilter: chunkFilter,
streamRateCalculator: streamRateCalculator,
writeFailures: writeFailures,
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
schemaconfig: &c,
}
i.mapper = newFPMapper(i.getLabelsFromFingerprint)
return i, err
}
// consumeChunk manually adds a chunk that was received during ingester chunk
// transfer.
func (i *instance) consumeChunk(ctx context.Context, ls labels.Labels, chunk *logproto.Chunk) error {
fp := i.getHashForLabels(ls)
s, _, _ := i.streams.LoadOrStoreNewByFP(fp,
func() (*stream, error) {
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
s, err := i.createStreamByFP(ls, fp)
s.chunkMtx.Lock() // Lock before return, because we have defer that unlocks it.
if err != nil {
return nil, err
}
return s, nil
},
func(s *stream) error {
s.chunkMtx.Lock()
return nil
},
)
defer s.chunkMtx.Unlock()
err := s.consumeChunk(ctx, chunk)
if err == nil {
i.metrics.memoryChunks.Inc()
}
return err
}
// Push will iterate over the given streams present in the PushRequest and attempt to store them.
//
// Although multiple streams are part of the PushRequest, the returned error only reflects what
// happened to *the last stream in the request*. Ex: if three streams are part of the PushRequest
// and all three failed, the returned error only describes what happened to the last processed stream.
func (i *instance) Push(ctx context.Context, req *logproto.PushRequest) error {
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
record := recordPool.GetRecord()
record.UserID = i.instanceID
defer recordPool.PutRecord(record)
rateLimitWholeStream := i.limiter.limits.ShardStreams(i.instanceID).Enabled
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
var appendErr error
for _, reqStream := range req.Streams {
s, _, err := i.streams.LoadOrStoreNew(reqStream.Labels,
func() (*stream, error) {
s, err := i.createStream(reqStream, record)
// Lock before adding to maps
if err == nil {
s.chunkMtx.Lock()
}
return s, err
},
func(s *stream) error {
s.chunkMtx.Lock()
return nil
},
)
if err != nil {
appendErr = err
continue
}
_, appendErr = s.Push(ctx, reqStream.Entries, record, 0, false, rateLimitWholeStream)
s.chunkMtx.Unlock()
}
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
if !record.IsEmpty() {
if err := i.wal.Log(record); err != nil {
if e, ok := err.(*os.PathError); ok && e.Err == syscall.ENOSPC {
i.metrics.walDiskFullFailures.Inc()
i.flushOnShutdownSwitch.TriggerAnd(func() {
level.Error(util_log.Logger).Log(
"msg",
"Error writing to WAL, disk full, no further messages will be logged for this error",
)
})
} else {
return err
}
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
}
}
return appendErr
}
func (i *instance) createStream(pushReqStream logproto.Stream, record *wal.Record) (*stream, error) {
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
// record is only nil when replaying WAL. We don't want to drop data when replaying a WAL after
// reducing the stream limits, for instance.
var err error
if record != nil {
err = i.limiter.AssertMaxStreamsPerUser(i.instanceID, i.streams.Len())
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
}
if err != nil {
if i.configs.LogStreamCreation(i.instanceID) {
level.Debug(util_log.Logger).Log(
"msg", "failed to create stream, exceeded limit",
"org_id", i.instanceID,
"err", err,
"stream", pushReqStream.Labels,
)
}
validation.DiscardedSamples.WithLabelValues(validation.StreamLimit, i.instanceID).Add(float64(len(pushReqStream.Entries)))
bytes := 0
for _, e := range pushReqStream.Entries {
bytes += len(e.Line)
}
validation.DiscardedBytes.WithLabelValues(validation.StreamLimit, i.instanceID).Add(float64(bytes))
return nil, httpgrpc.Errorf(http.StatusTooManyRequests, validation.StreamLimitErrorMsg, i.instanceID)
}
labels, err := syntax.ParseLabels(pushReqStream.Labels)
if err != nil {
if i.configs.LogStreamCreation(i.instanceID) {
level.Debug(util_log.Logger).Log(
"msg", "failed to create stream, failed to parse labels",
"org_id", i.instanceID,
"err", err,
"stream", pushReqStream.Labels,
)
}
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
fp := i.getHashForLabels(labels)
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
sortedLabels := i.index.Add(logproto.FromLabelsToLabelAdapters(labels), fp)
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
chunkfmt, headfmt, err := i.chunkFormatAt(minTs(&pushReqStream))
if err != nil {
return nil, fmt.Errorf("failed to create stream: %w", err)
}
s := newStream(chunkfmt, headfmt, i.cfg, i.limiter, i.instanceID, fp, sortedLabels, i.limiter.UnorderedWrites(i.instanceID), i.streamRateCalculator, i.metrics, i.writeFailures)
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
// record will be nil when replaying the wal (we don't want to rewrite wal entries as we replay them).
if record != nil {
record.Series = append(record.Series, tsdb_record.RefSeries{
Ref: chunks.HeadSeriesRef(fp),
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
Labels: sortedLabels,
})
} else {
// If the record is nil, this is a WAL recovery.
i.metrics.recoveredStreamsTotal.Inc()
}
memoryStreams.WithLabelValues(i.instanceID).Inc()
memoryStreamsLabelsBytes.Add(float64(len(s.labels.String())))
i.streamsCreatedTotal.Inc()
i.addTailersToNewStream(s)
streamsCountStats.Add(1)
if i.configs.LogStreamCreation(i.instanceID) {
level.Debug(util_log.Logger).Log(
"msg", "successfully created stream",
"org_id", i.instanceID,
"stream", pushReqStream.Labels,
)
}
return s, nil
}
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
func (i *instance) createStreamByFP(ls labels.Labels, fp model.Fingerprint) (*stream, error) {
sortedLabels := i.index.Add(logproto.FromLabelsToLabelAdapters(ls), fp)
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
chunkfmt, headfmt, err := i.chunkFormatAt(model.Now())
if err != nil {
return nil, fmt.Errorf("failed to create stream for fingerprint: %w", err)
}
s := newStream(chunkfmt, headfmt, i.cfg, i.limiter, i.instanceID, fp, sortedLabels, i.limiter.UnorderedWrites(i.instanceID), i.streamRateCalculator, i.metrics, i.writeFailures)
i.streamsCreatedTotal.Inc()
memoryStreams.WithLabelValues(i.instanceID).Inc()
memoryStreamsLabelsBytes.Add(float64(len(s.labels.String())))
i.addTailersToNewStream(s)
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
return s, nil
}
// chunkFormatAt returns chunk formats to use at given period of time.
func (i *instance) chunkFormatAt(at model.Time) (byte, chunkenc.HeadBlockFmt, error) {
// NOTE: We choose chunk formats for stream based on it's entries timestamp.
// Rationale being, a single (ingester) instance can be running across multiple schema period
// and choosing correct periodConfig during creation of stream is more accurate rather
// than choosing it during starting of instance itself.
periodConfig, err := i.schemaconfig.SchemaForTime(at)
if err != nil {
return 0, 0, err
}
chunkFormat, headblock, err := periodConfig.ChunkFormat()
if err != nil {
return 0, 0, err
}
return chunkFormat, headblock, nil
}
// getOrCreateStream returns the stream or creates it.
// It's safe to use this function if returned stream is not consistency sensitive to streamsMap(e.g. ingesterRecoverer),
// otherwise use streamsMap.LoadOrStoreNew with locking stream's chunkMtx inside.
func (i *instance) getOrCreateStream(pushReqStream logproto.Stream, record *wal.Record) (*stream, error) {
s, _, err := i.streams.LoadOrStoreNew(pushReqStream.Labels, func() (*stream, error) {
return i.createStream(pushReqStream, record)
}, nil)
return s, err
}
// removeStream removes a stream from the instance.
func (i *instance) removeStream(s *stream) {
if i.streams.Delete(s) {
i.index.Delete(s.labels, s.fp)
i.streamsRemovedTotal.Inc()
memoryStreams.WithLabelValues(i.instanceID).Dec()
memoryStreamsLabelsBytes.Sub(float64(len(s.labels.String())))
streamsCountStats.Add(-1)
}
}
func (i *instance) getHashForLabels(ls labels.Labels) model.Fingerprint {
var fp uint64
fp, i.buf = ls.HashWithoutLabels(i.buf, []string(nil)...)
return i.mapper.mapFP(model.Fingerprint(fp), ls)
}
// Return labels associated with given fingerprint. Used by fingerprint mapper.
func (i *instance) getLabelsFromFingerprint(fp model.Fingerprint) labels.Labels {
s, ok := i.streams.LoadByFP(fp)
if !ok {
return nil
}
return s.labels
}
func (i *instance) Query(ctx context.Context, req logql.SelectLogParams) (iter.EntryIterator, error) {
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
expr, err := req.LogSelector()
if err != nil {
return nil, err
}
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
5 years ago
pipeline, err := expr.Pipeline()
if err != nil {
return nil, err
}
pipeline, err = deletion.SetupPipeline(req, pipeline)
if err != nil {
return nil, err
}
stats := stats.FromContext(ctx)
var iters []iter.EntryIterator
shard, err := parseShardFromRequest(req.Shards)
Shard ingester queries. (#3852) * Shard ingester queries. This is still experimental but already yield from 2x to 6x faster for short period queries. I'm still playing with it but I want to share how I do it early. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notice of the code origin. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Align shards from ingester and storage. Utimately we should have a storage that relies on fingerprint, but that's harder to change. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove comment Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes delete index func Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test reverting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug causing non deterministic hash. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Fixes build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * got linted :( Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
5 years ago
if err != nil {
return nil, err
}
err = i.forMatchingStreams(
ctx,
req.Start,
expr.Matchers(),
Shard ingester queries. (#3852) * Shard ingester queries. This is still experimental but already yield from 2x to 6x faster for short period queries. I'm still playing with it but I want to share how I do it early. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notice of the code origin. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Align shards from ingester and storage. Utimately we should have a storage that relies on fingerprint, but that's harder to change. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove comment Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes delete index func Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test reverting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug causing non deterministic hash. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Fixes build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * got linted :( Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
5 years ago
shard,
func(stream *stream) error {
iter, err := stream.Iterator(ctx, stats, req.Start, req.End, req.Direction, pipeline.ForStream(stream.labels))
if err != nil {
return err
}
iters = append(iters, iter)
return nil
},
)
if err != nil {
return nil, err
}
return iter.NewSortEntryIterator(iters, req.Direction), nil
}
func (i *instance) QuerySample(ctx context.Context, req logql.SelectSampleParams) (iter.SampleIterator, error) {
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
expr, err := req.Expr()
if err != nil {
return nil, err
}
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
extractor, err := expr.Extractor()
if err != nil {
return nil, err
}
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
5 years ago
extractor, err = deletion.SetupExtractor(req, extractor)
if err != nil {
return nil, err
}
stats := stats.FromContext(ctx)
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
var iters []iter.SampleIterator
Shard ingester queries. (#3852) * Shard ingester queries. This is still experimental but already yield from 2x to 6x faster for short period queries. I'm still playing with it but I want to share how I do it early. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notice of the code origin. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Align shards from ingester and storage. Utimately we should have a storage that relies on fingerprint, but that's harder to change. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove comment Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes delete index func Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test reverting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug causing non deterministic hash. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Fixes build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * got linted :( Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
5 years ago
var shard *astmapper.ShardAnnotation
shards, err := logql.ParseShards(req.Shards)
if err != nil {
return nil, err
}
if len(shards) > 1 {
return nil, errors.New("only one shard per ingester query is supported")
}
if len(shards) == 1 {
shard = &shards[0]
}
selector, err := expr.Selector()
if err != nil {
return nil, err
}
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
err = i.forMatchingStreams(
ctx,
req.Start,
selector.Matchers(),
Shard ingester queries. (#3852) * Shard ingester queries. This is still experimental but already yield from 2x to 6x faster for short period queries. I'm still playing with it but I want to share how I do it early. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notice of the code origin. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Align shards from ingester and storage. Utimately we should have a storage that relies on fingerprint, but that's harder to change. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove comment Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes delete index func Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test reverting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug causing non deterministic hash. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Fixes build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * got linted :( Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
5 years ago
shard,
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
func(stream *stream) error {
iter, err := stream.SampleIterator(ctx, stats, req.Start, req.End, extractor.ForStream(stream.labels))
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
if err != nil {
return err
}
iters = append(iters, iter)
return nil
},
)
if err != nil {
return nil, err
}
return iter.NewSortSampleIterator(iters), nil
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
}
// Label returns the label names or values depending on the given request
// Without label matchers the label names and values are retrieved from the index directly.
// If label matchers are given only the matching streams are fetched from the index.
// The label names or values are then retrieved from those matching streams.
func (i *instance) Label(ctx context.Context, req *logproto.LabelRequest, matchers ...*labels.Matcher) (*logproto.LabelResponse, error) {
if len(matchers) == 0 {
var labels []string
if req.Values {
values, err := i.index.LabelValues(*req.Start, req.Name, nil)
if err != nil {
return nil, err
}
labels = make([]string, len(values))
copy(labels, values)
return &logproto.LabelResponse{
Values: labels,
}, nil
}
names, err := i.index.LabelNames(*req.Start, nil)
Shard ingester queries. (#3852) * Shard ingester queries. This is still experimental but already yield from 2x to 6x faster for short period queries. I'm still playing with it but I want to share how I do it early. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notice of the code origin. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Align shards from ingester and storage. Utimately we should have a storage that relies on fingerprint, but that's harder to change. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove comment Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes delete index func Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test reverting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug causing non deterministic hash. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Fixes build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * got linted :( Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
5 years ago
if err != nil {
return nil, err
}
labels = make([]string, len(names))
copy(labels, names)
return &logproto.LabelResponse{
Values: labels,
}, nil
}
labels := util.NewUniqueStrings(0)
err := i.forMatchingStreams(ctx, *req.Start, matchers, nil, func(s *stream) error {
for _, label := range s.labels {
if req.Values && label.Name == req.Name {
labels.Add(label.Value)
continue
}
if !req.Values {
labels.Add(label.Name)
}
}
return nil
})
if err != nil {
return nil, err
}
return &logproto.LabelResponse{
Values: labels.Strings(),
}, nil
}
func (i *instance) Series(ctx context.Context, req *logproto.SeriesRequest) (*logproto.SeriesResponse, error) {
groups, err := logql.MatchForSeriesRequest(req.GetGroups())
if err != nil {
return nil, err
}
shard, err := parseShardFromRequest(req.Shards)
if err != nil {
return nil, err
}
var series []logproto.SeriesIdentifier
// If no matchers were supplied we include all streams.
if len(groups) == 0 {
series = make([]logproto.SeriesIdentifier, 0, i.streams.Len())
err = i.forMatchingStreams(ctx, req.Start, nil, shard, func(stream *stream) error {
// consider the stream only if it overlaps the request time range
if shouldConsiderStream(stream, req.Start, req.End) {
series = append(series, logproto.SeriesIdentifier{
Labels: stream.labels.Map(),
})
}
return nil
})
if err != nil {
return nil, err
}
} else {
dedupedSeries := make(map[uint64]logproto.SeriesIdentifier)
for _, matchers := range groups {
err = i.forMatchingStreams(ctx, req.Start, matchers, shard, func(stream *stream) error {
// consider the stream only if it overlaps the request time range
if shouldConsiderStream(stream, req.Start, req.End) {
// exit early when this stream was added by an earlier group
key := stream.labelHash
Use series hash to identify uniques in merge. (#9985) **What this PR does / why we need it**: A run of the protobuf encoding for the querier showed a spike in memory usage in the `codec.MergeResponse` method. The routine was allocating a lot of memory for the `series.String()` method which is then hashed. Instead, we can use a `uint64` hash. ``` › go test -v -run=^$ -bench "Benchmark_MergeResponses$" -count=10 ./pkg/querier/queryrange ... │ before.txt │ after.txt │ │ sec/op │ sec/op vs base │ _MergeResponses-16 4.563 ± 1% 1.197 ± 1% -73.77% (p=0.000 n=10) │ before.txt │ after.txt │ │ B/op │ B/op vs base │ _MergeResponses-16 6578.715Mi ± 0% 2.362Mi ± 0% -99.96% (p=0.000 n=10) │ before.txt │ after.txt │ │ allocs/op │ allocs/op vs base │ _MergeResponses-16 40500.1k ± 0% 100.1k ± 0% -99.75% (p=0.000 n=10) ``` **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/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)
2 years ago
// TODO(karsten): Due to key collision this check is not
// enough. Ideally there is a comparison between the
// potentail duplicated series.
if _, found := dedupedSeries[key]; found {
return nil
}
dedupedSeries[key] = logproto.SeriesIdentifier{
Labels: stream.labels.Map(),
}
}
return nil
})
if err != nil {
return nil, err
}
}
series = make([]logproto.SeriesIdentifier, 0, len(dedupedSeries))
for _, v := range dedupedSeries {
series = append(series, v)
}
}
return &logproto.SeriesResponse{Series: series}, nil
}
func (i *instance) GetStats(ctx context.Context, req *logproto.IndexStatsRequest) (*logproto.IndexStatsResponse, error) {
matchers, err := syntax.ParseMatchers(req.Matchers, true)
if err != nil {
return nil, err
}
res := &logproto.IndexStatsResponse{}
from, through := req.From.Time(), req.Through.Time()
if err = i.forMatchingStreams(ctx, from, matchers, nil, func(s *stream) error {
// Consider streams which overlap our time range
if shouldConsiderStream(s, from, through) {
s.chunkMtx.RLock()
var hasChunkOverlap bool
for _, chk := range s.chunks {
// Consider chunks which overlap our time range
// and haven't been flushed.
// Flushed chunks will already be counted
// by the TSDB manager+shipper
chkFrom, chkThrough := chk.chunk.Bounds()
if chk.flushed.IsZero() && from.Before(chkThrough) && through.After(chkFrom) {
hasChunkOverlap = true
res.Chunks++
factor := util.GetFactorOfTime(from.UnixNano(), through.UnixNano(), chkFrom.UnixNano(), chkThrough.UnixNano())
res.Entries += uint64(factor * float64(chk.chunk.Size()))
res.Bytes += uint64(factor * float64(chk.chunk.UncompressedSize()))
}
}
if hasChunkOverlap {
res.Streams++
}
s.chunkMtx.RUnlock()
}
return nil
}); err != nil {
return nil, err
}
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogKV(
"function", "instance.GetStats",
"from", from,
"through", through,
"matchers", syntax.MatchersString(matchers),
"streams", res.Streams,
"chunks", res.Chunks,
"bytes", res.Bytes,
"entries", res.Entries,
)
}
return res, nil
}
func (i *instance) GetVolume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error) {
matchers, err := syntax.ParseMatchers(req.Matchers, true)
if err != nil && req.Matchers != seriesvolume.MatchAny {
return nil, err
}
targetLabels := req.TargetLabels
labelsToMatch, matchers, matchAny := util.PrepareLabelsAndMatchers(targetLabels, matchers)
matchAny = matchAny || len(matchers) == 0
seriesNames := make(map[uint64]string)
seriesLabels := labels.Labels(make([]labels.Label, 0, len(labelsToMatch)))
from, through := req.From.Time(), req.Through.Time()
volumes := make(map[string]uint64)
aggregateBySeries := seriesvolume.AggregateBySeries(req.AggregateBy) || req.AggregateBy == ""
if err = i.forMatchingStreams(ctx, from, matchers, nil, func(s *stream) error {
// Consider streams which overlap our time range
if shouldConsiderStream(s, from, through) {
s.chunkMtx.RLock()
var size uint64
for _, chk := range s.chunks {
// Consider chunks which overlap our time range
// and haven't been flushed.
// Flushed chunks will already be counted
// by the TSDB manager+shipper
chkFrom, chkThrough := chk.chunk.Bounds()
if chk.flushed.IsZero() && from.Before(chkThrough) && through.After(chkFrom) {
factor := util.GetFactorOfTime(from.UnixNano(), through.UnixNano(), chkFrom.UnixNano(), chkThrough.UnixNano())
size += uint64(float64(chk.chunk.UncompressedSize()) * factor)
}
}
var labelVolumes map[string]uint64
if aggregateBySeries {
seriesLabels = seriesLabels[:0]
for _, l := range s.labels {
if _, ok := labelsToMatch[l.Name]; matchAny || ok {
seriesLabels = append(seriesLabels, l)
}
}
} else {
labelVolumes = make(map[string]uint64, len(s.labels))
for _, l := range s.labels {
if len(targetLabels) > 0 {
if _, ok := labelsToMatch[l.Name]; matchAny || ok {
labelVolumes[l.Name] += size
}
} else {
labelVolumes[l.Name] += size
}
}
}
// If the labels are < 1k, this does not alloc
// https://github.com/prometheus/prometheus/pull/8025
hash := seriesLabels.Hash()
if _, ok := seriesNames[hash]; !ok {
seriesNames[hash] = seriesLabels.String()
}
if aggregateBySeries {
volumes[seriesNames[hash]] += size
} else {
for k, v := range labelVolumes {
volumes[k] += v
}
}
s.chunkMtx.RUnlock()
}
return nil
}); err != nil {
return nil, err
}
res := seriesvolume.MapToVolumeResponse(volumes, int(req.Limit))
return res, nil
}
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
func (i *instance) numStreams() int {
return i.streams.Len()
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
}
// forAllStreams will execute a function for all streams in the instance.
// It uses a function in order to enable generic stream access without accidentally leaking streams under the mutex.
func (i *instance) forAllStreams(ctx context.Context, fn func(*stream) error) error {
var chunkFilter chunk.Filterer
if i.chunkFilter != nil {
chunkFilter = i.chunkFilter.ForRequest(ctx)
}
err := i.streams.ForEach(func(s *stream) (bool, error) {
if chunkFilter != nil && chunkFilter.ShouldFilter(s.labels) {
return true, nil
}
err := fn(s)
if err != nil {
return false, err
}
return true, nil
})
if err != nil {
return err
}
return nil
}
// forMatchingStreams will execute a function for each stream that satisfies a set of requirements (time range, matchers, etc).
// It uses a function in order to enable generic stream access without accidentally leaking streams under the mutex.
func (i *instance) forMatchingStreams(
ctx context.Context,
// ts denotes the beginning of the request
// and is used to select the correct inverted index
ts time.Time,
matchers []*labels.Matcher,
Shard ingester queries. (#3852) * Shard ingester queries. This is still experimental but already yield from 2x to 6x faster for short period queries. I'm still playing with it but I want to share how I do it early. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notice of the code origin. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Align shards from ingester and storage. Utimately we should have a storage that relies on fingerprint, but that's harder to change. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove comment Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes delete index func Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test reverting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug causing non deterministic hash. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Fixes build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * got linted :( Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
5 years ago
shards *astmapper.ShardAnnotation,
fn func(*stream) error,
) error {
filters, matchers := util.SplitFiltersAndMatchers(matchers)
ids, err := i.index.Lookup(ts, matchers, shards)
Shard ingester queries. (#3852) * Shard ingester queries. This is still experimental but already yield from 2x to 6x faster for short period queries. I'm still playing with it but I want to share how I do it early. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notice of the code origin. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Align shards from ingester and storage. Utimately we should have a storage that relies on fingerprint, but that's harder to change. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Remove comment Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes delete index func Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test reverting. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug causing non deterministic hash. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Update pkg/ingester/index/index_test.go Co-authored-by: Owen Diehl <ow.diehl@gmail.com> * Fixes build. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * got linted :( Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Owen Diehl <ow.diehl@gmail.com>
5 years ago
if err != nil {
return err
}
var chunkFilter chunk.Filterer
if i.chunkFilter != nil {
chunkFilter = i.chunkFilter.ForRequest(ctx)
}
outer:
for _, streamID := range ids {
stream, ok := i.streams.LoadByFP(streamID)
if !ok {
// If a stream is missing here, it has already been flushed
// and is supposed to be picked up from storage by querier
continue
}
for _, filter := range filters {
if !filter.Matches(stream.labels.Get(filter.Name)) {
continue outer
}
}
if chunkFilter != nil && chunkFilter.ShouldFilter(stream.labels) {
continue
}
err := fn(stream)
if err != nil {
return err
}
}
return nil
}
func (i *instance) addNewTailer(ctx context.Context, t *tailer) error {
if err := i.forMatchingStreams(ctx, time.Now(), t.matchers, nil, func(s *stream) error {
s.addTailer(t)
return nil
}); err != nil {
return err
}
i.tailerMtx.Lock()
defer i.tailerMtx.Unlock()
i.tailers[t.getID()] = t
return nil
}
func (i *instance) addTailersToNewStream(stream *stream) {
i.tailerMtx.RLock()
defer i.tailerMtx.RUnlock()
for _, t := range i.tailers {
// we don't want to watch streams for closed tailers.
// When a new tail request comes in we will clean references to closed tailers
if t.isClosed() {
continue
}
var chunkFilter chunk.Filterer
if i.chunkFilter != nil {
chunkFilter = i.chunkFilter.ForRequest(t.conn.Context())
}
if isMatching(stream.labels, t.matchers) {
if chunkFilter != nil && chunkFilter.ShouldFilter(stream.labels) {
continue
}
stream.addTailer(t)
}
}
}
func (i *instance) checkClosedTailers() {
closedTailers := []uint32{}
i.tailerMtx.RLock()
for _, t := range i.tailers {
if t.isClosed() {
closedTailers = append(closedTailers, t.getID())
continue
}
}
i.tailerMtx.RUnlock()
if len(closedTailers) != 0 {
i.tailerMtx.Lock()
defer i.tailerMtx.Unlock()
for _, closedTailer := range closedTailers {
delete(i.tailers, closedTailer)
}
}
}
func (i *instance) closeTailers() {
i.tailerMtx.Lock()
defer i.tailerMtx.Unlock()
for _, t := range i.tailers {
t.close()
}
}
func (i *instance) openTailersCount() uint32 {
i.checkClosedTailers()
i.tailerMtx.RLock()
defer i.tailerMtx.RUnlock()
return uint32(len(i.tailers))
}
func parseShardFromRequest(reqShards []string) (*astmapper.ShardAnnotation, error) {
var shard *astmapper.ShardAnnotation
shards, err := logql.ParseShards(reqShards)
if err != nil {
return nil, err
}
if len(shards) > 1 {
return nil, errors.New("only one shard per ingester query is supported")
}
if len(shards) == 1 {
shard = &shards[0]
}
return shard, nil
}
func isDone(ctx context.Context) bool {
return ctx.Err() != nil
}
// QuerierQueryServer is the GRPC server stream we use to send batch of entries.
type QuerierQueryServer interface {
Context() context.Context
Send(res *logproto.QueryResponse) error
}
func sendBatches(ctx context.Context, i iter.EntryIterator, queryServer QuerierQueryServer, limit int32) error {
stats := stats.FromContext(ctx)
// send until the limit is reached.
for limit != 0 && !isDone(ctx) {
fetchSize := uint32(queryBatchSize)
if limit > 0 {
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
fetchSize = mathutil.MinUint32(queryBatchSize, uint32(limit))
}
batch, batchSize, err := iter.ReadBatch(i, fetchSize)
if err != nil {
return err
}
if limit > 0 {
limit -= int32(batchSize)
}
stats.AddIngesterBatch(int64(batchSize))
batch.Stats = stats.Ingester()
if isDone(ctx) {
break
}
if err := queryServer.Send(batch); err != nil && err != context.Canceled {
return err
}
// We check this after sending an empty batch to make sure stats are sent
if len(batch.Streams) == 0 {
return nil
}
stats.Reset()
}
return nil
}
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
func sendSampleBatches(ctx context.Context, it iter.SampleIterator, queryServer logproto.Querier_QuerySampleServer) error {
sp := opentracing.SpanFromContext(ctx)
stats := stats.FromContext(ctx)
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
for !isDone(ctx) {
batch, size, err := iter.ReadSampleBatch(it, queryBatchSampleSize)
if err != nil {
return err
}
stats.AddIngesterBatch(int64(size))
batch.Stats = stats.Ingester()
if isDone(ctx) {
break
}
if err := queryServer.Send(batch); err != nil && err != context.Canceled {
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
return err
}
// We check this after sending an empty batch to make sure stats are sent
if len(batch.Series) == 0 {
return nil
}
stats.Reset()
if sp != nil {
sp.LogKV("event", "sent batch", "size", size)
}
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
}
Improve metric queries by computing samples at the edges. (#2293) * First pass breaking the code appart. Wondering how we're going to achieve fast mutation of labels. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. I realize I need hash for deduping lines. going to benchmark somes. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Tested some hash and decided which one to use. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Wip Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Starting working on ingester. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Trying to find a better hash function. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More hash testing we have a winner. xxhash it is. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Settle on xxhash Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Better params interfacing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add interface for queryparams for things that exist in both type of params. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add storage sample iterator implementations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing tests and verifying we don't get collions for the hashing method. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing ingesters tests and refactoring utility function/tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing and testing that stats are still well computed. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixing more tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * More engine tests finished. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes sharding evaluator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes more engine tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix error tests in the engine. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish fixing all tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a bug where extractor was not passed in correctly. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add notes about upgrade. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Renamed and fix a bug. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add memchunk tests and starting test for sampleIterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Test heap sample iterator. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * working on test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finishing testing all new iterators. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Making sure all store functions are tested. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Benchmark and verify everything is working well. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Make the linter happy. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * use xxhash v2. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix a flaky test because of map. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * go.mod. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Edward Welch <edward.welch@grafana.com>
6 years ago
return nil
}
func shouldConsiderStream(stream *stream, reqFrom, reqThrough time.Time) bool {
Introduces per stream chunks mutex (#3000) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * adds chunkMtx to ingester streams * all flush locking locks streams -> chunks in order to prevent deadlocks * wal integration tests * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * cleans up merge * lint * inline functions to simplify unlocking
5 years ago
from, to := stream.Bounds()
if reqThrough.UnixNano() > from.UnixNano() && reqFrom.UnixNano() <= to.UnixNano() {
return true
}
return false
}
// OnceSwitch is an optimized switch that can only ever be switched "on" in a concurrent environment.
type OnceSwitch struct {
triggered atomic.Bool
}
func (o *OnceSwitch) Get() bool {
return o.triggered.Load()
}
func (o *OnceSwitch) Trigger() {
o.TriggerAnd(nil)
}
// TriggerAnd will ensure the switch is on and run the provided function if
// the switch was not already toggled on.
func (o *OnceSwitch) TriggerAnd(fn func()) {
triggeredPrior := o.triggered.Swap(true)
if !triggeredPrior && fn != nil {
fn()
}
}
Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
2 years ago
// minTs is a helper to return minimum Unix timestamp (as `model.Time`)
// across all the entries in a given `stream`.
func minTs(stream *logproto.Stream) model.Time {
// NOTE: We choose `min` timestamp because, the chunk is written once then
// added to the index buckets for may be different days. It would better rather to have
// some latest(say v13) indices reference older (say v12) compatible chunks than vice versa.
streamMinTs := int64(math.MaxInt64)
for _, entry := range stream.Entries {
ts := entry.Timestamp.UnixNano()
if streamMinTs > ts {
streamMinTs = ts
}
}
return model.TimeFromUnixNano(streamMinTs)
}