Merge pull request #1502 from prometheus/beorn7/storage7

Beorn7/storage7
This commit is contained in:
Björn Rabenstein 2016-03-24 14:23:16 +01:00
commit d43f380cf2
13 changed files with 1530 additions and 72 deletions

View file

@ -143,7 +143,7 @@ func init() {
)
cfg.fs.Var(
&local.DefaultChunkEncoding, "storage.local.chunk-encoding-version",
"Which chunk encoding version to use for newly created chunks. Currently supported is 0 (delta encoding) and 1 (double-delta encoding).",
"Which chunk encoding version to use for newly created chunks. Currently supported is 0 (delta encoding), 1 (double-delta encoding), and 2 (double-delta encoding with variable bit-width).",
)
// Index cache sizes.
cfg.fs.IntVar(

View file

@ -469,7 +469,7 @@ func (t *Test) clear() {
}
var closer testutil.Closer
t.storage, closer = local.NewTestStorage(t, 1)
t.storage, closer = local.NewTestStorage(t, 2)
t.closeStorage = closer.Close
t.queryEngine = NewEngine(t.storage, nil)

View file

@ -24,7 +24,7 @@ import (
)
func TestRuleEval(t *testing.T) {
storage, closer := local.NewTestStorage(t, 1)
storage, closer := local.NewTestStorage(t, 2)
defer closer.Close()
engine := promql.NewEngine(storage, nil)
now := model.Now()

View file

@ -15,6 +15,7 @@ package local
import (
"container/list"
"errors"
"fmt"
"io"
"sort"
@ -26,9 +27,11 @@ import (
"github.com/prometheus/prometheus/storage/metric"
)
// The DefaultChunkEncoding can be changed via a flag.
// DefaultChunkEncoding can be changed via a flag.
var DefaultChunkEncoding = doubleDelta
var errChunkBoundsExceeded = errors.New("attempted access outside of chunk boundaries")
type chunkEncoding byte
// String implements flag.Value.
@ -43,6 +46,8 @@ func (ce *chunkEncoding) Set(s string) error {
*ce = delta
case "1":
*ce = doubleDelta
case "2":
*ce = varbit
default:
return fmt.Errorf("invalid chunk encoding: %s", s)
}
@ -52,6 +57,7 @@ func (ce *chunkEncoding) Set(s string) error {
const (
delta chunkEncoding = iota
doubleDelta
varbit
)
// chunkDesc contains meta-data for a chunk. Pay special attention to the
@ -306,6 +312,21 @@ func rangeValues(it chunkIterator, in metric.Interval) ([]model.SamplePair, erro
return result, it.err()
}
// addToOverflowChunk is a utility function that creates a new chunk as overflow
// chunk, adds the provided sample to it, and returns a chunk slice containing
// the provided old chunk followed by the new overflow chunk.
func addToOverflowChunk(c chunk, s model.SamplePair) ([]chunk, error) {
overflowChunks, err := newChunk().add(s)
if err != nil {
return nil, err
}
return []chunk{c, overflowChunks[0]}, nil
}
// transcodeAndAdd is a utility function that transcodes the dst chunk into the
// provided src chunk (plus the necessary overflow chunks) and then adds the
// provided sample. It returns the new chunks (transcoded plus overflow) with
// the new sample at the end.
func transcodeAndAdd(dst chunk, src chunk, s model.SamplePair) ([]chunk, error) {
chunkOps.WithLabelValues(transcode).Inc()
@ -334,7 +355,7 @@ func transcodeAndAdd(dst chunk, src chunk, s model.SamplePair) ([]chunk, error)
}
// newChunk creates a new chunk according to the encoding set by the
// defaultChunkEncoding flag.
// DefaultChunkEncoding flag.
func newChunk() chunk {
chunk, err := newChunkForEncoding(DefaultChunkEncoding)
if err != nil {
@ -349,6 +370,8 @@ func newChunkForEncoding(encoding chunkEncoding) (chunk, error) {
return newDeltaEncodedChunk(d1, d0, true, chunkLen), nil
case doubleDelta:
return newDoubleDeltaEncodedChunk(d1, d0, true, chunkLen), nil
case varbit:
return newVarbitChunk(varbitZeroEncoding), nil
default:
return nil, fmt.Errorf("unknown chunk encoding: %v", encoding)
}

View file

@ -74,6 +74,7 @@ func newDeltaEncodedChunk(tb, vb deltaBytes, isInt bool, length int) *deltaEncod
// add implements chunk.
func (c deltaEncodedChunk) add(s model.SamplePair) ([]chunk, error) {
// TODO(beorn7): Since we return &c, this method might cause an unnecessary allocation.
if c.len() == 0 {
c = c[:deltaHeaderBytes]
binary.LittleEndian.PutUint64(c[deltaHeaderBaseTimeOffset:], uint64(s.Timestamp))
@ -86,11 +87,7 @@ func (c deltaEncodedChunk) add(s model.SamplePair) ([]chunk, error) {
// Do we generally have space for another sample in this chunk? If not,
// overflow into a new one.
if remainingBytes < sampleSize {
overflowChunks, err := newChunk().add(s)
if err != nil {
return nil, err
}
return []chunk{&c, overflowChunks[0]}, nil
return addToOverflowChunk(&c, s)
}
baseValue := c.baseValue()
@ -130,11 +127,7 @@ func (c deltaEncodedChunk) add(s model.SamplePair) ([]chunk, error) {
return transcodeAndAdd(newDeltaEncodedChunk(ntb, nvb, nInt, cap(c)), &c, s)
}
// Chunk is already half full. Better create a new one and save the transcoding efforts.
overflowChunks, err := newChunk().add(s)
if err != nil {
return nil, err
}
return []chunk{&c, overflowChunks[0]}, nil
return addToOverflowChunk(&c, s)
}
offset := len(c)

View file

@ -81,6 +81,7 @@ func newDoubleDeltaEncodedChunk(tb, vb deltaBytes, isInt bool, length int) *doub
// add implements chunk.
func (c doubleDeltaEncodedChunk) add(s model.SamplePair) ([]chunk, error) {
// TODO(beorn7): Since we return &c, this method might cause an unnecessary allocation.
if c.len() == 0 {
return c.addFirstSample(s), nil
}
@ -98,11 +99,7 @@ func (c doubleDeltaEncodedChunk) add(s model.SamplePair) ([]chunk, error) {
// Do we generally have space for another sample in this chunk? If not,
// overflow into a new one.
if remainingBytes < sampleSize {
overflowChunks, err := newChunk().add(s)
if err != nil {
return nil, err
}
return []chunk{&c, overflowChunks[0]}, nil
return addToOverflowChunk(&c, s)
}
projectedTime := c.baseTime() + model.Time(c.len())*c.baseTimeDelta()
@ -136,11 +133,7 @@ func (c doubleDeltaEncodedChunk) add(s model.SamplePair) ([]chunk, error) {
return transcodeAndAdd(newDoubleDeltaEncodedChunk(ntb, nvb, nInt, cap(c)), &c, s)
}
// Chunk is already half full. Better create a new one and save the transcoding efforts.
overflowChunks, err := newChunk().add(s)
if err != nil {
return nil, err
}
return []chunk{&c, overflowChunks[0]}, nil
return addToOverflowChunk(&c, s)
}
offset := len(c)

View file

@ -653,6 +653,10 @@ func TestCheckpointAndLoadSeriesMapAndHeadsChunkType1(t *testing.T) {
testCheckpointAndLoadSeriesMapAndHeads(t, 1)
}
func TestCheckpointAndLoadSeriesMapAndHeadsChunkType2(t *testing.T) {
testCheckpointAndLoadSeriesMapAndHeads(t, 2)
}
func TestCheckpointAndLoadFPMappings(t *testing.T) {
p, closer := newTestPersistence(t, 1)
defer closer.Close()
@ -758,6 +762,10 @@ func TestFingerprintsModifiedBeforeChunkType1(t *testing.T) {
testFingerprintsModifiedBefore(t, 1)
}
func TestFingerprintsModifiedBeforeChunkType2(t *testing.T) {
testFingerprintsModifiedBefore(t, 2)
}
func testDropArchivedMetric(t *testing.T, encoding chunkEncoding) {
p, closer := newTestPersistence(t, encoding)
defer closer.Close()
@ -822,6 +830,10 @@ func TestDropArchivedMetricChunkType1(t *testing.T) {
testDropArchivedMetric(t, 1)
}
func TestDropArchivedMetricChunkType2(t *testing.T) {
testDropArchivedMetric(t, 2)
}
type incrementalBatch struct {
fpToMetric index.FingerprintMetricMapping
expectedLnToLvs index.LabelNameLabelValuesMapping
@ -1002,6 +1014,10 @@ func TestIndexingChunkType1(t *testing.T) {
testIndexing(t, 1)
}
func TestIndexingChunkType2(t *testing.T) {
testIndexing(t, 2)
}
func verifyIndexedState(i int, t *testing.T, b incrementalBatch, indexedFpsToMetrics index.FingerprintMetricMapping, p *persistence) {
p.waitForIndexing()
for fp, m := range indexedFpsToMetrics {

View file

@ -31,7 +31,7 @@ import (
)
func TestMatches(t *testing.T) {
storage, closer := NewTestStorage(t, 1)
storage, closer := NewTestStorage(t, 2)
defer closer.Close()
storage.archiveHighWatermark = 90
@ -266,7 +266,7 @@ func TestMatches(t *testing.T) {
}
func TestFingerprintsForLabels(t *testing.T) {
storage, closer := NewTestStorage(t, 1)
storage, closer := NewTestStorage(t, 2)
defer closer.Close()
samples := make([]*model.Sample, 100)
@ -348,7 +348,7 @@ func TestFingerprintsForLabels(t *testing.T) {
var benchLabelMatchingRes map[model.Fingerprint]metric.Metric
func BenchmarkLabelMatching(b *testing.B) {
s, closer := NewTestStorage(b, 1)
s, closer := NewTestStorage(b, 2)
defer closer.Close()
h := fnv.New64a()
@ -444,7 +444,7 @@ func TestRetentionCutoff(t *testing.T) {
now := model.Now()
insertStart := now.Add(-2 * time.Hour)
s, closer := NewTestStorage(t, 1)
s, closer := NewTestStorage(t, 2)
defer closer.Close()
// Stop maintenance loop to prevent actual purging.
@ -498,7 +498,7 @@ func TestDropMetrics(t *testing.T) {
now := model.Now()
insertStart := now.Add(-2 * time.Hour)
s, closer := NewTestStorage(t, 1)
s, closer := NewTestStorage(t, 2)
defer closer.Close()
chunkFileExists := func(fp model.Fingerprint) (bool, error) {
@ -605,7 +605,7 @@ func TestQuarantineMetric(t *testing.T) {
now := model.Now()
insertStart := now.Add(-2 * time.Hour)
s, closer := NewTestStorage(t, 1)
s, closer := NewTestStorage(t, 2)
defer closer.Close()
chunkFileExists := func(fp model.Fingerprint) (bool, error) {
@ -749,7 +749,7 @@ func testChunk(t *testing.T, encoding chunkEncoding) {
for m := range s.fpToSeries.iter() {
s.fpLocker.Lock(m.fp)
defer s.fpLocker.Unlock(m.fp) // TODO remove, see below
var values []model.SamplePair
for _, cd := range m.series.chunkDescs {
if cd.isEvicted() {
@ -772,7 +772,7 @@ func testChunk(t *testing.T, encoding chunkEncoding) {
t.Errorf("%d. Got %v; want %v", i, v.Value, samples[i].Value)
}
}
s.fpLocker.Unlock(m.fp)
//s.fpLocker.Unlock(m.fp)
}
log.Info("test done, closing")
}
@ -785,6 +785,10 @@ func TestChunkType1(t *testing.T) {
testChunk(t, 1)
}
func TestChunkType2(t *testing.T) {
testChunk(t, 2)
}
func testValueAtOrBeforeTime(t *testing.T, encoding chunkEncoding) {
samples := make(model.Samples, 10000)
for i := range samples {
@ -859,6 +863,10 @@ func TestValueAtTimeChunkType1(t *testing.T) {
testValueAtOrBeforeTime(t, 1)
}
func TestValueAtTimeChunkType2(t *testing.T) {
testValueAtOrBeforeTime(t, 2)
}
func benchmarkValueAtOrBeforeTime(b *testing.B, encoding chunkEncoding) {
samples := make(model.Samples, 10000)
for i := range samples {
@ -937,6 +945,10 @@ func BenchmarkValueAtTimeChunkType1(b *testing.B) {
benchmarkValueAtOrBeforeTime(b, 1)
}
func BenchmarkValueAtTimeChunkType2(b *testing.B) {
benchmarkValueAtOrBeforeTime(b, 2)
}
func testRangeValues(t *testing.T, encoding chunkEncoding) {
samples := make(model.Samples, 10000)
for i := range samples {
@ -1089,6 +1101,10 @@ func TestRangeValuesChunkType1(t *testing.T) {
testRangeValues(t, 1)
}
func TestRangeValuesChunkType2(t *testing.T) {
testRangeValues(t, 2)
}
func benchmarkRangeValues(b *testing.B, encoding chunkEncoding) {
samples := make(model.Samples, 10000)
for i := range samples {
@ -1133,6 +1149,10 @@ func BenchmarkRangeValuesChunkType1(b *testing.B) {
benchmarkRangeValues(b, 1)
}
func BenchmarkRangeValuesChunkType2(b *testing.B) {
benchmarkRangeValues(b, 2)
}
func testEvictAndPurgeSeries(t *testing.T, encoding chunkEncoding) {
samples := make(model.Samples, 10000)
for i := range samples {
@ -1284,6 +1304,10 @@ func TestEvictAndPurgeSeriesChunkType1(t *testing.T) {
testEvictAndPurgeSeries(t, 1)
}
func TestEvictAndPurgeSeriesChunkType2(t *testing.T) {
testEvictAndPurgeSeries(t, 2)
}
func testEvictAndLoadChunkDescs(t *testing.T, encoding chunkEncoding) {
samples := make(model.Samples, 10000)
for i := range samples {
@ -1386,6 +1410,10 @@ func BenchmarkAppendType1(b *testing.B) {
benchmarkAppend(b, 1)
}
func BenchmarkAppendType2(b *testing.B) {
benchmarkAppend(b, 2)
}
// Append a large number of random samples and then check if we can get them out
// of the storage alright.
func testFuzz(t *testing.T, encoding chunkEncoding) {
@ -1402,7 +1430,10 @@ func testFuzz(t *testing.T, encoding chunkEncoding) {
for _, sample := range samples {
s.Append(sample)
}
return verifyStorage(t, s, samples, 24*7*time.Hour)
if !verifyStorageRandom(t, s, samples) {
return false
}
return verifyStorageSequential(t, s, samples)
}
if err := quick.Check(check, nil); err != nil {
@ -1418,6 +1449,10 @@ func TestFuzzChunkType1(t *testing.T) {
testFuzz(t, 1)
}
func TestFuzzChunkType2(t *testing.T) {
testFuzz(t, 2)
}
// benchmarkFuzz is the benchmark version of testFuzz. The storage options are
// set such that evictions, checkpoints, and purging will happen concurrently,
// too. This benchmark will have a very long runtime (up to minutes). You can
@ -1462,11 +1497,12 @@ func benchmarkFuzz(b *testing.B, encoding chunkEncoding) {
for _, sample := range samples[start:middle] {
s.Append(sample)
}
verifyStorage(b, s.(*memorySeriesStorage), samples[:middle], o.PersistenceRetentionPeriod)
verifyStorageRandom(b, s.(*memorySeriesStorage), samples[:middle])
for _, sample := range samples[middle:end] {
s.Append(sample)
}
verifyStorage(b, s.(*memorySeriesStorage), samples[:end], o.PersistenceRetentionPeriod)
verifyStorageRandom(b, s.(*memorySeriesStorage), samples[:end])
verifyStorageSequential(b, s.(*memorySeriesStorage), samples)
}
}
@ -1478,17 +1514,20 @@ func BenchmarkFuzzChunkType1(b *testing.B) {
benchmarkFuzz(b, 1)
}
func BenchmarkFuzzChunkType2(b *testing.B) {
benchmarkFuzz(b, 2)
}
func createRandomSamples(metricName string, minLen int) model.Samples {
type valueCreator func() model.SampleValue
type deltaApplier func(model.SampleValue) model.SampleValue
var (
maxMetrics = 5
maxStreakLength = 500
maxTimeDelta = 10000
maxTimeDeltaFactor = 10
timestamp = model.Now() - model.Time(maxTimeDelta*maxTimeDeltaFactor*minLen/4) // So that some timestamps are in the future.
generators = []struct {
maxMetrics = 5
maxStreakLength = 2000
maxTimeDelta = 10000
timestamp = model.Now() - model.Time(maxTimeDelta*minLen) // So that some timestamps are in the future.
generators = []struct {
createValue valueCreator
applyDelta []deltaApplier
}{
@ -1532,6 +1571,28 @@ func createRandomSamples(metricName string, minLen int) model.Samples {
},
},
}
timestampIncrementers = []func(baseDelta model.Time) model.Time{
// Regular increments.
func(delta model.Time) model.Time {
return delta
},
// Jittered increments. σ is 1/100 of delta, e.g. 10ms for 10s scrape interval.
func(delta model.Time) model.Time {
return delta + model.Time(rand.NormFloat64()*float64(delta)/100)
},
// Regular increments, but missing a scrape with 10% chance.
func(delta model.Time) model.Time {
i := rand.Intn(100)
if i < 90 {
return delta
}
if i < 99 {
return 2 * delta
}
return 3 * delta
// Ignoring the case with more than two missed scrapes in a row.
},
}
)
// Prefill result with two samples with colliding metrics (to test fingerprint mapping).
@ -1563,13 +1624,16 @@ func createRandomSamples(metricName string, minLen int) model.Samples {
}
for len(result) < minLen {
// Pick a metric for this cycle.
metric := metrics[rand.Intn(len(metrics))]
timeDelta := rand.Intn(maxTimeDelta) + 1
generator := generators[rand.Intn(len(generators))]
createValue := generator.createValue
applyDelta := generator.applyDelta[rand.Intn(len(generator.applyDelta))]
incTimestamp := func() { timestamp += model.Time(timeDelta * (rand.Intn(maxTimeDeltaFactor) + 1)) }
var (
// Pick a metric for this cycle.
metric = metrics[rand.Intn(len(metrics))]
timeDelta = model.Time(rand.Intn(maxTimeDelta) + 1)
generator = generators[rand.Intn(len(generators))]
createValue = generator.createValue
applyDelta = generator.applyDelta[rand.Intn(len(generator.applyDelta))]
incTimestamp = timestampIncrementers[rand.Intn(len(timestampIncrementers))]
)
switch rand.Intn(4) {
case 0: // A single sample.
result = append(result, &model.Sample{
@ -1577,7 +1641,7 @@ func createRandomSamples(metricName string, minLen int) model.Samples {
Value: createValue(),
Timestamp: timestamp,
})
incTimestamp()
timestamp += incTimestamp(timeDelta)
case 1: // A streak of random sample values.
for n := rand.Intn(maxStreakLength); n >= 0; n-- {
result = append(result, &model.Sample{
@ -1585,7 +1649,7 @@ func createRandomSamples(metricName string, minLen int) model.Samples {
Value: createValue(),
Timestamp: timestamp,
})
incTimestamp()
timestamp += incTimestamp(timeDelta)
}
case 2: // A streak of sample values with incremental changes.
value := createValue()
@ -1595,7 +1659,7 @@ func createRandomSamples(metricName string, minLen int) model.Samples {
Value: value,
Timestamp: timestamp,
})
incTimestamp()
timestamp += incTimestamp(timeDelta)
value = applyDelta(value)
}
case 3: // A streak of constant sample values.
@ -1606,7 +1670,7 @@ func createRandomSamples(metricName string, minLen int) model.Samples {
Value: value,
Timestamp: timestamp,
})
incTimestamp()
timestamp += incTimestamp(timeDelta)
}
}
}
@ -1614,34 +1678,32 @@ func createRandomSamples(metricName string, minLen int) model.Samples {
return result
}
func verifyStorage(t testing.TB, s *memorySeriesStorage, samples model.Samples, maxAge time.Duration) bool {
func verifyStorageRandom(t testing.TB, s *memorySeriesStorage, samples model.Samples) bool {
s.WaitForIndexing()
result := true
for _, i := range rand.Perm(len(samples)) {
sample := samples[i]
if sample.Timestamp.Before(model.TimeFromUnixNano(time.Now().Add(-maxAge).UnixNano())) {
continue
// TODO: Once we have a guaranteed cutoff at the
// retention period, we can verify here that no results
// are returned.
}
fp, err := s.mapper.mapFP(sample.Metric.FastFingerprint(), sample.Metric)
if err != nil {
t.Fatal(err)
}
p := s.NewPreloader()
it := p.PreloadRange(fp, sample.Timestamp, sample.Timestamp)
it := p.PreloadInstant(fp, sample.Timestamp, 0)
found := it.ValueAtOrBeforeTime(sample.Timestamp)
if found.Timestamp == model.Earliest {
t.Errorf("Sample %#v: Expected sample not found.", sample)
startTime := it.(*boundedIterator).start
switch {
case found.Timestamp != model.Earliest && sample.Timestamp.Before(startTime):
t.Errorf("Sample #%d %#v: Expected outdated sample to be excluded.", i, sample)
result = false
p.Close()
continue
}
if sample.Value != found.Value || sample.Timestamp != found.Timestamp {
case found.Timestamp == model.Earliest && !sample.Timestamp.Before(startTime):
t.Errorf("Sample #%d %#v: Expected sample not found.", i, sample)
result = false
case found.Timestamp == model.Earliest && sample.Timestamp.Before(startTime):
// All good. Outdated sample dropped.
case sample.Value != found.Value || sample.Timestamp != found.Timestamp:
t.Errorf(
"Value (or timestamp) mismatch, want %f (at time %v), got %f (at time %v).",
sample.Value, sample.Timestamp, found.Value, found.Timestamp,
"Sample #%d %#v: Value (or timestamp) mismatch, want %f (at time %v), got %f (at time %v).",
i, sample, sample.Value, sample.Timestamp, found.Value, found.Timestamp,
)
result = false
}
@ -1650,8 +1712,62 @@ func verifyStorage(t testing.TB, s *memorySeriesStorage, samples model.Samples,
return result
}
func verifyStorageSequential(t testing.TB, s *memorySeriesStorage, samples model.Samples) bool {
s.WaitForIndexing()
var (
result = true
fp model.Fingerprint
p = s.NewPreloader()
it SeriesIterator
r []model.SamplePair
j int
)
defer func() {
p.Close()
}()
for i, sample := range samples {
newFP, err := s.mapper.mapFP(sample.Metric.FastFingerprint(), sample.Metric)
if err != nil {
t.Fatal(err)
}
if it == nil || newFP != fp {
fp = newFP
p.Close()
p = s.NewPreloader()
it = p.PreloadRange(fp, sample.Timestamp, model.Latest)
r = it.RangeValues(metric.Interval{
OldestInclusive: sample.Timestamp,
NewestInclusive: model.Latest,
})
j = -1
}
startTime := it.(*boundedIterator).start
if sample.Timestamp.Before(startTime) {
continue
}
j++
if j >= len(r) {
t.Errorf(
"Sample #%d %v not found.",
i, sample,
)
result = false
continue
}
found := r[j]
if sample.Value != found.Value || sample.Timestamp != found.Timestamp {
t.Errorf(
"Sample #%d %v: Value (or timestamp) mismatch, want %f (at time %v), got %f (at time %v).",
i, sample, sample.Value, sample.Timestamp, found.Value, found.Timestamp,
)
result = false
}
}
return result
}
func TestAppendOutOfOrder(t *testing.T) {
s, closer := NewTestStorage(t, 1)
s, closer := NewTestStorage(t, 2)
defer closer.Close()
m := model.Metric{

1190
storage/local/varbit.go Normal file

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,75 @@
// Copyright 2016 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package local
import "github.com/prometheus/common/model"
var (
// bit masks for consecutive bits in a byte at various offsets.
bitMask = [][]byte{
{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}, // 0 bit
{0x80, 0x40, 0x20, 0x10, 0x08, 0x04, 0x02, 0x01}, // 1 bit
{0xC0, 0x60, 0x30, 0x18, 0x0C, 0x06, 0x03, 0x01}, // 2 bit
{0xE0, 0x70, 0x38, 0x1C, 0x0E, 0x07, 0x03, 0x01}, // 3 bit
{0xF0, 0x78, 0x3C, 0x1E, 0x0F, 0x07, 0x03, 0x01}, // 4 bit
{0xF8, 0x7C, 0x3E, 0x1F, 0x0F, 0x07, 0x03, 0x01}, // 5 bit
{0xFC, 0x7E, 0x3F, 0x1F, 0x0F, 0x07, 0x03, 0x01}, // 6 bit
{0xFE, 0x7F, 0x3F, 0x1F, 0x0F, 0x07, 0x03, 0x01}, // 7 bit
{0xFF, 0x7F, 0x3F, 0x1F, 0x0F, 0x07, 0x03, 0x01}, // 8 bit
}
)
// isInt32 returns true if v can be represented as an int32.
func isInt32(v model.SampleValue) bool {
return model.SampleValue(int32(v)) == v
}
// countBits returs the number of leading zero bits and the number of
// significant bits after that in the given bit pattern. The maximum number of
// leading zeros is 31 (so that it can be represented by a 5bit number). Leading
// zeros beyond that are considered part of the significant bits.
func countBits(pattern uint64) (leading, significant byte) {
// TODO(beorn7): This would probably be faster with ugly endless switch
// statements.
if pattern == 0 {
return
}
for pattern < 1<<63 {
leading++
pattern <<= 1
}
for pattern > 0 {
significant++
pattern <<= 1
}
if leading > 31 { // 5 bit limit.
significant += leading - 31
leading = 31
}
return
}
// isSignedIntN returns if n can be represented as a signed int with the given
// bit length.
func isSignedIntN(i int64, n byte) bool {
upper := int64(1) << (n - 1)
if i >= upper {
return false
}
lower := upper - (1 << n)
if i < lower {
return false
}
return true
}

View file

@ -0,0 +1,52 @@
// Copyright 2016 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package local
import "testing"
func TestCountBits(t *testing.T) {
for i := byte(0); i < 56; i++ {
for j := byte(0); j <= 8; j++ {
for k := byte(0); k < 8; k++ {
p := uint64(bitMask[j][k]) << i
gotLeading, gotSignificant := countBits(p)
wantLeading := 56 - i + k
wantSignificant := j
if j+k > 8 {
wantSignificant -= j + k - 8
}
if wantLeading > 31 {
wantSignificant += wantLeading - 31
wantLeading = 31
}
if p == 0 {
wantLeading = 0
wantSignificant = 0
}
if wantLeading != gotLeading {
t.Errorf(
"unexpected leading bit count for i=%d, j=%d, k=%d; want %d, got %d",
i, j, k, wantLeading, gotLeading,
)
}
if wantSignificant != gotSignificant {
t.Errorf(
"unexpected significant bit count for i=%d, j=%d, k=%d; want %d, got %d",
i, j, k, wantSignificant, gotSignificant,
)
}
}
}
}
}

View file

@ -189,7 +189,7 @@ func TestTemplateExpansion(t *testing.T) {
time := model.Time(0)
storage, closer := local.NewTestStorage(t, 1)
storage, closer := local.NewTestStorage(t, 2)
defer closer.Close()
storage.Append(&model.Sample{
Metric: model.Metric{

View file

@ -87,7 +87,7 @@ func TestQuery(t *testing.T) {
},
}
storage, closer := local.NewTestStorage(t, 1)
storage, closer := local.NewTestStorage(t, 2)
defer closer.Close()
storage.Append(&model.Sample{
Metric: model.Metric{