mirror of
https://github.com/prometheus/prometheus.git
synced 2024-11-10 07:34:04 -08:00
Merge pull request #1502 from prometheus/beorn7/storage7
Beorn7/storage7
This commit is contained in:
commit
d43f380cf2
|
@ -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(
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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,16 +1514,19 @@ 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
|
||||
maxStreakLength = 2000
|
||||
maxTimeDelta = 10000
|
||||
maxTimeDeltaFactor = 10
|
||||
timestamp = model.Now() - model.Time(maxTimeDelta*maxTimeDeltaFactor*minLen/4) // So that some timestamps are in the future.
|
||||
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 {
|
||||
var (
|
||||
// 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)) }
|
||||
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
1190
storage/local/varbit.go
Normal file
File diff suppressed because it is too large
Load diff
75
storage/local/varbit_helpers.go
Normal file
75
storage/local/varbit_helpers.go
Normal 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
|
||||
}
|
52
storage/local/varbit_test.go
Normal file
52
storage/local/varbit_test.go
Normal 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,
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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{
|
||||
|
|
|
@ -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{
|
||||
|
|
Loading…
Reference in a new issue