Pin `chunk` and `index` format to `schema` version. (#10213)

We pin all three `Chunk`, `HeadBlock` and `TSDB` Version to `schema`
version in period config.


This is the following mapping (after being discussed with @owen-d and
@sandeepsukhani )

`v12` (current existing schema) - ChunkFormatV3 (UnorderedHeadBlock) +
TSDBv2
`v13` (introducing new schema) - ChunkFormatV4
(UnorderedWithNonIndexedLabelsHeadBlockFmt) + TSDBv3

Note the new schema `v13` supports the latest chunk and index format.

**NOTES for Reviewer**

1. General approach is we removed the idea of `index.LiveFormat`,
`chunkenc.DefaultChunkFormat` and `chunkenc.DefaultHeadBlockFmt` and
made following two changes. These variables were used before to tie
chunk and tsdb formats specific to Loki versions. This PR remove that
coupling and pin these formats to `schema` version instead.

1. These variables were replaced with explicit chunk and index formats
within those packages (and it's tests)
2. If these variables were used outside it's own packages say by
ingester, compactor, etc. Then we extract correct chunk and index
versions from the `schema` config.

2. Add two methods to `periodConfig`. (1) `ChunkFormat()` returning
chunk and head format tied to schema (2) `TSDBFormat()` returning tsdb
format tied to schema.

2. Other ideas I thought of doing but didn't end up doing is make
`ChunkFormat` and `IndexFormat` as separate type (rather than `byte` and
`int` currently. Similar to `HeadBlockFmt` type). But didnt' do it
eventually to keep the PR small and don't want to complicate with lots
of changes.

4. Moved couple of test cases from `chunkenc` to `config` package,
because the test case was actually testing methods on `schemaconfig` and
it was creating cycling dependencies.

---------

Signed-off-by: Kaviraj <kavirajkanagaraj@gmail.com>
pull/10334/head
Kaviraj Kanagaraj 2 years ago committed by GitHub
parent 3303844cb0
commit bbfb13c3e4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 84
      pkg/chunkenc/memchunk.go
  2. 143
      pkg/chunkenc/memchunk_test.go
  3. 18
      pkg/chunkenc/unordered_test.go
  4. 2
      pkg/chunkenc/util_test.go
  5. 8
      pkg/ingester/checkpoint.go
  6. 2
      pkg/ingester/checkpoint_test.go
  7. 2
      pkg/ingester/chunk_test.go
  8. 8
      pkg/ingester/encoding_test.go
  9. 2
      pkg/ingester/flush_test.go
  10. 77
      pkg/ingester/instance.go
  11. 11
      pkg/ingester/instance_test.go
  12. 2
      pkg/ingester/recovery.go
  13. 33
      pkg/ingester/stream.go
  14. 49
      pkg/ingester/stream_test.go
  15. 5
      pkg/ingester/streams_map_test.go
  16. 18
      pkg/storage/async_store_test.go
  17. 1915
      pkg/storage/batch_test.go
  18. 121
      pkg/storage/chunk/chunk_test.go
  19. 37
      pkg/storage/config/schema_config.go
  20. 123
      pkg/storage/config/schema_config_test.go
  21. 23
      pkg/storage/hack/main.go
  22. 84
      pkg/storage/lazy_chunk_test.go
  23. 147
      pkg/storage/store_test.go
  24. 4
      pkg/storage/stores/indexshipper/compactor/retention/retention_test.go
  25. 12
      pkg/storage/stores/indexshipper/compactor/retention/util_test.go
  26. 4
      pkg/storage/stores/series/index/schema.go
  27. 12
      pkg/storage/stores/series/index/schema_config.go
  28. 17
      pkg/storage/stores/series_store_write_test.go
  29. 12
      pkg/storage/stores/shipper/index/compactor/compacted_index_test.go
  30. 33
      pkg/storage/stores/shipper/index/compactor/iterator_test.go
  31. 4
      pkg/storage/stores/shipper/index/compactor/util.go
  32. 4
      pkg/storage/stores/tsdb/builder_test.go
  33. 25
      pkg/storage/stores/tsdb/compactor.go
  34. 38
      pkg/storage/stores/tsdb/compactor_test.go
  35. 146
      pkg/storage/stores/tsdb/head_manager_test.go
  36. 9
      pkg/storage/stores/tsdb/index/index.go
  37. 8
      pkg/storage/stores/tsdb/index/index_test.go
  38. 2
      pkg/storage/stores/tsdb/index_shipper_querier.go
  39. 27
      pkg/storage/stores/tsdb/manager.go
  40. 2
      pkg/storage/stores/tsdb/querier_test.go
  41. 2
      pkg/storage/stores/tsdb/util_test.go
  42. 18
      pkg/storage/util_test.go
  43. 2
      tools/tsdb/migrate-versions/main.go
  44. 7
      tools/tsdb/tsdb-map/main.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

@ -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"),
))))

@ -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.

@ -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)

@ -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
}

@ -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,

@ -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) {

@ -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",

@ -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())

@ -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)
}

@ -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))
}
})
}

@ -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",

@ -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
}

@ -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
}

@ -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",

@ -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),
}))

File diff suppressed because it is too large Load Diff

@ -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")
}

@ -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
}

@ -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))
})
}
}

@ -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)

@ -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)
})
}
}
}

@ -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{}{}

@ -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

@ -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)

@ -590,3 +590,7 @@ func (v11Entries) GetLabelNamesForSeries(bucket Bucket, seriesID []byte) ([]Quer
type v12Entries struct {
v11Entries
}
type v13Entries struct {
v12Entries
}

@ -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

@ -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 {

@ -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)

@ -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)),
},

@ -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{

@ -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,

@ -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 {

@ -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{

@ -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()
})
}
}
}

@ -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 {

@ -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{}

@ -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)

@ -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 {

@ -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

@ -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)

@ -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)

@ -36,7 +36,7 @@ const (
)
var (
desiredVer = tsdb_index.LiveFormat
desiredVer = tsdb_index.FormatV3
tableNumMin, tableNumMax int64
newTablePrefix string
)

@ -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")

Loading…
Cancel
Save