feat: Introduce a new Object Storage WAL format. (#13253)

pull/13264/head^2
Cyril Tovena 11 months ago committed by GitHub
parent 467eb1bb1b
commit 1d6f8d51fc
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 9
      pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go
  2. 16
      pkg/storage/stores/shipper/indexshipper/tsdb/index/postings.go
  3. 4
      pkg/storage/stores/shipper/indexshipper/tsdb/index/postings_test.go
  4. 1
      pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go
  5. 183
      pkg/storage/wal/README.md
  6. 334
      pkg/storage/wal/chunks/chunks.go
  7. 345
      pkg/storage/wal/chunks/chunks_test.go
  8. 20
      pkg/storage/wal/chunks/meta.go
  9. 91
      pkg/storage/wal/index/buffer.go
  10. 1898
      pkg/storage/wal/index/index.go
  11. 624
      pkg/storage/wal/index/index_test.go
  12. 334
      pkg/storage/wal/segment.go
  13. 188
      pkg/storage/wal/segment_test.go
  14. 20000
      pkg/storage/wal/testdata/20kseries.json
  15. 1000
      pkg/storage/wal/testdata/agent-logfmt.txt
  16. 1000
      pkg/storage/wal/testdata/calico.txt
  17. 5000
      pkg/storage/wal/testdata/distributor-logfmt.txt
  18. 210
      pkg/storage/wal/testdata/drone-json.txt
  19. 100
      pkg/storage/wal/testdata/ingester-logfmt.txt
  20. 1000
      pkg/storage/wal/testdata/journald.txt
  21. 1000
      pkg/storage/wal/testdata/kafka.txt
  22. 1000
      pkg/storage/wal/testdata/kubernetes.txt
  23. 1000
      pkg/storage/wal/testdata/vault.txt

@ -176,7 +176,6 @@ func (m *Metadata) EnsureBounds(from, through int64) {
if m.Through == 0 || through > m.Through {
m.Through = through
}
}
// NewTOCFromByteSlice return parsed TOC from given index byte slice.
@ -1646,7 +1645,6 @@ func readFingerprintOffsetsTable(bs ByteSlice, off uint64) (FingerprintOffsets,
}
return res, d.Err()
}
// Close the reader and its underlying resources.
@ -2074,7 +2072,7 @@ func (dec *Decoder) Postings(b []byte) (int, Postings, error) {
if len(l) != 4*n {
return 0, nil, fmt.Errorf("unexpected postings length, should be %d bytes for %d postings, got %d bytes", 4*n, n, len(l))
}
return n, newBigEndianPostings(l), nil
return n, NewBigEndianPostings(l), nil
}
// LabelNamesOffsetsFor decodes the offsets of the name symbols for a given series.
@ -2335,7 +2333,6 @@ func (dec *Decoder) readChunkStatsV3(d *encoding.Decbuf, from, through int64) (r
}
return res, d.Err()
}
func (dec *Decoder) accumulateChunkStats(d *encoding.Decbuf, nChunks int, from, through int64) (res ChunkStats, err error) {
@ -2372,16 +2369,13 @@ func (dec *Decoder) readChunkStatsPriorV3(d *encoding.Decbuf, seriesRef storage.
} else if chk.MinTime >= through {
break
}
}
return res, nil
}
// Series decodes a series entry from the given byte slice into lset and chks.
func (dec *Decoder) Series(version int, b []byte, seriesRef storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta) (uint64, error) {
d, fprint, err := dec.prepSeries(b, lbls, chks)
if err != nil {
return 0, err
@ -2392,7 +2386,6 @@ func (dec *Decoder) Series(version int, b []byte, seriesRef storage.SeriesRef, f
return 0, errors.Wrapf(err, "series %s", lbls.String())
}
return fprint, nil
}
func (dec *Decoder) readChunks(version int, d *encoding.Decbuf, seriesRef storage.SeriesRef, from int64, through int64, chks *[]ChunkMeta) error {

@ -777,22 +777,22 @@ func (it *ListPostings) Err() error {
return nil
}
// bigEndianPostings implements the Postings interface over a byte stream of
// BigEndianPostings implements the Postings interface over a byte stream of
// big endian numbers.
type bigEndianPostings struct {
type BigEndianPostings struct {
list []byte
cur uint32
}
func newBigEndianPostings(list []byte) *bigEndianPostings {
return &bigEndianPostings{list: list}
func NewBigEndianPostings(list []byte) *BigEndianPostings {
return &BigEndianPostings{list: list}
}
func (it *bigEndianPostings) At() storage.SeriesRef {
func (it *BigEndianPostings) At() storage.SeriesRef {
return storage.SeriesRef(it.cur)
}
func (it *bigEndianPostings) Next() bool {
func (it *BigEndianPostings) Next() bool {
if len(it.list) >= 4 {
it.cur = binary.BigEndian.Uint32(it.list)
it.list = it.list[4:]
@ -801,7 +801,7 @@ func (it *bigEndianPostings) Next() bool {
return false
}
func (it *bigEndianPostings) Seek(x storage.SeriesRef) bool {
func (it *BigEndianPostings) Seek(x storage.SeriesRef) bool {
if storage.SeriesRef(it.cur) >= x {
return true
}
@ -821,7 +821,7 @@ func (it *bigEndianPostings) Seek(x storage.SeriesRef) bool {
return false
}
func (it *bigEndianPostings) Err() error {
func (it *BigEndianPostings) Err() error {
return nil
}

@ -716,7 +716,7 @@ func TestBigEndian(t *testing.T) {
}
t.Run("Iteration", func(t *testing.T) {
bep := newBigEndianPostings(beLst)
bep := NewBigEndianPostings(beLst)
for i := 0; i < num; i++ {
require.True(t, bep.Next())
require.Equal(t, storage.SeriesRef(ls[i]), bep.At())
@ -764,7 +764,7 @@ func TestBigEndian(t *testing.T) {
},
}
bep := newBigEndianPostings(beLst)
bep := NewBigEndianPostings(beLst)
for _, v := range table {
require.Equal(t, v.found, bep.Seek(storage.SeriesRef(v.seek)))

@ -281,7 +281,6 @@ func (c *IndexClient) Volume(ctx context.Context, userID string, from, through m
}
func (c *IndexClient) GetShards(ctx context.Context, userID string, from, through model.Time, targetBytesPerShard uint64, predicate chunk.Predicate) (*logproto.ShardsResponse, error) {
// TODO(owen-d): perf, this is expensive :(
var mtx sync.Mutex

@ -0,0 +1,183 @@
# Loki New Object Storage WAL
## Principles
- The WAL can be streamed to a file or remote object storage.
- When building WAL segments in the ingester, prioritize colocation first by tenant and then by series. This allows efficient reading during compaction and querying.
- At compaction, chunks from the WAL should be reusable and writable to the new block format without decompression.
We aim for at least 8MB WAL segments, preferably larger. In a cluster with a 32MB/s write rate, using 4 ingesters will suffice, halving the current ingester requirement.
## Overview
Multitenancy is achieved by storing the tenant as a label `__0_tenant_id__` in the index to ensure sorting by tenant first. This label is not exposed to users and is removed during compaction.
```
┌──────────────────────────────┐
│ Magic Header ("LOKW") │
│ (4 bytes) │
├──────────────────────────────┤
│ ┌──────────────────────────┐ │
│ │ Chunk 1 │ │
│ ├──────────────────────────┤ │
│ │ ... │ │
│ ├──────────────────────────┤ │
│ │ Chunk N │ │
│ └──────────────────────────┘ │
├──────────────────────────────┤
│ Index │
├──────────────────────────────┤
│ Index Len (4b) │
├──────────────────────────────┤
│ Version (1 byte) │
├──────────────────────────────┤
│ Magic Footer ("LOKW") │
│ (4 bytes) │
└──────────────────────────────┘
```
## Index
The index format is designed to enable efficient seeking to specific chunks required for recent queries. Inspired by the [Prometheus](https://github.com/prometheus/prometheus/blob/main/tsdb/docs/format/index.md) tsdb index, it has some key differences, particularly in the chunk reference within the Series tables. This reference contains sufficient information to seek directly to the chunk in the WAL (Write-Ahead Log).
```
┌────────────────────────────────────────────────────────────────────────────┐
│ len <uvarint>
├────────────────────────────────────────────────────────────────────────────┤
│ ┌────────────────────────────────────────────────────────────────────────┐ │
│ │ labels count <uvarint64> │ │
│ ├────────────────────────────────────────────────────────────────────────┤ │
│ │ ┌────────────────────────────────────────────────┐ │ │
│ │ │ ref(l_i.name) <uvarint32> │ │ │
│ │ ├────────────────────────────────────────────────┤ │ │
│ │ │ ref(l_i.value) <uvarint32> │ │ │
│ │ └────────────────────────────────────────────────┘ │ │
│ │ ... │ │
│ ├────────────────────────────────────────────────────────────────────────┤ │
│ │ chunks count <uvarint64> │ │
│ ├────────────────────────────────────────────────────────────────────────┤ │
│ │ ┌────────────────────────────────────────────────┐ │ │
│ │ │ c_0.mint <varint64> │ │ │
│ │ ├────────────────────────────────────────────────┤ │ │
│ │ │ c_0.maxt - c_0.mint <uvarint64> │ │ │
│ │ ├────────────────────────────────────────────────┤ │ │
│ │ │ ref(c_0.data) <uvarint64> │ │ │
│ │ ├────────────────────────────────────────────────┤ │ │
│ │ │ c_0.entries <uvarint32> │ │ │
│ │ └────────────────────────────────────────────────┘ │ │
│ │ ┌────────────────────────────────────────────────┐ │ │
│ │ │ c_i.mint - c_i-1.maxt <uvarint64> │ │ │
│ │ ├────────────────────────────────────────────────┤ │ │
│ │ │ c_i.maxt - c_i.mint <uvarint64> │ │ │
│ │ ├────────────────────────────────────────────────┤ │ │
│ │ │ ref(c_i.data) - ref(c_i-1.data) <varint64> │ │ │
│ │ ├────────────────────────────────────────────────┤ │ │
│ │ │ c_i.entries <uvarint32> │ │ │
│ │ └────────────────────────────────────────────────┘ │ │
│ │ ... │ │
│ ├────────────────────────────────────────────────────────────────────────┤ │
│ │ ┌────────────────────────────────────────────────┐ │ │
│ │ │ last_chunk.mint - prev_chunk.maxt <uvarint64> │ │ │
│ │ ├────────────────────────────────────────────────┤ │ │
│ │ │ last_chunk.maxt - last_chunk.mint <uvarint64> │ │ │
│ │ ├────────────────────────────────────────────────┤ │ │
│ │ │ ref(last_chunk.data) - ref(prev_chunk.data) │ │ │
│ │ │ <varint64> │ │ │
│ │ ├────────────────────────────────────────────────┤ │ │
│ │ │ last_chunk.entries <uvarint32> │ │ │
│ │ └────────────────────────────────────────────────┘ │ │
│ └────────────────────────────────────────────────────────────────────────┘ │
├────────────────────────────────────────────────────────────────────────────┤
│ CRC32 <4b>
└────────────────────────────────────────────────────────────────────────────┘
```
> Note: data_len for all entries except the last one is inferred from the offset of the next entry.
### Explanation
- **len <uvarint>**: The length of the series entry.
- **labels count <uvarint64>**: The number of labels in the series.
- **ref(l_i.name) <uvarint32>**: Reference to the label name in the symbol table.
- **ref(l_i.value) <uvarint32>**: Reference to the label value in the symbol table.
- **chunks count <uvarint64>**: The number of chunks in the series.
- **c_0.mint <varint64>**: Minimum timestamp of the first chunk.
- **c_0.maxt - c_0.mint <uvarint64>**: Time delta between the minimum and maximum timestamp of the first chunk.
- **ref(c_0.data) <uvarint64>**: Reference to the chunk data.
- **c_0.entries <uvarint32>**: Number of entries in the chunk.
- **c_i.mint - c_i-1.maxt <uvarint64>**: Time delta between the minimum timestamp of the current chunk and the maximum timestamp of the previous chunk.
- **c_i.maxt - c_i.mint <uvarint64>**: Time delta between the minimum and maximum timestamp of the current chunk.
- **ref(c_i.data) - ref(c_i-1.data) <varint64>**: Delta between the current chunk reference and the previous chunk reference.
- **c_i.entries <uvarint32>**: Number of entries in the chunk.
- **CRC32 <4b>**: CRC32 checksum of the series entry.
## Chunks
### Chunk Format Overview
The chunk format is structured to efficiently store and retrieve log data. It starts with a byte that indicates the encoding used for the raw logs, followed by a sequence of double-delta encoded timestamps and the lengths of each log line. Finally, it includes the raw compressed logs and a CRC32 checksum for the metadata.
#### Key Components of the Chunk Format
1. **Initial Byte**:
- Indicates the encoding (compression format, we'll start with 1 for snappy ) used for the raw logs.
2. **Timestamps and Lengths**:
- A sequence of double-delta encoded timestamps.
- Lengths of each log line.
3. **Raw Compressed Logs**:
- The actual log data, compressed for efficiency.
4. **CRC32 Checksum**:
- Ensures the integrity of the metadata.
Unlike the current Loki chunk format, this format does not use smaller blocks because WAL (Write-Ahead Log) segments are typically created within seconds.
### Structure of a Chunk
```
┌──────────────────────────────────────────────────────────────────────────┐
│ encoding (1 byte) │
├──────────────────────────────────────────────────────────────────────────┤
#entries <uvarint>
├──────────────────────────────────────────────────────────────────────────┤
│ ts_0 <uvarint>
├──────────────────────────────────────────────────────────────────────────┤
│ len_line_0 <uvarint>
├──────────────────────────────────────────────────────────────────────────┤
│ ts_1_delta <uvarint>
├──────────────────────────────────────────────────────────────────────────┤
│ len_line_1 <uvarint>
├──────────────────────────────────────────────────────────────────────────┤
│ ts_2_dod <varint>
├──────────────────────────────────────────────────────────────────────────┤
│ len_line_2 <uvarint>
├──────────────────────────────────────────────────────────────────────────┤
│ ... │
├──────────────────────────────────────────────────────────────────────────┤
│ compressed logs <bytes>
├──────────────────────────────────────────────────────────────────────────┤
| compressed logs offset <4b> |
├──────────────────────────────────────────────────────────────────────────┤
│ crc32 <4 bytes>
└──────────────────────────────────────────────────────────────────────────┘
```
#### Explanation
- **encoding (1 byte)**: Indicates the encoding used for the raw logs (e.g., 0 for no compression, 1 for gzip, etc.).
- **#entries <uvarint>**: The number of log entries in the chunk.
- **ts_0 <uvarint>**: The initial timestamp, with nanosecond precision.
- **len_line_0 <uvarint>**: The length of the first log line.
- **ts_1_delta <uvarint>**: The delta from the initial timestamp to the second timestamp.
- **len_line_1 <uvarint>**: The length of the second log line.
- **ts_2_dod <varint>**: The delta of deltas, representing the difference from the previous delta (i.e., double-delta encoding). Can be negative if the spacing between points is decreasing.
- **len_line_2 <uvarint>**: The length of the third log line.
- **compressed logs <bytes>**: The actual log data, compressed according to the specified encoding.
- **compressed logs offset <4 bytes>**: The offset of the compressed log data.
- **crc32 (4 bytes)**: CRC32 checksum for the metadata (excluding the compressed data), ensuring the integrity of the timestamp and length information.
The offset to the compressed logs is known from the index, allowing efficient access and decompression. The CRC32 checksum at the end verifies the integrity of the metadata, as the compressed data typically includes its own CRC for verification.
This structure ensures efficient storage and retrieval of log entries, utilizing double-delta encoding for timestamps and compressing the log data to save space. The timestamps are precise to nanoseconds, allowing for high-resolution time tracking.

@ -0,0 +1,334 @@
package chunks
import (
"bytes"
"encoding/binary"
"errors"
"fmt"
"hash"
"hash/crc32"
"io"
"sync"
"unsafe"
"github.com/golang/snappy"
"github.com/klauspost/compress/s2"
"github.com/grafana/loki/v3/pkg/chunkenc"
"github.com/grafana/loki/v3/pkg/logproto"
)
// EncodingType defines the type for encoding enums
type EncodingType byte
// Supported encoding types
const (
EncodingSnappy EncodingType = iota + 1
)
// Initialize the CRC32 table
var castagnoliTable *crc32.Table
func init() {
castagnoliTable = crc32.MakeTable(crc32.Castagnoli)
}
// newCRC32 initializes a CRC32 hash with the preconfigured polynomial
func newCRC32() hash.Hash32 {
return crc32.New(castagnoliTable)
}
// CRC32 pool
var crc32Pool = sync.Pool{
New: func() interface{} {
return newCRC32()
},
}
// S2 writer pool
var s2WriterPool = sync.Pool{
New: func() interface{} {
return snappy.NewBufferedWriter(nil)
},
}
// S2 reader pool
var s2ReaderPool = sync.Pool{
New: func() interface{} {
return s2.NewReader(nil)
},
}
type StatsWriter struct {
io.Writer
written int64
}
func (w *StatsWriter) Write(p []byte) (int, error) {
n, err := w.Writer.Write(p)
w.written += int64(n)
return n, err
}
// WriteChunk writes the log entries to the writer w with the specified encoding type.
func WriteChunk(writer io.Writer, entries []*logproto.Entry, encoding EncodingType) (int64, error) {
w := &StatsWriter{Writer: writer}
// Validate encoding type
if encoding != EncodingSnappy {
return 0, errors.New("unsupported encoding type")
}
// Get a CRC32 hash instance from the pool
crc := crc32Pool.Get().(hash.Hash32)
crc.Reset()
defer crc32Pool.Put(crc)
// Write encoding byte
if _, err := w.Write([]byte{byte(encoding)}); err != nil {
return w.written, err
}
crc.Write([]byte{byte(encoding)})
// Write number of entries
buf := make([]byte, binary.MaxVarintLen64)
n := binary.PutUvarint(buf, uint64(len(entries)))
if _, err := w.Write(buf[:n]); err != nil {
return w.written, err
}
crc.Write(buf[:n])
// todo: investigate delta+bitpacking from https://github.com/ronanh/intcomp or prometheus bitstream.
// Write timestamps and lengths
var prevT, prevDelta, t, delta uint64
for i, e := range entries {
t = uint64(e.Timestamp.UnixNano())
switch i {
case 0:
n = binary.PutUvarint(buf, t)
if _, err := w.Write(buf[:n]); err != nil {
return w.written, err
}
crc.Write(buf[:n])
case 1:
delta = t - prevT
n = binary.PutUvarint(buf, delta)
if _, err := w.Write(buf[:n]); err != nil {
return w.written, err
}
crc.Write(buf[:n])
default:
delta = t - prevT
dod := int64(delta - prevDelta)
n = binary.PutVarint(buf, dod)
if _, err := w.Write(buf[:n]); err != nil {
return w.written, err
}
crc.Write(buf[:n])
}
prevT = t
prevDelta = delta
// Write length of the line
lineLen := uint64(len(e.Line))
n = binary.PutUvarint(buf, lineLen)
if _, err := w.Write(buf[:n]); err != nil {
return w.written, err
}
crc.Write(buf[:n])
}
// Get the offset for the start of the compressed content
offset := w.written
// Get an S2 writer from the pool and reset it
s2w := s2WriterPool.Get().(*snappy.Writer)
s2w.Reset(w)
defer s2WriterPool.Put(s2w)
// Write compressed logs
for _, e := range entries {
n, err := s2w.Write(unsafeGetBytes(e.Line))
if err != nil {
return w.written, err
}
if n != len(e.Line) {
return w.written, fmt.Errorf("failed to write all bytes: %d != %d", n, len(e.Line))
}
}
if err := s2w.Close(); err != nil {
return w.written, err
}
// Reuse the buffer for offset and checksum
offsetChecksumBuf := make([]byte, 4)
// Write the offset using BigEndian
binary.BigEndian.PutUint32(offsetChecksumBuf, uint32(offset))
if _, err := w.Write(offsetChecksumBuf); err != nil {
return w.written, err
}
// Calculate and write CRC32 checksum at the end using BigEndian
checksum := crc.Sum32()
binary.BigEndian.PutUint32(offsetChecksumBuf, checksum)
if _, err := w.Write(offsetChecksumBuf); err != nil {
return w.written, err
}
return w.written, nil
}
// ChunkReader reads chunks from a byte slice
type ChunkReader struct {
b []byte
pos int
entries uint64
entryIdx uint64
dataPos int
reader io.Reader
prevDelta int64
err error
lineBuf []byte
ts int64
}
// NewChunkReader creates a new ChunkReader and performs CRC verification.
func NewChunkReader(b []byte) (*ChunkReader, error) {
if len(b) < 8 {
return nil, errors.New("invalid chunk: too short")
}
// Extract the CRC32 checksum at the end
crcValue := binary.BigEndian.Uint32(b[len(b)-4:])
// Extract the offset
offset := binary.BigEndian.Uint32(b[len(b)-8 : len(b)-4])
if int(offset) > len(b)-8 {
return nil, errors.New("invalid offset: out of bounds")
}
// Verify CRC32 checksum
if crc32.Checksum(b[:offset], castagnoliTable) != crcValue {
return nil, errors.New("CRC verification failed")
}
// Initialize ChunkReader
reader := &ChunkReader{
b: b[:offset],
}
// Read the chunk header
if err := reader.readChunkHeader(); err != nil {
return nil, err
}
// Initialize the decompression reader
compressedData := b[offset : len(b)-8]
s2Reader := s2ReaderPool.Get().(*s2.Reader)
s2Reader.Reset(bytes.NewReader(compressedData))
reader.reader = s2Reader
return reader, nil
}
// Close implements iter.EntryIterator.
func (r *ChunkReader) Close() error {
// Return the S2 reader to the pool
if r.reader != nil {
s2ReaderPool.Put(r.reader.(*s2.Reader))
r.reader = nil
}
return nil
}
// Entry implements iter.EntryIterator.
// Currrently the chunk reader returns the timestamp and the line, but it could returns all timestamps or/and all lines.
func (r *ChunkReader) At() (int64, []byte) {
return r.ts, r.lineBuf
}
// Err implements iter.EntryIterator.
func (r *ChunkReader) Err() error {
return r.err
}
// Next implements iter.EntryIterator. Reads the next entry from the chunk.
func (r *ChunkReader) Next() bool {
if r.entryIdx >= r.entries || r.err != nil {
return false
}
// Read timestamp
switch r.entryIdx {
case 0:
ts, n := binary.Uvarint(r.b[r.pos:])
r.pos += n
r.ts = int64(ts)
case 1:
delta, n := binary.Uvarint(r.b[r.pos:])
r.pos += n
r.prevDelta = int64(delta)
r.ts += r.prevDelta
default:
dod, n := binary.Varint(r.b[r.pos:])
r.pos += n
r.prevDelta += dod
r.ts += r.prevDelta
}
// Read line length
l, n := binary.Uvarint(r.b[r.pos:])
lineLen := int(l)
r.pos += n
// If the buffer is not yet initialize or too small, we get a new one.
if r.lineBuf == nil || lineLen > cap(r.lineBuf) {
// in case of a replacement we replace back the buffer in the pool
if r.lineBuf != nil {
chunkenc.BytesBufferPool.Put(r.lineBuf)
}
r.lineBuf = chunkenc.BytesBufferPool.Get(lineLen).([]byte)
if lineLen > cap(r.lineBuf) {
r.err = fmt.Errorf("could not get a line buffer of size %d, actual %d", lineLen, cap(r.lineBuf))
return false
}
}
r.lineBuf = r.lineBuf[:lineLen]
// Read line from decompressed data
if _, err := io.ReadFull(r.reader, r.lineBuf); err != nil {
if err != io.EOF {
r.err = err
}
return false
}
r.entryIdx++
return true
}
// readChunkHeader reads the chunk header and initializes the reader state
func (r *ChunkReader) readChunkHeader() error {
if len(r.b) < 1 {
return errors.New("invalid chunk header")
}
// Read encoding byte
encoding := r.b[r.pos]
r.pos++
// Read number of entries
entries, n := binary.Uvarint(r.b[r.pos:])
r.pos += n
r.entries = entries
// Validate encoding (assuming only Snappy is supported)
if EncodingType(encoding) != EncodingSnappy {
return errors.New("unsupported encoding type")
}
return nil
}
func unsafeGetBytes(s string) []byte {
return unsafe.Slice(unsafe.StringData(s), len(s))
}

@ -0,0 +1,345 @@
package chunks
import (
"bufio"
"bytes"
"fmt"
"os"
"path/filepath"
"strconv"
"testing"
"time"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/v3/pkg/logproto"
)
func TestChunkReaderWriter(t *testing.T) {
tests := []struct {
name string
entries []*logproto.Entry
}{
{
name: "Single entry",
entries: []*logproto.Entry{
{Timestamp: time.Now(), Line: "This is a single log entry."},
},
},
{
name: "Multiple entries",
entries: []*logproto.Entry{
{Timestamp: time.Now(), Line: "Log entry 1"},
{Timestamp: time.Now().Add(1 * time.Second), Line: "Log entry 2"},
{Timestamp: time.Now().Add(2 * time.Second), Line: "Log entry 3"},
},
},
{
name: "Different spacing",
entries: []*logproto.Entry{
{Timestamp: time.Unix(0, 1), Line: "Log entry 1"},
{Timestamp: time.Unix(0, 2), Line: "Log entry 2"},
{Timestamp: time.Unix(0, 4), Line: "Log entry 3"},
{Timestamp: time.Unix(0, 5), Line: "Log entry 4"},
},
},
{
// todo: fix dod for variable timestamp delta causing negative dod
name: "Many entries",
entries: func() []*logproto.Entry {
entries := make([]*logproto.Entry, 1000)
for i := 0; i < 1000; i++ {
entries[i] = &logproto.Entry{
Timestamp: time.Now().Add(time.Duration(i) * time.Second),
Line: "Log entry " + strconv.Itoa(i+1),
}
}
return entries
}(),
},
{
name: "Entries with varying lengths",
entries: []*logproto.Entry{
{Timestamp: time.Now(), Line: "Short"},
{Timestamp: time.Now().Add(1 * time.Second), Line: "A bit longer log entry"},
{Timestamp: time.Now().Add(2 * time.Second), Line: "An even longer log entry than the previous one"},
},
},
{
name: "Empty lines",
entries: []*logproto.Entry{
{Timestamp: time.Now(), Line: ""},
{Timestamp: time.Now().Add(1 * time.Second), Line: ""},
{Timestamp: time.Now().Add(2 * time.Second), Line: ""},
},
},
{
name: "Some Empty lines",
entries: []*logproto.Entry{
{Timestamp: time.Now(), Line: ""},
{Timestamp: time.Now().Add(1 * time.Second), Line: ""},
{Timestamp: time.Now().Add(2 * time.Second), Line: "foo"},
{Timestamp: time.Now().Add(4 * time.Second), Line: ""},
{Timestamp: time.Now().Add(9 * time.Second), Line: "bar"},
},
},
{
name: "No entries",
entries: []*logproto.Entry{},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
var buf bytes.Buffer
// Write the chunk
_, err := WriteChunk(&buf, tt.entries, EncodingSnappy)
require.NoError(t, err, "writeChunk failed")
// Read the chunk
reader, err := NewChunkReader(buf.Bytes())
require.NoError(t, err, "NewChunkReader failed")
defer reader.Close()
var readEntries []*logproto.Entry
for reader.Next() {
ts, l := reader.At()
readEntries = append(readEntries, &logproto.Entry{
Timestamp: time.Unix(0, ts),
Line: string(l),
})
}
require.NoError(t, reader.Err(), "reader encountered error")
require.Len(t, readEntries, len(tt.entries))
for i, entry := range tt.entries {
require.Equal(t, entry.Line, readEntries[i].Line, "Lines don't match", i)
require.Equal(t, entry.Timestamp.UnixNano(), readEntries[i].Timestamp.UnixNano(), "Timestamps don't match", i)
}
})
}
}
func TestChunkReaderWriterWithLogGenerator(t *testing.T) {
filenames := testDataFile()
for _, filename := range filenames {
t.Run(filename, func(t *testing.T) {
gen := newLogGenerator(t, filename)
defer gen.Close()
var entries []*logproto.Entry
for more, line := gen.Next(); more; more, line = gen.Next() {
entries = append(entries, &logproto.Entry{
Timestamp: time.Now(),
Line: string(line),
})
if len(entries) >= 10000 {
break
}
}
var buf bytes.Buffer
// Write the chunk
_, err := WriteChunk(&buf, entries, EncodingSnappy)
require.NoError(t, err, "writeChunk failed")
// Read the chunk
reader, err := NewChunkReader(buf.Bytes())
require.NoError(t, err, "NewChunkReader failed")
defer reader.Close()
var readEntries []*logproto.Entry
for reader.Next() {
ts, l := reader.At()
readEntries = append(readEntries, &logproto.Entry{
Timestamp: time.Unix(0, ts),
Line: string(l),
})
}
require.NoError(t, reader.Err(), "reader encountered error")
require.Len(t, readEntries, len(entries))
for i, entry := range entries {
require.Equal(t, entry.Line, readEntries[i].Line, "Lines don't match", i)
require.Equal(t, entry.Timestamp.UnixNano(), readEntries[i].Timestamp.UnixNano(), "Timestamps don't match", i)
}
})
}
}
// BenchmarkWriteChunk benchmarks the writeChunk function.
func BenchmarkWriteChunk(b *testing.B) {
// Generate sample log entries
entries := generateLogEntries(1000)
// Reset the buffer for each iteration
buf := bytes.NewBuffer(make([]byte, 0, 5<<20))
b.ReportAllocs()
b.ResetTimer()
// Run the benchmark
for n := 0; n < b.N; n++ {
buf.Reset()
// Call the writeChunk function
_, err := WriteChunk(buf, entries, EncodingSnappy)
if err != nil {
b.Fatalf("writeChunk failed: %v", err)
}
}
}
var (
lineBuf []byte
ts int64
)
// Benchmark reads with log generator
func BenchmarkReadChunkWithLogGenerator(b *testing.B) {
filenames := testDataFile()
for _, filename := range filenames {
b.Run(filename, func(b *testing.B) {
gen := newLogGenerator(b, filename)
defer gen.Close()
var entries []*logproto.Entry
for more, line := gen.Next(); more; more, line = gen.Next() {
entries = append(entries, &logproto.Entry{
Timestamp: time.Now(),
Line: string(line),
})
if len(entries) >= 100000 {
break
}
}
// Reset the buffer for each iteration
buf := bytes.NewBuffer(make([]byte, 0, 5<<20))
_, err := WriteChunk(buf, entries, EncodingSnappy)
if err != nil {
b.Fatalf("writeChunk failed: %v", err)
}
b.ReportAllocs()
b.ResetTimer()
// Run the benchmark
for n := 0; n < b.N; n++ {
reader, err := NewChunkReader(buf.Bytes())
require.NoError(b, err, "NewChunkReader failed")
for reader.Next() {
ts, lineBuf = reader.At()
}
reader.Close()
}
})
}
}
// Benchmark with log generator
func BenchmarkWriteChunkWithLogGenerator(b *testing.B) {
filenames := testDataFile()
for _, filename := range filenames {
for _, count := range []int{1000, 10000, 100000} {
b.Run(fmt.Sprintf("%s-%d", filename, count), func(b *testing.B) {
gen := newLogGenerator(b, filename)
defer gen.Close()
var entries []*logproto.Entry
for more, line := gen.Next(); more; more, line = gen.Next() {
entries = append(entries, &logproto.Entry{
Timestamp: time.Now(),
Line: string(line),
})
if len(entries) >= count {
break
}
}
// Reset the buffer for each iteration
buf := bytes.NewBuffer(make([]byte, 0, 5<<20))
b.ReportAllocs()
b.ResetTimer()
// Run the benchmark
for n := 0; n < b.N; n++ {
buf.Reset()
// Call the writeChunk function
_, err := WriteChunk(buf, entries, EncodingSnappy)
if err != nil {
b.Fatalf("writeChunk failed: %v", err)
}
}
})
}
}
}
func testDataFile() []string {
testdataDir := "../testdata"
files, err := os.ReadDir(testdataDir)
if err != nil {
panic(err)
}
var fileNames []string
for _, file := range files {
if !file.IsDir() {
filePath := filepath.Join(testdataDir, file.Name())
fileNames = append(fileNames, filePath)
}
}
return fileNames
}
// generateLogEntries generates a slice of logproto.Entry with the given count.
func generateLogEntries(count int) []*logproto.Entry {
entries := make([]*logproto.Entry, count)
for i := 0; i < count; i++ {
entries[i] = &logproto.Entry{
Timestamp: time.Now(),
Line: "This is a sample log entry.",
}
}
return entries
}
type logGenerator struct {
f *os.File
s *bufio.Scanner
}
func (g *logGenerator) Next() (bool, []byte) {
if g.s.Scan() {
return true, g.s.Bytes()
}
g.reset()
return g.s.Scan(), g.s.Bytes()
}
func (g *logGenerator) Close() {
if g.f != nil {
g.f.Close()
}
g.f = nil
}
func (g *logGenerator) reset() {
_, _ = g.f.Seek(0, 0)
g.s = bufio.NewScanner(g.f)
}
func newLogGenerator(t testing.TB, filename string) *logGenerator {
t.Helper()
file, err := os.Open(filename)
require.NoError(t, err)
return &logGenerator{
f: file,
s: bufio.NewScanner(file),
}
}

@ -0,0 +1,20 @@
package chunks
type ChunkRef uint64
type Meta struct {
// Start offset of the chunk
Ref ChunkRef
// Min and Max time nanoseconds precise.
MinTime, MaxTime int64
}
func NewChunkRef(offset, size uint64) ChunkRef {
return ChunkRef(offset<<32 | size)
}
func (b ChunkRef) Unpack() (int, int) {
offset := int(b >> 32)
size := int((b << 32) >> 32)
return offset, size
}

@ -0,0 +1,91 @@
package index
import (
"bytes"
"fmt"
"io"
"github.com/pkg/errors"
)
type BufferWriter struct {
buf *bytes.Buffer
pos uint64
}
// NewBufferWriter returns a new BufferWriter.
// todo: pooling memory
func NewBufferWriter() *BufferWriter {
return &BufferWriter{
buf: bytes.NewBuffer(nil),
pos: 0,
}
}
func (fw *BufferWriter) Pos() uint64 {
return fw.pos
}
func (fw *BufferWriter) Write(bufs ...[]byte) error {
for _, buf := range bufs {
n, err := fw.buf.Write(buf)
if err != nil {
return err
}
fw.pos += uint64(n)
}
return nil
}
func (fw *BufferWriter) Flush() error {
return nil
}
func (fw *BufferWriter) WriteAt(buf []byte, pos uint64) error {
if pos > fw.pos {
return fmt.Errorf("position out of range")
}
if pos+uint64(len(buf)) > fw.pos {
return fmt.Errorf("write exceeds buffer size")
}
copy(fw.buf.Bytes()[pos:], buf)
return nil
}
func (fw *BufferWriter) Read(buf []byte) (int, error) {
return fw.buf.Read(buf)
}
func (fw *BufferWriter) ReadFrom(r io.Reader) (int64, error) {
n, err := fw.buf.ReadFrom(r)
if err != nil {
return n, err
}
fw.pos += uint64(n)
return n, err
}
func (fw *BufferWriter) AddPadding(size int) error {
p := fw.pos % uint64(size)
if p == 0 {
return nil
}
p = uint64(size) - p
if err := fw.Write(make([]byte, p)); err != nil {
return errors.Wrap(err, "add padding")
}
return nil
}
func (fw *BufferWriter) Buffer() ([]byte, io.Closer, error) {
return fw.buf.Bytes(), io.NopCloser(nil), nil
}
func (fw *BufferWriter) Close() error {
return nil
}
func (fw *BufferWriter) Remove() error {
return nil
}

File diff suppressed because it is too large Load Diff

@ -0,0 +1,624 @@
// Copyright 2017 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package index
import (
"context"
"errors"
"fmt"
"hash/crc32"
"path/filepath"
"slices"
"sort"
"strconv"
"testing"
"github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index"
"github.com/grafana/loki/v3/pkg/storage/wal/chunks"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/encoding"
"github.com/prometheus/prometheus/util/testutil"
"github.com/stretchr/testify/require"
"go.uber.org/goleak"
)
func TestMain(m *testing.M) {
goleak.VerifyTestMain(m)
}
type series struct {
l labels.Labels
chunks []chunks.Meta
}
type mockIndex struct {
series map[storage.SeriesRef]series
postings map[labels.Label][]storage.SeriesRef
symbols map[string]struct{}
}
func newMockIndex() mockIndex {
ix := mockIndex{
series: make(map[storage.SeriesRef]series),
postings: make(map[labels.Label][]storage.SeriesRef),
symbols: make(map[string]struct{}),
}
ix.postings[labels.Label{Name: AllPostingsKey.Name, Value: AllPostingsKey.Value}] = []storage.SeriesRef{}
return ix
}
func (m mockIndex) Symbols() (map[string]struct{}, error) {
return m.symbols, nil
}
func (m mockIndex) AddSeries(ref storage.SeriesRef, l labels.Labels, chunks ...chunks.Meta) error {
name, value := index.AllPostingsKey()
if _, ok := m.series[ref]; ok {
return fmt.Errorf("series with reference %d already added", ref)
}
l.Range(func(lbl labels.Label) {
m.symbols[lbl.Name] = struct{}{}
m.symbols[lbl.Value] = struct{}{}
if _, ok := m.postings[lbl]; !ok {
m.postings[lbl] = []storage.SeriesRef{}
}
m.postings[lbl] = append(m.postings[lbl], ref)
})
m.postings[labels.Label{Name: name, Value: value}] = append(m.postings[labels.Label{Name: name, Value: value}], ref)
s := series{l: l}
// Actual chunk data is not stored in the index.
s.chunks = append(s.chunks, chunks...)
m.series[ref] = s
return nil
}
func (m mockIndex) Close() error {
return nil
}
func (m mockIndex) LabelValues(_ context.Context, name string) ([]string, error) {
values := []string{}
for l := range m.postings {
if l.Name == name {
values = append(values, l.Value)
}
}
return values, nil
}
func (m mockIndex) Postings(_ context.Context, name string, values ...string) (index.Postings, error) {
p := []index.Postings{}
for _, value := range values {
l := labels.Label{Name: name, Value: value}
p = append(p, m.SortedPostings(index.NewListPostings(m.postings[l])))
}
return index.Merge(p...), nil
}
func (m mockIndex) SortedPostings(p index.Postings) index.Postings {
ep, err := index.ExpandPostings(p)
if err != nil {
return index.ErrPostings(fmt.Errorf("expand postings: %w", err))
}
sort.Slice(ep, func(i, j int) bool {
return labels.Compare(m.series[ep[i]].l, m.series[ep[j]].l) < 0
})
return index.NewListPostings(ep)
}
func (m mockIndex) Series(ref storage.SeriesRef, builder *labels.ScratchBuilder, chks *[]chunks.Meta) error {
s, ok := m.series[ref]
if !ok {
return errors.New("not found")
}
builder.Assign(s.l)
*chks = append((*chks)[:0], s.chunks...)
return nil
}
func TestIndexRW_Create_Open(t *testing.T) {
// An empty index must still result in a readable file.
iw, err := NewWriter(context.Background())
require.NoError(t, err)
require.NoError(t, iw.Close())
buf, closer, err := iw.Buffer()
require.NoError(t, err)
defer closer.Close()
ir, err := NewReader(RealByteSlice(buf))
require.NoError(t, err)
require.NoError(t, ir.Close())
buf[0], buf[1] = 0, 0
_, err = NewReader(RealByteSlice(buf))
require.Error(t, err)
}
func TestIndexRW_Postings(t *testing.T) {
ctx := context.Background()
var input indexWriterSeriesSlice
for i := 1; i < 5; i++ {
input = append(input, &indexWriterSeries{
labels: labels.FromStrings("a", "1", "b", strconv.Itoa(i)),
})
}
ir, buf, _ := createReader(ctx, t, input)
p, err := ir.Postings(ctx, "a", "1")
require.NoError(t, err)
var c []chunks.Meta
var builder labels.ScratchBuilder
for i := 0; p.Next(); i++ {
err := ir.Series(p.At(), &builder, &c)
require.NoError(t, err)
require.Empty(t, c)
testutil.RequireEqual(t, input[i].labels, builder.Labels())
}
require.NoError(t, p.Err())
// The label indices are no longer used, so test them by hand here.
labelValuesOffsets := map[string]uint64{}
d := encoding.NewDecbufAt(ir.b, int(ir.toc.LabelIndicesTable), castagnoliTable)
cnt := d.Be32()
for d.Err() == nil && d.Len() > 0 && cnt > 0 {
require.Equal(t, 1, d.Uvarint(), "Unexpected number of keys for label indices table")
lbl := d.UvarintStr()
off := d.Uvarint64()
labelValuesOffsets[lbl] = off
cnt--
}
require.NoError(t, d.Err())
labelIndices := map[string][]string{}
for lbl, off := range labelValuesOffsets {
d := encoding.NewDecbufAt(ir.b, int(off), castagnoliTable)
require.Equal(t, 1, d.Be32int(), "Unexpected number of label indices table names")
for i := d.Be32(); i > 0 && d.Err() == nil; i-- {
v, err := ir.lookupSymbol(ctx, d.Be32())
require.NoError(t, err)
labelIndices[lbl] = append(labelIndices[lbl], v)
}
require.NoError(t, d.Err())
}
require.Equal(t, map[string][]string{
"a": {"1"},
"b": {"1", "2", "3", "4"},
}, labelIndices)
t.Run("ShardedPostings()", func(t *testing.T) {
ir, err := NewReader(RealByteSlice(buf))
require.NoError(t, err)
t.Cleanup(func() {
require.NoError(t, ir.Close())
})
// List all postings for a given label value. This is what we expect to get
// in output from all shards.
p, err = ir.Postings(ctx, "a", "1")
require.NoError(t, err)
var expected []storage.SeriesRef
for p.Next() {
expected = append(expected, p.At())
}
require.NoError(t, p.Err())
require.NotEmpty(t, expected)
// Query the same postings for each shard.
const shardCount = uint64(4)
actualShards := make(map[uint64][]storage.SeriesRef)
actualPostings := make([]storage.SeriesRef, 0, len(expected))
for shardIndex := uint64(0); shardIndex < shardCount; shardIndex++ {
p, err = ir.Postings(ctx, "a", "1")
require.NoError(t, err)
p = ir.ShardedPostings(p, shardIndex, shardCount)
for p.Next() {
ref := p.At()
actualShards[shardIndex] = append(actualShards[shardIndex], ref)
actualPostings = append(actualPostings, ref)
}
require.NoError(t, p.Err())
}
// We expect the postings merged out of shards is the exact same of the non sharded ones.
require.ElementsMatch(t, expected, actualPostings)
// We expect the series in each shard are the expected ones.
for shardIndex, ids := range actualShards {
for _, id := range ids {
var lbls labels.ScratchBuilder
require.NoError(t, ir.Series(id, &lbls, nil))
require.Equal(t, shardIndex, labels.StableHash(lbls.Labels())%shardCount)
}
}
})
}
func TestPostingsMany(t *testing.T) {
ctx := context.Background()
// Create a label in the index which has 999 values.
var input indexWriterSeriesSlice
for i := 1; i < 1000; i++ {
v := fmt.Sprintf("%03d", i)
input = append(input, &indexWriterSeries{
labels: labels.FromStrings("i", v, "foo", "bar"),
})
}
ir, _, symbols := createReader(ctx, t, input)
cases := []struct {
in []string
}{
// Simple cases, everything is present.
{in: []string{"002"}},
{in: []string{"031", "032", "033"}},
{in: []string{"032", "033"}},
{in: []string{"127", "128"}},
{in: []string{"127", "128", "129"}},
{in: []string{"127", "129"}},
{in: []string{"128", "129"}},
{in: []string{"998", "999"}},
{in: []string{"999"}},
// Before actual values.
{in: []string{"000"}},
{in: []string{"000", "001"}},
{in: []string{"000", "002"}},
// After actual values.
{in: []string{"999a"}},
{in: []string{"999", "999a"}},
{in: []string{"998", "999", "999a"}},
// In the middle of actual values.
{in: []string{"126a", "127", "128"}},
{in: []string{"127", "127a", "128"}},
{in: []string{"127", "127a", "128", "128a", "129"}},
{in: []string{"127", "128a", "129"}},
{in: []string{"128", "128a", "129"}},
{in: []string{"128", "129", "129a"}},
{in: []string{"126a", "126b", "127", "127a", "127b", "128", "128a", "128b", "129", "129a", "129b"}},
}
var builder labels.ScratchBuilder
for _, c := range cases {
it, err := ir.Postings(ctx, "i", c.in...)
require.NoError(t, err)
got := []string{}
var metas []chunks.Meta
for it.Next() {
require.NoError(t, ir.Series(it.At(), &builder, &metas))
got = append(got, builder.Labels().Get("i"))
}
require.NoError(t, it.Err())
exp := []string{}
for _, e := range c.in {
if _, ok := symbols[e]; ok && e != "l" {
exp = append(exp, e)
}
}
require.Equal(t, exp, got, fmt.Sprintf("input: %v", c.in))
}
}
func TestPersistence_index_e2e(t *testing.T) {
ctx := context.Background()
lbls, err := labels.ReadLabels(filepath.Join("..", "testdata", "20kseries.json"), 20000)
require.NoError(t, err)
// Sort labels as the index writer expects series in sorted order.
sort.Sort(labels.Slice(lbls))
var input indexWriterSeriesSlice
ref := uint64(0)
// Generate ChunkMetas for every label set.
for i, lset := range lbls {
var metas []chunks.Meta
for j := 0; j <= (i % 20); j++ {
ref++
metas = append(metas, chunks.Meta{
MinTime: int64(j * 10000),
MaxTime: int64((j+1)*10000) - 1,
Ref: chunks.ChunkRef(ref),
})
}
input = append(input, &indexWriterSeries{
labels: lset,
chunks: metas,
})
}
ir, _, _ := createReader(ctx, t, input)
// Population procedure as done by compaction.
var (
postings = index.NewMemPostings()
values = map[string]map[string]struct{}{}
)
mi := newMockIndex()
for i, s := range input {
require.NoError(t, mi.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...))
s.labels.Range(func(l labels.Label) {
valset, ok := values[l.Name]
if !ok {
valset = map[string]struct{}{}
values[l.Name] = valset
}
valset[l.Value] = struct{}{}
})
postings.Add(storage.SeriesRef(i), s.labels)
}
for p := range mi.postings {
gotp, err := ir.Postings(ctx, p.Name, p.Value)
require.NoError(t, err)
expp, err := mi.Postings(ctx, p.Name, p.Value)
require.NoError(t, err)
var chks, expchks []chunks.Meta
var builder, eBuilder labels.ScratchBuilder
for gotp.Next() {
require.True(t, expp.Next())
ref := gotp.At()
err := ir.Series(ref, &builder, &chks)
require.NoError(t, err)
err = mi.Series(expp.At(), &eBuilder, &expchks)
require.NoError(t, err)
testutil.RequireEqual(t, eBuilder.Labels(), builder.Labels())
require.Equal(t, expchks, chks)
}
require.False(t, expp.Next(), "Expected no more postings for %q=%q", p.Name, p.Value)
require.NoError(t, gotp.Err())
}
labelPairs := map[string][]string{}
for l := range mi.postings {
labelPairs[l.Name] = append(labelPairs[l.Name], l.Value)
}
for k, v := range labelPairs {
sort.Strings(v)
res, err := ir.SortedLabelValues(ctx, k)
require.NoError(t, err)
require.Equal(t, len(v), len(res))
for i := 0; i < len(v); i++ {
require.Equal(t, v[i], res[i])
}
}
gotSymbols := []string{}
it := ir.Symbols()
for it.Next() {
gotSymbols = append(gotSymbols, it.At())
}
require.NoError(t, it.Err())
expSymbols := []string{}
for s := range mi.symbols {
expSymbols = append(expSymbols, s)
}
sort.Strings(expSymbols)
require.Equal(t, expSymbols, gotSymbols)
}
func TestWriter_ShouldReturnErrorOnSeriesWithDuplicatedLabelNames(t *testing.T) {
w, err := NewWriter(context.Background())
require.NoError(t, err)
require.NoError(t, w.AddSymbol("__name__"))
require.NoError(t, w.AddSymbol("metric_1"))
require.NoError(t, w.AddSymbol("metric_2"))
require.NoError(t, w.AddSeries(0, labels.FromStrings("__name__", "metric_1", "__name__", "metric_2")))
err = w.Close()
require.Error(t, err)
require.ErrorContains(t, err, "corruption detected when writing postings to index")
}
func TestDecbufUvarintWithInvalidBuffer(t *testing.T) {
b := RealByteSlice([]byte{0x81, 0x81, 0x81, 0x81, 0x81, 0x81})
db := encoding.NewDecbufUvarintAt(b, 0, castagnoliTable)
require.Error(t, db.Err())
}
func TestReaderWithInvalidBuffer(t *testing.T) {
b := RealByteSlice([]byte{0x81, 0x81, 0x81, 0x81, 0x81, 0x81})
_, err := NewReader(b)
require.Error(t, err)
}
func TestSymbols(t *testing.T) {
buf := encoding.Encbuf{}
// Add prefix to the buffer to simulate symbols as part of larger buffer.
buf.PutUvarintStr("something")
symbolsStart := buf.Len()
buf.PutBE32int(204) // Length of symbols table.
buf.PutBE32int(100) // Number of symbols.
for i := 0; i < 100; i++ {
// i represents index in unicode characters table.
buf.PutUvarintStr(string(rune(i))) // Symbol.
}
checksum := crc32.Checksum(buf.Get()[symbolsStart+4:], castagnoliTable)
buf.PutBE32(checksum) // Check sum at the end.
s, err := NewSymbols(RealByteSlice(buf.Get()), FormatV2, symbolsStart)
require.NoError(t, err)
// We store only 4 offsets to symbols.
require.Equal(t, 32, s.Size())
for i := 99; i >= 0; i-- {
s, err := s.Lookup(uint32(i))
require.NoError(t, err)
require.Equal(t, string(rune(i)), s)
}
_, err = s.Lookup(100)
require.Error(t, err)
for i := 99; i >= 0; i-- {
r, err := s.ReverseLookup(string(rune(i)))
require.NoError(t, err)
require.Equal(t, uint32(i), r)
}
_, err = s.ReverseLookup(string(rune(100)))
require.Error(t, err)
iter := s.Iter()
i := 0
for iter.Next() {
require.Equal(t, string(rune(i)), iter.At())
i++
}
require.NoError(t, iter.Err())
}
func BenchmarkReader_ShardedPostings(b *testing.B) {
const (
numSeries = 10000
numShards = 16
)
ctx := context.Background()
var input indexWriterSeriesSlice
for i := 1; i <= numSeries; i++ {
input = append(input, &indexWriterSeries{
labels: labels.FromStrings("const", fmt.Sprintf("%10d", 1), "unique", fmt.Sprintf("%10d", i)),
})
}
ir, _, _ := createReader(ctx, b, input)
b.ResetTimer()
for n := 0; n < b.N; n++ {
allPostings, err := ir.Postings(ctx, "const", fmt.Sprintf("%10d", 1))
require.NoError(b, err)
ir.ShardedPostings(allPostings, uint64(n%numShards), numShards)
}
}
func TestDecoder_Postings_WrongInput(t *testing.T) {
_, _, err := (&Decoder{}).Postings([]byte("the cake is a lie"))
require.Error(t, err)
}
func TestChunksRefOrdering(t *testing.T) {
idx, err := NewWriter(context.Background())
require.NoError(t, err)
require.NoError(t, idx.AddSymbol("1"))
require.NoError(t, idx.AddSymbol("2"))
require.NoError(t, idx.AddSymbol("__name__"))
c50 := chunks.Meta{Ref: 50}
c100 := chunks.Meta{Ref: 100}
c200 := chunks.Meta{Ref: 200}
require.NoError(t, idx.AddSeries(1, labels.FromStrings("__name__", "1"), c100))
require.EqualError(t, idx.AddSeries(2, labels.FromStrings("__name__", "2"), c50), "unsorted chunk reference: 50, previous: 100")
require.NoError(t, idx.AddSeries(2, labels.FromStrings("__name__", "2"), c200))
require.NoError(t, idx.Close())
}
func TestChunksTimeOrdering(t *testing.T) {
idx, err := NewWriter(context.Background())
require.NoError(t, err)
require.NoError(t, idx.AddSymbol("1"))
require.NoError(t, idx.AddSymbol("2"))
require.NoError(t, idx.AddSymbol("__name__"))
require.NoError(t, idx.AddSeries(1, labels.FromStrings("__name__", "1"),
chunks.Meta{Ref: 1, MinTime: 0, MaxTime: 10}, // Also checks that first chunk can have MinTime: 0.
chunks.Meta{Ref: 2, MinTime: 11, MaxTime: 20},
chunks.Meta{Ref: 3, MinTime: 21, MaxTime: 30},
))
require.EqualError(t, idx.AddSeries(1, labels.FromStrings("__name__", "2"),
chunks.Meta{Ref: 10, MinTime: 0, MaxTime: 10},
chunks.Meta{Ref: 20, MinTime: 10, MaxTime: 20},
), "chunk minT 10 is not higher than previous chunk maxT 10")
require.EqualError(t, idx.AddSeries(1, labels.FromStrings("__name__", "2"),
chunks.Meta{Ref: 10, MinTime: 100, MaxTime: 30},
), "chunk maxT 30 is less than minT 100")
require.NoError(t, idx.Close())
}
// createFileReader creates a temporary index file. It writes the provided input to this file.
// It returns a Reader for this file, the file's name, and the symbol map.
func createReader(ctx context.Context, tb testing.TB, input indexWriterSeriesSlice) (*Reader, []byte, map[string]struct{}) {
tb.Helper()
iw, err := NewWriter(ctx)
require.NoError(tb, err)
symbols := map[string]struct{}{}
for _, s := range input {
s.labels.Range(func(l labels.Label) {
symbols[l.Name] = struct{}{}
symbols[l.Value] = struct{}{}
})
}
syms := []string{}
for s := range symbols {
syms = append(syms, s)
}
slices.Sort(syms)
for _, s := range syms {
require.NoError(tb, iw.AddSymbol(s))
}
for i, s := range input {
require.NoError(tb, iw.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...))
}
require.NoError(tb, iw.Close())
buf, closer, err := iw.Buffer()
require.NoError(tb, err)
ir, err := NewReader(RealByteSlice(buf))
require.NoError(tb, err)
tb.Cleanup(func() {
require.NoError(tb, ir.Close())
require.NoError(tb, closer.Close())
})
return ir, buf, symbols
}

@ -0,0 +1,334 @@
package wal
import (
"bytes"
"context"
"encoding/binary"
"errors"
"fmt"
"io"
"sort"
"github.com/dolthub/swiss"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/storage"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb"
tsdbindex "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index"
"github.com/grafana/loki/v3/pkg/storage/wal/chunks"
"github.com/grafana/loki/v3/pkg/storage/wal/index"
"github.com/grafana/loki/v3/pkg/util/encoding"
)
// LOKW is the magic number for the Loki WAL format.
var (
magicNumber = uint32(0x4C4F4B57)
magicBuf [4]byte
)
func init() {
binary.BigEndian.PutUint32(magicBuf[:], magicNumber)
}
type streamID struct {
labels, tenant string
}
type SegmentWriter struct {
streams *swiss.Map[streamID, *streamSegment]
buf1 encoding.Encbuf
}
type streamSegment struct {
lbls labels.Labels
entries []*logproto.Entry
tenantID string
maxt int64
}
// NewWalSegmentWriter creates a new WalSegmentWriter.
func NewWalSegmentWriter() *SegmentWriter {
return &SegmentWriter{
streams: swiss.NewMap[streamID, *streamSegment](64),
buf1: encoding.EncWith(make([]byte, 0, 4)),
}
}
// Labels are passed a string `{foo="bar",baz="qux"}` `{foo="foo",baz="foo"}`. labels.Labels => Symbols foo, baz , qux
func (b *SegmentWriter) Append(tenantID, labelsString string, lbls labels.Labels, entries []*logproto.Entry) {
if len(entries) == 0 {
return
}
id := streamID{labels: labelsString, tenant: tenantID}
s, ok := b.streams.Get(id)
if !ok {
if lbls.Get(tsdb.TenantLabel) == "" {
lbls = labels.NewBuilder(lbls).Set(tsdb.TenantLabel, tenantID).Labels()
}
s = &streamSegment{
// todo: should be pooled.
// prometheus bucketed pool
// https://pkg.go.dev/github.com/prometheus/prometheus/util/pool
entries: make([]*logproto.Entry, 0, 64),
lbls: lbls,
tenantID: tenantID,
}
s.maxt = entries[len(entries)-1].Timestamp.UnixNano()
s.entries = append(s.entries, entries...)
b.streams.Put(id, s)
return
}
for i, e := range entries {
if e.Timestamp.UnixNano() >= s.maxt {
s.entries = append(s.entries, entries[i])
s.maxt = e.Timestamp.UnixNano()
continue
}
// search for the right place to insert.
idx := sort.Search(len(s.entries), func(i int) bool {
return s.entries[i].Timestamp.UnixNano() > e.Timestamp.UnixNano()
})
// insert at the right place.
s.entries = append(s.entries, nil)
copy(s.entries[idx+1:], s.entries[idx:])
s.entries[idx] = e
}
}
func (b *SegmentWriter) WriteTo(w io.Writer) (int64, error) {
var (
total int64
streams = make([]*streamSegment, 0, b.streams.Count())
)
// Collect all streams and sort them by tenantID and labels.
b.streams.Iter(func(k streamID, v *streamSegment) bool {
streams = append(streams, v)
return false
})
sort.Slice(streams, func(i, j int) bool {
if streams[i].tenantID != streams[j].tenantID {
return streams[i].tenantID < streams[j].tenantID
}
return labels.Compare(streams[i].lbls, streams[j].lbls) < 0
})
idxw, err := index.NewWriter(context.TODO())
if err != nil {
return total, err
}
// Build symbols
symbolsMap := make(map[string]struct{})
for _, s := range streams {
for _, l := range s.lbls {
symbolsMap[l.Name] = struct{}{}
symbolsMap[l.Value] = struct{}{}
}
}
// Sort symbols
symbols := make([]string, 0, len(symbolsMap))
for s := range symbolsMap {
symbols = append(symbols, s)
}
sort.Strings(symbols)
// Add symbols
for _, symbol := range symbols {
if err := idxw.AddSymbol(symbol); err != nil {
return total, err
}
}
// Writes magic header
n, err := w.Write(magicBuf[:])
if err != nil {
return total, err
}
total += int64(n)
// Write all streams to the writer.
for i, s := range streams {
if len(s.entries) == 0 {
continue
}
n, err := s.WriteTo(w)
if err != nil {
return total, err
}
err = idxw.AddSeries(storage.SeriesRef(i), s.lbls, chunks.Meta{
MinTime: s.entries[0].Timestamp.UnixNano(),
MaxTime: s.entries[len(s.entries)-1].Timestamp.UnixNano(),
Ref: chunks.NewChunkRef(uint64(total), uint64(n)),
})
if err != nil {
return total, err
}
total += n
}
if err := idxw.Close(); err != nil {
return total, err
}
buf, closer, err := idxw.Buffer()
if err != nil {
return total, err
}
defer closer.Close()
n, err = w.Write(buf)
if err != nil {
return total, err
}
if n != len(buf) {
return total, errors.New("invalid written index len")
}
total += int64(n)
// write index len 4b
b.buf1.PutBE32int(n)
n, err = w.Write(b.buf1.Get())
if err != nil {
return total, err
}
total += int64(n)
// write the version
n, err = w.Write([]byte{1})
if err != nil {
return total, err
}
total += int64(n)
// Writes magic footer
n, err = w.Write(magicBuf[:])
if err != nil {
return total, err
}
total += int64(n)
return total, nil
}
func (s *streamSegment) WriteTo(w io.Writer) (n int64, err error) {
return chunks.WriteChunk(w, s.entries, chunks.EncodingSnappy)
}
// Reset clears the writer.
// After calling Reset, the writer can be reused.
func (b *SegmentWriter) Reset() {
b.streams.Clear()
b.buf1.Reset()
}
type SegmentReader struct {
idr *index.Reader
b []byte
}
func NewReader(b []byte) (*SegmentReader, error) {
if len(b) < 13 {
return nil, errors.New("segment too small")
}
if !bytes.Equal(magicBuf[:], b[:4]) {
return nil, errors.New("invalid segment header")
}
if !bytes.Equal(magicBuf[:], b[len(b)-4:]) {
return nil, errors.New("invalid segment footer")
}
n := 5
version := b[len(b)-n]
if version != 1 {
return nil, fmt.Errorf("invalid segment version: %d", version)
}
indexLen := binary.BigEndian.Uint32(b[len(b)-n-4 : len(b)-n])
n += 4
idr, err := index.NewReader(index.RealByteSlice(b[len(b)-n-int(indexLen) : len(b)-n]))
if err != nil {
return nil, err
}
return &SegmentReader{
idr: idr,
b: b[:len(b)-n-int(indexLen)],
}, nil
}
// todo: Evaluate/benchmark wal segment using apache arrow as format ?
type SeriesIter struct {
ir *index.Reader
ps tsdbindex.Postings
err error
curSeriesRef storage.SeriesRef
curLabels labels.Labels
labelsBuilder *labels.ScratchBuilder
chunksMeta []chunks.Meta
blocks []byte
}
func NewSeriesIter(ir *index.Reader, ps tsdbindex.Postings, blocks []byte) *SeriesIter {
return &SeriesIter{
ir: ir,
ps: ps,
blocks: blocks,
labelsBuilder: &labels.ScratchBuilder{},
chunksMeta: make([]chunks.Meta, 0, 1),
}
}
func (iter *SeriesIter) Next() bool {
if !iter.ps.Next() {
return false
}
if iter.ps.At() != iter.curSeriesRef {
iter.curSeriesRef = iter.ps.At()
err := iter.ir.Series(iter.curSeriesRef, iter.labelsBuilder, &iter.chunksMeta)
if err != nil {
iter.err = err
return false
}
iter.curLabels = iter.labelsBuilder.Labels()
}
return true
}
func (iter *SeriesIter) At() labels.Labels {
return iter.curLabels
}
func (iter *SeriesIter) Err() error {
return iter.err
}
func (iter *SeriesIter) ChunkReader(_ *chunks.ChunkReader) (*chunks.ChunkReader, error) {
if len(iter.chunksMeta) == 0 {
return nil, fmt.Errorf("no chunks found for series %d", iter.curSeriesRef)
}
if len(iter.chunksMeta) > 1 {
return nil, fmt.Errorf("multiple chunks found for series %d", iter.curSeriesRef)
}
offset, size := iter.chunksMeta[0].Ref.Unpack()
if offset < 0 || offset >= len(iter.blocks) || size < 0 || offset+size > len(iter.blocks) {
return nil, fmt.Errorf("invalid offset or size for series %d: offset %d, size %d, blocks len %d", iter.curSeriesRef, offset, size, len(iter.blocks))
}
return chunks.NewChunkReader(iter.blocks[offset : offset+size])
}
func (r *SegmentReader) Series(ctx context.Context) (*SeriesIter, error) {
ps, err := r.idr.Postings(ctx, index.AllPostingsKey.Name, index.AllPostingsKey.Value)
if err != nil {
return nil, err
}
if ps.Err() != nil {
return nil, ps.Err()
}
return NewSeriesIter(r.idr, ps, r.b), nil
}

@ -0,0 +1,188 @@
package wal
import (
"bytes"
"context"
"fmt"
"sort"
"testing"
"time"
"github.com/prometheus/prometheus/model/labels"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logql/syntax"
"github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb"
"github.com/grafana/loki/pkg/push"
)
func TestWalSegmentWriter_Append(t *testing.T) {
type batch struct {
tenant string
labels string
entries []*logproto.Entry
}
// Test cases
tests := []struct {
name string
batches [][]batch
expected []batch
}{
{
name: "add two streams",
batches: [][]batch{
{
{
labels: "foo",
tenant: "tenant1",
entries: []*logproto.Entry{
{Timestamp: time.Unix(1, 0), Line: "Entry 1"},
{Timestamp: time.Unix(3, 0), Line: "Entry 3"},
},
},
{
labels: "bar",
tenant: "tenant1",
entries: []*logproto.Entry{
{Timestamp: time.Unix(2, 0), Line: "Entry 2"},
{Timestamp: time.Unix(3, 0), Line: "Entry 3"},
},
},
},
{
{
labels: "foo",
tenant: "tenant1",
entries: []*logproto.Entry{
{Timestamp: time.Unix(2, 0), Line: "Entry 2"},
{Timestamp: time.Unix(3, 0), Line: "Entry 3"},
},
},
{
labels: "bar",
tenant: "tenant1",
entries: []*logproto.Entry{
{Timestamp: time.Unix(1, 0), Line: "Entry 1"},
},
},
},
},
expected: []batch{
{
labels: "foo",
tenant: "tenant1",
entries: []*logproto.Entry{
{Timestamp: time.Unix(1, 0), Line: "Entry 1"},
{Timestamp: time.Unix(2, 0), Line: "Entry 2"},
{Timestamp: time.Unix(3, 0), Line: "Entry 3"},
{Timestamp: time.Unix(3, 0), Line: "Entry 3"},
},
},
{
labels: "bar",
tenant: "tenant1",
entries: []*logproto.Entry{
{Timestamp: time.Unix(1, 0), Line: "Entry 1"},
{Timestamp: time.Unix(2, 0), Line: "Entry 2"},
{Timestamp: time.Unix(3, 0), Line: "Entry 3"},
},
},
},
},
}
// Run the test cases
for _, tt := range tests {
tt := tt
t.Run(tt.name, func(t *testing.T) {
t.Parallel()
// Create a new WalSegmentWriter
w := NewWalSegmentWriter()
// Append the entries
for _, batch := range tt.batches {
for _, stream := range batch {
labels, err := syntax.ParseLabels(stream.labels)
require.NoError(t, err)
w.Append(stream.tenant, stream.labels, labels, stream.entries)
}
}
require.NotEmpty(t, tt.expected, "expected entries are empty")
// Check the entries
for _, expected := range tt.expected {
stream, ok := w.streams.Get(streamID{labels: expected.labels, tenant: expected.tenant})
require.True(t, ok)
lbs, err := syntax.ParseLabels(expected.labels)
require.NoError(t, err)
lbs = append(lbs, labels.Label{Name: string(tsdb.TenantLabel), Value: expected.tenant})
sort.Sort(lbs)
require.Equal(t, lbs, stream.lbls)
require.Equal(t, expected.entries, stream.entries)
}
})
}
}
func TestMultiTenantWrite(t *testing.T) {
w := NewWalSegmentWriter()
dst := bytes.NewBuffer(nil)
lbls := []labels.Labels{
labels.FromStrings("container", "foo", "namespace", "dev"),
labels.FromStrings("container", "bar", "namespace", "staging"),
labels.FromStrings("container", "bar", "namespace", "prod"),
}
tenants := []string{"z", "c", "a", "b"}
for _, tenant := range tenants {
for _, lbl := range lbls {
lblString := lbl.String()
for i := 0; i < 10; i++ {
w.Append(tenant, lblString, lbl, []*push.Entry{
{Timestamp: time.Unix(0, int64(i)), Line: fmt.Sprintf("log line %d", i)},
})
}
}
}
n, err := w.WriteTo(dst)
require.NoError(t, err)
require.True(t, n > 0)
r, err := NewReader(dst.Bytes())
require.NoError(t, err)
iter, err := r.Series(context.Background())
require.NoError(t, err)
var expectedSeries, actualSeries []string
for _, tenant := range tenants {
for _, lbl := range lbls {
expectedSeries = append(expectedSeries, labels.NewBuilder(lbl).Set(tsdb.TenantLabel, tenant).Labels().String())
}
}
for iter.Next() {
actualSeries = append(actualSeries, iter.At().String())
chk, err := iter.ChunkReader(nil)
require.NoError(t, err)
// verify all lines
var i int
for chk.Next() {
ts, line := chk.At()
require.Equal(t, int64(i), ts)
require.Equal(t, fmt.Sprintf("log line %d", i), string(line))
i++
}
require.NoError(t, chk.Err())
require.NoError(t, chk.Close())
require.Equal(t, 10, i)
}
require.NoError(t, iter.Err())
require.ElementsMatch(t, expectedSeries, actualSeries)
}

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

@ -0,0 +1,210 @@
{"duration":20950259,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:03:32Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"q62wCcIkEOueqFKF","level":"debug","msg":"check capacity complete","time":"2024-04-16T15:03:28Z"}
{"id":"q62wCcIkEOueqFKF","level":"debug","msg":"no capacity changes required","time":"2024-04-16T15:03:28Z"}
{"id":"q62wCcIkEOueqFKF","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T15:03:28Z"}
{"id":"q62wCcIkEOueqFKF","level":"debug","msg":"calculate server capacity","time":"2024-04-16T15:03:28Z"}
{"id":"q62wCcIkEOueqFKF","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T15:03:28Z"}
{"duration":24479472,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:03:17Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":56560748,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:03:03Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"m6SpYHzdXrDAFqDR","level":"debug","msg":"check capacity complete","time":"2024-04-16T15:02:58Z"}
{"id":"m6SpYHzdXrDAFqDR","level":"debug","msg":"no capacity changes required","time":"2024-04-16T15:02:58Z"}
{"id":"m6SpYHzdXrDAFqDR","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T15:02:58Z"}
{"id":"m6SpYHzdXrDAFqDR","level":"debug","msg":"calculate server capacity","time":"2024-04-16T15:02:58Z"}
{"id":"m6SpYHzdXrDAFqDR","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T15:02:58Z"}
{"duration":36437134,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:02:47Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":36704414,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:02:32Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"T0I8Dsnw3uSi3Gal","level":"debug","msg":"check capacity complete","time":"2024-04-16T15:02:28Z"}
{"id":"T0I8Dsnw3uSi3Gal","level":"debug","msg":"no capacity changes required","time":"2024-04-16T15:02:28Z"}
{"id":"T0I8Dsnw3uSi3Gal","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T15:02:28Z"}
{"id":"T0I8Dsnw3uSi3Gal","level":"debug","msg":"calculate server capacity","time":"2024-04-16T15:02:28Z"}
{"id":"T0I8Dsnw3uSi3Gal","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T15:02:28Z"}
{"duration":49120197,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:02:18Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":69764560,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:02:03Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"9eA72xOtx8kzMhXn","level":"debug","msg":"check capacity complete","time":"2024-04-16T15:01:58Z"}
{"id":"9eA72xOtx8kzMhXn","level":"debug","msg":"no capacity changes required","time":"2024-04-16T15:01:58Z"}
{"id":"9eA72xOtx8kzMhXn","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T15:01:58Z"}
{"id":"9eA72xOtx8kzMhXn","level":"debug","msg":"calculate server capacity","time":"2024-04-16T15:01:58Z"}
{"id":"9eA72xOtx8kzMhXn","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T15:01:58Z"}
{"duration":93395725,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:01:48Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":27353687,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:01:32Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"pet7QVfO1yE8fk56","level":"debug","msg":"check capacity complete","time":"2024-04-16T15:01:28Z"}
{"id":"pet7QVfO1yE8fk56","level":"debug","msg":"no capacity changes required","time":"2024-04-16T15:01:28Z"}
{"id":"pet7QVfO1yE8fk56","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T15:01:28Z"}
{"id":"pet7QVfO1yE8fk56","level":"debug","msg":"calculate server capacity","time":"2024-04-16T15:01:27Z"}
{"id":"pet7QVfO1yE8fk56","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T15:01:27Z"}
{"duration":21783864,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:01:17Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":27700228,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:01:02Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"15eSzaEG0enf86Kl","level":"debug","msg":"check capacity complete","time":"2024-04-16T15:00:57Z"}
{"id":"15eSzaEG0enf86Kl","level":"debug","msg":"no capacity changes required","time":"2024-04-16T15:00:57Z"}
{"id":"15eSzaEG0enf86Kl","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T15:00:57Z"}
{"id":"15eSzaEG0enf86Kl","level":"debug","msg":"calculate server capacity","time":"2024-04-16T15:00:57Z"}
{"id":"15eSzaEG0enf86Kl","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T15:00:57Z"}
{"duration":20800183,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:00:47Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":114305939,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:00:33Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"JO1OT5ADoNA8NYqr","level":"debug","msg":"check capacity complete","time":"2024-04-16T15:00:27Z"}
{"id":"JO1OT5ADoNA8NYqr","level":"debug","msg":"no capacity changes required","time":"2024-04-16T15:00:27Z"}
{"id":"JO1OT5ADoNA8NYqr","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T15:00:27Z"}
{"id":"JO1OT5ADoNA8NYqr","level":"debug","msg":"calculate server capacity","time":"2024-04-16T15:00:27Z"}
{"id":"JO1OT5ADoNA8NYqr","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T15:00:27Z"}
{"duration":118009488,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:00:18Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":33663734,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T15:00:02Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"Xz2OCJhgeBSRFyoN","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:59:57Z"}
{"id":"Xz2OCJhgeBSRFyoN","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:59:57Z"}
{"id":"Xz2OCJhgeBSRFyoN","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:59:57Z"}
{"id":"Xz2OCJhgeBSRFyoN","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:59:57Z"}
{"id":"Xz2OCJhgeBSRFyoN","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:59:57Z"}
{"duration":48520224,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:59:48Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":25172573,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:59:32Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"pPc2ORUhHAhFgBg3","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:59:27Z"}
{"id":"pPc2ORUhHAhFgBg3","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:59:27Z"}
{"id":"pPc2ORUhHAhFgBg3","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:59:27Z"}
{"id":"pPc2ORUhHAhFgBg3","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:59:27Z"}
{"id":"pPc2ORUhHAhFgBg3","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:59:27Z"}
{"duration":215980677,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:59:18Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":58239998,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:59:03Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"4G6Srn6lSwzYrx19","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:58:57Z"}
{"id":"4G6Srn6lSwzYrx19","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:58:57Z"}
{"id":"4G6Srn6lSwzYrx19","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:58:57Z"}
{"id":"4G6Srn6lSwzYrx19","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:58:57Z"}
{"id":"4G6Srn6lSwzYrx19","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:58:57Z"}
{"duration":33198841,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:58:47Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":38097220,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:58:32Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"1Lu90T1fWzsWOKlc","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:58:27Z"}
{"id":"1Lu90T1fWzsWOKlc","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:58:27Z"}
{"id":"1Lu90T1fWzsWOKlc","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:58:27Z"}
{"id":"1Lu90T1fWzsWOKlc","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:58:27Z"}
{"id":"1Lu90T1fWzsWOKlc","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:58:27Z"}
{"duration":63956469,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:58:18Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":125762940,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:58:03Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"4XjwwNoOwZFaWePQ","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:57:57Z"}
{"id":"4XjwwNoOwZFaWePQ","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:57:57Z"}
{"id":"4XjwwNoOwZFaWePQ","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:57:57Z"}
{"id":"4XjwwNoOwZFaWePQ","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:57:57Z"}
{"id":"4XjwwNoOwZFaWePQ","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:57:56Z"}
{"duration":33644691,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:57:47Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":24328424,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:57:32Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"IQy23J3NON0BV10V","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:57:26Z"}
{"id":"IQy23J3NON0BV10V","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:57:26Z"}
{"id":"IQy23J3NON0BV10V","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:57:26Z"}
{"id":"IQy23J3NON0BV10V","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:57:26Z"}
{"id":"IQy23J3NON0BV10V","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:57:26Z"}
{"duration":20257078,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:57:17Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":28190860,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:57:02Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"FQ8wCQfaR9W387cH","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:56:56Z"}
{"id":"FQ8wCQfaR9W387cH","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:56:56Z"}
{"id":"FQ8wCQfaR9W387cH","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:56:56Z"}
{"id":"FQ8wCQfaR9W387cH","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:56:56Z"}
{"id":"FQ8wCQfaR9W387cH","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:56:56Z"}
{"duration":124540195,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:56:48Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":33392901,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:56:32Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"Hhwn7ecXjxF67DG6","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:56:26Z"}
{"id":"Hhwn7ecXjxF67DG6","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:56:26Z"}
{"id":"Hhwn7ecXjxF67DG6","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:56:26Z"}
{"id":"Hhwn7ecXjxF67DG6","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:56:26Z"}
{"id":"Hhwn7ecXjxF67DG6","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:56:26Z"}
{"duration":38704940,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:56:18Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":41929762,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:56:03Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"luflyGZvZnLzhQEH","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:55:56Z"}
{"id":"luflyGZvZnLzhQEH","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:55:56Z"}
{"id":"luflyGZvZnLzhQEH","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:55:56Z"}
{"id":"luflyGZvZnLzhQEH","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:55:56Z"}
{"id":"luflyGZvZnLzhQEH","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:55:56Z"}
{"duration":20307958,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:55:47Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":20185884,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:55:32Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"q20GZcvyzMwrTGx5","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:55:26Z"}
{"id":"q20GZcvyzMwrTGx5","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:55:26Z"}
{"id":"q20GZcvyzMwrTGx5","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:55:26Z"}
{"id":"q20GZcvyzMwrTGx5","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:55:26Z"}
{"id":"q20GZcvyzMwrTGx5","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:55:26Z"}
{"duration":62732272,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:55:18Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":233876906,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:55:03Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"3K61Yf6ImKYexoFx","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:54:56Z"}
{"id":"3K61Yf6ImKYexoFx","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:54:56Z"}
{"id":"3K61Yf6ImKYexoFx","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:54:56Z"}
{"id":"3K61Yf6ImKYexoFx","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:54:56Z"}
{"id":"3K61Yf6ImKYexoFx","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:54:53Z"}
{"duration":91919807,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:54:48Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":123141860,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:54:33Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"SmbOO0l5aADX9BaQ","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:54:23Z"}
{"id":"SmbOO0l5aADX9BaQ","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:54:23Z"}
{"id":"SmbOO0l5aADX9BaQ","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:54:23Z"}
{"id":"SmbOO0l5aADX9BaQ","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:54:23Z"}
{"id":"SmbOO0l5aADX9BaQ","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:54:23Z"}
{"duration":127118328,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:54:18Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":154732993,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:54:03Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"96TvvsMzSkkaW8oW","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:53:53Z"}
{"id":"96TvvsMzSkkaW8oW","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:53:53Z"}
{"id":"96TvvsMzSkkaW8oW","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:53:53Z"}
{"id":"96TvvsMzSkkaW8oW","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:53:53Z"}
{"id":"96TvvsMzSkkaW8oW","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:53:53Z"}
{"duration":118361504,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:53:48Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":137758190,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:53:33Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"C7aYn8cb4NCrkkYI","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:53:23Z"}
{"id":"C7aYn8cb4NCrkkYI","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:53:23Z"}
{"id":"C7aYn8cb4NCrkkYI","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:53:23Z"}
{"id":"C7aYn8cb4NCrkkYI","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:53:23Z"}
{"id":"C7aYn8cb4NCrkkYI","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:53:23Z"}
{"duration":68794582,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:53:18Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":74435188,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:53:03Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"CMG7ZwwYqNPBonAn","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:52:53Z"}
{"id":"CMG7ZwwYqNPBonAn","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:52:53Z"}
{"id":"CMG7ZwwYqNPBonAn","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:52:53Z"}
{"id":"CMG7ZwwYqNPBonAn","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:52:53Z"}
{"id":"CMG7ZwwYqNPBonAn","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:52:53Z"}
{"duration":50623103,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:52:48Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":55970830,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:52:33Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"focV9BzODwRbWwKE","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:52:23Z"}
{"id":"focV9BzODwRbWwKE","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:52:23Z"}
{"id":"focV9BzODwRbWwKE","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:52:23Z"}
{"id":"focV9BzODwRbWwKE","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:52:23Z"}
{"id":"focV9BzODwRbWwKE","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:52:23Z"}
{"duration":33285760,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:52:17Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":18306142,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:52:02Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"HphRnJOM8uYohf1p","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:51:53Z"}
{"id":"HphRnJOM8uYohf1p","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:51:53Z"}
{"id":"HphRnJOM8uYohf1p","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:51:53Z"}
{"id":"HphRnJOM8uYohf1p","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:51:53Z"}
{"id":"HphRnJOM8uYohf1p","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:51:53Z"}
{"duration":174783742,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:51:48Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":31536414,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:51:32Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"m3n8GndhG45uGIQA","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:51:23Z"}
{"id":"m3n8GndhG45uGIQA","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:51:23Z"}
{"id":"m3n8GndhG45uGIQA","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:51:23Z"}
{"id":"m3n8GndhG45uGIQA","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:51:22Z"}
{"id":"m3n8GndhG45uGIQA","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:51:22Z"}
{"duration":119507962,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:51:18Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":430410510,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:51:03Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"nTO38tWtnvRWRl1G","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:50:52Z"}
{"id":"nTO38tWtnvRWRl1G","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:50:52Z"}
{"id":"nTO38tWtnvRWRl1G","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:50:52Z"}
{"id":"nTO38tWtnvRWRl1G","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:50:52Z"}
{"id":"nTO38tWtnvRWRl1G","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:50:52Z"}
{"duration":23638914,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:50:47Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":20300837,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:50:32Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"5qEIzErDfiALVPAN","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:50:22Z"}
{"id":"5qEIzErDfiALVPAN","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:50:22Z"}
{"id":"5qEIzErDfiALVPAN","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:50:22Z"}
{"id":"5qEIzErDfiALVPAN","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:50:22Z"}
{"id":"5qEIzErDfiALVPAN","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:50:22Z"}
{"duration":33318175,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:50:17Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":19013978,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:50:02Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"q61oHTtF4MMiQVGH","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:49:52Z"}
{"id":"q61oHTtF4MMiQVGH","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:49:52Z"}
{"id":"q61oHTtF4MMiQVGH","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:49:52Z"}
{"id":"q61oHTtF4MMiQVGH","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:49:52Z"}
{"id":"q61oHTtF4MMiQVGH","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:49:52Z"}
{"duration":19815267,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:49:47Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":21037324,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:49:32Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"4rNxIlhDKxGgzBHe","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:49:22Z"}
{"id":"4rNxIlhDKxGgzBHe","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:49:22Z"}
{"id":"4rNxIlhDKxGgzBHe","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:49:22Z"}
{"id":"4rNxIlhDKxGgzBHe","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:49:22Z"}
{"id":"4rNxIlhDKxGgzBHe","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:49:22Z"}
{"duration":20219085,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:49:17Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"duration":24085452,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:49:02Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}
{"id":"D4Oh1ivB6cdLWa08","level":"debug","msg":"check capacity complete","time":"2024-04-16T14:48:52Z"}
{"id":"D4Oh1ivB6cdLWa08","level":"debug","msg":"no capacity changes required","time":"2024-04-16T14:48:52Z"}
{"id":"D4Oh1ivB6cdLWa08","level":"debug","max-pool":4,"min-pool":0,"msg":"check capacity","pending-builds":0,"running-builds":0,"server-buffer":0,"server-capacity":0,"server-count":0,"time":"2024-04-16T14:48:52Z"}
{"id":"D4Oh1ivB6cdLWa08","level":"debug","msg":"calculate server capacity","time":"2024-04-16T14:48:52Z"}
{"id":"D4Oh1ivB6cdLWa08","level":"debug","msg":"calculate unfinished jobs","time":"2024-04-16T14:48:52Z"}
{"duration":51986975,"level":"debug","method":"GET","msg":"request completed","referer":"","remote":"10.136.105.40:52702","request":"/metrics","status":200,"time":"2024-04-16T14:48:48Z","user-agent":"GrafanaAgent/v0.40.3 (flow; linux; helm)"}

@ -0,0 +1,100 @@
ts=2024-04-17T09:52:46.364803306Z caller=http.go:194 level=debug traceID=1ee220e024eb5b0a orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.368313ms"
ts=2024-04-17T09:52:46.364804708Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.619763ms"
ts=2024-04-17T09:52:46.364730588Z caller=http.go:194 level=debug traceID=116a1a427c28ef7c orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 944.101µs"
ts=2024-04-17T09:52:46.364808055Z caller=http.go:194 level=debug traceID=126a4a4108de5ce6 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 595.34µs"
ts=2024-04-17T09:52:46.364790582Z caller=http.go:194 level=debug traceID=126a4a4108de5ce6 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 845.635µs"
ts=2024-04-17T09:52:46.36478937Z caller=http.go:194 level=debug traceID=285eaac96a357b61 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 677.201µs"
ts=2024-04-17T09:52:46.36470023Z caller=http.go:194 level=debug traceID=4da0ab0295ab6c16 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.702772ms"
ts=2024-04-17T09:52:46.364312973Z caller=http.go:194 level=debug traceID=241f9222b132d55f orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.305663ms"
ts=2024-04-17T09:52:46.364492298Z caller=http.go:194 level=debug traceID=285eaac96a357b61 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 672.113µs"
ts=2024-04-17T09:52:46.36471937Z caller=http.go:194 level=debug traceID=6ff6d0cc340ca6e7 orgID=32133 msg="POST /ingester.v1.IngesterService/Push (200) 1.822532ms"
ts=2024-04-17T09:52:46.364561196Z caller=http.go:194 level=debug traceID=4da0ab0295ab6c16 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.468961ms"
ts=2024-04-17T09:52:46.36477128Z caller=head.go:216 level=debug tenant=987678 msg="profile is empty after delta computation" metricName=memory
ts=2024-04-17T09:52:46.364749251Z caller=http.go:194 level=debug traceID=6b72cb766bf5191f orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 8.508724ms"
ts=2024-04-17T09:52:46.364726948Z caller=head.go:216 level=debug tenant=987678 msg="profile is empty after delta computation" metricName=memory
ts=2024-04-17T09:52:46.364753592Z caller=http.go:194 level=debug traceID=4d4de2711f38c05c orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 2.564507ms"
ts=2024-04-17T09:52:46.364737024Z caller=http.go:194 level=debug traceID=285eaac96a357b61 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 967.106µs"
ts=2024-04-17T09:52:46.364734092Z caller=http.go:194 level=debug traceID=345399fe6745a50c orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 10.63998ms"
ts=2024-04-17T09:52:46.364684284Z caller=http.go:194 level=debug traceID=6141b48c238b3d5e orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 839.55µs"
ts=2024-04-17T09:52:46.364649689Z caller=http.go:194 level=debug traceID=285eaac96a357b61 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 783.914µs"
ts=2024-04-17T09:52:46.364650688Z caller=http.go:194 level=debug traceID=568f48b07fd4c3ef orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 874.107µs"
ts=2024-04-17T09:52:46.364616184Z caller=http.go:194 level=debug traceID=2b1bf144cf74bf28 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 17.261341ms"
ts=2024-04-17T09:52:46.364595982Z caller=http.go:194 level=debug traceID=6141b48c238b3d5e orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 914.739µs"
ts=2024-04-17T09:52:46.364584929Z caller=http.go:194 level=debug traceID=5b8bca4cdba21f43 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 2.142724ms"
ts=2024-04-17T09:52:46.364590656Z caller=http.go:194 level=debug traceID=6141b48c238b3d5e orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 895.782µs"
ts=2024-04-17T09:52:46.364657596Z caller=http.go:194 level=debug traceID=6141b48c238b3d5e orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.368331ms"
ts=2024-04-17T09:52:46.364622325Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 660.247µs"
ts=2024-04-17T09:52:46.364594116Z caller=http.go:194 level=debug traceID=1ee220e024eb5b0a orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 467.285µs"
ts=2024-04-17T09:52:46.364545621Z caller=http.go:194 level=debug traceID=1ee220e024eb5b0a orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 530.452µs"
ts=2024-04-17T09:52:46.36450376Z caller=http.go:194 level=debug traceID=74f2e8cf7e0c7fc7 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.135674ms"
ts=2024-04-17T09:52:46.36443397Z caller=http.go:194 level=debug traceID=6fb0c4a951fa6f0f orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.398122ms"
ts=2024-04-17T09:52:46.364464147Z caller=http.go:194 level=debug traceID=6141b48c238b3d5e orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.131431ms"
ts=2024-04-17T09:52:46.364440826Z caller=http.go:194 level=debug traceID=34467dc44e314b02 orgID=325123 msg="POST /ingester.v1.IngesterService/Push (200) 1.023612ms"
ts=2024-04-17T09:52:46.36432296Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.408666ms"
ts=2024-04-17T09:52:46.364405022Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.341907ms"
ts=2024-04-17T09:52:46.36444769Z caller=http.go:194 level=debug traceID=568f48b07fd4c3ef orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 632.404µs"
ts=2024-04-17T09:52:46.364429591Z caller=http.go:194 level=debug traceID=126a4a4108de5ce6 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 478.714µs"
ts=2024-04-17T09:52:46.364341351Z caller=http.go:194 level=debug traceID=4d4de2711f38c05c orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 2.09688ms"
ts=2024-04-17T09:52:46.364357027Z caller=http.go:194 level=debug traceID=4da0ab0295ab6c16 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 763.325µs"
ts=2024-04-17T09:52:46.364363024Z caller=http.go:194 level=debug traceID=6ff6d0cc340ca6e7 orgID=32133 msg="POST /ingester.v1.IngesterService/Push (200) 1.501374ms"
ts=2024-04-17T09:52:46.364308133Z caller=http.go:194 level=debug traceID=6fb0c4a951fa6f0f orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.102861ms"
ts=2024-04-17T09:52:46.36423841Z caller=http.go:194 level=debug traceID=071fd5ad1d0d0515 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.527951ms"
ts=2024-04-17T09:52:46.364312232Z caller=http.go:194 level=debug traceID=126a4a4108de5ce6 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 858.218µs"
ts=2024-04-17T09:52:46.36428579Z caller=http.go:194 level=debug traceID=6141b48c238b3d5e orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 492.16µs"
ts=2024-04-17T09:52:46.361802864Z caller=http.go:194 level=debug traceID=744100a66cc00de4 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.011408ms"
ts=2024-04-17T09:52:46.364289781Z caller=http.go:194 level=debug traceID=285eaac96a357b61 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 502.847µs"
ts=2024-04-17T09:52:46.364200562Z caller=http.go:194 level=debug traceID=62968dddfc09bbbe orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.494135ms"
ts=2024-04-17T09:52:46.364282918Z caller=http.go:194 level=debug traceID=6141b48c238b3d5e orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 704.283µs"
ts=2024-04-17T09:52:46.364279378Z caller=http.go:194 level=debug traceID=6141b48c238b3d5e orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 764.795µs"
ts=2024-04-17T09:52:46.364217907Z caller=http.go:194 level=debug traceID=1ee220e024eb5b0a orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 485.357µs"
ts=2024-04-17T09:52:46.364219229Z caller=http.go:194 level=debug traceID=6141b48c238b3d5e orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 837.055µs"
ts=2024-04-17T09:52:46.364112209Z caller=http.go:194 level=debug traceID=635516613bdc1d25 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 679.644µs"
ts=2024-04-17T09:52:46.364155917Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.279766ms"
ts=2024-04-17T09:52:46.364081008Z caller=http.go:194 level=debug traceID=64b8a1a256934690 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 3.267758ms"
ts=2024-04-17T09:52:46.364189774Z caller=http.go:194 level=debug traceID=34467dc44e314b02 orgID=325123 msg="POST /ingester.v1.IngesterService/Push (200) 880.75µs"
ts=2024-04-17T09:52:46.364145376Z caller=http.go:194 level=debug traceID=70121da20778353f orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.053066ms"
ts=2024-04-17T09:52:46.364093541Z caller=http.go:194 level=debug traceID=4da0ab0295ab6c16 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 660.622µs"
ts=2024-04-17T09:52:46.364068342Z caller=http.go:194 level=debug traceID=3dc61701564a23f7 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 5.097204ms"
ts=2024-04-17T09:52:46.36353869Z caller=http.go:194 level=debug traceID=4a7fddbe653f5299 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 2.804943ms"
ts=2024-04-17T09:52:46.362372357Z caller=http.go:194 level=debug traceID=0f348e7d30d12cff orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 2.009358ms"
ts=2024-04-17T09:52:46.364063967Z caller=http.go:194 level=debug traceID=4da0ab0295ab6c16 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.368053ms"
ts=2024-04-17T09:52:46.364085113Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 918.778µs"
ts=2024-04-17T09:52:46.364063374Z caller=http.go:194 level=debug traceID=1ee220e024eb5b0a orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 370.137µs"
ts=2024-04-17T09:52:46.358938595Z caller=http.go:194 level=debug traceID=694c1939d3e1fbc7 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 926.871µs"
ts=2024-04-17T09:52:46.364006323Z caller=http.go:194 level=debug traceID=241f9222b132d55f orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.061869ms"
ts=2024-04-17T09:52:46.363969935Z caller=http.go:194 level=debug traceID=4da0ab0295ab6c16 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 773.863µs"
ts=2024-04-17T09:52:46.36401476Z caller=http.go:194 level=debug traceID=46ecede10c2603f5 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 3.392645ms"
ts=2024-04-17T09:52:46.363873777Z caller=http.go:194 level=debug traceID=71aeed5818159731 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 805.376µs"
ts=2024-04-17T09:52:46.363974185Z caller=http.go:194 level=debug traceID=1b48f5156a61ca69 msg="GET /debug/pprof/delta_mutex (200) 1.161082ms"
ts=2024-04-17T09:52:46.363789152Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 764.051µs"
ts=2024-04-17T09:52:46.362529898Z caller=http.go:194 level=debug traceID=635516613bdc1d25 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 790.475µs"
ts=2024-04-17T09:52:46.364031457Z caller=http.go:194 level=debug traceID=126a4a4108de5ce6 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 476.579µs"
ts=2024-04-17T09:52:46.361323896Z caller=http.go:194 level=debug traceID=599c62a3ac7de78a orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.331755ms"
ts=2024-04-17T09:52:46.363943545Z caller=http.go:194 level=debug traceID=4da0ab0295ab6c16 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 582.609µs"
ts=2024-04-17T09:52:46.363907661Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 844.336µs"
ts=2024-04-17T09:52:46.363979594Z caller=http.go:194 level=debug traceID=6ff6d0cc340ca6e7 orgID=32133 msg="POST /ingester.v1.IngesterService/Push (200) 1.315545ms"
ts=2024-04-17T09:52:46.363939502Z caller=http.go:194 level=debug traceID=34467dc44e314b02 orgID=325123 msg="POST /ingester.v1.IngesterService/Push (200) 758.566µs"
ts=2024-04-17T09:52:46.363924229Z caller=http.go:194 level=debug traceID=1ee220e024eb5b0a orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 443.231µs"
ts=2024-04-17T09:52:46.363882565Z caller=http.go:194 level=debug traceID=71aeed5818159731 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 746.182µs"
ts=2024-04-17T09:52:46.363850696Z caller=http.go:194 level=debug traceID=6141b48c238b3d5e orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 487.255µs"
ts=2024-04-17T09:52:46.363769122Z caller=http.go:194 level=debug traceID=04ece9ea52939418 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 14.805487ms"
ts=2024-04-17T09:52:46.363816282Z caller=http.go:194 level=debug traceID=0052911cab18a35d orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 4.99224ms"
ts=2024-04-17T09:52:46.363754816Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 931.934µs"
ts=2024-04-17T09:52:46.363705611Z caller=http.go:194 level=debug traceID=126a4a4108de5ce6 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 744.005µs"
ts=2024-04-17T09:52:46.36377788Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 956.91µs"
ts=2024-04-17T09:52:46.363732729Z caller=http.go:194 level=debug traceID=6e6e579dac547044 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.036865ms"
ts=2024-04-17T09:52:46.36376503Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 851.996µs"
ts=2024-04-17T09:52:46.36375519Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 845.405µs"
ts=2024-04-17T09:52:46.363674007Z caller=http.go:194 level=debug traceID=0b0e675689e15909 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.177299ms"
ts=2024-04-17T09:52:46.363672895Z caller=http.go:194 level=debug traceID=554713b69cd1473a orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.611307ms"
ts=2024-04-17T09:52:46.36371749Z caller=http.go:194 level=debug traceID=70121da20778353f orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 846.904µs"
ts=2024-04-17T09:52:46.363695016Z caller=http.go:194 level=debug traceID=126a4a4108de5ce6 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 529.001µs"
ts=2024-04-17T09:52:46.363644321Z caller=http.go:194 level=debug traceID=6141b48c238b3d5e orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 440.298µs"
ts=2024-04-17T09:52:46.363603421Z caller=http.go:194 level=debug traceID=748df0ddf0b0f22d orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.562106ms"
ts=2024-04-17T09:52:46.363685875Z caller=http.go:194 level=debug traceID=71aeed5818159731 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 701.457µs"
ts=2024-04-17T09:52:46.363659588Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 729.084µs"
ts=2024-04-17T09:52:46.363561079Z caller=http.go:194 level=debug traceID=635516613bdc1d25 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 870.913µs"
ts=2024-04-17T09:52:46.363625435Z caller=http.go:194 level=debug traceID=2f549a41e3a5eb00 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 560.428µs"
ts=2024-04-17T09:52:46.36352246Z caller=http.go:194 level=debug traceID=0650b8c486d05d36 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.104667ms"
ts=2024-04-17T09:52:46.363534538Z caller=http.go:194 level=debug traceID=5269eebfe73cfedb orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 1.110652ms"
ts=2024-04-17T09:52:46.363579783Z caller=http.go:194 level=debug traceID=64b8a1a256934690 orgID=12372 msg="POST /ingester.v1.IngesterService/Push (200) 3.154879ms"

File diff suppressed because it is too large Load Diff

File diff suppressed because one or more lines are too long

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff
Loading…
Cancel
Save