prometheus/tsdb/chunkenc/xor.go
Bryan Boreham d166da7b59
tsdb: stop saving a copy of last 4 samples in memSeries (#11296)
* TSDB chunks: remove race between writing and reading

Because the data is stored as a bit-stream, the last byte in the stream
could change if the stream is appended to after an Iterator is obtained.
Copy the last byte when the Iterator is created, so we don't have to
read it later.

Clarify in comments that concurrent Iterator and Appender are allowed,
but the chunk must not be modified while an Iterator is created.
(This was already the case, in order to copy the bstream slice header.)

* TSDB: stop saving last 4 samples in memSeries

This extra copy of the last 4 samples was introduced to avoid a race
condition between reading the last byte of the chunk and writing to it.

But now we have fixed that by having `bstreamReader` copy the last byte,
we don't need to copy the last 4 samples.

This change saves 56 bytes per series, which is very worthwhile when
you have millions or tens of millions of series.

* TSDB: tidy up stopIterator re-use

Previous changes have left this code duplicating some lines; pull
them out to a separate function and tidy up.

* TSDB head_test: stop checking when iterators are wrapped

The behaviour has changed so chunk iterators are only wrapped when
transaction isolation requires them to stop short of the end.
This makes tests fail which are checking the type.

Tests should check the observable behaviour, not the type.

Signed-off-by: Bryan Boreham <bjboreham@gmail.com>
Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com>
Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com>
2022-09-27 19:32:05 +05:30

471 lines
11 KiB
Go

// Copyright 2017 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// The code in this file was largely written by Damian Gryski as part of
// https://github.com/dgryski/go-tsz and published under the license below.
// It was modified to accommodate reading from byte slices without modifying
// the underlying bytes, which would panic when reading from mmap'd
// read-only byte slices.
// Copyright (c) 2015,2016 Damian Gryski <damian@gryski.com>
// All rights reserved.
// Redistribution and use in source and binary forms, with or without
// modification, are permitted provided that the following conditions are met:
// * Redistributions of source code must retain the above copyright notice,
// this list of conditions and the following disclaimer.
//
// * Redistributions in binary form must reproduce the above copyright notice,
// this list of conditions and the following disclaimer in the documentation
// and/or other materials provided with the distribution.
//
// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
// DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
// FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
// DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
// SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
// CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
// OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
package chunkenc
import (
"encoding/binary"
"math"
"math/bits"
)
const (
chunkCompactCapacityThreshold = 32
)
// XORChunk holds XOR encoded sample data.
type XORChunk struct {
b bstream
}
// NewXORChunk returns a new chunk with XOR encoding of the given size.
func NewXORChunk() *XORChunk {
b := make([]byte, 2, 128)
return &XORChunk{b: bstream{stream: b, count: 0}}
}
// Encoding returns the encoding type.
func (c *XORChunk) Encoding() Encoding {
return EncXOR
}
// Bytes returns the underlying byte slice of the chunk.
func (c *XORChunk) Bytes() []byte {
return c.b.bytes()
}
// NumSamples returns the number of samples in the chunk.
func (c *XORChunk) NumSamples() int {
return int(binary.BigEndian.Uint16(c.Bytes()))
}
func (c *XORChunk) Compact() {
if l := len(c.b.stream); cap(c.b.stream) > l+chunkCompactCapacityThreshold {
buf := make([]byte, l)
copy(buf, c.b.stream)
c.b.stream = buf
}
}
// Appender implements the Chunk interface.
// It is not valid to call Appender() multiple times concurrently or to use multiple
// Appenders on the same chunk.
func (c *XORChunk) Appender() (Appender, error) {
it := c.iterator(nil)
// To get an appender we must know the state it would have if we had
// appended all existing data from scratch.
// We iterate through the end and populate via the iterator's state.
for it.Next() {
}
if err := it.Err(); err != nil {
return nil, err
}
a := &xorAppender{
b: &c.b,
t: it.t,
v: it.val,
tDelta: it.tDelta,
leading: it.leading,
trailing: it.trailing,
}
if binary.BigEndian.Uint16(a.b.bytes()) == 0 {
a.leading = 0xff
}
return a, nil
}
func (c *XORChunk) iterator(it Iterator) *xorIterator {
if xorIter, ok := it.(*xorIterator); ok {
xorIter.Reset(c.b.bytes())
return xorIter
}
return &xorIterator{
// The first 2 bytes contain chunk headers.
// We skip that for actual samples.
br: newBReader(c.b.bytes()[2:]),
numTotal: binary.BigEndian.Uint16(c.b.bytes()),
t: math.MinInt64,
}
}
// Iterator implements the Chunk interface.
// Iterator() must not be called concurrently with any modifications to the chunk,
// but after it returns you can use an Iterator concurrently with an Appender or
// other Iterators.
func (c *XORChunk) Iterator(it Iterator) Iterator {
return c.iterator(it)
}
type xorAppender struct {
b *bstream
t int64
v float64
tDelta uint64
leading uint8
trailing uint8
}
func (a *xorAppender) Append(t int64, v float64) {
var tDelta uint64
num := binary.BigEndian.Uint16(a.b.bytes())
if num == 0 {
buf := make([]byte, binary.MaxVarintLen64)
for _, b := range buf[:binary.PutVarint(buf, t)] {
a.b.writeByte(b)
}
a.b.writeBits(math.Float64bits(v), 64)
} else if num == 1 {
tDelta = uint64(t - a.t)
buf := make([]byte, binary.MaxVarintLen64)
for _, b := range buf[:binary.PutUvarint(buf, tDelta)] {
a.b.writeByte(b)
}
a.writeVDelta(v)
} else {
tDelta = uint64(t - a.t)
dod := int64(tDelta - a.tDelta)
// Gorilla has a max resolution of seconds, Prometheus milliseconds.
// Thus we use higher value range steps with larger bit size.
switch {
case dod == 0:
a.b.writeBit(zero)
case bitRange(dod, 14):
a.b.writeBits(0b10, 2)
a.b.writeBits(uint64(dod), 14)
case bitRange(dod, 17):
a.b.writeBits(0b110, 3)
a.b.writeBits(uint64(dod), 17)
case bitRange(dod, 20):
a.b.writeBits(0b1110, 4)
a.b.writeBits(uint64(dod), 20)
default:
a.b.writeBits(0b1111, 4)
a.b.writeBits(uint64(dod), 64)
}
a.writeVDelta(v)
}
a.t = t
a.v = v
binary.BigEndian.PutUint16(a.b.bytes(), num+1)
a.tDelta = tDelta
}
// bitRange returns whether the given integer can be represented by nbits.
// See docs/bstream.md.
func bitRange(x int64, nbits uint8) bool {
return -((1<<(nbits-1))-1) <= x && x <= 1<<(nbits-1)
}
func (a *xorAppender) writeVDelta(v float64) {
vDelta := math.Float64bits(v) ^ math.Float64bits(a.v)
if vDelta == 0 {
a.b.writeBit(zero)
return
}
a.b.writeBit(one)
leading := uint8(bits.LeadingZeros64(vDelta))
trailing := uint8(bits.TrailingZeros64(vDelta))
// Clamp number of leading zeros to avoid overflow when encoding.
if leading >= 32 {
leading = 31
}
if a.leading != 0xff && leading >= a.leading && trailing >= a.trailing {
a.b.writeBit(zero)
a.b.writeBits(vDelta>>a.trailing, 64-int(a.leading)-int(a.trailing))
} else {
a.leading, a.trailing = leading, trailing
a.b.writeBit(one)
a.b.writeBits(uint64(leading), 5)
// Note that if leading == trailing == 0, then sigbits == 64. But that value doesn't actually fit into the 6 bits we have.
// Luckily, we never need to encode 0 significant bits, since that would put us in the other case (vdelta == 0).
// So instead we write out a 0 and adjust it back to 64 on unpacking.
sigbits := 64 - leading - trailing
a.b.writeBits(uint64(sigbits), 6)
a.b.writeBits(vDelta>>trailing, int(sigbits))
}
}
type xorIterator struct {
br bstreamReader
numTotal uint16
numRead uint16
t int64
val float64
leading uint8
trailing uint8
tDelta uint64
err error
}
func (it *xorIterator) Seek(t int64) bool {
if it.err != nil {
return false
}
for t > it.t || it.numRead == 0 {
if !it.Next() {
return false
}
}
return true
}
func (it *xorIterator) At() (int64, float64) {
return it.t, it.val
}
func (it *xorIterator) Err() error {
return it.err
}
func (it *xorIterator) Reset(b []byte) {
// The first 2 bytes contain chunk headers.
// We skip that for actual samples.
it.br = newBReader(b[2:])
it.numTotal = binary.BigEndian.Uint16(b)
it.numRead = 0
it.t = 0
it.val = 0
it.leading = 0
it.trailing = 0
it.tDelta = 0
it.err = nil
}
func (it *xorIterator) Next() bool {
if it.err != nil || it.numRead == it.numTotal {
return false
}
if it.numRead == 0 {
t, err := binary.ReadVarint(&it.br)
if err != nil {
it.err = err
return false
}
v, err := it.br.readBits(64)
if err != nil {
it.err = err
return false
}
it.t = t
it.val = math.Float64frombits(v)
it.numRead++
return true
}
if it.numRead == 1 {
tDelta, err := binary.ReadUvarint(&it.br)
if err != nil {
it.err = err
return false
}
it.tDelta = tDelta
it.t = it.t + int64(it.tDelta)
return it.readValue()
}
var d byte
// read delta-of-delta
for i := 0; i < 4; i++ {
d <<= 1
bit, err := it.br.readBitFast()
if err != nil {
bit, err = it.br.readBit()
}
if err != nil {
it.err = err
return false
}
if bit == zero {
break
}
d |= 1
}
var sz uint8
var dod int64
switch d {
case 0b0:
// dod == 0
case 0b10:
sz = 14
case 0b110:
sz = 17
case 0b1110:
sz = 20
case 0b1111:
// Do not use fast because it's very unlikely it will succeed.
bits, err := it.br.readBits(64)
if err != nil {
it.err = err
return false
}
dod = int64(bits)
}
if sz != 0 {
bits, err := it.br.readBitsFast(sz)
if err != nil {
bits, err = it.br.readBits(sz)
}
if err != nil {
it.err = err
return false
}
// Account for negative numbers, which come back as high unsigned numbers.
// See docs/bstream.md.
if bits > (1 << (sz - 1)) {
bits -= 1 << sz
}
dod = int64(bits)
}
it.tDelta = uint64(int64(it.tDelta) + dod)
it.t = it.t + int64(it.tDelta)
return it.readValue()
}
func (it *xorIterator) readValue() bool {
bit, err := it.br.readBitFast()
if err != nil {
bit, err = it.br.readBit()
}
if err != nil {
it.err = err
return false
}
if bit == zero {
// it.val = it.val
} else {
bit, err := it.br.readBitFast()
if err != nil {
bit, err = it.br.readBit()
}
if err != nil {
it.err = err
return false
}
if bit == zero {
// reuse leading/trailing zero bits
// it.leading, it.trailing = it.leading, it.trailing
} else {
bits, err := it.br.readBitsFast(5)
if err != nil {
bits, err = it.br.readBits(5)
}
if err != nil {
it.err = err
return false
}
it.leading = uint8(bits)
bits, err = it.br.readBitsFast(6)
if err != nil {
bits, err = it.br.readBits(6)
}
if err != nil {
it.err = err
return false
}
mbits := uint8(bits)
// 0 significant bits here means we overflowed and we actually need 64; see comment in encoder
if mbits == 0 {
mbits = 64
}
it.trailing = 64 - it.leading - mbits
}
mbits := 64 - it.leading - it.trailing
bits, err := it.br.readBitsFast(mbits)
if err != nil {
bits, err = it.br.readBits(mbits)
}
if err != nil {
it.err = err
return false
}
vbits := math.Float64bits(it.val)
vbits ^= bits << it.trailing
it.val = math.Float64frombits(vbits)
}
it.numRead++
return true
}
// OOOXORChunk holds a XORChunk and overrides the Encoding() method.
type OOOXORChunk struct {
*XORChunk
}
func (c *OOOXORChunk) Encoding() Encoding {
return EncOOOXOR
}