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

627 lines
15 KiB

package ingester
import (
"context"
"net/http"
"os"
"sync"
"syscall"
"github.com/go-kit/kit/log/level"
"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/pkg/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
tsdb_record "github.com/prometheus/prometheus/tsdb/record"
"github.com/weaveworks/common/httpgrpc"
"go.uber.org/atomic"
"github.com/cortexproject/cortex/pkg/ingester/client"
"github.com/cortexproject/cortex/pkg/ingester/index"
"github.com/cortexproject/cortex/pkg/util"
cutil "github.com/cortexproject/cortex/pkg/util"
"github.com/grafana/loki/pkg/helpers"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/stats"
"github.com/grafana/loki/pkg/util/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 (
queryBatchSize = 128
queryBatchSampleSize = 512
)
// Errors returned on Query.
var (
ErrStreamMissing = errors.New("Stream missing")
)
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"})
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"})
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
streamsMtx sync.RWMutex
buf []byte // buffer used to compute fps.
streams map[string]*stream
streamsByFP map[model.Fingerprint]*stream
index *index.InvertedIndex
mapper *fpMapper // using of mapper needs streamsMtx because it calls back
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
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
}
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 newInstance(
cfg *Config,
instanceID string,
limiter *Limiter,
wal WAL,
metrics *ingesterMetrics,
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
) *instance {
i := &instance{
cfg: cfg,
streams: map[string]*stream{},
streamsByFP: map[model.Fingerprint]*stream{},
buf: make([]byte, 0, 1024),
index: index.New(),
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,
wal: wal,
metrics: metrics,
flushOnShutdownSwitch: flushOnShutdownSwitch,
}
i.mapper = newFPMapper(i.getLabelsFromFingerprint)
return i
}
// 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 {
i.streamsMtx.Lock()
defer i.streamsMtx.Unlock()
fp := i.getHashForLabels(ls)
stream, ok := i.streamsByFP[fp]
if !ok {
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(client.FromLabelsToLabelAdapters(ls), 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
stream = newStream(i.cfg, fp, sortedLabels, i.metrics)
i.streamsByFP[fp] = stream
i.streams[stream.labelsString] = stream
i.streamsCreatedTotal.Inc()
memoryStreams.WithLabelValues(i.instanceID).Inc()
i.addTailersToNewStream(stream)
}
err := stream.consumeChunk(ctx, chunk)
if err == nil {
memoryChunks.Inc()
}
return err
}
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)
i.streamsMtx.Lock()
defer i.streamsMtx.Unlock()
var appendErr error
for _, s := range req.Streams {
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
stream, err := i.getOrCreateStream(s, false, record)
if err != nil {
appendErr = err
continue
}
if _, err := stream.Push(ctx, s.Entries, record); err != nil {
appendErr = err
continue
}
}
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.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
}
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
}
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
// getOrCreateStream returns the stream or creates it. Must hold streams mutex if not asked to lock.
func (i *instance) getOrCreateStream(pushReqStream logproto.Stream, lock bool, record *WALRecord) (*stream, error) {
if lock {
i.streamsMtx.Lock()
defer i.streamsMtx.Unlock()
}
stream, ok := i.streams[pushReqStream.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
if ok {
return stream, 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
// 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, len(i.streams))
}
if err != nil {
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())
}
labels, err := logql.ParseLabels(pushReqStream.Labels)
if err != nil {
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(client.FromLabelsToLabelAdapters(labels), fp)
stream = newStream(i.cfg, fp, sortedLabels, i.metrics)
i.streams[pushReqStream.Labels] = stream
i.streamsByFP[fp] = stream
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: uint64(fp),
Labels: sortedLabels,
})
} else {
// If the record is nil, this is a WAL recovery.
i.metrics.recoveredStreamsTotal.Inc()
}
memoryStreams.WithLabelValues(i.instanceID).Inc()
i.streamsCreatedTotal.Inc()
i.addTailersToNewStream(stream)
return stream, nil
}
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. Must hold streamsMtx.
func (i *instance) getLabelsFromFingerprint(fp model.Fingerprint) labels.Labels {
s := i.streamsByFP[fp]
if s == nil {
return nil
}
return s.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
func (i *instance) Query(ctx context.Context, req logql.SelectLogParams) ([]iter.EntryIterator, error) {
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
}
ingStats := stats.GetIngesterData(ctx)
var iters []iter.EntryIterator
err = i.forMatchingStreams(
expr.Matchers(),
func(stream *stream) error {
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
iter, err := stream.Iterator(ctx, ingStats, 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 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
func (i *instance) QuerySample(ctx context.Context, req logql.SelectSampleParams) ([]iter.SampleIterator, error) {
expr, err := req.Expr()
if err != nil {
return nil, err
}
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
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
ingStats := stats.GetIngesterData(ctx)
var iters []iter.SampleIterator
err = i.forMatchingStreams(
expr.Selector().Matchers(),
func(stream *stream) error {
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
iter, err := stream.SampleIterator(ctx, ingStats, 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 iters, nil
}
func (i *instance) Label(_ context.Context, req *logproto.LabelRequest) (*logproto.LabelResponse, error) {
var labels []string
if req.Values {
values := i.index.LabelValues(req.Name)
labels = make([]string, len(values))
for i := 0; i < len(values); i++ {
labels[i] = values[i]
}
} else {
names := i.index.LabelNames()
labels = make([]string, len(names))
for i := 0; i < len(names); i++ {
labels[i] = names[i]
}
}
return &logproto.LabelResponse{
Values: labels,
}, nil
}
func (i *instance) Series(_ context.Context, req *logproto.SeriesRequest) (*logproto.SeriesResponse, error) {
groups, err := loghttp.Match(req.GetGroups())
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, len(i.streams))
err = i.forAllStreams(func(stream *stream) error {
// consider the stream only if it overlaps the request time range
if shouldConsiderStream(stream, req) {
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(matchers, func(stream *stream) error {
// consider the stream only if it overlaps the request time range
if shouldConsiderStream(stream, req) {
// exit early when this stream was added by an earlier group
key := stream.labels.Hash()
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
}
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 {
i.streamsMtx.RLock()
defer i.streamsMtx.RUnlock()
return len(i.streams)
}
// 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(fn func(*stream) error) error {
i.streamsMtx.RLock()
defer i.streamsMtx.RUnlock()
for _, stream := range i.streams {
err := fn(stream)
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(
matchers []*labels.Matcher,
fn func(*stream) error,
) error {
i.streamsMtx.RLock()
defer i.streamsMtx.RUnlock()
filters, matchers := cutil.SplitFiltersAndMatchers(matchers)
ids := i.index.Lookup(matchers)
outer:
for _, streamID := range ids {
stream, ok := i.streamsByFP[streamID]
if !ok {
return ErrStreamMissing
}
for _, filter := range filters {
if !filter.Matches(stream.labels.Get(filter.Name)) {
continue outer
}
}
err := fn(stream)
if err != nil {
return err
}
}
return nil
}
func (i *instance) addNewTailer(t *tailer) error {
if err := i.forMatchingStreams(t.matchers, 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
}
if isMatching(stream.labels, t.matchers) {
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 isDone(ctx context.Context) bool {
select {
case <-ctx.Done():
return true
default:
return false
}
}
func sendBatches(ctx context.Context, i iter.EntryIterator, queryServer logproto.Querier_QueryServer, limit uint32) error {
ingStats := stats.GetIngesterData(ctx)
fluent-bit shared object go plugin (#847) * Import fluent-bit-go-loki plugin sources * Add fluent-bit-go package constraint * Ensure dependencies with dep ``` $ dep ensure ``` * Update .gitignore * Move fluent-bit-go-loki to fluent-bit/fluent-bit-go-loki directory * Add documentation for fluent-bit-go-loki * Fix for lint * Fix for goimports errors * Display fluent-bit-go-loki plugin version * Use more descriptive description for fluent-bit Loki plugin * Remove needless newlines * Rectify fluent-bit-go-loki makefile Remove needless parts and rename meaningless target * Use more descriptive description in README * refactor(fluent-bit): move make targets into root Makefile * Organize imports * Reorder imports with goimports $ goimports -w out_loki.go * Remove needless LICENSE file Because it is the same for Loki repository. * Rely on Loki client retry mechanism * Make descriptions more clearly * Change default unit * BatchWait: msec -> sec * BatchSize: KiB -> Byte * Use logql to parse Loki native labels format * Use logger instead of fmt.Printf * Update documentation to reflect recent changes * Use prometheus version module to print version * Dump error message with logger * Use DYN_GO_FLAGS instead of custom FLAGS for fluent-bit-plugin * Support RemoveKeys parameter * use logql.ParseMatchers() * refactor fluent-bit go plugin * add ci step for docker image * fix lint issues * fix a failing url test * fix a failing url test * add build images for drone ci * Adding labelmap file to complex record to labels Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add helm chart and fix a bug with labelmap * New push path * dep sync * Move fluent-bit folder and use jsonnet for drone * fix build * Adds more documentation and include a basic config in the container Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * remove binaries * Some readme grammar fixes * docs: really nitpicky grammar change in clients README * Add helm documentation * fixes bad config * bump chart * fixes .gitignore
6 years ago
if limit == 0 {
// send all batches.
for !isDone(ctx) {
batch, size, err := iter.ReadBatch(i, queryBatchSize)
if err != nil {
return err
}
if len(batch.Streams) == 0 {
return nil
}
if err := queryServer.Send(batch); err != nil {
return err
}
ingStats.TotalLinesSent += int64(size)
ingStats.TotalBatches++
}
return nil
}
// send until the limit is reached.
sent := uint32(0)
for sent < limit && !isDone(queryServer.Context()) {
batch, batchSize, err := iter.ReadBatch(i, helpers.MinUint32(queryBatchSize, limit-sent))
if err != nil {
return err
}
sent += batchSize
if len(batch.Streams) == 0 {
return nil
}
if err := queryServer.Send(batch); err != nil {
return err
}
ingStats.TotalLinesSent += int64(batchSize)
ingStats.TotalBatches++
}
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 {
ingStats := stats.GetIngesterData(ctx)
for !isDone(ctx) {
batch, size, err := iter.ReadSampleBatch(it, queryBatchSampleSize)
if err != nil {
return err
}
if len(batch.Series) == 0 {
return nil
}
if err := queryServer.Send(batch); err != nil {
return err
}
ingStats.TotalLinesSent += int64(size)
ingStats.TotalBatches++
}
return nil
}
func shouldConsiderStream(stream *stream, req *logproto.SeriesRequest) 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()
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
if req.End.UnixNano() > from.UnixNano() && req.Start.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()
}
}