prometheus/tsdb/chunkenc/xor.go

509 lines
12 KiB
Go
Raw Normal View History

2017-04-10 18:59:45 +00:00
// 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.
2018-04-08 09:28:30 +00:00
// 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.
2017-04-28 09:34:35 +00:00
// 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
2016-11-15 09:33:34 +00:00
import (
"encoding/binary"
2016-11-15 09:33:34 +00:00
"math"
"math/bits"
Style cleanup of all the changes in sparsehistogram so far A lot of this code was hacked together, literally during a hackathon. This commit intends not to change the code substantially, but just make the code obey the usual style practices. A (possibly incomplete) list of areas: * Generally address linter warnings. * The `pgk` directory is deprecated as per dev-summit. No new packages should be added to it. I moved the new `pkg/histogram` package to `model` anticipating what's proposed in #9478. * Make the naming of the Sparse Histogram more consistent. Including abbreviations, there were just too many names for it: SparseHistogram, Histogram, Histo, hist, his, shs, h. The idea is to call it "Histogram" in general. Only add "Sparse" if it is needed to avoid confusion with conventional Histograms (which is rare because the TSDB really has no notion of conventional Histograms). Use abbreviations only in local scope, and then really abbreviate (not just removing three out of seven letters like in "Histo"). This is in the spirit of https://github.com/golang/go/wiki/CodeReviewComments#variable-names * Several other minor name changes. * A lot of formatting of doc comments. For one, following https://github.com/golang/go/wiki/CodeReviewComments#comment-sentences , but also layout question, anticipating how things will look like when rendered by `godoc` (even where `godoc` doesn't render them right now because they are for unexported types or not a doc comment at all but just a normal code comment - consistency is queen!). * Re-enabled `TestQueryLog` and `TestEndopints` (they pass now, leaving them disabled was presumably an oversight). * Bucket iterator for histogram.Histogram is now created with a method. * HistogramChunk.iterator now allows iterator recycling. (I think @dieterbe only commented it out because he was confused by the question in the comment.) * HistogramAppender.Append panics now because we decided to treat staleness marker differently. Signed-off-by: beorn7 <beorn@grafana.com>
2021-10-09 13:57:07 +00:00
"github.com/prometheus/prometheus/model/histogram"
2016-11-15 09:33:34 +00:00
)
const (
chunkCompactCapacityThreshold = 32
)
2016-11-15 09:33:34 +00:00
// XORChunk holds XOR encoded sample data.
type XORChunk struct {
b bstream
2016-11-15 09:33:34 +00:00
}
// NewXORChunk returns a new chunk with XOR encoding of the given size.
2016-12-31 09:10:27 +00:00
func NewXORChunk() *XORChunk {
2016-12-09 19:45:46 +00:00
b := make([]byte, 2, 128)
return &XORChunk{b: bstream{stream: b, count: 0}}
}
2017-03-19 16:05:01 +00:00
// Encoding returns the encoding type.
2016-12-09 19:45:46 +00:00
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()
2016-11-15 09:33:34 +00:00
}
// NumSamples returns the number of samples in the chunk.
func (c *XORChunk) NumSamples() int {
return int(binary.BigEndian.Uint16(c.Bytes()))
}
// Compact implements the Chunk interface.
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
}
}
2016-11-15 09:33:34 +00:00
// Appender implements the Chunk interface.
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 14:02:05 +00:00
// 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() != ValNone {
}
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 it.numTotal == 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.
2016-12-09 19:45:46 +00:00
br: newBReader(c.b.bytes()[2:]),
numTotal: binary.BigEndian.Uint16(c.b.bytes()),
storage: Added Chunks{Queryable/Querier/SeriesSet/Series/Iteratable. Added generic Merge{SeriesSet/Querier} implementation. (#7005) * storage: Added Chunks{Queryable/Querier/SeriesSet/Series/Iteratable. Added generic Merge{SeriesSet/Querier} implementation. ## Rationales: In many places (e.g. chunk Remote read, Thanos Receive fetching chunk from TSDB), we operate on encoded chunks not samples. This means that we unnecessary decode/encode, wasting CPU, time and memory. This PR adds chunk iterator interfaces and makes the merge code to be reused between both seriesSets I will make the use of it in following PR inside tsdb itself. For now fanout implements it and mergers. All merges now also allows passing series mergers. This opens doors for custom deduplications other than TSDB vertical ones (e.g. offline one we have in Thanos). ## Changes * Added Chunk versions of all iterating methods. It all starts in Querier/ChunkQuerier. The plan is that Storage will implement both chunked and samples. * Added Seek to chunks.Iterator interface for iterating over chunks. * NewMergeChunkQuerier was added; Both this and NewMergeQuerier are now using generigMergeQuerier to share the code. Generic code was added. * Improved tests. * Added some TODO for further simplifications in next PRs. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Addressed Brian's comments. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Moved s/Labeled/SeriesLabels as per Krasi suggestion. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Addressed Krasi's comments. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Second iteration of Krasi comments. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Another round of comments. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com>
2020-03-24 20:15:47 +00:00
t: math.MinInt64,
}
2016-11-15 09:33:34 +00:00
}
// Iterator implements the Chunk interface.
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 14:02:05 +00:00
// 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)
2016-11-15 09:33:34 +00:00
}
type xorAppender struct {
b *bstream
2016-11-15 09:33:34 +00:00
t int64
v float64
tDelta uint64
2016-11-20 13:33:00 +00:00
leading uint8
trailing uint8
2016-11-15 09:33:34 +00:00
}
func (a *xorAppender) AppendHistogram(t int64, h *histogram.Histogram) {
Style cleanup of all the changes in sparsehistogram so far A lot of this code was hacked together, literally during a hackathon. This commit intends not to change the code substantially, but just make the code obey the usual style practices. A (possibly incomplete) list of areas: * Generally address linter warnings. * The `pgk` directory is deprecated as per dev-summit. No new packages should be added to it. I moved the new `pkg/histogram` package to `model` anticipating what's proposed in #9478. * Make the naming of the Sparse Histogram more consistent. Including abbreviations, there were just too many names for it: SparseHistogram, Histogram, Histo, hist, his, shs, h. The idea is to call it "Histogram" in general. Only add "Sparse" if it is needed to avoid confusion with conventional Histograms (which is rare because the TSDB really has no notion of conventional Histograms). Use abbreviations only in local scope, and then really abbreviate (not just removing three out of seven letters like in "Histo"). This is in the spirit of https://github.com/golang/go/wiki/CodeReviewComments#variable-names * Several other minor name changes. * A lot of formatting of doc comments. For one, following https://github.com/golang/go/wiki/CodeReviewComments#comment-sentences , but also layout question, anticipating how things will look like when rendered by `godoc` (even where `godoc` doesn't render them right now because they are for unexported types or not a doc comment at all but just a normal code comment - consistency is queen!). * Re-enabled `TestQueryLog` and `TestEndopints` (they pass now, leaving them disabled was presumably an oversight). * Bucket iterator for histogram.Histogram is now created with a method. * HistogramChunk.iterator now allows iterator recycling. (I think @dieterbe only commented it out because he was confused by the question in the comment.) * HistogramAppender.Append panics now because we decided to treat staleness marker differently. Signed-off-by: beorn7 <beorn@grafana.com>
2021-10-09 13:57:07 +00:00
panic("appended a histogram to an xor chunk")
}
func (a *xorAppender) AppendFloatHistogram(t int64, h *histogram.FloatHistogram) {
panic("appended a float histogram to an xor chunk")
}
2016-12-31 09:10:27 +00:00
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)
2016-11-20 13:33:00 +00:00
} 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)
2016-11-20 13:33:00 +00:00
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.
Style cleanup of all the changes in sparsehistogram so far A lot of this code was hacked together, literally during a hackathon. This commit intends not to change the code substantially, but just make the code obey the usual style practices. A (possibly incomplete) list of areas: * Generally address linter warnings. * The `pgk` directory is deprecated as per dev-summit. No new packages should be added to it. I moved the new `pkg/histogram` package to `model` anticipating what's proposed in #9478. * Make the naming of the Sparse Histogram more consistent. Including abbreviations, there were just too many names for it: SparseHistogram, Histogram, Histo, hist, his, shs, h. The idea is to call it "Histogram" in general. Only add "Sparse" if it is needed to avoid confusion with conventional Histograms (which is rare because the TSDB really has no notion of conventional Histograms). Use abbreviations only in local scope, and then really abbreviate (not just removing three out of seven letters like in "Histo"). This is in the spirit of https://github.com/golang/go/wiki/CodeReviewComments#variable-names * Several other minor name changes. * A lot of formatting of doc comments. For one, following https://github.com/golang/go/wiki/CodeReviewComments#comment-sentences , but also layout question, anticipating how things will look like when rendered by `godoc` (even where `godoc` doesn't render them right now because they are for unexported types or not a doc comment at all but just a normal code comment - consistency is queen!). * Re-enabled `TestQueryLog` and `TestEndopints` (they pass now, leaving them disabled was presumably an oversight). * Bucket iterator for histogram.Histogram is now created with a method. * HistogramChunk.iterator now allows iterator recycling. (I think @dieterbe only commented it out because he was confused by the question in the comment.) * HistogramAppender.Append panics now because we decided to treat staleness marker differently. Signed-off-by: beorn7 <beorn@grafana.com>
2021-10-09 13:57:07 +00:00
//
// TODO(beorn7): This seems to needlessly jump to large bit
// sizes even for very small deviations from zero. Timestamp
// compression can probably benefit from some smaller bit
// buckets. See also what was done for histogram encoding in
// varbit.go.
2016-11-20 13:33:00 +00:00
switch {
case dod == 0:
a.b.writeBit(zero)
2016-12-07 14:37:37 +00:00
case bitRange(dod, 14):
a.b.writeBits(0b10, 2)
a.b.writeBits(uint64(dod), 14)
2016-12-07 14:37:37 +00:00
case bitRange(dod, 17):
a.b.writeBits(0b110, 3)
a.b.writeBits(uint64(dod), 17)
2016-12-07 14:37:37 +00:00
case bitRange(dod, 20):
a.b.writeBits(0b1110, 4)
a.b.writeBits(uint64(dod), 20)
2016-11-20 13:33:00 +00:00
default:
a.b.writeBits(0b1111, 4)
a.b.writeBits(uint64(dod), 64)
2016-11-15 09:33:34 +00:00
}
2016-11-20 13:33:00 +00:00
a.writeVDelta(v)
2016-11-15 09:33:34 +00:00
}
2016-11-20 13:33:00 +00:00
a.t = t
a.v = v
binary.BigEndian.PutUint16(a.b.bytes(), num+1)
a.tDelta = tDelta
2016-11-15 09:33:34 +00:00
}
// bitRange returns whether the given integer can be represented by nbits.
// See docs/bstream.md.
2016-12-07 14:37:37 +00:00
func bitRange(x int64, nbits uint8) bool {
return -((1<<(nbits-1))-1) <= x && x <= 1<<(nbits-1)
}
2016-11-20 13:33:00 +00:00
func (a *xorAppender) writeVDelta(v float64) {
xorWrite(a.b, v, a.v, &a.leading, &a.trailing)
}
2016-11-15 09:33:34 +00:00
type xorIterator struct {
br bstreamReader
2016-11-20 13:33:00 +00:00
numTotal uint16
numRead uint16
t int64
val float64
leading uint8
trailing uint8
tDelta uint64
2016-11-20 13:33:00 +00:00
err error
}
func (it *xorIterator) Seek(t int64) ValueType {
if it.err != nil {
return ValNone
}
for t > it.t || it.numRead == 0 {
if it.Next() == ValNone {
return ValNone
}
}
return ValFloat
}
2017-01-02 12:27:52 +00:00
func (it *xorIterator) At() (int64, float64) {
2016-11-20 13:33:00 +00:00
return it.t, it.val
}
func (it *xorIterator) AtHistogram() (int64, *histogram.Histogram) {
panic("cannot call xorIterator.AtHistogram")
}
func (it *xorIterator) AtFloatHistogram() (int64, *histogram.FloatHistogram) {
panic("cannot call xorIterator.AtFloatHistogram")
}
func (it *xorIterator) AtT() int64 {
return it.t
}
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() ValueType {
2016-11-20 13:33:00 +00:00
if it.err != nil || it.numRead == it.numTotal {
return ValNone
2016-11-20 13:33:00 +00:00
}
if it.numRead == 0 {
t, err := binary.ReadVarint(&it.br)
2016-11-20 13:33:00 +00:00
if err != nil {
it.err = err
return ValNone
2016-11-20 13:33:00 +00:00
}
v, err := it.br.readBits(64)
if err != nil {
it.err = err
return ValNone
2016-11-20 13:33:00 +00:00
}
it.t = t
2016-11-20 13:33:00 +00:00
it.val = math.Float64frombits(v)
it.numRead++
return ValFloat
2016-11-20 13:33:00 +00:00
}
if it.numRead == 1 {
tDelta, err := binary.ReadUvarint(&it.br)
2016-11-20 13:33:00 +00:00
if err != nil {
it.err = err
return ValNone
2016-11-20 13:33:00 +00:00
}
it.tDelta = tDelta
it.t = it.t + int64(it.tDelta)
2016-11-20 13:33:00 +00:00
return it.readValue()
2016-11-20 13:33:00 +00:00
}
var d byte
2016-11-20 13:33:00 +00:00
// 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()
}
2016-11-20 13:33:00 +00:00
if err != nil {
it.err = err
return ValNone
2016-11-20 13:33:00 +00:00
}
if bit == zero {
break
}
d |= 1
}
var sz uint8
var dod int64
2016-11-20 13:33:00 +00:00
switch d {
case 0b0:
2016-11-20 13:33:00 +00:00
// dod == 0
case 0b10:
2016-11-20 13:33:00 +00:00
sz = 14
case 0b110:
2016-11-20 13:33:00 +00:00
sz = 17
case 0b1110:
2016-11-20 13:33:00 +00:00
sz = 20
case 0b1111:
// Do not use fast because it's very unlikely it will succeed.
2016-11-20 13:33:00 +00:00
bits, err := it.br.readBits(64)
if err != nil {
it.err = err
return ValNone
2016-11-20 13:33:00 +00:00
}
dod = int64(bits)
2016-11-20 13:33:00 +00:00
}
if sz != 0 {
bits, err := it.br.readBitsFast(sz)
if err != nil {
bits, err = it.br.readBits(sz)
}
2016-11-20 13:33:00 +00:00
if err != nil {
it.err = err
return ValNone
2016-11-20 13:33:00 +00:00
}
// Account for negative numbers, which come back as high unsigned numbers.
// See docs/bstream.md.
2016-11-20 13:33:00 +00:00
if bits > (1 << (sz - 1)) {
bits -= 1 << sz
2016-11-20 13:33:00 +00:00
}
dod = int64(bits)
2016-11-20 13:33:00 +00:00
}
it.tDelta = uint64(int64(it.tDelta) + dod)
it.t = it.t + int64(it.tDelta)
2016-11-20 13:33:00 +00:00
return it.readValue()
}
2016-11-20 13:33:00 +00:00
func (it *xorIterator) readValue() ValueType {
err := xorRead(&it.br, &it.val, &it.leading, &it.trailing)
2016-11-20 13:33:00 +00:00
if err != nil {
it.err = err
return ValNone
2016-11-20 13:33:00 +00:00
}
it.numRead++
return ValFloat
}
func xorWrite(b *bstream, newValue, currentValue float64, leading, trailing *uint8) {
delta := math.Float64bits(newValue) ^ math.Float64bits(currentValue)
if delta == 0 {
b.writeBit(zero)
return
}
b.writeBit(one)
newLeading := uint8(bits.LeadingZeros64(delta))
newTrailing := uint8(bits.TrailingZeros64(delta))
// Clamp number of leading zeros to avoid overflow when encoding.
if newLeading >= 32 {
newLeading = 31
}
if *leading != 0xff && newLeading >= *leading && newTrailing >= *trailing {
// In this case, we stick with the current leading/trailing.
b.writeBit(zero)
b.writeBits(delta>>*trailing, 64-int(*leading)-int(*trailing))
return
}
// Update leading/trailing for the caller.
*leading, *trailing = newLeading, newTrailing
b.writeBit(one)
b.writeBits(uint64(newLeading), 5)
// Note that if newLeading == newTrailing == 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 - newLeading - newTrailing
b.writeBits(uint64(sigbits), 6)
b.writeBits(delta>>newTrailing, int(sigbits))
}
func xorRead(br *bstreamReader, value *float64, leading, trailing *uint8) error {
bit, err := br.readBitFast()
if err != nil {
bit, err = br.readBit()
}
if err != nil {
return err
}
if bit == zero {
return nil
}
bit, err = br.readBitFast()
if err != nil {
bit, err = br.readBit()
}
if err != nil {
return err
}
var (
bits uint64
newLeading, newTrailing, mbits uint8
)
2016-11-20 13:33:00 +00:00
if bit == zero {
// Reuse leading/trailing zero bits.
newLeading, newTrailing = *leading, *trailing
mbits = 64 - newLeading - newTrailing
2016-11-20 13:33:00 +00:00
} else {
bits, err = br.readBitsFast(5)
if err != nil {
bits, err = br.readBits(5)
}
if err != nil {
return err
2016-11-20 13:33:00 +00:00
}
newLeading = uint8(bits)
2016-11-20 13:33:00 +00:00
bits, err = br.readBitsFast(6)
if err != nil {
bits, err = br.readBits(6)
}
2016-11-20 13:33:00 +00:00
if err != nil {
return err
}
mbits = uint8(bits)
// 0 significant bits here means we overflowed and we actually
// need 64; see comment in xrWrite.
if mbits == 0 {
mbits = 64
2016-11-20 13:33:00 +00:00
}
newTrailing = 64 - newLeading - mbits
// Update leading/trailing zero bits for the caller.
*leading, *trailing = newLeading, newTrailing
2016-11-20 13:33:00 +00:00
}
bits, err = br.readBitsFast(mbits)
if err != nil {
bits, err = br.readBits(mbits)
}
if err != nil {
return err
}
vbits := math.Float64bits(*value)
vbits ^= bits << newTrailing
*value = math.Float64frombits(vbits)
return nil
2016-11-15 09:33:34 +00:00
}