From bbfb13c3e4ddec80eee70bef042a55d8b9d12c3e Mon Sep 17 00:00:00 2001 From: Kaviraj Kanagaraj Date: Thu, 24 Aug 2023 09:52:50 +0200 Subject: [PATCH] Pin `chunk` and `index` format to `schema` version. (#10213) We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema` version in period config. This is the following mapping (after being discussed with @owen-d and @sandeepsukhani ) `v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) + TSDBv2 `v13` (introducing new schema) - ChunkFormatV4 (UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3 Note the new schema `v13` supports the latest chunk and index format. **NOTES for Reviewer** 1. General approach is we removed the idea of `index.LiveFormat`, `chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and made following two changes. These variables were used before to tie chunk and tsdb formats specific to Loki versions. This PR remove that coupling and pin these formats to `schema` version instead. 1. These variables were replaced with explicit chunk and index formats within those packages (and it's tests) 2. If these variables were used outside it's own packages say by ingester, compactor, etc. Then we extract correct chunk and index versions from the `schema` config. 2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb format tied to schema. 2. Other ideas I thought of doing but didn't end up doing is make `ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and `int` currently. Similar to `HeadBlockFmt` type). But didnt' do it eventually to keep the PR small and don't want to complicate with lots of changes. 4. Moved couple of test cases from `chunkenc` to `config` package, because the test case was actually testing methods on `schemaconfig` and it was creating cycling dependencies. --------- Signed-off-by: Kaviraj --- pkg/chunkenc/memchunk.go | 84 +- pkg/chunkenc/memchunk_test.go | 143 +- pkg/chunkenc/unordered_test.go | 18 +- pkg/chunkenc/util_test.go | 2 +- pkg/ingester/checkpoint.go | 8 +- pkg/ingester/checkpoint_test.go | 2 +- pkg/ingester/chunk_test.go | 2 +- pkg/ingester/encoding_test.go | 8 +- pkg/ingester/flush_test.go | 2 +- pkg/ingester/instance.go | 77 +- pkg/ingester/instance_test.go | 11 +- pkg/ingester/recovery.go | 2 +- pkg/ingester/stream.go | 33 +- pkg/ingester/stream_test.go | 49 +- pkg/ingester/streams_map_test.go | 5 + pkg/storage/async_store_test.go | 18 +- pkg/storage/batch_test.go | 1915 +++++++++-------- pkg/storage/chunk/chunk_test.go | 121 +- pkg/storage/config/schema_config.go | 37 + pkg/storage/config/schema_config_test.go | 123 ++ pkg/storage/hack/main.go | 23 +- pkg/storage/lazy_chunk_test.go | 84 +- pkg/storage/store_test.go | 147 +- .../compactor/retention/retention_test.go | 4 +- .../compactor/retention/util_test.go | 12 + pkg/storage/stores/series/index/schema.go | 4 + .../stores/series/index/schema_config.go | 12 +- pkg/storage/stores/series_store_write_test.go | 17 +- .../index/compactor/compacted_index_test.go | 12 +- .../shipper/index/compactor/iterator_test.go | 33 +- .../stores/shipper/index/compactor/util.go | 4 +- pkg/storage/stores/tsdb/builder_test.go | 4 +- pkg/storage/stores/tsdb/compactor.go | 25 +- pkg/storage/stores/tsdb/compactor_test.go | 38 +- pkg/storage/stores/tsdb/head_manager_test.go | 146 +- pkg/storage/stores/tsdb/index/index.go | 9 +- pkg/storage/stores/tsdb/index/index_test.go | 8 +- .../stores/tsdb/index_shipper_querier.go | 2 +- pkg/storage/stores/tsdb/manager.go | 27 +- pkg/storage/stores/tsdb/querier_test.go | 2 +- pkg/storage/stores/tsdb/util_test.go | 2 +- pkg/storage/util_test.go | 18 +- tools/tsdb/migrate-versions/main.go | 2 +- tools/tsdb/tsdb-map/main.go | 7 +- 44 files changed, 1907 insertions(+), 1395 deletions(-) diff --git a/pkg/chunkenc/memchunk.go b/pkg/chunkenc/memchunk.go index 90d8ef8d32..ffcde05cf6 100644 --- a/pkg/chunkenc/memchunk.go +++ b/pkg/chunkenc/memchunk.go @@ -28,12 +28,10 @@ import ( const ( _ byte = iota - chunkFormatV1 - chunkFormatV2 - chunkFormatV3 - chunkFormatV4 - - DefaultChunkFormat = chunkFormatV4 // the currently used chunk format + ChunkFormatV1 + ChunkFormatV2 + ChunkFormatV3 + ChunkFormatV4 blocksPerChunk = 10 maxLineLength = 1024 * 1024 * 1024 @@ -84,10 +82,22 @@ const ( OrderedHeadBlockFmt UnorderedHeadBlockFmt UnorderedWithNonIndexedLabelsHeadBlockFmt - - DefaultHeadBlockFmt = UnorderedWithNonIndexedLabelsHeadBlockFmt ) +// ChunkHeadFormatFor returns corresponding head block format for the given `chunkfmt`. +func ChunkHeadFormatFor(chunkfmt byte) HeadBlockFmt { + if chunkfmt < ChunkFormatV3 { + return OrderedHeadBlockFmt + } + + if chunkfmt == ChunkFormatV3 { + return UnorderedHeadBlockFmt + } + + // return the latest head format for all chunkformat >v3 + return UnorderedWithNonIndexedLabelsHeadBlockFmt +} + var magicNumber = uint32(0x12EE56A) // The table gets initialized with sync.Once but may still cause a race @@ -293,7 +303,7 @@ func (hb *headBlock) LoadBytes(b []byte) error { return errors.Wrap(db.err(), "verifying headblock header") } switch version { - case chunkFormatV1, chunkFormatV2, chunkFormatV3, chunkFormatV4: + case ChunkFormatV1, ChunkFormatV2, ChunkFormatV3, ChunkFormatV4: default: return errors.Errorf("incompatible headBlock version (%v), only V1,V2,V3 is currently supported", version) } @@ -344,15 +354,16 @@ type entry struct { } // NewMemChunk returns a new in-mem chunk. -func NewMemChunk(enc Encoding, head HeadBlockFmt, blockSize, targetSize int) *MemChunk { - return newMemChunkWithFormat(DefaultChunkFormat, enc, head, blockSize, targetSize) +func NewMemChunk(chunkFormat byte, enc Encoding, head HeadBlockFmt, blockSize, targetSize int) *MemChunk { + return newMemChunkWithFormat(chunkFormat, enc, head, blockSize, targetSize) } func panicIfInvalidFormat(chunkFmt byte, head HeadBlockFmt) { - if chunkFmt == chunkFormatV2 && head != OrderedHeadBlockFmt { + if chunkFmt == ChunkFormatV2 && head != OrderedHeadBlockFmt { panic("only OrderedHeadBlockFmt is supported for V2 chunks") } - if chunkFmt == chunkFormatV4 && head != UnorderedWithNonIndexedLabelsHeadBlockFmt { + if chunkFmt == ChunkFormatV4 && head != UnorderedWithNonIndexedLabelsHeadBlockFmt { + fmt.Println("received head fmt", head.String()) panic("only UnorderedWithNonIndexedLabelsHeadBlockFmt is supported for V4 chunks") } } @@ -401,9 +412,9 @@ func newByteChunk(b []byte, blockSize, targetSize int, fromCheckpoint bool) (*Me } bc.format = version switch version { - case chunkFormatV1: + case ChunkFormatV1: bc.encoding = EncGZIP - case chunkFormatV2, chunkFormatV3, chunkFormatV4: + case ChunkFormatV2, ChunkFormatV3, ChunkFormatV4: // format v2+ has a byte for block encoding. enc := Encoding(db.byte()) if db.err() != nil { @@ -414,6 +425,9 @@ func newByteChunk(b []byte, blockSize, targetSize int, fromCheckpoint bool) (*Me return nil, errors.Errorf("invalid version %d", version) } + // Set the correct headblock format based on chunk format + bc.headFmt = ChunkHeadFormatFor(version) + // readSectionLenAndOffset reads len and offset for different sections within the chunk. // Starting from chunk version 4, we have started writing offset and length of various sections within the chunk. // These len and offset pairs would be stored together at the end of the chunk. @@ -427,7 +441,7 @@ func newByteChunk(b []byte, blockSize, targetSize int, fromCheckpoint bool) (*Me metasOffset := uint64(0) metasLen := uint64(0) - if version >= chunkFormatV4 { + if version >= ChunkFormatV4 { // version >= 4 starts writing length of sections after their offsets metasLen, metasOffset = readSectionLenAndOffset(chunkMetasSectionIdx) } else { @@ -458,7 +472,7 @@ func newByteChunk(b []byte, blockSize, targetSize int, fromCheckpoint bool) (*Me // Read offset and length. blk.offset = db.uvarint() - if version >= chunkFormatV3 { + if version >= ChunkFormatV3 { blk.uncompressedSize = db.uvarint() } l := db.uvarint() @@ -481,7 +495,7 @@ func newByteChunk(b []byte, blockSize, targetSize int, fromCheckpoint bool) (*Me } } - if version >= chunkFormatV4 { + if version >= ChunkFormatV4 { nonIndexedLabelsLen, nonIndexedLabelsOffset := readSectionLenAndOffset(chunkNonIndexedLabelsSectionIdx) lb := b[nonIndexedLabelsOffset : nonIndexedLabelsOffset+nonIndexedLabelsLen] // non-indexed labels Offset + checksum db = decbuf{b: lb} @@ -526,7 +540,7 @@ func (c *MemChunk) Bytes() ([]byte, error) { func (c *MemChunk) BytesSize() int { size := 4 // magic number size++ // format - if c.format > chunkFormatV1 { + if c.format > ChunkFormatV1 { size++ // chunk format v2+ has a byte for encoding. } @@ -538,7 +552,7 @@ func (c *MemChunk) BytesSize() int { size += binary.MaxVarintLen64 // mint size += binary.MaxVarintLen64 // maxt size += binary.MaxVarintLen32 // offset - if c.format >= chunkFormatV3 { + if c.format >= ChunkFormatV3 { size += binary.MaxVarintLen32 // uncompressed size } size += binary.MaxVarintLen32 // len(b) @@ -550,7 +564,7 @@ func (c *MemChunk) BytesSize() int { size += crc32.Size // metablock crc size += 8 // metaoffset - if c.format >= chunkFormatV4 { + if c.format >= ChunkFormatV4 { size += 8 // metablock length size += c.symbolizer.CheckpointSize() // non-indexed labels block @@ -586,7 +600,7 @@ func (c *MemChunk) writeTo(w io.Writer, forCheckpoint bool) (int64, error) { // Write the header (magicNum + version). eb.putBE32(magicNumber) eb.putByte(c.format) - if c.format > chunkFormatV1 { + if c.format > ChunkFormatV1 { // chunk format v2+ has a byte for encoding. eb.putByte(byte(c.encoding)) } @@ -599,7 +613,7 @@ func (c *MemChunk) writeTo(w io.Writer, forCheckpoint bool) (int64, error) { nonIndexedLabelsOffset := offset nonIndexedLabelsLen := 0 - if c.format >= chunkFormatV4 { + if c.format >= ChunkFormatV4 { var ( n int crcHash []byte @@ -655,7 +669,7 @@ func (c *MemChunk) writeTo(w io.Writer, forCheckpoint bool) (int64, error) { eb.putVarint64(b.mint) eb.putVarint64(b.maxt) eb.putUvarint(b.offset) - if c.format >= chunkFormatV3 { + if c.format >= ChunkFormatV3 { eb.putUvarint(b.uncompressedSize) } eb.putUvarint(len(b.b)) @@ -669,7 +683,7 @@ func (c *MemChunk) writeTo(w io.Writer, forCheckpoint bool) (int64, error) { } offset += int64(n) - if c.format >= chunkFormatV4 { + if c.format >= ChunkFormatV4 { // Write non-indexed labels offset and length eb.reset() eb.putBE64int(nonIndexedLabelsLen) @@ -683,7 +697,7 @@ func (c *MemChunk) writeTo(w io.Writer, forCheckpoint bool) (int64, error) { // Write the metasOffset. eb.reset() - if c.format >= chunkFormatV4 { + if c.format >= ChunkFormatV4 { eb.putBE64int(metasLen) } eb.putBE64int(int(metasOffset)) @@ -763,7 +777,7 @@ func (c *MemChunk) SpaceFor(e *logproto.Entry) bool { // a great check, but it will guarantee we are always under the target size newHBSize := c.head.UncompressedSize() + len(e.Line) nonIndexedLabelsSize := 0 - if c.format >= chunkFormatV4 { + if c.format >= ChunkFormatV4 { newHBSize += metaLabelsLen(logproto.FromLabelAdaptersToLabels(e.NonIndexedLabels)) // non-indexed labels are compressed while serializing the chunk so we don't know what their size would be after compression. // As adoption increases, their overall size can be non-trivial so we can't ignore them while calculating chunk size. @@ -786,7 +800,7 @@ func (c *MemChunk) UncompressedSize() int { size += b.uncompressedSize } - if c.format >= chunkFormatV4 { + if c.format >= ChunkFormatV4 { size += c.symbolizer.UncompressedSize() } @@ -802,7 +816,7 @@ func (c *MemChunk) CompressedSize() int { size := 0 // Better to account for any uncompressed data than ignore it even though this isn't accurate. size += c.head.UncompressedSize() - if c.format >= chunkFormatV4 { + if c.format >= ChunkFormatV4 { size += c.symbolizer.UncompressedSize() // length of each symbol } @@ -829,7 +843,7 @@ func (c *MemChunk) Append(entry *logproto.Entry) error { return ErrOutOfOrder } - if c.format < chunkFormatV4 { + if c.format < ChunkFormatV4 { entry.NonIndexedLabels = nil } if err := c.head.Append(entryTimestamp, entry.Line, logproto.FromLabelAdaptersToLabels(entry.NonIndexedLabels)); err != nil { @@ -940,7 +954,7 @@ func (c *MemChunk) Iterator(ctx context.Context, mintT, maxtT time.Time, directi mint, maxt := mintT.UnixNano(), maxtT.UnixNano() blockItrs := make([]iter.EntryIterator, 0, len(c.blocks)+1) - if c.format >= chunkFormatV4 { + if c.format >= ChunkFormatV4 { stats := stats.FromContext(ctx) stats.AddCompressedBytes(int64(c.symbolizer.CompressedSize())) decompressedSize := int64(c.symbolizer.DecompressedSize()) @@ -1025,7 +1039,7 @@ func (c *MemChunk) SampleIterator(ctx context.Context, from, through time.Time, mint, maxt := from.UnixNano(), through.UnixNano() its := make([]iter.SampleIterator, 0, len(c.blocks)+1) - if c.format >= chunkFormatV4 { + if c.format >= ChunkFormatV4 { stats := stats.FromContext(ctx) stats.AddCompressedBytes(int64(c.symbolizer.CompressedSize())) decompressedSize := int64(c.symbolizer.DecompressedSize()) @@ -1095,12 +1109,12 @@ func (c *MemChunk) Rebound(start, end time.Time, filter filter.Func) (Chunk, err // as close as possible, respect the block/target sizes specified. However, // if the blockSize is not set, use reasonable defaults. if c.blockSize > 0 { - newChunk = NewMemChunk(c.Encoding(), DefaultHeadBlockFmt, c.blockSize, c.targetSize) + newChunk = NewMemChunk(c.format, c.Encoding(), c.headFmt, c.blockSize, c.targetSize) } else { // Using defaultBlockSize for target block size. // The alternative here could be going over all the blocks and using the size of the largest block as target block size but I(Sandeep) feel that it is not worth the complexity. // For target chunk size I am using compressed size of original chunk since the newChunk should anyways be lower in size than that. - newChunk = NewMemChunk(c.Encoding(), DefaultHeadBlockFmt, defaultBlockSize, c.CompressedSize()) + newChunk = NewMemChunk(c.format, c.Encoding(), c.headFmt, defaultBlockSize, c.CompressedSize()) } for itr.Next() { @@ -1423,7 +1437,7 @@ func (si *bufferedIterator) moveNext() (int64, []byte, labels.Labels, bool) { decompressedBytes += int64(lineSize) - if si.format < chunkFormatV4 { + if si.format < ChunkFormatV4 { si.stats.AddDecompressedBytes(decompressedBytes) si.stats.AddDecompressedLines(1) return ts, si.buf[:lineSize], nil, true diff --git a/pkg/chunkenc/memchunk_test.go b/pkg/chunkenc/memchunk_test.go index efc9c180c8..66a6a92191 100644 --- a/pkg/chunkenc/memchunk_test.go +++ b/pkg/chunkenc/memchunk_test.go @@ -57,35 +57,42 @@ var ( }{ { headBlockFmt: OrderedHeadBlockFmt, - chunkFormat: chunkFormatV2, + chunkFormat: ChunkFormatV2, }, { headBlockFmt: OrderedHeadBlockFmt, - chunkFormat: chunkFormatV3, + chunkFormat: ChunkFormatV3, }, { headBlockFmt: UnorderedHeadBlockFmt, - chunkFormat: chunkFormatV3, + chunkFormat: ChunkFormatV3, }, { headBlockFmt: UnorderedWithNonIndexedLabelsHeadBlockFmt, - chunkFormat: chunkFormatV4, + chunkFormat: ChunkFormatV4, }, } ) -const DefaultTestHeadBlockFmt = DefaultHeadBlockFmt +const DefaultTestHeadBlockFmt = UnorderedWithNonIndexedLabelsHeadBlockFmt func TestBlocksInclusive(t *testing.T) { - chk := NewMemChunk(EncNone, DefaultTestHeadBlockFmt, testBlockSize, testTargetSize) - err := chk.Append(logprotoEntry(1, "1")) - require.Nil(t, err) - err = chk.cut() - require.Nil(t, err) + for _, enc := range testEncoding { + enc := enc + for _, format := range allPossibleFormats { + chunkfmt, headfmt := format.chunkFormat, format.headBlockFmt + chk := NewMemChunk(chunkfmt, enc, headfmt, testBlockSize, testTargetSize) + err := chk.Append(logprotoEntry(1, "1")) + require.Nil(t, err) + err = chk.cut() + require.Nil(t, err) + + blocks := chk.Blocks(time.Unix(0, 1), time.Unix(0, 1)) + require.Equal(t, 1, len(blocks)) + require.Equal(t, 1, blocks[0].Entries()) + } + } - blocks := chk.Blocks(time.Unix(0, 1), time.Unix(0, 1)) - require.Equal(t, 1, len(blocks)) - require.Equal(t, 1, blocks[0].Entries()) } func TestBlock(t *testing.T) { @@ -182,7 +189,7 @@ func TestBlock(t *testing.T) { require.Equal(t, cases[idx].ts, e.Timestamp.UnixNano()) require.Equal(t, cases[idx].str, e.Line) require.Empty(t, e.NonIndexedLabels) - if chunkFormat < chunkFormatV4 { + if chunkFormat < ChunkFormatV4 { require.Equal(t, labels.EmptyLabels().String(), it.Labels()) } else { expectedLabels := logproto.FromLabelAdaptersToLabels(cases[idx].lbs).String() @@ -240,43 +247,47 @@ func TestBlock(t *testing.T) { func TestCorruptChunk(t *testing.T) { for _, enc := range testEncoding { enc := enc - t.Run(enc.String(), func(t *testing.T) { - t.Parallel() + for _, format := range allPossibleFormats { + chunkfmt, headfmt := format.chunkFormat, format.headBlockFmt - chk := NewMemChunk(enc, DefaultTestHeadBlockFmt, testBlockSize, testTargetSize) - cases := []struct { - data []byte - }{ - // Data that should not decode as lines from a chunk in any encoding. - {data: []byte{0}}, - {data: []byte{1}}, - {data: []byte("asdfasdfasdfqwyteqwtyeq")}, - } + t.Run(enc.String(), func(t *testing.T) { + t.Parallel() + + chk := NewMemChunk(chunkfmt, enc, headfmt, testBlockSize, testTargetSize) + cases := []struct { + data []byte + }{ + // Data that should not decode as lines from a chunk in any encoding. + {data: []byte{0}}, + {data: []byte{1}}, + {data: []byte("asdfasdfasdfqwyteqwtyeq")}, + } - ctx, start, end := context.Background(), time.Unix(0, 0), time.Unix(0, math.MaxInt64) - for i, c := range cases { - chk.blocks = []block{{b: c.data}} - it, err := chk.Iterator(ctx, start, end, logproto.FORWARD, noopStreamPipeline) - require.NoError(t, err, "case %d", i) + ctx, start, end := context.Background(), time.Unix(0, 0), time.Unix(0, math.MaxInt64) + for i, c := range cases { + chk.blocks = []block{{b: c.data}} + it, err := chk.Iterator(ctx, start, end, logproto.FORWARD, noopStreamPipeline) + require.NoError(t, err, "case %d", i) - idx := 0 - for it.Next() { - idx++ + idx := 0 + for it.Next() { + idx++ + } + require.Error(t, it.Error(), "case %d", i) + require.NoError(t, it.Close()) } - require.Error(t, it.Error(), "case %d", i) - require.NoError(t, it.Close()) - } - }) + }) + } } } func TestReadFormatV1(t *testing.T) { t.Parallel() - c := NewMemChunk(EncGZIP, DefaultTestHeadBlockFmt, testBlockSize, testTargetSize) + c := NewMemChunk(ChunkFormatV3, EncGZIP, DefaultTestHeadBlockFmt, testBlockSize, testTargetSize) fillChunk(c) - // overrides default v2 format - c.format = chunkFormatV1 + // overrides to v1 for testing that specific version. + c.format = ChunkFormatV1 b, err := c.Bytes() if err != nil { @@ -369,14 +380,14 @@ func testNameWithFormats(enc Encoding, chunkFormat byte, headBlockFmt HeadBlockF } func TestRoundtripV3(t *testing.T) { - for _, f := range HeadBlockFmts { - for _, enc := range testEncoding { - enc := enc - t.Run(fmt.Sprintf("%v-%v", f, enc), func(t *testing.T) { + for _, enc := range testEncoding { + enc := enc + for _, format := range allPossibleFormats { + chunkfmt, headfmt := format.chunkFormat, format.headBlockFmt + t.Run(fmt.Sprintf("%v-%v", format, enc), func(t *testing.T) { t.Parallel() - c := NewMemChunk(enc, f, testBlockSize, testTargetSize) - c.format = chunkFormatV3 + c := NewMemChunk(chunkfmt, enc, headfmt, testBlockSize, testTargetSize) _ = fillChunk(c) b, err := c.Bytes() @@ -408,7 +419,7 @@ func TestSerialization(t *testing.T) { t.Run(testName, func(t *testing.T) { t.Parallel() - chk := NewMemChunk(enc, testData.headBlockFmt, testBlockSize, testTargetSize) + chk := NewMemChunk(testData.chunkFormat, enc, testData.headBlockFmt, testBlockSize, testTargetSize) chk.format = testData.chunkFormat numSamples := 50000 var entry *logproto.Entry @@ -437,7 +448,7 @@ func TestSerialization(t *testing.T) { require.Equal(t, int64(i), e.Timestamp.UnixNano()) require.Equal(t, strconv.Itoa(i), e.Line) require.Nil(t, e.NonIndexedLabels) - if appendWithNonIndexedLabels && testData.chunkFormat >= chunkFormatV4 { + if appendWithNonIndexedLabels && testData.chunkFormat >= ChunkFormatV4 { require.Equal(t, labels.FromStrings("foo", strconv.Itoa(i)).String(), it.Labels()) } else { require.Equal(t, labels.EmptyLabels().String(), it.Labels()) @@ -460,7 +471,7 @@ func TestSerialization(t *testing.T) { s := sampleIt.Sample() require.Equal(t, int64(i), s.Timestamp) require.Equal(t, 1., s.Value) - if appendWithNonIndexedLabels && testData.chunkFormat >= chunkFormatV4 { + if appendWithNonIndexedLabels && testData.chunkFormat >= ChunkFormatV4 { require.Equal(t, labels.FromStrings("foo", strconv.Itoa(i)).String(), sampleIt.Labels()) } else { require.Equal(t, labels.EmptyLabels().String(), sampleIt.Labels()) @@ -525,7 +536,7 @@ func TestChunkFilling(t *testing.T) { func TestGZIPChunkTargetSize(t *testing.T) { t.Parallel() - chk := NewMemChunk(EncGZIP, DefaultTestHeadBlockFmt, testBlockSize, testTargetSize) + chk := NewMemChunk(ChunkFormatV3, EncGZIP, DefaultTestHeadBlockFmt, testBlockSize, testTargetSize) lineSize := 512 entry := &logproto.Entry{ @@ -622,7 +633,7 @@ func TestMemChunk_AppendOutOfOrder(t *testing.T) { t.Run(testName, func(t *testing.T) { t.Parallel() - tester(t, NewMemChunk(EncGZIP, f, testBlockSize, testTargetSize)) + tester(t, NewMemChunk(ChunkFormatV3, EncGZIP, f, testBlockSize, testTargetSize)) }) } } @@ -667,7 +678,7 @@ func TestChunkSize(t *testing.T) { } func TestChunkStats(t *testing.T) { - c := NewMemChunk(EncSnappy, DefaultTestHeadBlockFmt, testBlockSize, 0) + c := NewMemChunk(ChunkFormatV4, EncSnappy, DefaultTestHeadBlockFmt, testBlockSize, 0) first := time.Now() entry := &logproto.Entry{ Timestamp: first, @@ -793,7 +804,7 @@ func BenchmarkWrite(b *testing.B) { b.Run(name, func(b *testing.B) { uncompressedBytes, compressedBytes := 0, 0 for n := 0; n < b.N; n++ { - c := NewMemChunk(enc, f, testBlockSize, testTargetSize) + c := NewMemChunk(ChunkFormatV3, enc, f, testBlockSize, testTargetSize) // adds until full so we trigger cut which serialize using gzip for c.SpaceFor(entry) { _ = c.Append(entry) @@ -885,7 +896,7 @@ func BenchmarkBackwardIterator(b *testing.B) { for _, bs := range testBlockSizes { b.Run(humanize.Bytes(uint64(bs)), func(b *testing.B) { b.ReportAllocs() - c := NewMemChunk(EncSnappy, DefaultTestHeadBlockFmt, bs, testTargetSize) + c := NewMemChunk(ChunkFormatV3, EncSnappy, DefaultTestHeadBlockFmt, bs, testTargetSize) _ = fillChunk(c) b.ResetTimer() for n := 0; n < b.N; n++ { @@ -996,7 +1007,7 @@ func BenchmarkHeadBlockSampleIterator(b *testing.B) { func TestMemChunk_IteratorBounds(t *testing.T) { createChunk := func() *MemChunk { t.Helper() - c := NewMemChunk(EncNone, DefaultTestHeadBlockFmt, 1e6, 1e6) + c := NewMemChunk(ChunkFormatV3, EncNone, DefaultTestHeadBlockFmt, 1e6, 1e6) if err := c.Append(&logproto.Entry{ Timestamp: time.Unix(0, 1), @@ -1060,7 +1071,7 @@ func TestMemchunkLongLine(t *testing.T) { t.Run(enc.String(), func(t *testing.T) { t.Parallel() - c := NewMemChunk(enc, DefaultTestHeadBlockFmt, testBlockSize, testTargetSize) + c := NewMemChunk(ChunkFormatV3, enc, DefaultTestHeadBlockFmt, testBlockSize, testTargetSize) for i := 1; i <= 10; i++ { require.NoError(t, c.Append(&logproto.Entry{Timestamp: time.Unix(0, int64(i)), Line: strings.Repeat("e", 200000)})) } @@ -1078,9 +1089,9 @@ func TestMemchunkLongLine(t *testing.T) { func TestBytesWith(t *testing.T) { t.Parallel() - exp, err := NewMemChunk(EncNone, DefaultTestHeadBlockFmt, testBlockSize, testTargetSize).BytesWith(nil) + exp, err := NewMemChunk(ChunkFormatV3, EncNone, DefaultTestHeadBlockFmt, testBlockSize, testTargetSize).BytesWith(nil) require.Nil(t, err) - out, err := NewMemChunk(EncNone, DefaultTestHeadBlockFmt, testBlockSize, testTargetSize).BytesWith([]byte{1, 2, 3}) + out, err := NewMemChunk(ChunkFormatV3, EncNone, DefaultTestHeadBlockFmt, testBlockSize, testTargetSize).BytesWith([]byte{1, 2, 3}) require.Nil(t, err) require.Equal(t, exp, out) @@ -1125,7 +1136,7 @@ func TestCheckpointEncoding(t *testing.T) { cpy, err = MemchunkFromCheckpoint(chk.Bytes(), head.Bytes(), f.headBlockFmt, blockSize, targetSize) require.Nil(t, err) - if f.chunkFormat <= chunkFormatV2 { + if f.chunkFormat <= ChunkFormatV2 { for i := range c.blocks { c.blocks[i].uncompressedSize = 0 } @@ -1154,7 +1165,7 @@ func TestCheckpointEncoding(t *testing.T) { cpy, err = MemchunkFromCheckpoint(chk.Bytes(), head.Bytes(), f.headBlockFmt, blockSize, targetSize) require.Nil(t, err) - if f.chunkFormat <= chunkFormatV2 { + if f.chunkFormat <= ChunkFormatV2 { for i := range c.blocks { c.blocks[i].uncompressedSize = 0 } @@ -1173,7 +1184,7 @@ var ( func BenchmarkBufferedIteratorLabels(b *testing.B) { for _, f := range HeadBlockFmts { b.Run(f.String(), func(b *testing.B) { - c := NewMemChunk(EncSnappy, f, testBlockSize, testTargetSize) + c := NewMemChunk(ChunkFormatV3, EncSnappy, f, testBlockSize, testTargetSize) _ = fillChunk(c) labelsSet := []labels.Labels{ @@ -1387,7 +1398,7 @@ func TestMemChunk_Rebound(t *testing.T) { } func buildTestMemChunk(t *testing.T, from, through time.Time) *MemChunk { - chk := NewMemChunk(EncGZIP, DefaultTestHeadBlockFmt, defaultBlockSize, 0) + chk := NewMemChunk(ChunkFormatV3, EncGZIP, DefaultTestHeadBlockFmt, defaultBlockSize, 0) for ; from.Before(through); from = from.Add(time.Second) { err := chk.Append(&logproto.Entry{ Line: from.String(), @@ -1466,7 +1477,7 @@ func TestMemChunk_ReboundAndFilter_with_filter(t *testing.T) { } func buildFilterableTestMemChunk(t *testing.T, from, through time.Time, matchingFrom, matchingTo *time.Time) *MemChunk { - chk := NewMemChunk(EncGZIP, DefaultTestHeadBlockFmt, defaultBlockSize, 0) + chk := NewMemChunk(ChunkFormatV3, EncGZIP, DefaultTestHeadBlockFmt, defaultBlockSize, 0) t.Logf("from : %v", from.String()) t.Logf("through: %v", through.String()) for from.Before(through) { @@ -1598,7 +1609,7 @@ func TestMemChunk_SpaceFor(t *testing.T) { expectFunc: func(chunkFormat byte, _ HeadBlockFmt) bool { // Succeed unless we're using chunk format v4, which should // take the non-indexed labels into account. - return chunkFormat < chunkFormatV4 + return chunkFormat < ChunkFormatV4 }, }, } { @@ -1633,7 +1644,7 @@ func TestMemChunk_IteratorWithNonIndexedLabels(t *testing.T) { streamLabels := labels.Labels{ {Name: "job", Value: "fake"}, } - chk := newMemChunkWithFormat(chunkFormatV4, enc, UnorderedWithNonIndexedLabelsHeadBlockFmt, testBlockSize, testTargetSize) + chk := newMemChunkWithFormat(ChunkFormatV4, enc, UnorderedWithNonIndexedLabelsHeadBlockFmt, testBlockSize, testTargetSize) require.NoError(t, chk.Append(logprotoEntryWithNonIndexedLabels(1, "lineA", []logproto.LabelAdapter{ {Name: "traceID", Value: "123"}, {Name: "user", Value: "a"}, @@ -1843,7 +1854,7 @@ func TestMemChunk_IteratorWithNonIndexedLabels(t *testing.T) { } func TestMemChunk_IteratorOptions(t *testing.T) { - chk := newMemChunkWithFormat(chunkFormatV4, EncNone, UnorderedWithNonIndexedLabelsHeadBlockFmt, testBlockSize, testTargetSize) + chk := newMemChunkWithFormat(ChunkFormatV4, EncNone, UnorderedWithNonIndexedLabelsHeadBlockFmt, testBlockSize, testTargetSize) require.NoError(t, chk.Append(logprotoEntryWithNonIndexedLabels(0, "0", logproto.FromLabelsToLabelAdapters( labels.FromStrings("a", "0"), )))) diff --git a/pkg/chunkenc/unordered_test.go b/pkg/chunkenc/unordered_test.go index 5a3f0cc874..2a7e34944f 100644 --- a/pkg/chunkenc/unordered_test.go +++ b/pkg/chunkenc/unordered_test.go @@ -344,7 +344,7 @@ func TestHeadBlockInterop(t *testing.T) { // ensure backwards compatibility from when chunk format // and head block format was split func TestChunkBlockFmt(t *testing.T) { - require.Equal(t, chunkFormatV3, byte(OrderedHeadBlockFmt)) + require.Equal(t, ChunkFormatV3, byte(OrderedHeadBlockFmt)) } func BenchmarkHeadBlockWrites(b *testing.B) { @@ -428,7 +428,7 @@ func BenchmarkHeadBlockWrites(b *testing.B) { } func TestUnorderedChunkIterators(t *testing.T) { - c := NewMemChunk(EncSnappy, UnorderedWithNonIndexedLabelsHeadBlockFmt, testBlockSize, testTargetSize) + c := NewMemChunk(ChunkFormatV4, EncSnappy, UnorderedWithNonIndexedLabelsHeadBlockFmt, testBlockSize, testTargetSize) for i := 0; i < 100; i++ { // push in reverse order require.Nil(t, c.Append(&logproto.Entry{ @@ -484,11 +484,11 @@ func TestUnorderedChunkIterators(t *testing.T) { } func BenchmarkUnorderedRead(b *testing.B) { - legacy := NewMemChunk(EncSnappy, OrderedHeadBlockFmt, testBlockSize, testTargetSize) + legacy := NewMemChunk(ChunkFormatV3, EncSnappy, OrderedHeadBlockFmt, testBlockSize, testTargetSize) fillChunkClose(legacy, false) - ordered := NewMemChunk(EncSnappy, UnorderedHeadBlockFmt, testBlockSize, testTargetSize) + ordered := NewMemChunk(ChunkFormatV3, EncSnappy, UnorderedHeadBlockFmt, testBlockSize, testTargetSize) fillChunkClose(ordered, false) - unordered := NewMemChunk(EncSnappy, UnorderedHeadBlockFmt, testBlockSize, testTargetSize) + unordered := NewMemChunk(ChunkFormatV3, EncSnappy, UnorderedHeadBlockFmt, testBlockSize, testTargetSize) fillChunkRandomOrder(unordered, false) tcs := []struct { @@ -546,7 +546,7 @@ func BenchmarkUnorderedRead(b *testing.B) { } func TestUnorderedIteratorCountsAllEntries(t *testing.T) { - c := NewMemChunk(EncSnappy, UnorderedWithNonIndexedLabelsHeadBlockFmt, testBlockSize, testTargetSize) + c := NewMemChunk(ChunkFormatV4, EncSnappy, UnorderedWithNonIndexedLabelsHeadBlockFmt, testBlockSize, testTargetSize) fillChunkRandomOrder(c, false) ct := 0 @@ -583,7 +583,7 @@ func TestUnorderedIteratorCountsAllEntries(t *testing.T) { } func chunkFrom(xs []logproto.Entry) ([]byte, error) { - c := NewMemChunk(EncSnappy, DefaultHeadBlockFmt, testBlockSize, testTargetSize) + c := NewMemChunk(ChunkFormatV4, EncSnappy, UnorderedWithNonIndexedLabelsHeadBlockFmt, testBlockSize, testTargetSize) for _, x := range xs { if err := c.Append(&x); err != nil { return nil, err @@ -643,7 +643,7 @@ func TestReorder(t *testing.T) { }, } { t.Run(tc.desc, func(t *testing.T) { - c := NewMemChunk(EncSnappy, DefaultHeadBlockFmt, testBlockSize, testTargetSize) + c := NewMemChunk(ChunkFormatV4, EncSnappy, UnorderedWithNonIndexedLabelsHeadBlockFmt, testBlockSize, testTargetSize) for _, x := range tc.input { require.Nil(t, c.Append(&x)) } @@ -660,7 +660,7 @@ func TestReorder(t *testing.T) { } func TestReorderAcrossBlocks(t *testing.T) { - c := NewMemChunk(EncSnappy, DefaultHeadBlockFmt, testBlockSize, testTargetSize) + c := NewMemChunk(ChunkFormatV4, EncSnappy, UnorderedWithNonIndexedLabelsHeadBlockFmt, testBlockSize, testTargetSize) for _, batch := range [][]int{ // ensure our blocks have overlapping bounds and must be reordered // before closing. diff --git a/pkg/chunkenc/util_test.go b/pkg/chunkenc/util_test.go index 43c96ae99a..e048b2c8a8 100644 --- a/pkg/chunkenc/util_test.go +++ b/pkg/chunkenc/util_test.go @@ -30,7 +30,7 @@ func generateData(enc Encoding, chunksCount, blockSize, targetSize int) ([]Chunk for n := 0; n < chunksCount; n++ { entry := logprotoEntry(0, testdata.LogString(0)) - c := NewMemChunk(enc, DefaultHeadBlockFmt, blockSize, targetSize) + c := NewMemChunk(ChunkFormatV4, enc, UnorderedWithNonIndexedLabelsHeadBlockFmt, blockSize, targetSize) for c.SpaceFor(entry) { size += uint64(len(entry.Line)) _ = c.Append(entry) diff --git a/pkg/ingester/checkpoint.go b/pkg/ingester/checkpoint.go index 604d5c93bf..68d36e8340 100644 --- a/pkg/ingester/checkpoint.go +++ b/pkg/ingester/checkpoint.go @@ -90,7 +90,7 @@ func toWireChunks(descs []chunkDesc, wireChunks []chunkWithBuffer) ([]chunkWithB return wireChunks, nil } -func fromWireChunks(conf *Config, wireChunks []Chunk) ([]chunkDesc, error) { +func fromWireChunks(conf *Config, headfmt chunkenc.HeadBlockFmt, wireChunks []Chunk) ([]chunkDesc, error) { descs := make([]chunkDesc, 0, len(wireChunks)) for _, c := range wireChunks { desc := chunkDesc{ @@ -100,11 +100,7 @@ func fromWireChunks(conf *Config, wireChunks []Chunk) ([]chunkDesc, error) { lastUpdated: c.LastUpdated, } - // Always use Unordered headblocks during replay - // to ensure Loki can effectively replay an unordered-friendly - // WAL into a new configuration that disables unordered writes. - hbType := chunkenc.DefaultHeadBlockFmt - mc, err := chunkenc.MemchunkFromCheckpoint(c.Data, c.Head, hbType, conf.BlockSize, conf.TargetChunkSize) + mc, err := chunkenc.MemchunkFromCheckpoint(c.Data, c.Head, headfmt, conf.BlockSize, conf.TargetChunkSize) if err != nil { return nil, err } diff --git a/pkg/ingester/checkpoint_test.go b/pkg/ingester/checkpoint_test.go index a81f407917..7fe442acfc 100644 --- a/pkg/ingester/checkpoint_test.go +++ b/pkg/ingester/checkpoint_test.go @@ -556,7 +556,7 @@ func buildChunks(t testing.TB, size int) []Chunk { for i := 0; i < size; i++ { // build chunks of 256k blocks, 1.5MB target size. Same as default config. - c := chunkenc.NewMemChunk(chunkenc.EncGZIP, chunkenc.UnorderedHeadBlockFmt, 256*1024, 1500*1024) + c := chunkenc.NewMemChunk(chunkenc.ChunkFormatV3, chunkenc.EncGZIP, chunkenc.UnorderedHeadBlockFmt, 256*1024, 1500*1024) fillChunk(t, c) descs = append(descs, chunkDesc{ chunk: c, diff --git a/pkg/ingester/chunk_test.go b/pkg/ingester/chunk_test.go index a615b9030d..2e42bc6cc2 100644 --- a/pkg/ingester/chunk_test.go +++ b/pkg/ingester/chunk_test.go @@ -49,7 +49,7 @@ func TestIterator(t *testing.T) { }{ {"dumbChunk", chunkenc.NewDumbChunk}, {"gzipChunk", func() chunkenc.Chunk { - return chunkenc.NewMemChunk(chunkenc.EncGZIP, chunkenc.DefaultHeadBlockFmt, 256*1024, 0) + return chunkenc.NewMemChunk(chunkenc.ChunkFormatV4, chunkenc.EncGZIP, chunkenc.UnorderedWithNonIndexedLabelsHeadBlockFmt, 256*1024, 0) }}, } { t.Run(chk.name, func(t *testing.T) { diff --git a/pkg/ingester/encoding_test.go b/pkg/ingester/encoding_test.go index af6a63bdda..a1378626f6 100644 --- a/pkg/ingester/encoding_test.go +++ b/pkg/ingester/encoding_test.go @@ -56,7 +56,7 @@ func Test_EncodingChunks(t *testing.T) { t.Run(fmt.Sprintf("%v-%s", close, tc.desc), func(t *testing.T) { conf := tc.conf - c := chunkenc.NewMemChunk(chunkenc.EncGZIP, chunkenc.DefaultHeadBlockFmt, conf.BlockSize, conf.TargetChunkSize) + c := chunkenc.NewMemChunk(chunkenc.ChunkFormatV4, chunkenc.EncGZIP, chunkenc.UnorderedWithNonIndexedLabelsHeadBlockFmt, conf.BlockSize, conf.TargetChunkSize) fillChunk(t, c) if close { require.Nil(t, c.Close()) @@ -90,7 +90,9 @@ func Test_EncodingChunks(t *testing.T) { } } - backAgain, err := fromWireChunks(&conf, chunks) + _, headfmt := defaultChunkFormat(t) + + backAgain, err := fromWireChunks(&conf, headfmt, chunks) require.Nil(t, err) for i, to := range backAgain { @@ -117,7 +119,7 @@ func Test_EncodingChunks(t *testing.T) { func Test_EncodingCheckpoint(t *testing.T) { conf := dummyConf() - c := chunkenc.NewMemChunk(chunkenc.EncGZIP, chunkenc.DefaultHeadBlockFmt, conf.BlockSize, conf.TargetChunkSize) + c := chunkenc.NewMemChunk(chunkenc.ChunkFormatV4, chunkenc.EncGZIP, chunkenc.UnorderedWithNonIndexedLabelsHeadBlockFmt, conf.BlockSize, conf.TargetChunkSize) require.Nil(t, c.Append(&logproto.Entry{ Timestamp: time.Unix(1, 0), Line: "hi there", diff --git a/pkg/ingester/flush_test.go b/pkg/ingester/flush_test.go index f20090bd78..8b7567bbc2 100644 --- a/pkg/ingester/flush_test.go +++ b/pkg/ingester/flush_test.go @@ -124,7 +124,7 @@ func buildChunkDecs(t testing.TB) []*chunkDesc { for i := range res { res[i] = &chunkDesc{ closed: true, - chunk: chunkenc.NewMemChunk(chunkenc.EncSnappy, chunkenc.DefaultHeadBlockFmt, dummyConf().BlockSize, dummyConf().TargetChunkSize), + chunk: chunkenc.NewMemChunk(chunkenc.ChunkFormatV4, chunkenc.EncSnappy, chunkenc.UnorderedWithNonIndexedLabelsHeadBlockFmt, dummyConf().BlockSize, dummyConf().TargetChunkSize), } fillChunk(t, res[i].chunk) require.NoError(t, res[i].chunk.Close()) diff --git a/pkg/ingester/instance.go b/pkg/ingester/instance.go index 2d82f95fac..75c9b3ffc0 100644 --- a/pkg/ingester/instance.go +++ b/pkg/ingester/instance.go @@ -2,6 +2,8 @@ package ingester import ( "context" + "fmt" + "math" "net/http" "os" "sync" @@ -21,6 +23,7 @@ import ( "go.uber.org/atomic" "github.com/grafana/loki/pkg/analytics" + "github.com/grafana/loki/pkg/chunkenc" "github.com/grafana/loki/pkg/distributor/writefailures" "github.com/grafana/loki/pkg/ingester/index" "github.com/grafana/loki/pkg/ingester/wal" @@ -37,7 +40,7 @@ import ( "github.com/grafana/loki/pkg/util" "github.com/grafana/loki/pkg/util/deletion" util_log "github.com/grafana/loki/pkg/util/log" - "github.com/grafana/loki/pkg/util/math" + mathutil "github.com/grafana/loki/pkg/util/math" "github.com/grafana/loki/pkg/validation" ) @@ -108,6 +111,8 @@ type instance struct { streamRateCalculator *StreamRateCalculator writeFailures *writefailures.Manager + + schemaconfig *config.SchemaConfig } func newInstance( @@ -127,6 +132,8 @@ func newInstance( if err != nil { return nil, err } + + c := config.SchemaConfig{Configs: periodConfigs} i := &instance{ cfg: cfg, streams: newStreamsMap(), @@ -150,6 +157,7 @@ func newInstance( streamRateCalculator: streamRateCalculator, writeFailures: writeFailures, + schemaconfig: &c, } i.mapper = newFPMapper(i.getLabelsFromFingerprint) return i, err @@ -162,8 +170,11 @@ func (i *instance) consumeChunk(ctx context.Context, ls labels.Labels, chunk *lo s, _, _ := i.streams.LoadOrStoreNewByFP(fp, func() (*stream, error) { - s := i.createStreamByFP(ls, fp) - s.chunkMtx.Lock() + s, err := i.createStreamByFP(ls, fp) + s.chunkMtx.Lock() // Lock before return, because we have defer that unlocks it. + if err != nil { + return nil, err + } return s, nil }, func(s *stream) error { @@ -279,7 +290,13 @@ func (i *instance) createStream(pushReqStream logproto.Stream, record *wal.Recor fp := i.getHashForLabels(labels) sortedLabels := i.index.Add(logproto.FromLabelsToLabelAdapters(labels), fp) - s := newStream(i.cfg, i.limiter, i.instanceID, fp, sortedLabels, i.limiter.UnorderedWrites(i.instanceID), i.streamRateCalculator, i.metrics, i.writeFailures) + + chunkfmt, headfmt, err := i.chunkFormatAt(minTs(&pushReqStream)) + if err != nil { + return nil, fmt.Errorf("failed to create stream: %w", err) + } + + s := newStream(chunkfmt, headfmt, i.cfg, i.limiter, i.instanceID, fp, sortedLabels, i.limiter.UnorderedWrites(i.instanceID), i.streamRateCalculator, i.metrics, i.writeFailures) // record will be nil when replaying the wal (we don't want to rewrite wal entries as we replay them). if record != nil { @@ -309,16 +326,43 @@ func (i *instance) createStream(pushReqStream logproto.Stream, record *wal.Recor return s, nil } -func (i *instance) createStreamByFP(ls labels.Labels, fp model.Fingerprint) *stream { +func (i *instance) createStreamByFP(ls labels.Labels, fp model.Fingerprint) (*stream, error) { sortedLabels := i.index.Add(logproto.FromLabelsToLabelAdapters(ls), fp) - s := newStream(i.cfg, i.limiter, i.instanceID, fp, sortedLabels, i.limiter.UnorderedWrites(i.instanceID), i.streamRateCalculator, i.metrics, i.writeFailures) + + chunkfmt, headfmt, err := i.chunkFormatAt(model.Now()) + if err != nil { + return nil, fmt.Errorf("failed to create stream for fingerprint: %w", err) + } + + s := newStream(chunkfmt, headfmt, i.cfg, i.limiter, i.instanceID, fp, sortedLabels, i.limiter.UnorderedWrites(i.instanceID), i.streamRateCalculator, i.metrics, i.writeFailures) i.streamsCreatedTotal.Inc() memoryStreams.WithLabelValues(i.instanceID).Inc() memoryStreamsLabelsBytes.Add(float64(len(s.labels.String()))) i.addTailersToNewStream(s) - return s + return s, nil +} + +// chunkFormatAt returns chunk formats to use at given period of time. +func (i *instance) chunkFormatAt(at model.Time) (byte, chunkenc.HeadBlockFmt, error) { + // NOTE: We choose chunk formats for stream based on it's entries timestamp. + // Rationale being, a single (ingester) instance can be running across multiple schema period + // and choosing correct periodConfig during creation of stream is more accurate rather + // than choosing it during starting of instance itself. + + periodConfig, err := i.schemaconfig.SchemaForTime(at) + if err != nil { + return 0, 0, err + } + + chunkFormat, headblock, err := periodConfig.ChunkFormat() + if err != nil { + return 0, 0, err + } + + return chunkFormat, headblock, nil + } // getOrCreateStream returns the stream or creates it. @@ -888,7 +932,7 @@ func sendBatches(ctx context.Context, i iter.EntryIterator, queryServer QuerierQ for limit != 0 && !isDone(ctx) { fetchSize := uint32(queryBatchSize) if limit > 0 { - fetchSize = math.MinUint32(queryBatchSize, uint32(limit)) + fetchSize = mathutil.MinUint32(queryBatchSize, uint32(limit)) } batch, batchSize, err := iter.ReadBatch(i, fetchSize) if err != nil { @@ -982,3 +1026,20 @@ func (o *OnceSwitch) TriggerAnd(fn func()) { fn() } } + +// minTs is a helper to return minimum Unix timestamp (as `model.Time`) +// across all the entries in a given `stream`. +func minTs(stream *logproto.Stream) model.Time { + // NOTE: We choose `min` timestamp because, the chunk is written once then + // added to the index buckets for may be different days. It would better rather to have + // some latest(say v13) indices reference older (say v12) compatible chunks than vice versa. + + streamMinTs := int64(math.MaxInt64) + for _, entry := range stream.Entries { + ts := entry.Timestamp.UnixNano() + if streamMinTs > ts { + streamMinTs = ts + } + } + return model.TimeFromUnixNano(streamMinTs) +} diff --git a/pkg/ingester/instance_test.go b/pkg/ingester/instance_test.go index 511ee32175..ff54db7c33 100644 --- a/pkg/ingester/instance_test.go +++ b/pkg/ingester/instance_test.go @@ -52,6 +52,7 @@ var defaultPeriodConfigs = []config.PeriodConfig{ { From: MustParseDayTime("1900-01-01"), IndexType: config.StorageTypeBigTable, + Schema: "v13", }, } @@ -295,7 +296,9 @@ func setupTestStreams(t *testing.T) (*instance, time.Time, int) { for _, testStream := range testStreams { stream, err := instance.getOrCreateStream(testStream, recordPool.GetRecord()) require.NoError(t, err) - chunk := newStream(cfg, limiter, "fake", 0, nil, true, NewStreamRateCalculator(), NilMetrics, nil).NewChunk() + chunkfmt, headfmt, err := instance.chunkFormatAt(minTs(&testStream)) + require.NoError(t, err) + chunk := newStream(chunkfmt, headfmt, cfg, limiter, "fake", 0, nil, true, NewStreamRateCalculator(), NilMetrics, nil).NewChunk() for _, entry := range testStream.Entries { err = chunk.Append(&entry) require.NoError(t, err) @@ -546,9 +549,13 @@ func Benchmark_instance_addNewTailer(b *testing.B) { } }) lbs := makeRandomLabels() + + chunkfmt, headfmt, err := inst.chunkFormatAt(model.Now()) + require.NoError(b, err) + b.Run("addTailersToNewStream", func(b *testing.B) { for n := 0; n < b.N; n++ { - inst.addTailersToNewStream(newStream(nil, limiter, "fake", 0, lbs, true, NewStreamRateCalculator(), NilMetrics, nil)) + inst.addTailersToNewStream(newStream(chunkfmt, headfmt, nil, limiter, "fake", 0, lbs, true, NewStreamRateCalculator(), NilMetrics, nil)) } }) } diff --git a/pkg/ingester/recovery.go b/pkg/ingester/recovery.go index 285652cc83..8f170b8543 100644 --- a/pkg/ingester/recovery.go +++ b/pkg/ingester/recovery.go @@ -210,7 +210,7 @@ func (r *ingesterRecoverer) Close() { s.unorderedWrites = isAllowed if !isAllowed && old { - err := s.chunks[len(s.chunks)-1].chunk.ConvertHead(headBlockType(isAllowed)) + err := s.chunks[len(s.chunks)-1].chunk.ConvertHead(headBlockType(s.chunkFormat, isAllowed)) if err != nil { level.Warn(util_log.Logger).Log( "msg", "error converting headblock", diff --git a/pkg/ingester/stream.go b/pkg/ingester/stream.go index c1ee9f3272..4c6aa4f9a1 100644 --- a/pkg/ingester/stream.go +++ b/pkg/ingester/stream.go @@ -74,6 +74,9 @@ type stream struct { streamRateCalculator *StreamRateCalculator writeFailures *writefailures.Manager + + chunkFormat byte + chunkHeadBlockFormat chunkenc.HeadBlockFmt } type chunkDesc struct { @@ -91,7 +94,19 @@ type entryWithError struct { e error } -func newStream(cfg *Config, limits RateLimiterStrategy, tenant string, fp model.Fingerprint, labels labels.Labels, unorderedWrites bool, streamRateCalculator *StreamRateCalculator, metrics *ingesterMetrics, writeFailures *writefailures.Manager) *stream { +func newStream( + chunkFormat byte, + headBlockFmt chunkenc.HeadBlockFmt, + cfg *Config, + limits RateLimiterStrategy, + tenant string, + fp model.Fingerprint, + labels labels.Labels, + unorderedWrites bool, + streamRateCalculator *StreamRateCalculator, + metrics *ingesterMetrics, + writeFailures *writefailures.Manager, +) *stream { hashNoShard, _ := labels.HashWithoutLabels(make([]byte, 0, 1024), ShardLbName) return &stream{ limiter: NewStreamRateLimiter(limits, tenant, 10*time.Second), @@ -106,8 +121,10 @@ func newStream(cfg *Config, limits RateLimiterStrategy, tenant string, fp model. tenant: tenant, streamRateCalculator: streamRateCalculator, - unorderedWrites: unorderedWrites, - writeFailures: writeFailures, + unorderedWrites: unorderedWrites, + writeFailures: writeFailures, + chunkFormat: chunkFormat, + chunkHeadBlockFormat: headBlockFmt, } } @@ -132,7 +149,7 @@ func (s *stream) consumeChunk(_ context.Context, chunk *logproto.Chunk) error { func (s *stream) setChunks(chunks []Chunk) (bytesAdded, entriesAdded int, err error) { s.chunkMtx.Lock() defer s.chunkMtx.Unlock() - chks, err := fromWireChunks(s.cfg, chunks) + chks, err := fromWireChunks(s.cfg, s.chunkHeadBlockFormat, chunks) if err != nil { return 0, 0, err } @@ -145,7 +162,7 @@ func (s *stream) setChunks(chunks []Chunk) (bytesAdded, entriesAdded int, err er } func (s *stream) NewChunk() *chunkenc.MemChunk { - return chunkenc.NewMemChunk(s.cfg.parsedEncoding, headBlockType(s.unorderedWrites), s.cfg.BlockSize, s.cfg.TargetChunkSize) + return chunkenc.NewMemChunk(s.chunkFormat, s.cfg.parsedEncoding, s.chunkHeadBlockFormat, s.cfg.BlockSize, s.cfg.TargetChunkSize) } func (s *stream) Push( @@ -592,9 +609,11 @@ func (s *stream) resetCounter() { s.entryCt = 0 } -func headBlockType(unorderedWrites bool) chunkenc.HeadBlockFmt { +func headBlockType(chunkfmt byte, unorderedWrites bool) chunkenc.HeadBlockFmt { if unorderedWrites { - return chunkenc.DefaultHeadBlockFmt + if chunkfmt >= chunkenc.ChunkFormatV3 { + return chunkenc.ChunkHeadFormatFor(chunkfmt) + } } return chunkenc.OrderedHeadBlockFmt } diff --git a/pkg/ingester/stream_test.go b/pkg/ingester/stream_test.go index 705119984a..641fd1c926 100644 --- a/pkg/ingester/stream_test.go +++ b/pkg/ingester/stream_test.go @@ -52,7 +52,11 @@ func TestMaxReturnedStreamsErrors(t *testing.T) { t.Run(tc.name, func(t *testing.T) { cfg := defaultConfig() cfg.MaxReturnedErrors = tc.limit + + chunkfmt, headfmt := defaultChunkFormat(t) s := newStream( + chunkfmt, + headfmt, cfg, limiter, "fake", @@ -100,7 +104,11 @@ func TestPushDeduplication(t *testing.T) { require.NoError(t, err) limiter := NewLimiter(limits, NilMetrics, &ringCountMock{count: 1}, 1) + chunkfmt, headfmt := defaultChunkFormat(t) + s := newStream( + chunkfmt, + headfmt, defaultConfig(), limiter, "fake", @@ -131,7 +139,11 @@ func TestPushRejectOldCounter(t *testing.T) { require.NoError(t, err) limiter := NewLimiter(limits, NilMetrics, &ringCountMock{count: 1}, 1) + chunkfmt, headfmt := defaultChunkFormat(t) + s := newStream( + chunkfmt, + headfmt, defaultConfig(), limiter, "fake", @@ -179,7 +191,9 @@ func TestStreamIterator(t *testing.T) { new func() *chunkenc.MemChunk }{ {"gzipChunk", func() *chunkenc.MemChunk { - return chunkenc.NewMemChunk(chunkenc.EncGZIP, chunkenc.DefaultHeadBlockFmt, 256*1024, 0) + chunkfmt, headfmt := defaultChunkFormat(t) + + return chunkenc.NewMemChunk(chunkfmt, chunkenc.EncGZIP, headfmt, 256*1024, 0) }}, } { t.Run(chk.name, func(t *testing.T) { @@ -231,7 +245,11 @@ func TestEntryErrorCorrectlyReported(t *testing.T) { require.NoError(t, err) limiter := NewLimiter(limits, NilMetrics, &ringCountMock{count: 1}, 1) + chunkfmt, headfmt := defaultChunkFormat(t) + s := newStream( + chunkfmt, + headfmt, &cfg, limiter, "fake", @@ -262,7 +280,11 @@ func TestUnorderedPush(t *testing.T) { require.NoError(t, err) limiter := NewLimiter(limits, NilMetrics, &ringCountMock{count: 1}, 1) + chunkfmt, headfmt := defaultChunkFormat(t) + s := newStream( + chunkfmt, + headfmt, &cfg, limiter, "fake", @@ -360,7 +382,11 @@ func TestPushRateLimit(t *testing.T) { require.NoError(t, err) limiter := NewLimiter(limits, NilMetrics, &ringCountMock{count: 1}, 1) + chunkfmt, headfmt := defaultChunkFormat(t) + s := newStream( + chunkfmt, + headfmt, defaultConfig(), limiter, "fake", @@ -394,8 +420,11 @@ func TestPushRateLimitAllOrNothing(t *testing.T) { limiter := NewLimiter(limits, NilMetrics, &ringCountMock{count: 1}, 1) cfg := defaultConfig() + chunkfmt, headfmt := defaultChunkFormat(t) s := newStream( + chunkfmt, + headfmt, cfg, limiter, "fake", @@ -428,8 +457,11 @@ func TestReplayAppendIgnoresValidityWindow(t *testing.T) { cfg := defaultConfig() cfg.MaxChunkAge = time.Minute + chunkfmt, headfmt := defaultChunkFormat(t) s := newStream( + chunkfmt, + headfmt, cfg, limiter, "fake", @@ -489,8 +521,9 @@ func Benchmark_PushStream(b *testing.B) { limits, err := validation.NewOverrides(defaultLimitsTestConfig(), nil) require.NoError(b, err) limiter := NewLimiter(limits, NilMetrics, &ringCountMock{count: 1}, 1) + chunkfmt, headfmt := defaultChunkFormat(b) - s := newStream(&Config{MaxChunkAge: 24 * time.Hour}, limiter, "fake", model.Fingerprint(0), ls, true, NewStreamRateCalculator(), NilMetrics, nil) + s := newStream(chunkfmt, headfmt, &Config{MaxChunkAge: 24 * time.Hour}, limiter, "fake", model.Fingerprint(0), ls, true, NewStreamRateCalculator(), NilMetrics, nil) t, err := newTailer("foo", `{namespace="loki-dev"}`, &fakeTailServer{}, 10) require.NoError(b, err) @@ -510,3 +543,15 @@ func Benchmark_PushStream(b *testing.B) { recordPool.PutRecord(rec) } } + +func defaultChunkFormat(t testing.TB) (byte, chunkenc.HeadBlockFmt) { + t.Helper() + + cfg := defaultPeriodConfigs[0] + + chunkfmt, headfmt, err := cfg.ChunkFormat() + + require.NoError(t, err) + + return chunkfmt, headfmt +} diff --git a/pkg/ingester/streams_map_test.go b/pkg/ingester/streams_map_test.go index 2769d9b6b2..2468ffd7c7 100644 --- a/pkg/ingester/streams_map_test.go +++ b/pkg/ingester/streams_map_test.go @@ -14,9 +14,12 @@ func TestStreamsMap(t *testing.T) { limits, err := validation.NewOverrides(defaultLimitsTestConfig(), nil) require.NoError(t, err) limiter := NewLimiter(limits, NilMetrics, &ringCountMock{count: 1}, 1) + chunkfmt, headfmt := defaultChunkFormat(t) ss := []*stream{ newStream( + chunkfmt, + headfmt, defaultConfig(), limiter, "fake", @@ -30,6 +33,8 @@ func TestStreamsMap(t *testing.T) { nil, ), newStream( + chunkfmt, + headfmt, defaultConfig(), limiter, "fake", diff --git a/pkg/storage/async_store_test.go b/pkg/storage/async_store_test.go index ae2ef9f6bd..18e14164be 100644 --- a/pkg/storage/async_store_test.go +++ b/pkg/storage/async_store_test.go @@ -77,18 +77,24 @@ func buildMockChunkRef(t *testing.T, num int) []chunk.Chunk { now := time.Now() var chunks []chunk.Chunk + periodConfig := config.PeriodConfig{ + + From: config.DayTime{Time: 0}, + Schema: "v11", + RowShards: 16, + } + s := config.SchemaConfig{ Configs: []config.PeriodConfig{ - { - From: config.DayTime{Time: 0}, - Schema: "v11", - RowShards: 16, - }, + periodConfig, }, } + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + for i := 0; i < num; i++ { - chk := newChunk(buildTestStreams(fooLabelsWithName, timeRange{ + chk := newChunk(chunkfmt, headfmt, buildTestStreams(fooLabelsWithName, timeRange{ from: now.Add(time.Duration(i) * time.Minute), to: now.Add(time.Duration(i+1) * time.Minute), })) diff --git a/pkg/storage/batch_test.go b/pkg/storage/batch_test.go index b3416d838e..d2ae096976 100644 --- a/pkg/storage/batch_test.go +++ b/pkg/storage/batch_test.go @@ -38,17 +38,22 @@ func Test_batchIterSafeStart(t *testing.T) { }, }, } + periodConfig := config.PeriodConfig{ + From: config.DayTime{Time: 0}, + Schema: "v11", + RowShards: 16, + } + + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + chks := []*LazyChunk{ - newLazyChunk(stream), + newLazyChunk(chunkfmt, headfmt, stream), } s := config.SchemaConfig{ Configs: []config.PeriodConfig{ - { - From: config.DayTime{Time: 0}, - Schema: "v11", - RowShards: 16, - }, + periodConfig, }, } @@ -56,928 +61,971 @@ func Test_batchIterSafeStart(t *testing.T) { // if it was started already, we should see a panic before this time.Sleep(time.Millisecond) - - // ensure idempotency - batch.Start() - batch.Start() - - require.NotNil(t, batch.Next()) -} - -func Test_newLogBatchChunkIterator(t *testing.T) { - tests := map[string]struct { - chunks []*LazyChunk - expected []logproto.Stream - matchers string - start, end time.Time - direction logproto.Direction - batchSize int - }{ - "forward with overlap": { - []*LazyChunk{ - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - { - Timestamp: from.Add(4 * time.Millisecond), - Line: "5", - }, - }, - }), - }, - []logproto.Stream{ - { - Labels: fooLabels.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - }, - }, - }, - fooLabelsWithName.String(), - from, from.Add(4 * time.Millisecond), - logproto.FORWARD, - 2, - }, - "forward all overlap and all chunks have a from time less than query from time": { - []*LazyChunk{ - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - { - Timestamp: from.Add(4 * time.Millisecond), - Line: "5", - }, - }, - }), - }, - []logproto.Stream{ - { - Labels: fooLabels.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - { - Timestamp: from.Add(4 * time.Millisecond), - Line: "5", - }, - }, - }, - }, - fooLabelsWithName.String(), - from.Add(1 * time.Millisecond), from.Add(5 * time.Millisecond), - logproto.FORWARD, - 2, - }, - "forward with overlapping non-continuous entries": { - []*LazyChunk{ - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - }, - }), - }, - []logproto.Stream{ - { - Labels: fooLabels.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - }, - }, - }, - fooLabelsWithName.String(), - from, from.Add(3 * time.Millisecond), - logproto.FORWARD, - 2, - }, - "backward with overlap": { - []*LazyChunk{ - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - { - Timestamp: from.Add(4 * time.Millisecond), - Line: "5", - }, - }, - }), - }, - []logproto.Stream{ - { - Labels: fooLabels.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from, - Line: "1", - }, - }, - }, - }, - fooLabelsWithName.String(), - from, from.Add(4 * time.Millisecond), - logproto.BACKWARD, - 2, - }, - "backward all overlap and all chunks have a through time greater than query through time": { - []*LazyChunk{ - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(4 * time.Millisecond), - Line: "5", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(4 * time.Millisecond), - Line: "5", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(4 * time.Millisecond), - Line: "5", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - { - Timestamp: from.Add(4 * time.Millisecond), - Line: "5", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - { - Timestamp: from.Add(4 * time.Millisecond), - Line: "5", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - { - Timestamp: from.Add(4 * time.Millisecond), - Line: "5", - }, - }, - }), - }, - []logproto.Stream{ - { - Labels: fooLabels.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from, - Line: "1", - }, - }, - }, - }, - fooLabelsWithName.String(), - from, from.Add(4 * time.Millisecond), - logproto.BACKWARD, - 2, - }, - "backward with overlapping non-continuous entries": { - []*LazyChunk{ - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(0 * time.Millisecond), - Line: "0", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "3", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(1 * time.Millisecond), - Line: "1", - }, - { - Timestamp: from.Add(6 * time.Millisecond), - Line: "6", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(5 * time.Millisecond), - Line: "5", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(4 * time.Millisecond), - Line: "4", - }, - { - Timestamp: from.Add(7 * time.Millisecond), - Line: "7", - }, - }, - }), - }, - []logproto.Stream{ - { - Labels: fooLabels.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(7 * time.Millisecond), - Line: "7", - }, - { - Timestamp: from.Add(6 * time.Millisecond), - Line: "6", - }, - { - Timestamp: from.Add(5 * time.Millisecond), - Line: "5", - }, - { - Timestamp: from.Add(4 * time.Millisecond), - Line: "4", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(1 * time.Millisecond), - Line: "1", - }, - { - Timestamp: from.Add(0 * time.Millisecond), - Line: "0", + + // ensure idempotency + batch.Start() + batch.Start() + + require.NotNil(t, batch.Next()) +} + +func Test_newLogBatchChunkIterator(t *testing.T) { + periodConfigs := []config.PeriodConfig{ + { + From: config.DayTime{Time: 0}, + Schema: "v11", + RowShards: 16, + }, + { + From: config.DayTime{Time: 0}, + Schema: "v12", + RowShards: 16, + }, + { + From: config.DayTime{Time: 0}, + Schema: "v13", + RowShards: 16, + }, + } + + type testCase struct { + chunks []*LazyChunk + expected []logproto.Stream + matchers string + start, end time.Time + direction logproto.Direction + batchSize int + } + + var tests map[string]testCase + + for _, periodConfig := range periodConfigs { + periodConfig := periodConfig + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + + tests = map[string]testCase{ + "forward with overlap": { + []*LazyChunk{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + { + Timestamp: from.Add(4 * time.Millisecond), + Line: "5", + }, + }, + }), + }, + []logproto.Stream{ + { + Labels: fooLabels.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, }, }, }, + fooLabelsWithName.String(), + from, from.Add(4 * time.Millisecond), + logproto.FORWARD, + 2, }, - fooLabelsWithName.String(), - from, from.Add(8 * time.Millisecond), - logproto.BACKWARD, - 2, - }, - "forward without overlap": { - []*LazyChunk{ - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", + "forward all overlap and all chunks have a from time less than query from time": { + []*LazyChunk{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + { + Timestamp: from.Add(4 * time.Millisecond), + Line: "5", + }, + }, + }), + }, + []logproto.Stream{ + { + Labels: fooLabels.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + { + Timestamp: from.Add(4 * time.Millisecond), + Line: "5", + }, }, }, - }), + }, + fooLabelsWithName.String(), + from.Add(1 * time.Millisecond), from.Add(5 * time.Millisecond), + logproto.FORWARD, + 2, }, - []logproto.Stream{ - { - Labels: fooLabels.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", + "forward with overlapping non-continuous entries": { + []*LazyChunk{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + }, + []logproto.Stream{ + { + Labels: fooLabels.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, }, }, }, + fooLabelsWithName.String(), + from, from.Add(3 * time.Millisecond), + logproto.FORWARD, + 2, }, - fooLabelsWithName.String(), - from, from.Add(3 * time.Millisecond), - logproto.FORWARD, - 2, - }, - "backward without overlap": { - []*LazyChunk{ - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", + "backward with overlap": { + []*LazyChunk{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + { + Timestamp: from.Add(4 * time.Millisecond), + Line: "5", + }, + }, + }), + }, + []logproto.Stream{ + { + Labels: fooLabels.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from, + Line: "1", + }, }, }, - }), + }, + fooLabelsWithName.String(), + from, from.Add(4 * time.Millisecond), + logproto.BACKWARD, + 2, }, - []logproto.Stream{ - { - Labels: fooLabels.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(2 * time.Millisecond), - Line: "3", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from, - Line: "1", + "backward all overlap and all chunks have a through time greater than query through time": { + []*LazyChunk{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(4 * time.Millisecond), + Line: "5", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(4 * time.Millisecond), + Line: "5", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(4 * time.Millisecond), + Line: "5", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + { + Timestamp: from.Add(4 * time.Millisecond), + Line: "5", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + { + Timestamp: from.Add(4 * time.Millisecond), + Line: "5", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + { + Timestamp: from.Add(4 * time.Millisecond), + Line: "5", + }, + }, + }), + }, + []logproto.Stream{ + { + Labels: fooLabels.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from, + Line: "1", + }, }, }, }, + fooLabelsWithName.String(), + from, from.Add(4 * time.Millisecond), + logproto.BACKWARD, + 2, }, - fooLabelsWithName.String(), - from, from.Add(3 * time.Millisecond), - logproto.BACKWARD, - 2, - }, - // This test is rather complex under the hood. - // It should cause three sub batches in the iterator. - // The first batch has no overlap -- it cannot as the first. It has bounds [1,2) - // The second batch has one chunk overlap, but it includes no entries in the overlap. - // It has bounds [2,4). - // The third batch finally consumes the overlap, with bounds [4,max). - // Notably it also ends up testing the code paths for increasing batch sizes past - // the default due to nextChunks with the same start timestamp. - "forward identicals": { - []*LazyChunk{ - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(time.Millisecond), - Line: "2", + "backward with overlapping non-continuous entries": { + []*LazyChunk{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(0 * time.Millisecond), + Line: "0", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "3", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(1 * time.Millisecond), + Line: "1", + }, + { + Timestamp: from.Add(6 * time.Millisecond), + Line: "6", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(5 * time.Millisecond), + Line: "5", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(4 * time.Millisecond), + Line: "4", + }, + { + Timestamp: from.Add(7 * time.Millisecond), + Line: "7", + }, + }, + }), + }, + []logproto.Stream{ + { + Labels: fooLabels.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(7 * time.Millisecond), + Line: "7", + }, + { + Timestamp: from.Add(6 * time.Millisecond), + Line: "6", + }, + { + Timestamp: from.Add(5 * time.Millisecond), + Line: "5", + }, + { + Timestamp: from.Add(4 * time.Millisecond), + Line: "4", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(1 * time.Millisecond), + Line: "1", + }, + { + Timestamp: from.Add(0 * time.Millisecond), + Line: "0", + }, }, }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(time.Millisecond), - Line: "2", + }, + fooLabelsWithName.String(), + from, from.Add(8 * time.Millisecond), + logproto.BACKWARD, + 2, + }, + "forward without overlap": { + []*LazyChunk{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + }, + []logproto.Stream{ + { + Labels: fooLabels.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, }, }, - }), - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Entries: []logproto.Entry{ - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", + }, + fooLabelsWithName.String(), + from, from.Add(3 * time.Millisecond), + logproto.FORWARD, + 2, + }, + "backward without overlap": { + []*LazyChunk{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + }, + []logproto.Stream{ + { + Labels: fooLabels.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(2 * time.Millisecond), + Line: "3", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from, + Line: "1", + }, }, }, - }), + }, + fooLabelsWithName.String(), + from, from.Add(3 * time.Millisecond), + logproto.BACKWARD, + 2, }, - []logproto.Stream{ - { - Labels: fooLabels.String(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - { - Timestamp: from.Add(time.Millisecond), - Line: "2", - }, - { - Timestamp: from.Add(3 * time.Millisecond), - Line: "4", + // This test is rather complex under the hood. + // It should cause three sub batches in the iterator. + // The first batch has no overlap -- it cannot as the first. It has bounds [1,2) + // The second batch has one chunk overlap, but it includes no entries in the overlap. + // It has bounds [2,4). + // The third batch finally consumes the overlap, with bounds [4,max). + // Notably it also ends up testing the code paths for increasing batch sizes past + // the default due to nextChunks with the same start timestamp. + "forward identicals": { + []*LazyChunk{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + }, + }), + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Entries: []logproto.Entry{ + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, + }, + }), + }, + []logproto.Stream{ + { + Labels: fooLabels.String(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + { + Timestamp: from.Add(time.Millisecond), + Line: "2", + }, + { + Timestamp: from.Add(3 * time.Millisecond), + Line: "4", + }, }, }, }, + fooLabelsWithName.String(), + from, from.Add(4 * time.Millisecond), + logproto.FORWARD, + 1, }, - fooLabelsWithName.String(), - from, from.Add(4 * time.Millisecond), - logproto.FORWARD, - 1, - }, + } + } - s := config.SchemaConfig{ - Configs: []config.PeriodConfig{ - { - From: config.DayTime{Time: 0}, - Schema: "v11", - RowShards: 16, - }, + schemaConfigs := []config.SchemaConfig{ + { + Configs: []config.PeriodConfig{periodConfigs[0]}, + }, + { + Configs: []config.PeriodConfig{periodConfigs[1]}, + }, + { + Configs: []config.PeriodConfig{periodConfigs[2]}, }, } - for name, tt := range tests { - tt := tt - t.Run(name, func(t *testing.T) { - it, err := newLogBatchIterator(context.Background(), s, NilMetrics, tt.chunks, tt.batchSize, newMatchers(tt.matchers), log.NewNoopPipeline(), tt.direction, tt.start, tt.end, nil) - require.NoError(t, err) - streams, _, err := iter.ReadBatch(it, 1000) - _ = it.Close() - if err != nil { - t.Fatalf("error reading batch %s", err) - } + for _, schemaConfig := range schemaConfigs { + s := schemaConfig + for name, tt := range tests { + tt := tt + t.Run(name, func(t *testing.T) { + it, err := newLogBatchIterator(context.Background(), s, NilMetrics, tt.chunks, tt.batchSize, newMatchers(tt.matchers), log.NewNoopPipeline(), tt.direction, tt.start, tt.end, nil) + require.NoError(t, err) + streams, _, err := iter.ReadBatch(it, 1000) + _ = it.Close() + if err != nil { + t.Fatalf("error reading batch %s", err) + } - assertStream(t, tt.expected, streams.Streams) - }) + assertStream(t, tt.expected, streams.Streams) + }) + } } } func Test_newSampleBatchChunkIterator(t *testing.T) { + periodConfig := config.PeriodConfig{ + From: config.DayTime{Time: 0}, + Schema: "v11", + RowShards: 16, + } + + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + tests := map[string]struct { chunks []*LazyChunk expected []logproto.Series @@ -987,7 +1035,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }{ "forward with overlap": { []*LazyChunk{ - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1000,7 +1048,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1013,7 +1061,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1026,7 +1074,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1039,7 +1087,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1052,7 +1100,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1099,7 +1147,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, "forward with overlapping non-continuous entries": { []*LazyChunk{ - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1116,7 +1164,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1129,7 +1177,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1142,7 +1190,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1184,7 +1232,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, "forward last chunk boundaries equal to end": { []*LazyChunk{ - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1197,7 +1245,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1210,7 +1258,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1247,7 +1295,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, "forward last chunk boundaries equal to end and start": { []*LazyChunk{ - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1260,7 +1308,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1297,7 +1345,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, "forward without overlap": { []*LazyChunk{ - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1310,7 +1358,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1319,7 +1367,7 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { }, }, }), - newLazyChunk(logproto.Stream{ + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1387,8 +1435,17 @@ func Test_newSampleBatchChunkIterator(t *testing.T) { } func TestPartitionOverlappingchunks(t *testing.T) { + periodConfig := config.PeriodConfig{ + From: config.DayTime{Time: 0}, + Schema: "v11", + RowShards: 16, + } + + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + var ( - oneThroughFour = newLazyChunk(logproto.Stream{ + oneThroughFour = newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1401,7 +1458,7 @@ func TestPartitionOverlappingchunks(t *testing.T) { }, }, }) - two = newLazyChunk(logproto.Stream{ + two = newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1410,7 +1467,7 @@ func TestPartitionOverlappingchunks(t *testing.T) { }, }, }) - three = newLazyChunk(logproto.Stream{ + three = newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1468,8 +1525,18 @@ func TestPartitionOverlappingchunks(t *testing.T) { } func TestBuildHeapIterator(t *testing.T) { + + periodConfig := config.PeriodConfig{ + From: config.DayTime{Time: 0}, + Schema: "v11", + RowShards: 16, + } + + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + var ( - firstChunk = newLazyChunk(logproto.Stream{ + firstChunk = newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: "{foo=\"bar\"}", Entries: []logproto.Entry{ { @@ -1486,7 +1553,7 @@ func TestBuildHeapIterator(t *testing.T) { }, }, }) - secondChunk = newLazyInvalidChunk(logproto.Stream{ + secondChunk = newLazyInvalidChunk(chunkfmt, headfmt, logproto.Stream{ Labels: "{foo=\"bar\"}", Entries: []logproto.Entry{ { @@ -1499,7 +1566,7 @@ func TestBuildHeapIterator(t *testing.T) { }, }, }) - thirdChunk = newLazyChunk(logproto.Stream{ + thirdChunk = newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: "{foo=\"bar\"}", Entries: []logproto.Entry{ { @@ -1632,8 +1699,17 @@ func Test_IsInvalidChunkError(t *testing.T) { } func TestBatchCancel(t *testing.T) { + periodConfig := config.PeriodConfig{ + From: config.DayTime{Time: 0}, + Schema: "v11", + RowShards: 16, + } + + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + createChunk := func(from time.Time) *LazyChunk { - return newLazyChunk(logproto.Stream{ + return newLazyChunk(chunkfmt, headfmt, logproto.Stream{ Labels: fooLabelsWithName.String(), Entries: []logproto.Entry{ { @@ -1674,13 +1750,22 @@ func TestBatchCancel(t *testing.T) { var entry logproto.Entry func Benchmark_store_OverlappingChunks(b *testing.B) { + periodConfig := config.PeriodConfig{ + From: config.DayTime{Time: 0}, + Schema: "v11", + RowShards: 16, + } + + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(b, err) + b.ReportAllocs() st := &store{ chunkMetrics: NilMetrics, cfg: Config{ MaxChunkBatchSize: 50, }, - Store: newMockChunkStore(newOverlappingStreams(200, 200)), + Store: newMockChunkStore(chunkfmt, headfmt, newOverlappingStreams(200, 200)), } b.ResetTimer() statsCtx, ctx := stats.NewContext(user.InjectOrgID(context.Background(), "fake")) diff --git a/pkg/storage/chunk/chunk_test.go b/pkg/storage/chunk/chunk_test.go index 7f75c1ac10..b80d19bfb2 100644 --- a/pkg/storage/chunk/chunk_test.go +++ b/pkg/storage/chunk/chunk_test.go @@ -12,7 +12,6 @@ import ( "github.com/grafana/loki/pkg/ingester/client" "github.com/grafana/loki/pkg/logproto" - "github.com/grafana/loki/pkg/storage/config" ) const userID = "userID" @@ -65,6 +64,8 @@ func dummyChunkFor(now model.Time, metric labels.Labels) Chunk { func TestChunkCodec(t *testing.T) { dummy := dummyChunk(model.Now()) decodeContext := NewDecodeContext() + key := fmt.Sprintf("%s/%x:%x:%x:%x", dummy.ChunkRef.UserID, dummy.ChunkRef.Fingerprint, int64(dummy.ChunkRef.From), int64(dummy.ChunkRef.Through), dummy.ChunkRef.Checksum) + for i, c := range []struct { chunk Chunk err error @@ -107,17 +108,7 @@ func TestChunkCodec(t *testing.T) { encoded, err := c.chunk.Encoded() require.NoError(t, err) - s := config.SchemaConfig{ - Configs: []config.PeriodConfig{ - { - From: config.DayTime{Time: 0}, - Schema: "v11", - RowShards: 16, - }, - }, - } - - have, err := ParseExternalKey(userID, s.ExternalKey(c.chunk.ChunkRef)) + have, err := ParseExternalKey(userID, key) require.NoError(t, err) buf := make([]byte, len(encoded)) @@ -136,53 +127,6 @@ func TestChunkCodec(t *testing.T) { } } -const fixedTimestamp = model.Time(1557654321000) - -func TestChunkDecodeBackwardsCompatibility(t *testing.T) { - // lets build a new chunk same as what was built using code at commit b1777a50ab19 - c, _ := NewForEncoding(Bigchunk) - nc, err := c.(*bigchunk).Add(model.SamplePair{Timestamp: fixedTimestamp, Value: 0}) - require.NoError(t, err) - require.Equal(t, nil, nc, "returned chunk should be nil") - - chunk := NewChunk( - userID, - client.Fingerprint(labelsForDummyChunks), - labelsForDummyChunks, - c, - fixedTimestamp.Add(-time.Hour), - fixedTimestamp, - ) - // Force checksum calculation. - require.NoError(t, chunk.Encode()) - - // Chunk encoded using code at commit b1777a50ab19 - rawData := []byte("\x00\x00\x00\xb7\xff\x06\x00\x00sNaPpY\x01\xa5\x00\x00\xfcB\xb4\xc9{\"fingerprint\":18245339272195143978,\"userID\":\"userID\",\"from\":1557650721,\"through\":1557654321,\"metric\":{\"__name__\":\"foo\",\"bar\":\"baz\",\"toms\":\"code\"},\"encoding\":0}\n\x00\x00\x00\x15\x01\x00\x11\x00\x00\x01\xd0\xdd\xf5\xb6\xd5Z\x00\x00\x00\x00\x00\x00\x00\x00\x00") - decodeContext := NewDecodeContext() - have, err := ParseExternalKey(userID, "userID/fd3477666dacf92a:16aab37c8e8:16aab6eb768:70b431bb") - require.NoError(t, err) - require.NoError(t, have.Decode(decodeContext, rawData)) - want := chunk - // We can't just compare these two chunks, since the Bigchunk internals are different on construction and read-in. - // Compare the serialised version instead - require.NoError(t, have.Encode()) - require.NoError(t, want.Encode()) - haveEncoded, _ := have.Encoded() - wantEncoded, _ := want.Encoded() - require.Equal(t, haveEncoded, wantEncoded) - - s := config.SchemaConfig{ - Configs: []config.PeriodConfig{ - { - From: config.DayTime{Time: 0}, - Schema: "v11", - RowShards: 16, - }, - }, - } - require.Equal(t, s.ExternalKey(have.ChunkRef), s.ExternalKey(want.ChunkRef)) -} - func TestParseExternalKey(t *testing.T) { for _, c := range []struct { key string @@ -285,65 +229,6 @@ func benchmarkDecode(b *testing.B, batchSize int) { } } -func TestChunkKeys(t *testing.T) { - for _, tc := range []struct { - name string - chunk Chunk - schemaCfg config.SchemaConfig - }{ - { - name: "Legacy key (pre-checksum)", - chunk: Chunk{ - ChunkRef: logproto.ChunkRef{ - Fingerprint: 100, - UserID: "fake", - From: model.TimeFromUnix(1000), - Through: model.TimeFromUnix(5000), - Checksum: 12345, - }, - }, - schemaCfg: config.SchemaConfig{ - Configs: []config.PeriodConfig{ - { - From: config.DayTime{Time: 0}, - Schema: "v11", - RowShards: 16, - }, - }, - }, - }, - { - name: "Newer key (post-v12)", - chunk: Chunk{ - ChunkRef: logproto.ChunkRef{ - Fingerprint: 100, - UserID: "fake", - From: model.TimeFromUnix(1000), - Through: model.TimeFromUnix(5000), - Checksum: 12345, - }, - }, - schemaCfg: config.SchemaConfig{ - Configs: []config.PeriodConfig{ - { - From: config.DayTime{Time: 0}, - Schema: "v12", - RowShards: 16, - }, - }, - }, - }, - } { - t.Run(tc.name, func(t *testing.T) { - key := tc.schemaCfg.ExternalKey(tc.chunk.ChunkRef) - newChunk, err := ParseExternalKey("fake", key) - require.NoError(t, err) - require.Equal(t, tc.chunk, newChunk) - require.Equal(t, key, tc.schemaCfg.ExternalKey(newChunk.ChunkRef)) - }) - } -} - func BenchmarkParseNewerExternalKey(b *testing.B) { benchmarkParseExternalKey(b, "fake/57f628c7f6d57aad/162c699f000:162c69a07eb:eb242d99") } diff --git a/pkg/storage/config/schema_config.go b/pkg/storage/config/schema_config.go index 13850dba42..370a26f28d 100644 --- a/pkg/storage/config/schema_config.go +++ b/pkg/storage/config/schema_config.go @@ -17,7 +17,9 @@ import ( "github.com/prometheus/common/model" yaml "gopkg.in/yaml.v2" + "github.com/grafana/loki/pkg/chunkenc" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/grafana/loki/pkg/util/log" ) @@ -371,6 +373,38 @@ func (cfg *PeriodConfig) applyDefaults() { } } +// ChunkFormat returns chunk format including it's headBlockFormat corresponding to the `schema` version +// in the given `PeriodConfig`. +func (cfg *PeriodConfig) ChunkFormat() (byte, chunkenc.HeadBlockFmt, error) { + sver, err := cfg.VersionAsInt() + if err != nil { + return 0, 0, fmt.Errorf("failed to get chunk format: %w", err) + } + + switch { + case sver <= 12: + return chunkenc.ChunkFormatV3, chunkenc.ChunkHeadFormatFor(chunkenc.ChunkFormatV3), nil + default: // for v13 and above + return chunkenc.ChunkFormatV4, chunkenc.ChunkHeadFormatFor(chunkenc.ChunkFormatV4), nil + } +} + +// TSDBFormat returns index format corresponding to the `schema` version +// in the given `PeriodConfig`. +func (cfg *PeriodConfig) TSDBFormat() (int, error) { + sver, err := cfg.VersionAsInt() + if err != nil { + return 0, fmt.Errorf("failed to get index format: %w", err) + } + + switch { + case sver <= 12: + return index.FormatV2, nil + default: // for v13 and above + return index.FormatV3, nil + } +} + // Validate the period config. func (cfg PeriodConfig) validate() error { validateError := validateChunks(cfg) @@ -431,6 +465,9 @@ func (cfg *PeriodConfig) VersionAsInt() (int, error) { v := strings.Trim(cfg.Schema, "v") n, err := strconv.Atoi(v) + if err != nil { + err = fmt.Errorf("invalid schema version: %w", err) + } cfg.schemaInt = &n return n, err } diff --git a/pkg/storage/config/schema_config_test.go b/pkg/storage/config/schema_config_test.go index 1004f5b7ea..9412c5b5f0 100644 --- a/pkg/storage/config/schema_config_test.go +++ b/pkg/storage/config/schema_config_test.go @@ -7,9 +7,14 @@ import ( "time" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" yaml "gopkg.in/yaml.v2" + + "github.com/grafana/loki/pkg/ingester/client" + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/storage/chunk" ) func TestChunkTableFor(t *testing.T) { @@ -984,3 +989,121 @@ func TestGetIndexStoreTableRanges(t *testing.T) { }, }, GetIndexStoreTableRanges(TSDBType, schemaConfig.Configs)) } + +const ( + fixedTimestamp = model.Time(1557654321000) + userID = "userID" +) + +var ( + labelsForDummyChunks = labels.Labels{ + {Name: labels.MetricName, Value: "foo"}, + {Name: "bar", Value: "baz"}, + {Name: "toms", Value: "code"}, + } +) + +func TestChunkDecodeBackwardsCompatibility(t *testing.T) { + // lets build a new chunk same as what was built using code at commit b1777a50ab19 + c, err := chunk.NewForEncoding(chunk.Bigchunk) + require.NoError(t, err) + nc, err := c.Add(model.SamplePair{Timestamp: fixedTimestamp, Value: 0}) + require.NoError(t, err) + require.Equal(t, nil, nc, "returned chunk should be nil") + + chnk := chunk.NewChunk( + userID, + client.Fingerprint(labelsForDummyChunks), + labelsForDummyChunks, + c, + fixedTimestamp.Add(-time.Hour), + fixedTimestamp, + ) + // Force checksum calculation. + require.NoError(t, chnk.Encode()) + + // Chunk encoded using code at commit b1777a50ab19 + rawData := []byte("\x00\x00\x00\xb7\xff\x06\x00\x00sNaPpY\x01\xa5\x00\x00\xfcB\xb4\xc9{\"fingerprint\":18245339272195143978,\"userID\":\"userID\",\"from\":1557650721,\"through\":1557654321,\"metric\":{\"__name__\":\"foo\",\"bar\":\"baz\",\"toms\":\"code\"},\"encoding\":0}\n\x00\x00\x00\x15\x01\x00\x11\x00\x00\x01\xd0\xdd\xf5\xb6\xd5Z\x00\x00\x00\x00\x00\x00\x00\x00\x00") + decodeContext := chunk.NewDecodeContext() + have, err := chunk.ParseExternalKey(userID, "userID/fd3477666dacf92a:16aab37c8e8:16aab6eb768:70b431bb") + require.NoError(t, err) + require.NoError(t, have.Decode(decodeContext, rawData)) + want := chnk + // We can't just compare these two chunks, since the Bigchunk internals are different on construction and read-in. + // Compare the serialised version instead + require.NoError(t, have.Encode()) + require.NoError(t, want.Encode()) + haveEncoded, _ := have.Encoded() + wantEncoded, _ := want.Encoded() + require.Equal(t, haveEncoded, wantEncoded) + + s := SchemaConfig{ + Configs: []PeriodConfig{ + { + From: DayTime{Time: 0}, + Schema: "v11", + RowShards: 16, + }, + }, + } + require.Equal(t, s.ExternalKey(have.ChunkRef), s.ExternalKey(want.ChunkRef)) +} + +func TestChunkKeys(t *testing.T) { + for _, tc := range []struct { + name string + chunk chunk.Chunk + schemaCfg SchemaConfig + }{ + { + name: "Legacy key (pre-checksum)", + chunk: chunk.Chunk{ + ChunkRef: logproto.ChunkRef{ + Fingerprint: 100, + UserID: "fake", + From: model.TimeFromUnix(1000), + Through: model.TimeFromUnix(5000), + Checksum: 12345, + }, + }, + schemaCfg: SchemaConfig{ + Configs: []PeriodConfig{ + { + From: DayTime{Time: 0}, + Schema: "v11", + RowShards: 16, + }, + }, + }, + }, + { + name: "Newer key (post-v12)", + chunk: chunk.Chunk{ + ChunkRef: logproto.ChunkRef{ + Fingerprint: 100, + UserID: "fake", + From: model.TimeFromUnix(1000), + Through: model.TimeFromUnix(5000), + Checksum: 12345, + }, + }, + schemaCfg: SchemaConfig{ + Configs: []PeriodConfig{ + { + From: DayTime{Time: 0}, + Schema: "v12", + RowShards: 16, + }, + }, + }, + }, + } { + t.Run(tc.name, func(t *testing.T) { + key := tc.schemaCfg.ExternalKey(tc.chunk.ChunkRef) + newChunk, err := chunk.ParseExternalKey("fake", key) + require.NoError(t, err) + require.Equal(t, tc.chunk, newChunk) + require.Equal(t, key, tc.schemaCfg.ExternalKey(newChunk.ChunkRef)) + }) + } +} diff --git a/pkg/storage/hack/main.go b/pkg/storage/hack/main.go index b9802d1678..e3ad3b6ac4 100644 --- a/pkg/storage/hack/main.go +++ b/pkg/storage/hack/main.go @@ -43,7 +43,7 @@ func main() { } } -func getStore(cm storage.ClientMetrics) (storage.Store, error) { +func getStore(cm storage.ClientMetrics) (storage.Store, *config.SchemaConfig, error) { storeConfig := storage.Config{ BoltDBConfig: local.BoltDBConfig{Directory: "/tmp/benchmark/index"}, FSConfig: local.FSConfig{Directory: "/tmp/benchmark/chunks"}, @@ -55,7 +55,7 @@ func getStore(cm storage.ClientMetrics) (storage.Store, error) { From: config.DayTime{Time: start}, IndexType: "boltdb", ObjectType: "filesystem", - Schema: "v9", + Schema: "v13", IndexTables: config.PeriodicTableConfig{ Prefix: "index_", Period: time.Hour * 168, @@ -64,16 +64,27 @@ func getStore(cm storage.ClientMetrics) (storage.Store, error) { }, } - return storage.NewStore(storeConfig, config.ChunkStoreConfig{}, schemaCfg, &validation.Overrides{}, cm, prometheus.DefaultRegisterer, util_log.Logger) + store, err := storage.NewStore(storeConfig, config.ChunkStoreConfig{}, schemaCfg, &validation.Overrides{}, cm, prometheus.DefaultRegisterer, util_log.Logger) + return store, &schemaCfg, err } func fillStore(cm storage.ClientMetrics) error { - store, err := getStore(cm) + store, schemacfg, err := getStore(cm) if err != nil { return err } defer store.Stop() + periodcfg, err := schemacfg.SchemaForTime(start) + if err != nil { + return err + } + + chunkfmt, headfmt, err := periodcfg.ChunkFormat() + if err != nil { + return err + } + var wgPush sync.WaitGroup var flushCount int // insert 5 streams with a random logs every nanoseconds @@ -91,7 +102,7 @@ func fillStore(cm storage.ClientMetrics) error { labelsBuilder.Set(labels.MetricName, "logs") metric := labelsBuilder.Labels() fp := client.Fingerprint(lbs) - chunkEnc := chunkenc.NewMemChunk(chunkenc.EncLZ4_4M, chunkenc.DefaultHeadBlockFmt, 262144, 1572864) + chunkEnc := chunkenc.NewMemChunk(chunkfmt, chunkenc.EncLZ4_4M, headfmt, 262144, 1572864) for ts := start.UnixNano(); ts < start.UnixNano()+time.Hour.Nanoseconds(); ts = ts + time.Millisecond.Nanoseconds() { entry := &logproto.Entry{ Timestamp: time.Unix(0, ts), @@ -114,7 +125,7 @@ func fillStore(cm storage.ClientMetrics) error { if flushCount >= maxChunks { return } - chunkEnc = chunkenc.NewMemChunk(chunkenc.EncLZ4_64k, chunkenc.DefaultHeadBlockFmt, 262144, 1572864) + chunkEnc = chunkenc.NewMemChunk(chunkenc.ChunkFormatV4, chunkenc.EncLZ4_64k, chunkenc.UnorderedWithNonIndexedLabelsHeadBlockFmt, 262144, 1572864) } } }(i) diff --git a/pkg/storage/lazy_chunk_test.go b/pkg/storage/lazy_chunk_test.go index 5b712f2717..cd7059954a 100644 --- a/pkg/storage/lazy_chunk_test.go +++ b/pkg/storage/lazy_chunk_test.go @@ -14,48 +14,74 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql/log" "github.com/grafana/loki/pkg/storage/chunk" + "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/util" ) func TestLazyChunkIterator(t *testing.T) { - for i, tc := range []struct { - chunk *LazyChunk - expected []logproto.Stream - }{ - // TODO: Add tests for metadata labels. + periodConfigs := []config.PeriodConfig{ { - newLazyChunk(logproto.Stream{ - Labels: fooLabelsWithName.String(), - Hash: fooLabelsWithName.Hash(), - Entries: []logproto.Entry{ - { - Timestamp: from, - Line: "1", - }, - }, - }), - []logproto.Stream{ - { - Labels: fooLabels.String(), - Hash: fooLabels.Hash(), + From: config.DayTime{Time: 0}, + Schema: "v11", + RowShards: 16, + }, + { + From: config.DayTime{Time: 0}, + Schema: "v12", + RowShards: 16, + }, + { + From: config.DayTime{Time: 0}, + Schema: "v13", + RowShards: 16, + }, + } + + for _, periodConfig := range periodConfigs { + periodConfig := periodConfig + + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + + for i, tc := range []struct { + chunk *LazyChunk + expected []logproto.Stream + }{ + // TODO: Add tests for metadata labels. + { + newLazyChunk(chunkfmt, headfmt, logproto.Stream{ + Labels: fooLabelsWithName.String(), + Hash: fooLabelsWithName.Hash(), Entries: []logproto.Entry{ { Timestamp: from, Line: "1", }, }, + }), + []logproto.Stream{ + { + Labels: fooLabels.String(), + Hash: fooLabels.Hash(), + Entries: []logproto.Entry{ + { + Timestamp: from, + Line: "1", + }, + }, + }, }, }, - }, - } { - t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { - it, err := tc.chunk.Iterator(context.Background(), time.Unix(0, 0), time.Unix(1000, 0), logproto.FORWARD, log.NewNoopPipeline().ForStream(labels.Labels{labels.Label{Name: "foo", Value: "bar"}}), nil) - require.Nil(t, err) - streams, _, err := iter.ReadBatch(it, 1000) - require.Nil(t, err) - _ = it.Close() - require.Equal(t, tc.expected, streams.Streams) - }) + } { + t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { + it, err := tc.chunk.Iterator(context.Background(), time.Unix(0, 0), time.Unix(1000, 0), logproto.FORWARD, log.NewNoopPipeline().ForStream(labels.Labels{labels.Label{Name: "foo", Value: "bar"}}), nil) + require.Nil(t, err) + streams, _, err := iter.ReadBatch(it, 1000) + require.Nil(t, err) + _ = it.Close() + require.Equal(t, tc.expected, streams.Streams) + }) + } } } diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index d98eb78034..17a47c55d1 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -884,6 +884,14 @@ func Test_ChunkFilterer(t *testing.T) { } func Test_store_GetSeries(t *testing.T) { + periodConfig := config.PeriodConfig{ + From: config.DayTime{Time: 0}, + Schema: "v11", + } + + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + tests := []struct { name string req *logproto.QueryRequest @@ -928,7 +936,7 @@ func Test_store_GetSeries(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { s := &store{ - Store: newMockChunkStore(streamsFixture), + Store: newMockChunkStore(chunkfmt, headfmt, streamsFixture), cfg: Config{ MaxChunkBatchSize: tt.batchSize, }, @@ -1017,18 +1025,21 @@ func TestStore_indexPrefixChange(t *testing.T) { firstPeriodDate := parseDate("2019-01-01") secondPeriodDate := parseDate("2019-01-02") + + periodConfig := config.PeriodConfig{ + From: config.DayTime{Time: timeToModelTime(firstPeriodDate)}, + IndexType: config.TSDBType, + ObjectType: config.StorageTypeFileSystem, + Schema: "v9", + IndexTables: config.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + } + schemaConfig := config.SchemaConfig{ Configs: []config.PeriodConfig{ - { - From: config.DayTime{Time: timeToModelTime(firstPeriodDate)}, - IndexType: config.TSDBType, - ObjectType: config.StorageTypeFileSystem, - Schema: "v9", - IndexTables: config.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, - }, - }, + periodConfig, }, } @@ -1053,9 +1064,16 @@ func TestStore_indexPrefixChange(t *testing.T) { // build and add chunks to the store addedChunkIDs := map[string]struct{}{} for _, tr := range chunksToBuildForTimeRanges { - chk := newChunk(buildTestStreams(fooLabelsWithName, tr)) + periodConfig, err := schemaConfig.SchemaForTime(timeToModelTime(tr.from)) + require.NoError(t, err) + require.NotNil(t, periodConfig) - err := store.PutOne(ctx, chk.From, chk.Through, chk) + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + + chk := newChunk(chunkfmt, headfmt, buildTestStreams(fooLabelsWithName, tr)) + + err = store.PutOne(ctx, chk.From, chk.Through, chk) require.NoError(t, err) addedChunkIDs[schemaConfig.ExternalKey(chk.ChunkRef)] = struct{}{} @@ -1079,7 +1097,7 @@ func TestStore_indexPrefixChange(t *testing.T) { } // update schema with a new period that uses different index prefix - schemaConfig.Configs = append(schemaConfig.Configs, config.PeriodConfig{ + periodConfig2 := config.PeriodConfig{ From: config.DayTime{Time: timeToModelTime(secondPeriodDate)}, IndexType: config.TSDBType, ObjectType: "named-store", @@ -1089,7 +1107,8 @@ func TestStore_indexPrefixChange(t *testing.T) { Period: time.Hour * 24, }, RowShards: 2, - }) + } + schemaConfig.Configs = append(schemaConfig.Configs, periodConfig2) // time ranges adding a chunk to the new period and one that overlaps both chunksToBuildForTimeRanges = []timeRange{ @@ -1113,9 +1132,16 @@ func TestStore_indexPrefixChange(t *testing.T) { // build and add chunks to the store for _, tr := range chunksToBuildForTimeRanges { - chk := newChunk(buildTestStreams(fooLabelsWithName, tr)) + periodConfig, err := schemaConfig.SchemaForTime(timeToModelTime(tr.from)) + require.NoError(t, err) + require.NotNil(t, periodConfig) + + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) - err := store.PutOne(ctx, chk.From, chk.Through, chk) + chk := newChunk(chunkfmt, headfmt, buildTestStreams(fooLabelsWithName, tr)) + + err = store.PutOne(ctx, chk.From, chk.Through, chk) require.NoError(t, err) addedChunkIDs[schemaConfig.ExternalKey(chk.ChunkRef)] = struct{}{} @@ -1176,29 +1202,33 @@ func TestStore_MultiPeriod(t *testing.T) { } require.NoError(t, cfg.NamedStores.validate()) + periodConfigV9 := config.PeriodConfig{ + From: config.DayTime{Time: timeToModelTime(firstStoreDate)}, + IndexType: indexes[0], + ObjectType: config.StorageTypeFileSystem, + Schema: "v9", + IndexTables: config.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + } + + periodConfigV11 := config.PeriodConfig{ + From: config.DayTime{Time: timeToModelTime(secondStoreDate)}, + IndexType: indexes[1], + ObjectType: "named-store", + Schema: "v11", + IndexTables: config.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 2, + } + schemaConfig := config.SchemaConfig{ Configs: []config.PeriodConfig{ - { - From: config.DayTime{Time: timeToModelTime(firstStoreDate)}, - IndexType: indexes[0], - ObjectType: config.StorageTypeFileSystem, - Schema: "v9", - IndexTables: config.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, - }, - }, - { - From: config.DayTime{Time: timeToModelTime(secondStoreDate)}, - IndexType: indexes[1], - ObjectType: "named-store", - Schema: "v11", - IndexTables: config.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, - }, - RowShards: 2, - }, + periodConfigV9, + periodConfigV11, }, } @@ -1228,9 +1258,14 @@ func TestStore_MultiPeriod(t *testing.T) { // build and add chunks to the store addedChunkIDs := map[string]struct{}{} for _, tr := range chunksToBuildForTimeRanges { - chk := newChunk(buildTestStreams(fooLabelsWithName, tr)) + periodConfig, err := schemaConfig.SchemaForTime(timeToModelTime(tr.from)) + require.NoError(t, err) + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + + chk := newChunk(chunkfmt, headfmt, buildTestStreams(fooLabelsWithName, tr)) - err := store.PutOne(ctx, chk.From, chk.Through, chk) + err = store.PutOne(ctx, chk.From, chk.Through, chk) require.NoError(t, err) addedChunkIDs[schemaConfig.ExternalKey(chk.ChunkRef)] = struct{}{} @@ -1306,15 +1341,23 @@ func timeToModelTime(t time.Time) model.Time { } func Test_OverlappingChunks(t *testing.T) { + periodConfig := config.PeriodConfig{ + From: config.DayTime{Time: 0}, + Schema: "v11", + } + + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + chunks := []chunk.Chunk{ - newChunk(logproto.Stream{ + newChunk(chunkfmt, headfmt, logproto.Stream{ Labels: `{foo="bar"}`, Entries: []logproto.Entry{ {Timestamp: time.Unix(0, 1), Line: "1"}, {Timestamp: time.Unix(0, 4), Line: "4"}, }, }), - newChunk(logproto.Stream{ + newChunk(chunkfmt, headfmt, logproto.Stream{ Labels: `{foo="bar"}`, Entries: []logproto.Entry{ {Timestamp: time.Unix(0, 2), Line: "2"}, @@ -1355,9 +1398,17 @@ func Test_OverlappingChunks(t *testing.T) { } func Test_GetSeries(t *testing.T) { + periodConfig := config.PeriodConfig{ + From: config.DayTime{Time: 0}, + Schema: "v11", + } + + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + var ( store = &store{ - Store: newMockChunkStore([]*logproto.Stream{ + Store: newMockChunkStore(chunkfmt, headfmt, []*logproto.Stream{ { Labels: `{foo="bar",buzz="boo"}`, Entries: []logproto.Entry{ @@ -1535,9 +1586,15 @@ func TestStore_BoltdbTsdbSameIndexPrefix(t *testing.T) { // build and add chunks to the store addedChunkIDs := map[string]struct{}{} for _, tr := range chunksToBuildForTimeRanges { - chk := newChunk(buildTestStreams(fooLabelsWithName, tr)) + periodConfig, err := schemaConfig.SchemaForTime(timeToModelTime(tr.from)) + require.NoError(t, err) + + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + + chk := newChunk(chunkfmt, headfmt, buildTestStreams(fooLabelsWithName, tr)) - err := store.PutOne(ctx, chk.From, chk.Through, chk) + err = store.PutOne(ctx, chk.From, chk.Through, chk) require.NoError(t, err) addedChunkIDs[schemaConfig.ExternalKey(chk.ChunkRef)] = struct{}{} diff --git a/pkg/storage/stores/indexshipper/compactor/retention/retention_test.go b/pkg/storage/stores/indexshipper/compactor/retention/retention_test.go index e3e4479343..057d0c3818 100644 --- a/pkg/storage/stores/indexshipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/indexshipper/compactor/retention/retention_test.go @@ -216,7 +216,7 @@ func createChunk(t testing.TB, userID string, lbs labels.Labels, from model.Time labelsBuilder.Set(labels.MetricName, "logs") metric := labelsBuilder.Labels() fp := ingesterclient.Fingerprint(lbs) - chunkEnc := chunkenc.NewMemChunk(chunkenc.EncSnappy, chunkenc.DefaultHeadBlockFmt, blockSize, targetSize) + chunkEnc := chunkenc.NewMemChunk(chunkenc.ChunkFormatV4, chunkenc.EncSnappy, chunkenc.UnorderedWithNonIndexedLabelsHeadBlockFmt, blockSize, targetSize) for ts := from; !ts.After(through); ts = ts.Add(1 * time.Minute) { require.NoError(t, chunkEnc.Append(&logproto.Entry{ @@ -278,7 +278,7 @@ func labelsString(ls labels.Labels) string { func TestChunkRewriter(t *testing.T) { minListMarkDelay = 1 * time.Second now := model.Now() - schema := allSchemas[3] + schema := allSchemas[3] // v12 todaysTableInterval := ExtractIntervalFromTableName(schema.config.IndexTables.TableFor(now)) type tableResp struct { mustDeleteLines bool diff --git a/pkg/storage/stores/indexshipper/compactor/retention/util_test.go b/pkg/storage/stores/indexshipper/compactor/retention/util_test.go index bab0e8f117..85269a4d37 100644 --- a/pkg/storage/stores/indexshipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/indexshipper/compactor/retention/util_test.go @@ -84,6 +84,17 @@ var ( }, RowShards: 16, }, + { + From: dayFromTime(start.Add(125 * time.Hour)), + IndexType: "tsdb", + ObjectType: "filesystem", + Schema: "v12", + IndexTables: config.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, + }, }, } allSchemas = []struct { @@ -95,6 +106,7 @@ var ( {"v10", schemaCfg.Configs[1].From.Time, schemaCfg.Configs[1]}, {"v11", schemaCfg.Configs[2].From.Time, schemaCfg.Configs[2]}, {"v12", schemaCfg.Configs[3].From.Time, schemaCfg.Configs[3]}, + {"v13", schemaCfg.Configs[3].From.Time, schemaCfg.Configs[4]}, } sweepMetrics = newSweeperMetrics(prometheus.DefaultRegisterer) diff --git a/pkg/storage/stores/series/index/schema.go b/pkg/storage/stores/series/index/schema.go index aefd18a8ba..24ff305a06 100644 --- a/pkg/storage/stores/series/index/schema.go +++ b/pkg/storage/stores/series/index/schema.go @@ -590,3 +590,7 @@ func (v11Entries) GetLabelNamesForSeries(bucket Bucket, seriesID []byte) ([]Quer type v12Entries struct { v11Entries } + +type v13Entries struct { + v12Entries +} diff --git a/pkg/storage/stores/series/index/schema_config.go b/pkg/storage/stores/series/index/schema_config.go index 99d54416f5..ca5265e7b3 100644 --- a/pkg/storage/stores/series/index/schema_config.go +++ b/pkg/storage/stores/series/index/schema_config.go @@ -44,16 +44,18 @@ func CreateSchema(cfg config.PeriodConfig) (SeriesStoreSchema, error) { } v10 := v10Entries{rowShards: cfg.RowShards} - if cfg.Schema == "v10" { + switch cfg.Schema { + case "v10": return newSeriesStoreSchema(buckets, v10), nil - } else if cfg.Schema == "v11" { + case "v11": return newSeriesStoreSchema(buckets, v11Entries{v10}), nil - } else { // v12 + case "v12": return newSeriesStoreSchema(buckets, v12Entries{v11Entries{v10}}), nil + case "v13": + return newSeriesStoreSchema(buckets, v13Entries{v12Entries{v11Entries{v10}}}), nil } - default: - return nil, errInvalidSchemaVersion } + return nil, errInvalidSchemaVersion } // Bucket describes a range of time with a tableName and hashKey diff --git a/pkg/storage/stores/series_store_write_test.go b/pkg/storage/stores/series_store_write_test.go index e96b568771..9c8c2f4069 100644 --- a/pkg/storage/stores/series_store_write_test.go +++ b/pkg/storage/stores/series_store_write_test.go @@ -76,16 +76,19 @@ func (m *mockChunksClient) IsRetryableErr(_ error) bool { } func TestChunkWriter_PutOne(t *testing.T) { + periodConfig := config.PeriodConfig{ + From: config.DayTime{Time: 0}, + Schema: "v13", + } + schemaConfig := config.SchemaConfig{ - Configs: []config.PeriodConfig{ - { - From: config.DayTime{Time: 0}, - Schema: "v11", - }, - }, + Configs: []config.PeriodConfig{periodConfig}, } - memchk := chunkenc.NewMemChunk(chunkenc.EncGZIP, chunkenc.DefaultHeadBlockFmt, 256*1024, 0) + chunkfmt, headfmt, err := periodConfig.ChunkFormat() + require.NoError(t, err) + + memchk := chunkenc.NewMemChunk(chunkfmt, chunkenc.EncGZIP, headfmt, 256*1024, 0) chk := chunk.NewChunk("fake", model.Fingerprint(0), []labels.Label{{Name: "foo", Value: "bar"}}, chunkenc.NewFacade(memchk, 0, 0), 100, 400) for name, tc := range map[string]struct { diff --git a/pkg/storage/stores/shipper/index/compactor/compacted_index_test.go b/pkg/storage/stores/shipper/index/compactor/compacted_index_test.go index 1ddf908718..e662c48649 100644 --- a/pkg/storage/stores/shipper/index/compactor/compacted_index_test.go +++ b/pkg/storage/stores/shipper/index/compactor/compacted_index_test.go @@ -29,9 +29,11 @@ func TestCompactedIndex_IndexProcessor(t *testing.T) { defer cm.Unregister() testSchema := config.SchemaConfig{Configs: []config.PeriodConfig{tt.config}} store := newTestStore(t, cm) - c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, tt.from, tt.from.Add(1*time.Hour)) - c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "bar"}, labels.Label{Name: "fizz", Value: "buzz"}}, tt.from, tt.from.Add(1*time.Hour)) - c3 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "buzz"}}, tt.from, tt.from.Add(1*time.Hour)) + chunkfmt, headfmt, err := tt.config.ChunkFormat() + require.NoError(t, err) + c1 := createChunk(t, chunkfmt, headfmt, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, tt.from, tt.from.Add(1*time.Hour)) + c2 := createChunk(t, chunkfmt, headfmt, "2", labels.Labels{labels.Label{Name: "foo", Value: "bar"}, labels.Label{Name: "fizz", Value: "buzz"}}, tt.from, tt.from.Add(1*time.Hour)) + c3 := createChunk(t, chunkfmt, headfmt, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "buzz"}}, tt.from, tt.from.Add(1*time.Hour)) require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ c1, c2, c3, @@ -45,8 +47,8 @@ func TestCompactedIndex_IndexProcessor(t *testing.T) { compactedIndex := newCompactedIndex(tables[0].DB, tables[0].name, t.TempDir(), tt.config, util_log.Logger) // remove c1, c2 chunk and index c4 with same labels as c2 - c4 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "bar"}, labels.Label{Name: "fizz", Value: "buzz"}}, tt.from, tt.from.Add(30*time.Minute)) - err := compactedIndex.ForEachChunk(context.Background(), func(entry retention.ChunkEntry) (deleteChunk bool, err error) { + c4 := createChunk(t, chunkfmt, headfmt, "2", labels.Labels{labels.Label{Name: "foo", Value: "bar"}, labels.Label{Name: "fizz", Value: "buzz"}}, tt.from, tt.from.Add(30*time.Minute)) + err = compactedIndex.ForEachChunk(context.Background(), func(entry retention.ChunkEntry) (deleteChunk bool, err error) { if entry.Labels.Get("fizz") == "buzz" { chunkIndexed, err := compactedIndex.IndexChunk(c4) require.NoError(t, err) diff --git a/pkg/storage/stores/shipper/index/compactor/iterator_test.go b/pkg/storage/stores/shipper/index/compactor/iterator_test.go index 09d2fb904d..4767429d70 100644 --- a/pkg/storage/stores/shipper/index/compactor/iterator_test.go +++ b/pkg/storage/stores/shipper/index/compactor/iterator_test.go @@ -29,8 +29,11 @@ func Test_ChunkIterator(t *testing.T) { cm := storage.NewClientMetrics() defer cm.Unregister() store := newTestStore(t, cm) - c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, tt.from, tt.from.Add(1*time.Hour)) - c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, tt.from, tt.from.Add(1*time.Hour)) + chunkfmt, headfmt, err := tt.config.ChunkFormat() + require.NoError(t, err) + + c1 := createChunk(t, chunkfmt, headfmt, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, tt.from, tt.from.Add(1*time.Hour)) + c2 := createChunk(t, chunkfmt, headfmt, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, tt.from, tt.from.Add(1*time.Hour)) require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ c1, c2, @@ -41,7 +44,7 @@ func Test_ChunkIterator(t *testing.T) { tables := store.indexTables() require.Len(t, tables, 1) var actual []retention.ChunkEntry - err := tables[0].DB.Update(func(tx *bbolt.Tx) error { + err = tables[0].DB.Update(func(tx *bbolt.Tx) error { return ForEachChunk(context.Background(), tx.Bucket(local.IndexBucketName), tt.config, func(entry retention.ChunkEntry) (deleteChunk bool, err error) { actual = append(actual, entry) return len(actual) == 2, nil @@ -75,8 +78,11 @@ func Test_ChunkIteratorContextCancelation(t *testing.T) { store := newTestStore(t, cm) from := schemaCfg.Configs[0].From.Time - c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, from, from.Add(1*time.Hour)) - c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, from, from.Add(1*time.Hour)) + chunkfmt, headfmt, err := schemaCfg.Configs[0].ChunkFormat() + require.NoError(t, err) + + c1 := createChunk(t, chunkfmt, headfmt, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, from, from.Add(1*time.Hour)) + c2 := createChunk(t, chunkfmt, headfmt, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, from, from.Add(1*time.Hour)) require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{c1, c2})) store.Stop() @@ -88,7 +94,7 @@ func Test_ChunkIteratorContextCancelation(t *testing.T) { defer cancel() var actual []retention.ChunkEntry - err := tables[0].DB.Update(func(tx *bbolt.Tx) error { + err = tables[0].DB.Update(func(tx *bbolt.Tx) error { return ForEachChunk(ctx, tx.Bucket(local.IndexBucketName), schemaCfg.Configs[0], func(entry retention.ChunkEntry) (deleteChunk bool, err error) { actual = append(actual, entry) cancel() @@ -108,9 +114,12 @@ func Test_SeriesCleaner(t *testing.T) { defer cm.Unregister() testSchema := config.SchemaConfig{Configs: []config.PeriodConfig{tt.config}} store := newTestStore(t, cm) - c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, tt.from, tt.from.Add(1*time.Hour)) - c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, tt.from, tt.from.Add(1*time.Hour)) - c3 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "buzz"}}, tt.from, tt.from.Add(1*time.Hour)) + chunkfmt, headfmt, err := tt.config.ChunkFormat() + require.NoError(t, err) + + c1 := createChunk(t, chunkfmt, headfmt, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, tt.from, tt.from.Add(1*time.Hour)) + c2 := createChunk(t, chunkfmt, headfmt, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, tt.from, tt.from.Add(1*time.Hour)) + c3 := createChunk(t, chunkfmt, headfmt, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "buzz"}}, tt.from, tt.from.Add(1*time.Hour)) require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ c1, c2, c3, @@ -121,7 +130,7 @@ func Test_SeriesCleaner(t *testing.T) { tables := store.indexTables() require.Len(t, tables, 1) // remove c1, c2 chunk - err := tables[0].DB.Update(func(tx *bbolt.Tx) error { + err = tables[0].DB.Update(func(tx *bbolt.Tx) error { return ForEachChunk(context.Background(), tx.Bucket(local.IndexBucketName), tt.config, func(entry retention.ChunkEntry) (deleteChunk bool, err error) { return entry.Labels.Get("bar") == "foo", nil }) @@ -227,10 +236,12 @@ func Benchmark_ChunkIterator(b *testing.B) { cm := storage.NewClientMetrics() defer cm.Unregister() store := newTestStore(b, cm) + chunkfmt, headfmt, err := allSchemas[0].config.ChunkFormat() + require.NoError(b, err) for i := 0; i < 100; i++ { require.NoError(b, store.Put(context.TODO(), []chunk.Chunk{ - createChunk(b, "1", + createChunk(b, chunkfmt, headfmt, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}, labels.Label{Name: "i", Value: fmt.Sprintf("%d", i)}}, allSchemas[0].from, allSchemas[0].from.Add(1*time.Hour)), }, diff --git a/pkg/storage/stores/shipper/index/compactor/util.go b/pkg/storage/stores/shipper/index/compactor/util.go index 2cfc693d04..27897df0dd 100644 --- a/pkg/storage/stores/shipper/index/compactor/util.go +++ b/pkg/storage/stores/shipper/index/compactor/util.go @@ -21,7 +21,7 @@ func unsafeGetString(buf []byte) string { return *((*string)(unsafe.Pointer(&buf))) } -func createChunk(t testing.TB, userID string, lbs labels.Labels, from model.Time, through model.Time) chunk.Chunk { +func createChunk(t testing.TB, chunkFormat byte, headBlockFmt chunkenc.HeadBlockFmt, userID string, lbs labels.Labels, from model.Time, through model.Time) chunk.Chunk { t.Helper() const ( targetSize = 1500 * 1024 @@ -31,7 +31,7 @@ func createChunk(t testing.TB, userID string, lbs labels.Labels, from model.Time labelsBuilder.Set(labels.MetricName, "logs") metric := labelsBuilder.Labels() fp := ingesterclient.Fingerprint(lbs) - chunkEnc := chunkenc.NewMemChunk(chunkenc.EncSnappy, chunkenc.DefaultHeadBlockFmt, blockSize, targetSize) + chunkEnc := chunkenc.NewMemChunk(chunkFormat, chunkenc.EncSnappy, headBlockFmt, blockSize, targetSize) for ts := from; !ts.After(through); ts = ts.Add(1 * time.Minute) { require.NoError(t, chunkEnc.Append(&logproto.Entry{ diff --git a/pkg/storage/stores/tsdb/builder_test.go b/pkg/storage/stores/tsdb/builder_test.go index 52accda20a..a3d6aebfd6 100644 --- a/pkg/storage/stores/tsdb/builder_test.go +++ b/pkg/storage/stores/tsdb/builder_test.go @@ -46,7 +46,7 @@ func Test_Build(t *testing.T) { } t.Run("writes index to disk with from/through bounds of series in filename", func(t *testing.T) { - ctx, builder, tmpDir := setup(index.LiveFormat) + ctx, builder, tmpDir := setup(index.FormatV3) _, err := builder.Build(ctx, tmpDir, func(from, through model.Time, checksum uint32) Identifier { return &fakeIdentifier{ @@ -66,7 +66,7 @@ func Test_Build(t *testing.T) { }) t.Run("sorts symbols before writing to the index", func(t *testing.T) { - ctx, builder, tmpDir := setup(index.LiveFormat) + ctx, builder, tmpDir := setup(index.FormatV3) _, err := builder.Build(ctx, tmpDir, func(from, through model.Time, checksum uint32) Identifier { return &fakeIdentifier{ parentPath: tmpDir, diff --git a/pkg/storage/stores/tsdb/compactor.go b/pkg/storage/stores/tsdb/compactor.go index 4b19983e87..e6cf35e5cd 100644 --- a/pkg/storage/stores/tsdb/compactor.go +++ b/pkg/storage/stores/tsdb/compactor.go @@ -47,7 +47,12 @@ func (i indexProcessor) OpenCompactedIndexFile(ctx context.Context, path, tableN } }() - builder := NewBuilder(index.LiveFormat) + indexFormat, err := periodConfig.TSDBFormat() + if err != nil { + return nil, err + } + + builder := NewBuilder(indexFormat) err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) { builder.AddSeries(lbls.Copy(), fp, chks) }, labels.MustNewMatcher(labels.MatchEqual, "", "")) @@ -149,7 +154,12 @@ func (t *tableCompactor) CompactTable() error { } } - builder, err := setupBuilder(t.ctx, userID, existingUserIndexSet, multiTenantIndices) + indexType, err := t.periodConfig.TSDBFormat() + if err != nil { + return err + } + + builder, err := setupBuilder(t.ctx, indexType, userID, existingUserIndexSet, multiTenantIndices) if err != nil { return err } @@ -169,7 +179,12 @@ func (t *tableCompactor) CompactTable() error { continue } - builder, err := setupBuilder(t.ctx, userID, srcIdxSet, []Index{}) + indexType, err := t.periodConfig.TSDBFormat() + if err != nil { + return err + } + + builder, err := setupBuilder(t.ctx, indexType, userID, srcIdxSet, []Index{}) if err != nil { return err } @@ -191,9 +206,9 @@ func (t *tableCompactor) CompactTable() error { // setupBuilder creates a Builder for a single user. // It combines the users index from multiTenantIndexes and its existing compacted index(es) -func setupBuilder(ctx context.Context, userID string, sourceIndexSet compactor.IndexSet, multiTenantIndexes []Index) (*Builder, error) { +func setupBuilder(ctx context.Context, indexType int, userID string, sourceIndexSet compactor.IndexSet, multiTenantIndexes []Index) (*Builder, error) { sourceIndexes := sourceIndexSet.ListSourceFiles() - builder := NewBuilder(index.LiveFormat) + builder := NewBuilder(indexType) // add users index from multi-tenant indexes to the builder for _, idx := range multiTenantIndexes { diff --git a/pkg/storage/stores/tsdb/compactor_test.go b/pkg/storage/stores/tsdb/compactor_test.go index 0b6b2d09d8..226eeb0036 100644 --- a/pkg/storage/stores/tsdb/compactor_test.go +++ b/pkg/storage/stores/tsdb/compactor_test.go @@ -115,9 +115,9 @@ func (m *mockIndexSet) SetCompactedIndex(compactedIndex compactor.CompactedIndex return nil } -func setupMultiTenantIndex(t *testing.T, userStreams map[string][]stream, destDir string, ts time.Time) string { +func setupMultiTenantIndex(t *testing.T, indexFormat int, userStreams map[string][]stream, destDir string, ts time.Time) string { require.NoError(t, util.EnsureDirectory(destDir)) - b := NewBuilder(index.LiveFormat) + b := NewBuilder(indexFormat) for userID, streams := range userStreams { for _, stream := range streams { lb := labels.NewBuilder(stream.labels) @@ -153,9 +153,9 @@ func setupMultiTenantIndex(t *testing.T, userStreams map[string][]stream, destDi return dst.Path() } -func setupPerTenantIndex(t *testing.T, streams []stream, destDir string, ts time.Time) string { +func setupPerTenantIndex(t *testing.T, indexFormat int, streams []stream, destDir string, ts time.Time) string { require.NoError(t, util.EnsureDirectory(destDir)) - b := NewBuilder(index.LiveFormat) + b := NewBuilder(indexFormat) for _, stream := range streams { b.AddSeries( stream.labels, @@ -496,8 +496,8 @@ func TestCompactor_Compact(t *testing.T) { t.Run(name, func(t *testing.T) { tempDir := t.TempDir() objectStoragePath := filepath.Join(tempDir, objectsStorageDirName) - tablePathInStorage := filepath.Join(objectStoragePath, tableName) - tableWorkingDirectory := filepath.Join(tempDir, workingDirName, tableName) + tablePathInStorage := filepath.Join(objectStoragePath, tableName.prefix) + tableWorkingDirectory := filepath.Join(tempDir, workingDirName, tableName.prefix) require.NoError(t, util.EnsureDirectory(objectStoragePath)) require.NoError(t, util.EnsureDirectory(tablePathInStorage)) @@ -520,7 +520,9 @@ func TestCompactor_Compact(t *testing.T) { userStreams[userID] = append(userStreams[userID], stream) } } - setupMultiTenantIndex(t, userStreams, tablePathInStorage, multiTenantIndexConfig.createdAt) + indexFormat, err := periodConfig.TSDBFormat() + require.NoError(t, err) + setupMultiTenantIndex(t, indexFormat, userStreams, tablePathInStorage, multiTenantIndexConfig.createdAt) } // setup per-tenant indexes i.e compacted ones @@ -538,7 +540,9 @@ func TestCompactor_Compact(t *testing.T) { stream = buildStream(streamConfig.labels, streamConfig.chunkMetas, "") streams = append(streams, stream) } - setupPerTenantIndex(t, streams, filepath.Join(tablePathInStorage, userID), perTenantIndexConfig.createdAt) + indexFormat, err := periodConfig.TSDBFormat() + require.NoError(t, err) + setupPerTenantIndex(t, indexFormat, streams, filepath.Join(tablePathInStorage, userID), perTenantIndexConfig.createdAt) } } @@ -546,7 +550,7 @@ func TestCompactor_Compact(t *testing.T) { objectClient, err := local.NewFSObjectClient(local.FSConfig{Directory: objectStoragePath}) require.NoError(t, err) - _, commonPrefixes, err := objectClient.List(context.Background(), tableName, "/") + _, commonPrefixes, err := objectClient.List(context.Background(), tableName.prefix, "/") require.NoError(t, err) initializedIndexSets := map[string]compactor.IndexSet{} @@ -554,26 +558,26 @@ func TestCompactor_Compact(t *testing.T) { existingUserIndexSets := make(map[string]compactor.IndexSet, len(commonPrefixes)) for _, commonPrefix := range commonPrefixes { userID := path.Base(string(commonPrefix)) - idxSet, err := newMockIndexSet(userID, tableName, filepath.Join(tableWorkingDirectory, userID), objectClient) + idxSet, err := newMockIndexSet(userID, tableName.prefix, filepath.Join(tableWorkingDirectory, userID), objectClient) require.NoError(t, err) existingUserIndexSets[userID] = idxSet initializedIndexSets[userID] = idxSet } - commonIndexSet, err := newMockIndexSet("", tableName, tableWorkingDirectory, objectClient) + commonIndexSet, err := newMockIndexSet("", tableName.prefix, tableWorkingDirectory, objectClient) require.NoError(t, err) // build TableCompactor and compact the index tCompactor := newTableCompactor(context.Background(), commonIndexSet, existingUserIndexSets, func(userID string) (compactor.IndexSet, error) { - idxSet, err := newMockIndexSet(userID, tableName, filepath.Join(tableWorkingDirectory, userID), objectClient) + idxSet, err := newMockIndexSet(userID, tableName.prefix, filepath.Join(tableWorkingDirectory, userID), objectClient) require.NoError(t, err) initializedIndexSetsMtx.Lock() defer initializedIndexSetsMtx.Unlock() initializedIndexSets[userID] = idxSet return idxSet, nil - }, config.PeriodConfig{}) + }, periodConfig) require.NoError(t, tCompactor.CompactTable()) @@ -869,7 +873,7 @@ func setupCompactedIndex(t *testing.T) *testContext { } indexBuckets := indexBuckets(now, now, []config.TableRange{periodConfig.GetIndexTableNumberRange(config.DayTime{Time: now})}) tableName := indexBuckets[0] - tableInterval := retention.ExtractIntervalFromTableName(tableName) + tableInterval := retention.ExtractIntervalFromTableName(tableName.prefix) // shiftTableStart shift tableInterval.Start by the given amount of milliseconds. // It is used for building chunkmetas relative to start time of the table. shiftTableStart := func(ms int64) int64 { @@ -881,7 +885,9 @@ func setupCompactedIndex(t *testing.T) *testContext { userID := buildUserID(0) buildCompactedIndex := func() *compactedIndex { - builder := NewBuilder(index.LiveFormat) + indexFormat, err := periodConfig.TSDBFormat() + require.NoError(t, err) + builder := NewBuilder(indexFormat) stream := buildStream(lbls1, buildChunkMetas(shiftTableStart(0), shiftTableStart(10)), "") builder.AddSeries(stream.labels, stream.fp, stream.chunks) @@ -890,7 +896,7 @@ func setupCompactedIndex(t *testing.T) *testContext { builder.FinalizeChunks() - return newCompactedIndex(context.Background(), tableName, buildUserID(0), t.TempDir(), periodConfig, builder) + return newCompactedIndex(context.Background(), tableName.prefix, buildUserID(0), t.TempDir(), periodConfig, builder) } expectedChunkEntries := map[string][]retention.ChunkEntry{ diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index d9a715452f..2837c8d602 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -382,23 +382,71 @@ func TestBuildLegacyWALs(t *testing.T) { dir := t.TempDir() secondStoreDate := parseDate("2023-01-02") - schemaCfg := config.SchemaConfig{ - Configs: []config.PeriodConfig{ - { - IndexType: config.TSDBType, - ObjectType: config.StorageTypeFileSystem, - IndexTables: config.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, + schemaCfgs := []config.SchemaConfig{ + { + Configs: []config.PeriodConfig{ + { + Schema: "v11", + IndexType: config.TSDBType, + ObjectType: config.StorageTypeFileSystem, + IndexTables: config.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + }, + { + Schema: "v11", + From: config.DayTime{Time: timeToModelTime(secondStoreDate)}, + IndexType: config.TSDBType, + ObjectType: config.StorageTypeFileSystem, + IndexTables: config.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, }, }, - { - From: config.DayTime{Time: timeToModelTime(secondStoreDate)}, - IndexType: config.TSDBType, - ObjectType: config.StorageTypeFileSystem, - IndexTables: config.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, + }, { + Configs: []config.PeriodConfig{ + { + Schema: "v12", + IndexType: config.TSDBType, + ObjectType: config.StorageTypeFileSystem, + IndexTables: config.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + }, + { + Schema: "v12", + From: config.DayTime{Time: timeToModelTime(secondStoreDate)}, + IndexType: config.TSDBType, + ObjectType: config.StorageTypeFileSystem, + IndexTables: config.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + }, + }, + }, { + Configs: []config.PeriodConfig{ + { + Schema: "v13", + IndexType: config.TSDBType, + ObjectType: config.StorageTypeFileSystem, + IndexTables: config.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + }, + { + Schema: "v13", + From: config.DayTime{Time: timeToModelTime(secondStoreDate)}, + IndexType: config.TSDBType, + ObjectType: config.StorageTypeFileSystem, + IndexTables: config.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, }, }, }, @@ -460,39 +508,41 @@ func TestBuildLegacyWALs(t *testing.T) { shipperCfg.ActiveIndexDirectory = filepath.Join(dir) shipperCfg.CacheLocation = filepath.Join(dir, "cache") - for _, tc := range []struct { - name, store string - tableRange config.TableRange - expectedChunks []logproto.ChunkRef - }{ - { - name: "query-period-1", - store: "period-1", - tableRange: schemaCfg.Configs[0].GetIndexTableNumberRange(config.DayTime{Time: timeToModelTime(secondStoreDate.Add(-time.Millisecond))}), - expectedChunks: chunkMetasToLogProtoChunkRefs(c.User, c.Labels.Hash(), c.Chunks[:2]), - }, - { - name: "query-period-2", - store: "period-2", - tableRange: schemaCfg.Configs[1].GetIndexTableNumberRange(config.DayTime{Time: math.MaxInt64}), - expectedChunks: chunkMetasToLogProtoChunkRefs(c.User, c.Labels.Hash(), c.Chunks[1:]), - }, - } { - t.Run(tc.name, func(t *testing.T) { - store, stop, err := NewStore(tc.store, IndexCfg{Config: shipperCfg}, schemaCfg, nil, fsObjectClient, &zeroValueLimits{}, tc.tableRange, nil, nil, log.NewNopLogger(), nil) - require.Nil(t, err) - - refs, err := store.GetChunkRefs( - context.Background(), - c.User, - 0, timeToModelTime(secondStoreDate.Add(48*time.Hour)), - labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), - ) - require.Nil(t, err) - require.Equal(t, tc.expectedChunks, refs) - - stop() - }) + for _, schema := range schemaCfgs { + schemaCfg := schema + for _, tc := range []struct { + name, store string + tableRange config.TableRange + expectedChunks []logproto.ChunkRef + }{ + { + name: "query-period-1", + store: "period-1", + tableRange: schemaCfg.Configs[0].GetIndexTableNumberRange(config.DayTime{Time: timeToModelTime(secondStoreDate.Add(-time.Millisecond))}), + expectedChunks: chunkMetasToLogProtoChunkRefs(c.User, c.Labels.Hash(), c.Chunks[:2]), + }, + { + name: "query-period-2", + store: "period-2", + tableRange: schemaCfg.Configs[1].GetIndexTableNumberRange(config.DayTime{Time: math.MaxInt64}), + expectedChunks: chunkMetasToLogProtoChunkRefs(c.User, c.Labels.Hash(), c.Chunks[1:]), + }, + } { + t.Run(tc.name, func(t *testing.T) { + store, stop, err := NewStore(tc.store, IndexCfg{Config: shipperCfg}, schemaCfg, nil, fsObjectClient, &zeroValueLimits{}, tc.tableRange, nil, nil, log.NewNopLogger(), nil) + require.Nil(t, err) + refs, err := store.GetChunkRefs( + context.Background(), + c.User, + 0, timeToModelTime(secondStoreDate.Add(48*time.Hour)), + labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), + ) + require.Nil(t, err) + require.Equal(t, tc.expectedChunks, refs) + + stop() + }) + } } } diff --git a/pkg/storage/stores/tsdb/index/index.go b/pkg/storage/stores/tsdb/index/index.go index b6e2c749cf..eb9681160f 100644 --- a/pkg/storage/stores/tsdb/index/index.go +++ b/pkg/storage/stores/tsdb/index/index.go @@ -61,9 +61,6 @@ const ( fingerprintInterval = 1 << 10 millisecondsInHour = int64(time.Hour / time.Millisecond) - - // The format that will be written by this process - LiveFormat = FormatV3 ) type indexWriterStage uint8 @@ -269,9 +266,9 @@ func NewWriterWithVersion(ctx context.Context, version int, fn string) (*Writer, return iw, nil } -// NewWriter returns a new Writer to the given filename. It serializes data according to the `LiveFormat` version -func NewWriter(ctx context.Context, fn string) (*Writer, error) { - return NewWriterWithVersion(ctx, LiveFormat, fn) +// NewWriter returns a new Writer to the given filename. +func NewWriter(ctx context.Context, indexFormat int, fn string) (*Writer, error) { + return NewWriterWithVersion(ctx, indexFormat, fn) } func (w *Writer) write(bufs ...[]byte) error { diff --git a/pkg/storage/stores/tsdb/index/index_test.go b/pkg/storage/stores/tsdb/index/index_test.go index 9f650caba1..6ef9ebee01 100644 --- a/pkg/storage/stores/tsdb/index/index_test.go +++ b/pkg/storage/stores/tsdb/index/index_test.go @@ -129,7 +129,7 @@ func TestIndexRW_Create_Open(t *testing.T) { fn := filepath.Join(dir, IndexFilename) // An empty index must still result in a readable file. - iw, err := NewWriter(context.Background(), fn) + iw, err := NewWriter(context.Background(), FormatV3, fn) require.NoError(t, err) require.NoError(t, iw.Close()) @@ -153,7 +153,7 @@ func TestIndexRW_Postings(t *testing.T) { fn := filepath.Join(dir, IndexFilename) - iw, err := NewWriter(context.Background(), fn) + iw, err := NewWriter(context.Background(), FormatV3, fn) require.NoError(t, err) series := []labels.Labels{ @@ -233,7 +233,7 @@ func TestPostingsMany(t *testing.T) { fn := filepath.Join(dir, IndexFilename) - iw, err := NewWriter(context.Background(), fn) + iw, err := NewWriter(context.Background(), FormatV3, fn) require.NoError(t, err) // Create a label in the index which has 999 values. @@ -367,7 +367,7 @@ func TestPersistence_index_e2e(t *testing.T) { }) } - iw, err := NewWriter(context.Background(), filepath.Join(dir, IndexFilename)) + iw, err := NewWriter(context.Background(), FormatV3, filepath.Join(dir, IndexFilename)) require.NoError(t, err) syms := []string{} diff --git a/pkg/storage/stores/tsdb/index_shipper_querier.go b/pkg/storage/stores/tsdb/index_shipper_querier.go index 218baa7c13..e72f1c9053 100644 --- a/pkg/storage/stores/tsdb/index_shipper_querier.go +++ b/pkg/storage/stores/tsdb/index_shipper_querier.go @@ -42,7 +42,7 @@ func (i *indexShipperQuerier) indices(ctx context.Context, from, through model.T // Ensure we query both per tenant and multitenant TSDBs idxBuckets := indexBuckets(from, through, []config.TableRange{i.tableRange}) for _, bkt := range idxBuckets { - if err := i.shipper.ForEachConcurrent(ctx, bkt, user, func(multitenant bool, idx shipper_index.Index) error { + if err := i.shipper.ForEachConcurrent(ctx, bkt.prefix, user, func(multitenant bool, idx shipper_index.Index) error { impl, ok := idx.(Index) if !ok { return fmt.Errorf("unexpected shipper index type: %T", idx) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index bbe25d7f73..f4ea7978d9 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -151,18 +151,26 @@ func (m *tsdbManager) Start() (err error) { return nil } +type chunkInfo struct { + chunkMetas index.ChunkMetas + tsdbFormat int +} + func (m *tsdbManager) buildFromHead(heads *tenantHeads, shipper indexshipper.IndexShipper, tableRanges []config.TableRange) (err error) { periods := make(map[string]*Builder) if err := heads.forAll(func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas) error { // chunks may overlap index period bounds, in which case they're written to multiple - pds := make(map[string]index.ChunkMetas) + pds := make(map[string]chunkInfo) for _, chk := range chks { idxBuckets := indexBuckets(chk.From(), chk.Through(), tableRanges) for _, bucket := range idxBuckets { - pds[bucket] = append(pds[bucket], chk) + chkinfo := pds[bucket.prefix] + chkinfo.chunkMetas = append(chkinfo.chunkMetas, chk) + chkinfo.tsdbFormat = bucket.tsdbFormat + pds[bucket.prefix] = chkinfo } } @@ -172,10 +180,11 @@ func (m *tsdbManager) buildFromHead(heads *tenantHeads, shipper indexshipper.Ind withTenant := lb.Labels() // Add the chunks to all relevant builders - for pd, matchingChks := range pds { + for pd, chkinfo := range pds { + matchingChks := chkinfo.chunkMetas b, ok := periods[pd] if !ok { - b = NewBuilder(index.LiveFormat) + b = NewBuilder(chkinfo.tsdbFormat) periods[pd] = b } @@ -290,13 +299,19 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier, legacy boo return nil } -func indexBuckets(from, through model.Time, tableRanges config.TableRanges) (res []string) { +type indexInfo struct { + prefix string + tsdbFormat int +} + +func indexBuckets(from, through model.Time, tableRanges config.TableRanges) (res []indexInfo) { start := from.Time().UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) end := through.Time().UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) for cur := start; cur <= end; cur++ { cfg := tableRanges.ConfigForTableNumber(cur) if cfg != nil { - res = append(res, cfg.IndexTables.Prefix+strconv.Itoa(int(cur))) + tsdbFormat, _ := cfg.TSDBFormat() // Ignoring error, as any valid period config should return valid format. + res = append(res, indexInfo{prefix: cfg.IndexTables.Prefix + strconv.Itoa(int(cur)), tsdbFormat: tsdbFormat}) } } if len(res) == 0 { diff --git a/pkg/storage/stores/tsdb/querier_test.go b/pkg/storage/stores/tsdb/querier_test.go index e83fa824e3..09d978c27e 100644 --- a/pkg/storage/stores/tsdb/querier_test.go +++ b/pkg/storage/stores/tsdb/querier_test.go @@ -24,7 +24,7 @@ func mustParseLabels(s string) labels.Labels { func TestQueryIndex(t *testing.T) { dir := t.TempDir() - b := NewBuilder(index.LiveFormat) + b := NewBuilder(index.FormatV3) cases := []struct { labels labels.Labels chunks []index.ChunkMeta diff --git a/pkg/storage/stores/tsdb/util_test.go b/pkg/storage/stores/tsdb/util_test.go index 8fa8ee3588..dab02b1d2c 100644 --- a/pkg/storage/stores/tsdb/util_test.go +++ b/pkg/storage/stores/tsdb/util_test.go @@ -18,7 +18,7 @@ type LoadableSeries struct { } func BuildIndex(t testing.TB, dir string, cases []LoadableSeries, opts IndexOpts) *TSDBFile { - b := NewBuilder(index.LiveFormat) + b := NewBuilder(index.FormatV3) for _, s := range cases { b.AddSeries(s.Labels, model.Fingerprint(s.Labels.Hash()), s.Chunks) diff --git a/pkg/storage/util_test.go b/pkg/storage/util_test.go index 52e77d0134..c71323323f 100644 --- a/pkg/storage/util_test.go +++ b/pkg/storage/util_test.go @@ -78,23 +78,23 @@ func assertSeries(t *testing.T, expected, actual []logproto.Series) { } } -func newLazyChunk(stream logproto.Stream) *LazyChunk { +func newLazyChunk(chunkFormat byte, headfmt chunkenc.HeadBlockFmt, stream logproto.Stream) *LazyChunk { return &LazyChunk{ Fetcher: nil, IsValid: true, - Chunk: newChunk(stream), + Chunk: newChunk(chunkFormat, headfmt, stream), } } -func newLazyInvalidChunk(stream logproto.Stream) *LazyChunk { +func newLazyInvalidChunk(chunkFormat byte, headfmt chunkenc.HeadBlockFmt, stream logproto.Stream) *LazyChunk { return &LazyChunk{ Fetcher: nil, IsValid: false, - Chunk: newChunk(stream), + Chunk: newChunk(chunkFormat, headfmt, stream), } } -func newChunk(stream logproto.Stream) chunk.Chunk { +func newChunk(chunkFormat byte, headBlockFmt chunkenc.HeadBlockFmt, stream logproto.Stream) chunk.Chunk { lbs, err := syntax.ParseLabels(stream.Labels) if err != nil { panic(err) @@ -105,7 +105,7 @@ func newChunk(stream logproto.Stream) chunk.Chunk { lbs = builder.Labels() } from, through := loki_util.RoundToMilliseconds(stream.Entries[0].Timestamp, stream.Entries[len(stream.Entries)-1].Timestamp) - chk := chunkenc.NewMemChunk(chunkenc.EncGZIP, chunkenc.DefaultHeadBlockFmt, 256*1024, 0) + chk := chunkenc.NewMemChunk(chunkFormat, chunkenc.EncGZIP, headBlockFmt, 256*1024, 0) for _, e := range stream.Entries { _ = chk.Append(&e) } @@ -165,10 +165,10 @@ var ( _ chunkclient.Client = &mockChunkStoreClient{} ) -func newMockChunkStore(streams []*logproto.Stream) *mockChunkStore { +func newMockChunkStore(chunkFormat byte, headfmt chunkenc.HeadBlockFmt, streams []*logproto.Stream) *mockChunkStore { chunks := make([]chunk.Chunk, 0, len(streams)) for _, s := range streams { - chunks = append(chunks, newChunk(*s)) + chunks = append(chunks, newChunk(chunkFormat, headfmt, *s)) } return &mockChunkStore{schemas: config.SchemaConfig{}, chunks: chunks, client: &mockChunkStoreClient{chunks: chunks, scfg: config.SchemaConfig{}}} } @@ -383,4 +383,4 @@ var streamsFixture = []*logproto.Stream{ }, }, } -var storeFixture = newMockChunkStore(streamsFixture) +var storeFixture = newMockChunkStore(chunkenc.ChunkFormatV3, chunkenc.UnorderedWithNonIndexedLabelsHeadBlockFmt, streamsFixture) diff --git a/tools/tsdb/migrate-versions/main.go b/tools/tsdb/migrate-versions/main.go index 8edf8b0f48..ebf62c388f 100644 --- a/tools/tsdb/migrate-versions/main.go +++ b/tools/tsdb/migrate-versions/main.go @@ -36,7 +36,7 @@ const ( ) var ( - desiredVer = tsdb_index.LiveFormat + desiredVer = tsdb_index.FormatV3 tableNumMin, tableNumMax int64 newTablePrefix string ) diff --git a/tools/tsdb/tsdb-map/main.go b/tools/tsdb/tsdb-map/main.go index 5b65e0857b..9078c877f2 100644 --- a/tools/tsdb/tsdb-map/main.go +++ b/tools/tsdb/tsdb-map/main.go @@ -67,7 +67,12 @@ func main() { panic(err) } - builder := tsdb.NewBuilder(index.LiveFormat) + indexFormat, err := periodConfig.TSDBFormat() + if err != nil { + panic(err) + } + + builder := tsdb.NewBuilder(indexFormat) log.Println("Loading index into memory")