Weed out all the [Gg]et* method names.

The only exception is getNumChunksToPersist to avoid naming the struct
member numChunksToPersist in a weird way.
This commit is contained in:
beorn7 2015-05-20 19:13:06 +02:00
parent 81b190bf45
commit 3b9c421a69
15 changed files with 269 additions and 268 deletions

View file

@ -73,7 +73,7 @@ func (a *Analyzer) Analyze(ctx context.Context) error {
switch n := node.(type) { switch n := node.(type) {
case *VectorSelector: case *VectorSelector:
pt := getPreloadTimes(n.Offset) pt := getPreloadTimes(n.Offset)
fpts := a.Storage.GetFingerprintsForLabelMatchers(n.LabelMatchers) fpts := a.Storage.FingerprintsForLabelMatchers(n.LabelMatchers)
n.fingerprints = fpts n.fingerprints = fpts
n.metrics = map[clientmodel.Fingerprint]clientmodel.COWMetric{} n.metrics = map[clientmodel.Fingerprint]clientmodel.COWMetric{}
n.iterators = map[clientmodel.Fingerprint]local.SeriesIterator{} n.iterators = map[clientmodel.Fingerprint]local.SeriesIterator{}
@ -84,11 +84,11 @@ func (a *Analyzer) Analyze(ctx context.Context) error {
if _, alreadyInRanges := pt.ranges[fp]; !alreadyInRanges { if _, alreadyInRanges := pt.ranges[fp]; !alreadyInRanges {
pt.instants[fp] = struct{}{} pt.instants[fp] = struct{}{}
} }
n.metrics[fp] = a.Storage.GetMetricForFingerprint(fp) n.metrics[fp] = a.Storage.MetricForFingerprint(fp)
} }
case *MatrixSelector: case *MatrixSelector:
pt := getPreloadTimes(n.Offset) pt := getPreloadTimes(n.Offset)
fpts := a.Storage.GetFingerprintsForLabelMatchers(n.LabelMatchers) fpts := a.Storage.FingerprintsForLabelMatchers(n.LabelMatchers)
n.fingerprints = fpts n.fingerprints = fpts
n.metrics = map[clientmodel.Fingerprint]clientmodel.COWMetric{} n.metrics = map[clientmodel.Fingerprint]clientmodel.COWMetric{}
n.iterators = map[clientmodel.Fingerprint]local.SeriesIterator{} n.iterators = map[clientmodel.Fingerprint]local.SeriesIterator{}
@ -100,7 +100,7 @@ func (a *Analyzer) Analyze(ctx context.Context) error {
// an instant for the same fingerprint, should we have one. // an instant for the same fingerprint, should we have one.
delete(pt.instants, fp) delete(pt.instants, fp)
} }
n.metrics[fp] = a.Storage.GetMetricForFingerprint(fp) n.metrics[fp] = a.Storage.MetricForFingerprint(fp)
} }
} }
return true return true

View file

@ -644,7 +644,7 @@ func (ev *evaluator) eval(expr Expr) Value {
func (ev *evaluator) vectorSelector(node *VectorSelector) Vector { func (ev *evaluator) vectorSelector(node *VectorSelector) Vector {
vec := Vector{} vec := Vector{}
for fp, it := range node.iterators { for fp, it := range node.iterators {
sampleCandidates := it.GetValueAtTime(ev.Timestamp.Add(-node.Offset)) sampleCandidates := it.ValueAtTime(ev.Timestamp.Add(-node.Offset))
samplePair := chooseClosestSample(sampleCandidates, ev.Timestamp.Add(-node.Offset)) samplePair := chooseClosestSample(sampleCandidates, ev.Timestamp.Add(-node.Offset))
if samplePair != nil { if samplePair != nil {
vec = append(vec, &Sample{ vec = append(vec, &Sample{
@ -666,7 +666,7 @@ func (ev *evaluator) matrixSelector(node *MatrixSelector) Matrix {
sampleStreams := make([]*SampleStream, 0, len(node.iterators)) sampleStreams := make([]*SampleStream, 0, len(node.iterators))
for fp, it := range node.iterators { for fp, it := range node.iterators {
samplePairs := it.GetRangeValues(interval) samplePairs := it.RangeValues(interval)
if len(samplePairs) == 0 { if len(samplePairs) == 0 {
continue continue
} }
@ -695,7 +695,7 @@ func (ev *evaluator) matrixSelectorBounds(node *MatrixSelector) Matrix {
sampleStreams := make([]*SampleStream, 0, len(node.iterators)) sampleStreams := make([]*SampleStream, 0, len(node.iterators))
for fp, it := range node.iterators { for fp, it := range node.iterators {
samplePairs := it.GetBoundaryValues(interval) samplePairs := it.BoundaryValues(interval)
if len(samplePairs) == 0 { if len(samplePairs) == 0 {
continue continue
} }

View file

@ -41,8 +41,8 @@ const (
// goroutine-safe proxies for chunk methods. // goroutine-safe proxies for chunk methods.
type chunkDesc struct { type chunkDesc struct {
sync.Mutex sync.Mutex
chunk chunk // nil if chunk is evicted. c chunk // nil if chunk is evicted.
refCount int rCnt int
chunkFirstTime clientmodel.Timestamp // Used if chunk is evicted. chunkFirstTime clientmodel.Timestamp // Used if chunk is evicted.
chunkLastTime clientmodel.Timestamp // Used if chunk is evicted. chunkLastTime clientmodel.Timestamp // Used if chunk is evicted.
@ -59,14 +59,14 @@ func newChunkDesc(c chunk) *chunkDesc {
chunkOps.WithLabelValues(createAndPin).Inc() chunkOps.WithLabelValues(createAndPin).Inc()
atomic.AddInt64(&numMemChunks, 1) atomic.AddInt64(&numMemChunks, 1)
numMemChunkDescs.Inc() numMemChunkDescs.Inc()
return &chunkDesc{chunk: c, refCount: 1} return &chunkDesc{c: c, rCnt: 1}
} }
func (cd *chunkDesc) add(s *metric.SamplePair) []chunk { func (cd *chunkDesc) add(s *metric.SamplePair) []chunk {
cd.Lock() cd.Lock()
defer cd.Unlock() defer cd.Unlock()
return cd.chunk.add(s) return cd.c.add(s)
} }
// pin increments the refCount by one. Upon increment from 0 to 1, this // pin increments the refCount by one. Upon increment from 0 to 1, this
@ -76,11 +76,11 @@ func (cd *chunkDesc) pin(evictRequests chan<- evictRequest) {
cd.Lock() cd.Lock()
defer cd.Unlock() defer cd.Unlock()
if cd.refCount == 0 { if cd.rCnt == 0 {
// Remove ourselves from the evict list. // Remove ourselves from the evict list.
evictRequests <- evictRequest{cd, false} evictRequests <- evictRequest{cd, false}
} }
cd.refCount++ cd.rCnt++
} }
// unpin decrements the refCount by one. Upon decrement from 1 to 0, this // unpin decrements the refCount by one. Upon decrement from 1 to 0, this
@ -90,69 +90,69 @@ func (cd *chunkDesc) unpin(evictRequests chan<- evictRequest) {
cd.Lock() cd.Lock()
defer cd.Unlock() defer cd.Unlock()
if cd.refCount == 0 { if cd.rCnt == 0 {
panic("cannot unpin already unpinned chunk") panic("cannot unpin already unpinned chunk")
} }
cd.refCount-- cd.rCnt--
if cd.refCount == 0 { if cd.rCnt == 0 {
// Add ourselves to the back of the evict list. // Add ourselves to the back of the evict list.
evictRequests <- evictRequest{cd, true} evictRequests <- evictRequest{cd, true}
} }
} }
func (cd *chunkDesc) getRefCount() int { func (cd *chunkDesc) refCount() int {
cd.Lock() cd.Lock()
defer cd.Unlock() defer cd.Unlock()
return cd.refCount return cd.rCnt
} }
func (cd *chunkDesc) firstTime() clientmodel.Timestamp { func (cd *chunkDesc) firstTime() clientmodel.Timestamp {
cd.Lock() cd.Lock()
defer cd.Unlock() defer cd.Unlock()
if cd.chunk == nil { if cd.c == nil {
return cd.chunkFirstTime return cd.chunkFirstTime
} }
return cd.chunk.firstTime() return cd.c.firstTime()
} }
func (cd *chunkDesc) lastTime() clientmodel.Timestamp { func (cd *chunkDesc) lastTime() clientmodel.Timestamp {
cd.Lock() cd.Lock()
defer cd.Unlock() defer cd.Unlock()
if cd.chunk == nil { if cd.c == nil {
return cd.chunkLastTime return cd.chunkLastTime
} }
return cd.chunk.newIterator().getLastTimestamp() return cd.c.newIterator().lastTimestamp()
} }
func (cd *chunkDesc) isEvicted() bool { func (cd *chunkDesc) isEvicted() bool {
cd.Lock() cd.Lock()
defer cd.Unlock() defer cd.Unlock()
return cd.chunk == nil return cd.c == nil
} }
func (cd *chunkDesc) contains(t clientmodel.Timestamp) bool { func (cd *chunkDesc) contains(t clientmodel.Timestamp) bool {
return !t.Before(cd.firstTime()) && !t.After(cd.lastTime()) return !t.Before(cd.firstTime()) && !t.After(cd.lastTime())
} }
func (cd *chunkDesc) getChunk() chunk { func (cd *chunkDesc) chunk() chunk {
cd.Lock() cd.Lock()
defer cd.Unlock() defer cd.Unlock()
return cd.chunk return cd.c
} }
func (cd *chunkDesc) setChunk(c chunk) { func (cd *chunkDesc) setChunk(c chunk) {
cd.Lock() cd.Lock()
defer cd.Unlock() defer cd.Unlock()
if cd.chunk != nil { if cd.c != nil {
panic("chunk already set") panic("chunk already set")
} }
cd.chunk = c cd.c = c
} }
// maybeEvict evicts the chunk if the refCount is 0. It returns whether the chunk // maybeEvict evicts the chunk if the refCount is 0. It returns whether the chunk
@ -162,15 +162,15 @@ func (cd *chunkDesc) maybeEvict() bool {
cd.Lock() cd.Lock()
defer cd.Unlock() defer cd.Unlock()
if cd.chunk == nil { if cd.c == nil {
return true return true
} }
if cd.refCount != 0 { if cd.rCnt != 0 {
return false return false
} }
cd.chunkFirstTime = cd.chunk.firstTime() cd.chunkFirstTime = cd.c.firstTime()
cd.chunkLastTime = cd.chunk.newIterator().getLastTimestamp() cd.chunkLastTime = cd.c.newIterator().lastTimestamp()
cd.chunk = nil cd.c = nil
chunkOps.WithLabelValues(evict).Inc() chunkOps.WithLabelValues(evict).Inc()
atomic.AddInt64(&numMemChunks, -1) atomic.AddInt64(&numMemChunks, -1)
return true return true
@ -202,21 +202,21 @@ type chunkIterator interface {
// length returns the number of samples in the chunk. // length returns the number of samples in the chunk.
length() int length() int
// Gets the timestamp of the n-th sample in the chunk. // Gets the timestamp of the n-th sample in the chunk.
getTimestampAtIndex(int) clientmodel.Timestamp timestampAtIndex(int) clientmodel.Timestamp
// Gets the last timestamp in the chunk. // Gets the last timestamp in the chunk.
getLastTimestamp() clientmodel.Timestamp lastTimestamp() clientmodel.Timestamp
// Gets the sample value of the n-th sample in the chunk. // Gets the sample value of the n-th sample in the chunk.
getSampleValueAtIndex(int) clientmodel.SampleValue sampleValueAtIndex(int) clientmodel.SampleValue
// Gets the last sample value in the chunk. // Gets the last sample value in the chunk.
getLastSampleValue() clientmodel.SampleValue lastSampleValue() clientmodel.SampleValue
// Gets the two values that are immediately adjacent to a given time. In // Gets the two values that are immediately adjacent to a given time. In
// case a value exist at precisely the given time, only that single // case a value exist at precisely the given time, only that single
// value is returned. Only the first or last value is returned (as a // value is returned. Only the first or last value is returned (as a
// single value), if the given time is before or after the first or last // single value), if the given time is before or after the first or last
// value, respectively. // value, respectively.
getValueAtTime(clientmodel.Timestamp) metric.Values valueAtTime(clientmodel.Timestamp) metric.Values
// Gets all values contained within a given interval. // Gets all values contained within a given interval.
getRangeValues(metric.Interval) metric.Values rangeValues(metric.Interval) metric.Values
// Whether a given timestamp is contained between first and last value // Whether a given timestamp is contained between first and last value
// in the chunk. // in the chunk.
contains(clientmodel.Timestamp) bool contains(clientmodel.Timestamp) bool

View file

@ -318,7 +318,7 @@ func (p *persistence) sanitizeSeries(
return fp, true return fp, true
} }
// This series is supposed to be archived. // This series is supposed to be archived.
metric, err := p.getArchivedMetric(fp) metric, err := p.archivedMetric(fp)
if err != nil { if err != nil {
glog.Errorf( glog.Errorf(
"Fingerprint %v assumed archived but couldn't be looked up in archived index: %s", "Fingerprint %v assumed archived but couldn't be looked up in archived index: %s",

View file

@ -285,52 +285,52 @@ type deltaEncodedChunkIterator struct {
// length implements chunkIterator. // length implements chunkIterator.
func (it *deltaEncodedChunkIterator) length() int { return it.len } func (it *deltaEncodedChunkIterator) length() int { return it.len }
// getValueAtTime implements chunkIterator. // valueAtTime implements chunkIterator.
func (it *deltaEncodedChunkIterator) getValueAtTime(t clientmodel.Timestamp) metric.Values { func (it *deltaEncodedChunkIterator) valueAtTime(t clientmodel.Timestamp) metric.Values {
i := sort.Search(it.len, func(i int) bool { i := sort.Search(it.len, func(i int) bool {
return !it.getTimestampAtIndex(i).Before(t) return !it.timestampAtIndex(i).Before(t)
}) })
switch i { switch i {
case 0: case 0:
return metric.Values{metric.SamplePair{ return metric.Values{metric.SamplePair{
Timestamp: it.getTimestampAtIndex(0), Timestamp: it.timestampAtIndex(0),
Value: it.getSampleValueAtIndex(0), Value: it.sampleValueAtIndex(0),
}} }}
case it.len: case it.len:
return metric.Values{metric.SamplePair{ return metric.Values{metric.SamplePair{
Timestamp: it.getTimestampAtIndex(it.len - 1), Timestamp: it.timestampAtIndex(it.len - 1),
Value: it.getSampleValueAtIndex(it.len - 1), Value: it.sampleValueAtIndex(it.len - 1),
}} }}
default: default:
ts := it.getTimestampAtIndex(i) ts := it.timestampAtIndex(i)
if ts.Equal(t) { if ts.Equal(t) {
return metric.Values{metric.SamplePair{ return metric.Values{metric.SamplePair{
Timestamp: ts, Timestamp: ts,
Value: it.getSampleValueAtIndex(i), Value: it.sampleValueAtIndex(i),
}} }}
} }
return metric.Values{ return metric.Values{
metric.SamplePair{ metric.SamplePair{
Timestamp: it.getTimestampAtIndex(i - 1), Timestamp: it.timestampAtIndex(i - 1),
Value: it.getSampleValueAtIndex(i - 1), Value: it.sampleValueAtIndex(i - 1),
}, },
metric.SamplePair{ metric.SamplePair{
Timestamp: ts, Timestamp: ts,
Value: it.getSampleValueAtIndex(i), Value: it.sampleValueAtIndex(i),
}, },
} }
} }
} }
// getRangeValues implements chunkIterator. // rangeValues implements chunkIterator.
func (it *deltaEncodedChunkIterator) getRangeValues(in metric.Interval) metric.Values { func (it *deltaEncodedChunkIterator) rangeValues(in metric.Interval) metric.Values {
oldest := sort.Search(it.len, func(i int) bool { oldest := sort.Search(it.len, func(i int) bool {
return !it.getTimestampAtIndex(i).Before(in.OldestInclusive) return !it.timestampAtIndex(i).Before(in.OldestInclusive)
}) })
newest := sort.Search(it.len, func(i int) bool { newest := sort.Search(it.len, func(i int) bool {
return it.getTimestampAtIndex(i).After(in.NewestInclusive) return it.timestampAtIndex(i).After(in.NewestInclusive)
}) })
if oldest == it.len { if oldest == it.len {
@ -340,8 +340,8 @@ func (it *deltaEncodedChunkIterator) getRangeValues(in metric.Interval) metric.V
result := make(metric.Values, 0, newest-oldest) result := make(metric.Values, 0, newest-oldest)
for i := oldest; i < newest; i++ { for i := oldest; i < newest; i++ {
result = append(result, metric.SamplePair{ result = append(result, metric.SamplePair{
Timestamp: it.getTimestampAtIndex(i), Timestamp: it.timestampAtIndex(i),
Value: it.getSampleValueAtIndex(i), Value: it.sampleValueAtIndex(i),
}) })
} }
return result return result
@ -349,7 +349,7 @@ func (it *deltaEncodedChunkIterator) getRangeValues(in metric.Interval) metric.V
// contains implements chunkIterator. // contains implements chunkIterator.
func (it *deltaEncodedChunkIterator) contains(t clientmodel.Timestamp) bool { func (it *deltaEncodedChunkIterator) contains(t clientmodel.Timestamp) bool {
return !t.Before(it.baseT) && !t.After(it.getTimestampAtIndex(it.len-1)) return !t.Before(it.baseT) && !t.After(it.timestampAtIndex(it.len-1))
} }
// values implements chunkIterator. // values implements chunkIterator.
@ -358,8 +358,8 @@ func (it *deltaEncodedChunkIterator) values() <-chan *metric.SamplePair {
go func() { go func() {
for i := 0; i < it.len; i++ { for i := 0; i < it.len; i++ {
valuesChan <- &metric.SamplePair{ valuesChan <- &metric.SamplePair{
Timestamp: it.getTimestampAtIndex(i), Timestamp: it.timestampAtIndex(i),
Value: it.getSampleValueAtIndex(i), Value: it.sampleValueAtIndex(i),
} }
} }
close(valuesChan) close(valuesChan)
@ -367,8 +367,8 @@ func (it *deltaEncodedChunkIterator) values() <-chan *metric.SamplePair {
return valuesChan return valuesChan
} }
// getTimestampAtIndex implements chunkIterator. // timestampAtIndex implements chunkIterator.
func (it *deltaEncodedChunkIterator) getTimestampAtIndex(idx int) clientmodel.Timestamp { func (it *deltaEncodedChunkIterator) timestampAtIndex(idx int) clientmodel.Timestamp {
offset := deltaHeaderBytes + idx*int(it.tBytes+it.vBytes) offset := deltaHeaderBytes + idx*int(it.tBytes+it.vBytes)
switch it.tBytes { switch it.tBytes {
@ -386,13 +386,13 @@ func (it *deltaEncodedChunkIterator) getTimestampAtIndex(idx int) clientmodel.Ti
} }
} }
// getLastTimestamp implements chunkIterator. // lastTimestamp implements chunkIterator.
func (it *deltaEncodedChunkIterator) getLastTimestamp() clientmodel.Timestamp { func (it *deltaEncodedChunkIterator) lastTimestamp() clientmodel.Timestamp {
return it.getTimestampAtIndex(it.len - 1) return it.timestampAtIndex(it.len - 1)
} }
// getSampleValueAtIndex implements chunkIterator. // sampleValueAtIndex implements chunkIterator.
func (it *deltaEncodedChunkIterator) getSampleValueAtIndex(idx int) clientmodel.SampleValue { func (it *deltaEncodedChunkIterator) sampleValueAtIndex(idx int) clientmodel.SampleValue {
offset := deltaHeaderBytes + idx*int(it.tBytes+it.vBytes) + int(it.tBytes) offset := deltaHeaderBytes + idx*int(it.tBytes+it.vBytes) + int(it.tBytes)
if it.isInt { if it.isInt {
@ -422,7 +422,7 @@ func (it *deltaEncodedChunkIterator) getSampleValueAtIndex(idx int) clientmodel.
} }
} }
// getLastSampleValue implements chunkIterator. // lastSampleValue implements chunkIterator.
func (it *deltaEncodedChunkIterator) getLastSampleValue() clientmodel.SampleValue { func (it *deltaEncodedChunkIterator) lastSampleValue() clientmodel.SampleValue {
return it.getSampleValueAtIndex(it.len - 1) return it.sampleValueAtIndex(it.len - 1)
} }

View file

@ -396,52 +396,52 @@ type doubleDeltaEncodedChunkIterator struct {
// length implements chunkIterator. // length implements chunkIterator.
func (it *doubleDeltaEncodedChunkIterator) length() int { return it.len } func (it *doubleDeltaEncodedChunkIterator) length() int { return it.len }
// getValueAtTime implements chunkIterator. // valueAtTime implements chunkIterator.
func (it *doubleDeltaEncodedChunkIterator) getValueAtTime(t clientmodel.Timestamp) metric.Values { func (it *doubleDeltaEncodedChunkIterator) valueAtTime(t clientmodel.Timestamp) metric.Values {
i := sort.Search(it.len, func(i int) bool { i := sort.Search(it.len, func(i int) bool {
return !it.getTimestampAtIndex(i).Before(t) return !it.timestampAtIndex(i).Before(t)
}) })
switch i { switch i {
case 0: case 0:
return metric.Values{metric.SamplePair{ return metric.Values{metric.SamplePair{
Timestamp: it.getTimestampAtIndex(0), Timestamp: it.timestampAtIndex(0),
Value: it.getSampleValueAtIndex(0), Value: it.sampleValueAtIndex(0),
}} }}
case it.len: case it.len:
return metric.Values{metric.SamplePair{ return metric.Values{metric.SamplePair{
Timestamp: it.getTimestampAtIndex(it.len - 1), Timestamp: it.timestampAtIndex(it.len - 1),
Value: it.getSampleValueAtIndex(it.len - 1), Value: it.sampleValueAtIndex(it.len - 1),
}} }}
default: default:
ts := it.getTimestampAtIndex(i) ts := it.timestampAtIndex(i)
if ts.Equal(t) { if ts.Equal(t) {
return metric.Values{metric.SamplePair{ return metric.Values{metric.SamplePair{
Timestamp: ts, Timestamp: ts,
Value: it.getSampleValueAtIndex(i), Value: it.sampleValueAtIndex(i),
}} }}
} }
return metric.Values{ return metric.Values{
metric.SamplePair{ metric.SamplePair{
Timestamp: it.getTimestampAtIndex(i - 1), Timestamp: it.timestampAtIndex(i - 1),
Value: it.getSampleValueAtIndex(i - 1), Value: it.sampleValueAtIndex(i - 1),
}, },
metric.SamplePair{ metric.SamplePair{
Timestamp: ts, Timestamp: ts,
Value: it.getSampleValueAtIndex(i), Value: it.sampleValueAtIndex(i),
}, },
} }
} }
} }
// getRangeValues implements chunkIterator. // rangeValues implements chunkIterator.
func (it *doubleDeltaEncodedChunkIterator) getRangeValues(in metric.Interval) metric.Values { func (it *doubleDeltaEncodedChunkIterator) rangeValues(in metric.Interval) metric.Values {
oldest := sort.Search(it.len, func(i int) bool { oldest := sort.Search(it.len, func(i int) bool {
return !it.getTimestampAtIndex(i).Before(in.OldestInclusive) return !it.timestampAtIndex(i).Before(in.OldestInclusive)
}) })
newest := sort.Search(it.len, func(i int) bool { newest := sort.Search(it.len, func(i int) bool {
return it.getTimestampAtIndex(i).After(in.NewestInclusive) return it.timestampAtIndex(i).After(in.NewestInclusive)
}) })
if oldest == it.len { if oldest == it.len {
@ -451,8 +451,8 @@ func (it *doubleDeltaEncodedChunkIterator) getRangeValues(in metric.Interval) me
result := make(metric.Values, 0, newest-oldest) result := make(metric.Values, 0, newest-oldest)
for i := oldest; i < newest; i++ { for i := oldest; i < newest; i++ {
result = append(result, metric.SamplePair{ result = append(result, metric.SamplePair{
Timestamp: it.getTimestampAtIndex(i), Timestamp: it.timestampAtIndex(i),
Value: it.getSampleValueAtIndex(i), Value: it.sampleValueAtIndex(i),
}) })
} }
return result return result
@ -460,7 +460,7 @@ func (it *doubleDeltaEncodedChunkIterator) getRangeValues(in metric.Interval) me
// contains implements chunkIterator. // contains implements chunkIterator.
func (it *doubleDeltaEncodedChunkIterator) contains(t clientmodel.Timestamp) bool { func (it *doubleDeltaEncodedChunkIterator) contains(t clientmodel.Timestamp) bool {
return !t.Before(it.baseT) && !t.After(it.getTimestampAtIndex(it.len-1)) return !t.Before(it.baseT) && !t.After(it.timestampAtIndex(it.len-1))
} }
// values implements chunkIterator. // values implements chunkIterator.
@ -469,8 +469,8 @@ func (it *doubleDeltaEncodedChunkIterator) values() <-chan *metric.SamplePair {
go func() { go func() {
for i := 0; i < it.len; i++ { for i := 0; i < it.len; i++ {
valuesChan <- &metric.SamplePair{ valuesChan <- &metric.SamplePair{
Timestamp: it.getTimestampAtIndex(i), Timestamp: it.timestampAtIndex(i),
Value: it.getSampleValueAtIndex(i), Value: it.sampleValueAtIndex(i),
} }
} }
close(valuesChan) close(valuesChan)
@ -478,8 +478,8 @@ func (it *doubleDeltaEncodedChunkIterator) values() <-chan *metric.SamplePair {
return valuesChan return valuesChan
} }
// getTimestampAtIndex implements chunkIterator. // timestampAtIndex implements chunkIterator.
func (it *doubleDeltaEncodedChunkIterator) getTimestampAtIndex(idx int) clientmodel.Timestamp { func (it *doubleDeltaEncodedChunkIterator) timestampAtIndex(idx int) clientmodel.Timestamp {
if idx == 0 { if idx == 0 {
return it.baseT return it.baseT
} }
@ -515,13 +515,13 @@ func (it *doubleDeltaEncodedChunkIterator) getTimestampAtIndex(idx int) clientmo
} }
} }
// getLastTimestamp implements chunkIterator. // lastTimestamp implements chunkIterator.
func (it *doubleDeltaEncodedChunkIterator) getLastTimestamp() clientmodel.Timestamp { func (it *doubleDeltaEncodedChunkIterator) lastTimestamp() clientmodel.Timestamp {
return it.getTimestampAtIndex(it.len - 1) return it.timestampAtIndex(it.len - 1)
} }
// getSampleValueAtIndex implements chunkIterator. // sampleValueAtIndex implements chunkIterator.
func (it *doubleDeltaEncodedChunkIterator) getSampleValueAtIndex(idx int) clientmodel.SampleValue { func (it *doubleDeltaEncodedChunkIterator) sampleValueAtIndex(idx int) clientmodel.SampleValue {
if idx == 0 { if idx == 0 {
return it.baseV return it.baseV
} }
@ -572,7 +572,7 @@ func (it *doubleDeltaEncodedChunkIterator) getSampleValueAtIndex(idx int) client
} }
} }
// getLastSampleValue implements chunkIterator. // lastSampleValue implements chunkIterator.
func (it *doubleDeltaEncodedChunkIterator) getLastSampleValue() clientmodel.SampleValue { func (it *doubleDeltaEncodedChunkIterator) lastSampleValue() clientmodel.SampleValue {
return it.getSampleValueAtIndex(it.len - 1) return it.sampleValueAtIndex(it.len - 1)
} }

View file

@ -38,11 +38,11 @@ type Storage interface {
NewPreloader() Preloader NewPreloader() Preloader
// Get all of the metric fingerprints that are associated with the // Get all of the metric fingerprints that are associated with the
// provided label matchers. // provided label matchers.
GetFingerprintsForLabelMatchers(metric.LabelMatchers) clientmodel.Fingerprints FingerprintsForLabelMatchers(metric.LabelMatchers) clientmodel.Fingerprints
// Get all of the label values that are associated with a given label name. // Get all of the label values that are associated with a given label name.
GetLabelValuesForLabelName(clientmodel.LabelName) clientmodel.LabelValues LabelValuesForLabelName(clientmodel.LabelName) clientmodel.LabelValues
// Get the metric associated with the provided fingerprint. // Get the metric associated with the provided fingerprint.
GetMetricForFingerprint(clientmodel.Fingerprint) clientmodel.COWMetric MetricForFingerprint(clientmodel.Fingerprint) clientmodel.COWMetric
// Construct an iterator for a given fingerprint. // Construct an iterator for a given fingerprint.
NewIterator(clientmodel.Fingerprint) SeriesIterator NewIterator(clientmodel.Fingerprint) SeriesIterator
// Run the various maintenance loops in goroutines. Returns when the // Run the various maintenance loops in goroutines. Returns when the
@ -53,8 +53,8 @@ type Storage interface {
// operations, stops all maintenance loops,and frees all resources. // operations, stops all maintenance loops,and frees all resources.
Stop() error Stop() error
// WaitForIndexing returns once all samples in the storage are // WaitForIndexing returns once all samples in the storage are
// indexed. Indexing is needed for GetFingerprintsForLabelMatchers and // indexed. Indexing is needed for FingerprintsForLabelMatchers and
// GetLabelValuesForLabelName and may lag behind. // LabelValuesForLabelName and may lag behind.
WaitForIndexing() WaitForIndexing()
} }
@ -69,12 +69,12 @@ type SeriesIterator interface {
// value is returned. Only the first or last value is returned (as a // value is returned. Only the first or last value is returned (as a
// single value), if the given time is before or after the first or last // single value), if the given time is before or after the first or last
// value, respectively. // value, respectively.
GetValueAtTime(clientmodel.Timestamp) metric.Values ValueAtTime(clientmodel.Timestamp) metric.Values
// Gets the boundary values of an interval: the first and last value // Gets the boundary values of an interval: the first and last value
// within a given interval. // within a given interval.
GetBoundaryValues(metric.Interval) metric.Values BoundaryValues(metric.Interval) metric.Values
// Gets all values contained within a given interval. // Gets all values contained within a given interval.
GetRangeValues(metric.Interval) metric.Values RangeValues(metric.Interval) metric.Values
} }
// A Preloader preloads series data necessary for a query into memory and pins // A Preloader preloads series data necessary for a query into memory and pins

View file

@ -91,7 +91,7 @@ func (r *fpMapper) mapFP(fp clientmodel.Fingerprint, m clientmodel.Metric) (clie
// If we are here, FP does not exist in memory and is either not mapped // If we are here, FP does not exist in memory and is either not mapped
// at all, or existing mappings for FP are not for m. Check if we have // at all, or existing mappings for FP are not for m. Check if we have
// something for FP in the archive. // something for FP in the archive.
archivedMetric, err := r.p.getArchivedMetric(fp) archivedMetric, err := r.p.archivedMetric(fp)
if err != nil { if err != nil {
return fp, err return fp, err
} }

View file

@ -319,11 +319,11 @@ func (p *persistence) setDirty(dirty bool) {
} }
} }
// getFingerprintsForLabelPair returns the fingerprints for the given label // fingerprintsForLabelPair returns the fingerprints for the given label
// pair. This method is goroutine-safe but take into account that metrics queued // pair. This method is goroutine-safe but take into account that metrics queued
// for indexing with IndexMetric might not have made it into the index // for indexing with IndexMetric might not have made it into the index
// yet. (Same applies correspondingly to UnindexMetric.) // yet. (Same applies correspondingly to UnindexMetric.)
func (p *persistence) getFingerprintsForLabelPair(lp metric.LabelPair) (clientmodel.Fingerprints, error) { func (p *persistence) fingerprintsForLabelPair(lp metric.LabelPair) (clientmodel.Fingerprints, error) {
fps, _, err := p.labelPairToFingerprints.Lookup(lp) fps, _, err := p.labelPairToFingerprints.Lookup(lp)
if err != nil { if err != nil {
return nil, err return nil, err
@ -331,11 +331,11 @@ func (p *persistence) getFingerprintsForLabelPair(lp metric.LabelPair) (clientmo
return fps, nil return fps, nil
} }
// getLabelValuesForLabelName returns the label values for the given label // labelValuesForLabelName returns the label values for the given label
// name. This method is goroutine-safe but take into account that metrics queued // name. This method is goroutine-safe but take into account that metrics queued
// for indexing with IndexMetric might not have made it into the index // for indexing with IndexMetric might not have made it into the index
// yet. (Same applies correspondingly to UnindexMetric.) // yet. (Same applies correspondingly to UnindexMetric.)
func (p *persistence) getLabelValuesForLabelName(ln clientmodel.LabelName) (clientmodel.LabelValues, error) { func (p *persistence) labelValuesForLabelName(ln clientmodel.LabelName) (clientmodel.LabelValues, error) {
lvs, _, err := p.labelNameToLabelValues.Lookup(ln) lvs, _, err := p.labelNameToLabelValues.Lookup(ln)
if err != nil { if err != nil {
return nil, err return nil, err
@ -632,10 +632,10 @@ func (p *persistence) checkpointSeriesMapAndHeads(fingerprintToSeries *seriesMap
} }
} else { } else {
// This is the non-persisted head chunk. Fully marshal it. // This is the non-persisted head chunk. Fully marshal it.
if err = w.WriteByte(byte(chunkDesc.chunk.encoding())); err != nil { if err = w.WriteByte(byte(chunkDesc.c.encoding())); err != nil {
return return
} }
if err = chunkDesc.chunk.marshal(w); err != nil { if err = chunkDesc.c.marshal(w); err != nil {
return return
} }
} }
@ -880,7 +880,7 @@ func (p *persistence) dropAndPersistChunks(
// too old. If that's the case, the chunks in the series file // too old. If that's the case, the chunks in the series file
// are all too old, too. // are all too old, too.
i := 0 i := 0
for ; i < len(chunks) && chunks[i].newIterator().getLastTimestamp().Before(beforeTime); i++ { for ; i < len(chunks) && chunks[i].newIterator().lastTimestamp().Before(beforeTime); i++ {
} }
if i < len(chunks) { if i < len(chunks) {
firstTimeNotDropped = chunks[i].firstTime() firstTimeNotDropped = chunks[i].firstTime()
@ -1017,10 +1017,10 @@ func (p *persistence) deleteSeriesFile(fp clientmodel.Fingerprint) (int, error)
return numChunks, nil return numChunks, nil
} }
// getSeriesFileModTime returns the modification time of the series file // seriesFileModTime returns the modification time of the series file belonging
// belonging to the provided fingerprint. In case of an error, the zero value of // to the provided fingerprint. In case of an error, the zero value of time.Time
// time.Time is returned. // is returned.
func (p *persistence) getSeriesFileModTime(fp clientmodel.Fingerprint) time.Time { func (p *persistence) seriesFileModTime(fp clientmodel.Fingerprint) time.Time {
var modTime time.Time var modTime time.Time
if fi, err := os.Stat(p.fileNameForFingerprint(fp)); err == nil { if fi, err := os.Stat(p.fileNameForFingerprint(fp)); err == nil {
return fi.ModTime() return fi.ModTime()
@ -1029,17 +1029,17 @@ func (p *persistence) getSeriesFileModTime(fp clientmodel.Fingerprint) time.Time
} }
// indexMetric queues the given metric for addition to the indexes needed by // indexMetric queues the given metric for addition to the indexes needed by
// getFingerprintsForLabelPair, getLabelValuesForLabelName, and // fingerprintsForLabelPair, labelValuesForLabelName, and
// getFingerprintsModifiedBefore. If the queue is full, this method blocks // fingerprintsModifiedBefore. If the queue is full, this method blocks until
// until the metric can be queued. This method is goroutine-safe. // the metric can be queued. This method is goroutine-safe.
func (p *persistence) indexMetric(fp clientmodel.Fingerprint, m clientmodel.Metric) { func (p *persistence) indexMetric(fp clientmodel.Fingerprint, m clientmodel.Metric) {
p.indexingQueue <- indexingOp{fp, m, add} p.indexingQueue <- indexingOp{fp, m, add}
} }
// unindexMetric queues references to the given metric for removal from the // unindexMetric queues references to the given metric for removal from the
// indexes used for getFingerprintsForLabelPair, getLabelValuesForLabelName, and // indexes used for fingerprintsForLabelPair, labelValuesForLabelName, and
// getFingerprintsModifiedBefore. The index of fingerprints to archived metrics // fingerprintsModifiedBefore. The index of fingerprints to archived metrics is
// is not affected by this removal. (In fact, never call this method for an // not affected by this removal. (In fact, never call this method for an
// archived metric. To purge an archived metric, call purgeArchivedFingerprint.) // archived metric. To purge an archived metric, call purgeArchivedFingerprint.)
// If the queue is full, this method blocks until the metric can be queued. This // If the queue is full, this method blocks until the metric can be queued. This
// method is goroutine-safe. // method is goroutine-safe.
@ -1097,10 +1097,10 @@ func (p *persistence) updateArchivedTimeRange(
return p.archivedFingerprintToTimeRange.Put(codable.Fingerprint(fp), codable.TimeRange{First: first, Last: last}) return p.archivedFingerprintToTimeRange.Put(codable.Fingerprint(fp), codable.TimeRange{First: first, Last: last})
} }
// getFingerprintsModifiedBefore returns the fingerprints of archived timeseries // fingerprintsModifiedBefore returns the fingerprints of archived timeseries
// that have live samples before the provided timestamp. This method is // that have live samples before the provided timestamp. This method is
// goroutine-safe. // goroutine-safe.
func (p *persistence) getFingerprintsModifiedBefore(beforeTime clientmodel.Timestamp) ([]clientmodel.Fingerprint, error) { func (p *persistence) fingerprintsModifiedBefore(beforeTime clientmodel.Timestamp) ([]clientmodel.Fingerprint, error) {
var fp codable.Fingerprint var fp codable.Fingerprint
var tr codable.TimeRange var tr codable.TimeRange
fps := []clientmodel.Fingerprint{} fps := []clientmodel.Fingerprint{}
@ -1119,9 +1119,9 @@ func (p *persistence) getFingerprintsModifiedBefore(beforeTime clientmodel.Times
return fps, nil return fps, nil
} }
// getArchivedMetric retrieves the archived metric with the given // archivedMetric retrieves the archived metric with the given fingerprint. This
// fingerprint. This method is goroutine-safe. // method is goroutine-safe.
func (p *persistence) getArchivedMetric(fp clientmodel.Fingerprint) (clientmodel.Metric, error) { func (p *persistence) archivedMetric(fp clientmodel.Fingerprint) (clientmodel.Metric, error) {
metric, _, err := p.archivedFingerprintToMetrics.Lookup(fp) metric, _, err := p.archivedFingerprintToMetrics.Lookup(fp)
return metric, err return metric, err
} }
@ -1137,7 +1137,7 @@ func (p *persistence) purgeArchivedMetric(fp clientmodel.Fingerprint) (err error
} }
}() }()
metric, err := p.getArchivedMetric(fp) metric, err := p.archivedMetric(fp)
if err != nil || metric == nil { if err != nil || metric == nil {
return err return err
} }
@ -1573,7 +1573,7 @@ func writeChunkHeader(w io.Writer, c chunk) error {
) )
binary.LittleEndian.PutUint64( binary.LittleEndian.PutUint64(
header[chunkHeaderLastTimeOffset:], header[chunkHeaderLastTimeOffset:],
uint64(c.newIterator().getLastTimestamp()), uint64(c.newIterator().lastTimestamp()),
) )
_, err := w.Write(header) _, err := w.Write(header)
return err return err

View file

@ -397,7 +397,7 @@ func testCheckpointAndLoadSeriesMapAndHeads(t *testing.T, encoding chunkEncoding
if !reflect.DeepEqual(loadedS1.metric, m1) { if !reflect.DeepEqual(loadedS1.metric, m1) {
t.Errorf("want metric %v, got %v", m1, loadedS1.metric) t.Errorf("want metric %v, got %v", m1, loadedS1.metric)
} }
if !reflect.DeepEqual(loadedS1.head().chunk, s1.head().chunk) { if !reflect.DeepEqual(loadedS1.head().c, s1.head().c) {
t.Error("head chunks differ") t.Error("head chunks differ")
} }
if loadedS1.chunkDescsOffset != 0 { if loadedS1.chunkDescsOffset != 0 {
@ -413,7 +413,7 @@ func testCheckpointAndLoadSeriesMapAndHeads(t *testing.T, encoding chunkEncoding
if !reflect.DeepEqual(loadedS3.metric, m3) { if !reflect.DeepEqual(loadedS3.metric, m3) {
t.Errorf("want metric %v, got %v", m3, loadedS3.metric) t.Errorf("want metric %v, got %v", m3, loadedS3.metric)
} }
if loadedS3.head().chunk != nil { if loadedS3.head().c != nil {
t.Error("head chunk not evicted") t.Error("head chunk not evicted")
} }
if loadedS3.chunkDescsOffset != -1 { if loadedS3.chunkDescsOffset != -1 {
@ -515,7 +515,7 @@ func TestCheckpointAndLoadFPMappings(t *testing.T) {
} }
} }
func testGetFingerprintsModifiedBefore(t *testing.T, encoding chunkEncoding) { func testFingerprintsModifiedBefore(t *testing.T, encoding chunkEncoding) {
p, closer := newTestPersistence(t, encoding) p, closer := newTestPersistence(t, encoding)
defer closer.Close() defer closer.Close()
@ -537,7 +537,7 @@ func testGetFingerprintsModifiedBefore(t *testing.T, encoding chunkEncoding) {
} }
for ts, want := range expectedFPs { for ts, want := range expectedFPs {
got, err := p.getFingerprintsModifiedBefore(ts) got, err := p.fingerprintsModifiedBefore(ts)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -575,7 +575,7 @@ func testGetFingerprintsModifiedBefore(t *testing.T, encoding chunkEncoding) {
} }
for ts, want := range expectedFPs { for ts, want := range expectedFPs {
got, err := p.getFingerprintsModifiedBefore(ts) got, err := p.fingerprintsModifiedBefore(ts)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -585,12 +585,12 @@ func testGetFingerprintsModifiedBefore(t *testing.T, encoding chunkEncoding) {
} }
} }
func TestGetFingerprintsModifiedBeforeChunkType0(t *testing.T) { func TestFingerprintsModifiedBeforeChunkType0(t *testing.T) {
testGetFingerprintsModifiedBefore(t, 0) testFingerprintsModifiedBefore(t, 0)
} }
func TestGetFingerprintsModifiedBeforeChunkType1(t *testing.T) { func TestFingerprintsModifiedBeforeChunkType1(t *testing.T) {
testGetFingerprintsModifiedBefore(t, 1) testFingerprintsModifiedBefore(t, 1)
} }
func testDropArchivedMetric(t *testing.T, encoding chunkEncoding) { func testDropArchivedMetric(t *testing.T, encoding chunkEncoding) {
@ -605,7 +605,7 @@ func testDropArchivedMetric(t *testing.T, encoding chunkEncoding) {
p.indexMetric(2, m2) p.indexMetric(2, m2)
p.waitForIndexing() p.waitForIndexing()
outFPs, err := p.getFingerprintsForLabelPair(metric.LabelPair{Name: "n1", Value: "v1"}) outFPs, err := p.fingerprintsForLabelPair(metric.LabelPair{Name: "n1", Value: "v1"})
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -613,7 +613,7 @@ func testDropArchivedMetric(t *testing.T, encoding chunkEncoding) {
if !reflect.DeepEqual(outFPs, want) { if !reflect.DeepEqual(outFPs, want) {
t.Errorf("want %#v, got %#v", want, outFPs) t.Errorf("want %#v, got %#v", want, outFPs)
} }
outFPs, err = p.getFingerprintsForLabelPair(metric.LabelPair{Name: "n2", Value: "v2"}) outFPs, err = p.fingerprintsForLabelPair(metric.LabelPair{Name: "n2", Value: "v2"})
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -637,7 +637,7 @@ func testDropArchivedMetric(t *testing.T, encoding chunkEncoding) {
} }
p.waitForIndexing() p.waitForIndexing()
outFPs, err = p.getFingerprintsForLabelPair(metric.LabelPair{Name: "n1", Value: "v1"}) outFPs, err = p.fingerprintsForLabelPair(metric.LabelPair{Name: "n1", Value: "v1"})
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -645,7 +645,7 @@ func testDropArchivedMetric(t *testing.T, encoding chunkEncoding) {
if !reflect.DeepEqual(outFPs, want) { if !reflect.DeepEqual(outFPs, want) {
t.Errorf("want %#v, got %#v", want, outFPs) t.Errorf("want %#v, got %#v", want, outFPs)
} }
outFPs, err = p.getFingerprintsForLabelPair(metric.LabelPair{Name: "n2", Value: "v2"}) outFPs, err = p.fingerprintsForLabelPair(metric.LabelPair{Name: "n2", Value: "v2"})
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -858,7 +858,7 @@ func verifyIndexedState(i int, t *testing.T, b incrementalBatch, indexedFpsToMet
p.waitForIndexing() p.waitForIndexing()
for fp, m := range indexedFpsToMetrics { for fp, m := range indexedFpsToMetrics {
// Compare archived metrics with input metrics. // Compare archived metrics with input metrics.
mOut, err := p.getArchivedMetric(fp) mOut, err := p.archivedMetric(fp)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -884,7 +884,7 @@ func verifyIndexedState(i int, t *testing.T, b incrementalBatch, indexedFpsToMet
// Compare label name -> label values mappings. // Compare label name -> label values mappings.
for ln, lvs := range b.expectedLnToLvs { for ln, lvs := range b.expectedLnToLvs {
outLvs, err := p.getLabelValuesForLabelName(ln) outLvs, err := p.labelValuesForLabelName(ln)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -901,7 +901,7 @@ func verifyIndexedState(i int, t *testing.T, b incrementalBatch, indexedFpsToMet
// Compare label pair -> fingerprints mappings. // Compare label pair -> fingerprints mappings.
for lp, fps := range b.expectedLpToFps { for lp, fps := range b.expectedLpToFps {
outFPs, err := p.getFingerprintsForLabelPair(lp) outFPs, err := p.fingerprintsForLabelPair(lp)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }

View file

@ -40,8 +40,8 @@ func (p *memorySeriesPreloader) PreloadRange(
} }
/* /*
// GetMetricAtTime implements Preloader. // MetricAtTime implements Preloader.
func (p *memorySeriesPreloader) GetMetricAtTime(fp clientmodel.Fingerprint, t clientmodel.Timestamp) error { func (p *memorySeriesPreloader) MetricAtTime(fp clientmodel.Fingerprint, t clientmodel.Timestamp) error {
cds, err := p.storage.preloadChunks(fp, &timeSelector{ cds, err := p.storage.preloadChunks(fp, &timeSelector{
from: t, from: t,
through: t, through: t,
@ -53,8 +53,8 @@ func (p *memorySeriesPreloader) GetMetricAtTime(fp clientmodel.Fingerprint, t cl
return nil return nil
} }
// GetMetricAtInterval implements Preloader. // MetricAtInterval implements Preloader.
func (p *memorySeriesPreloader) GetMetricAtInterval(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval time.Duration) error { func (p *memorySeriesPreloader) MetricAtInterval(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval time.Duration) error {
cds, err := p.storage.preloadChunks(fp, &timeSelector{ cds, err := p.storage.preloadChunks(fp, &timeSelector{
from: from, from: from,
through: through, through: through,
@ -67,8 +67,8 @@ func (p *memorySeriesPreloader) GetMetricAtInterval(fp clientmodel.Fingerprint,
return return
} }
// GetMetricRange implements Preloader. // MetricRange implements Preloader.
func (p *memorySeriesPreloader) GetMetricRange(fp clientmodel.Fingerprint, t clientmodel.Timestamp, rangeDuration time.Duration) error { func (p *memorySeriesPreloader) MetricRange(fp clientmodel.Fingerprint, t clientmodel.Timestamp, rangeDuration time.Duration) error {
cds, err := p.storage.preloadChunks(fp, &timeSelector{ cds, err := p.storage.preloadChunks(fp, &timeSelector{
from: t, from: t,
through: t, through: t,
@ -81,8 +81,8 @@ func (p *memorySeriesPreloader) GetMetricRange(fp clientmodel.Fingerprint, t cli
return return
} }
// GetMetricRangeAtInterval implements Preloader. // MetricRangeAtInterval implements Preloader.
func (p *memorySeriesPreloader) GetMetricRangeAtInterval(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval, rangeDuration time.Duration) error { func (p *memorySeriesPreloader) MetricRangeAtInterval(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval, rangeDuration time.Duration) error {
cds, err := p.storage.preloadChunks(fp, &timeSelector{ cds, err := p.storage.preloadChunks(fp, &timeSelector{
from: from, from: from,
through: through, through: through,

View file

@ -208,7 +208,7 @@ func (s *memorySeries) add(v *metric.SamplePair) int {
newHead := newChunkDesc(newChunk()) newHead := newChunkDesc(newChunk())
s.chunkDescs = append(s.chunkDescs, newHead) s.chunkDescs = append(s.chunkDescs, newHead)
s.headChunkClosed = false s.headChunkClosed = false
} else if s.headChunkUsedByIterator && s.head().getRefCount() > 1 { } else if s.headChunkUsedByIterator && s.head().refCount() > 1 {
// We only need to clone the head chunk if the current head // We only need to clone the head chunk if the current head
// chunk was used in an iterator at all and if the refCount is // chunk was used in an iterator at all and if the refCount is
// still greater than the 1 we always have because the head // still greater than the 1 we always have because the head
@ -221,12 +221,12 @@ func (s *memorySeries) add(v *metric.SamplePair) int {
chunkOps.WithLabelValues(clone).Inc() chunkOps.WithLabelValues(clone).Inc()
// No locking needed here because a non-persisted head chunk can // No locking needed here because a non-persisted head chunk can
// not get evicted concurrently. // not get evicted concurrently.
s.head().chunk = s.head().chunk.clone() s.head().c = s.head().c.clone()
s.headChunkUsedByIterator = false s.headChunkUsedByIterator = false
} }
chunks := s.head().add(v) chunks := s.head().add(v)
s.head().chunk = chunks[0] s.head().c = chunks[0]
for _, c := range chunks[1:] { for _, c := range chunks[1:] {
s.chunkDescs = append(s.chunkDescs, newChunkDesc(c)) s.chunkDescs = append(s.chunkDescs, newChunkDesc(c))
@ -418,7 +418,7 @@ func (s *memorySeries) preloadChunksForRange(
func (s *memorySeries) newIterator() SeriesIterator { func (s *memorySeries) newIterator() SeriesIterator {
chunks := make([]chunk, 0, len(s.chunkDescs)) chunks := make([]chunk, 0, len(s.chunkDescs))
for i, cd := range s.chunkDescs { for i, cd := range s.chunkDescs {
if chunk := cd.getChunk(); chunk != nil { if chunk := cd.chunk(); chunk != nil {
if i == len(s.chunkDescs)-1 && !s.headChunkClosed { if i == len(s.chunkDescs)-1 && !s.headChunkClosed {
s.headChunkUsedByIterator = true s.headChunkUsedByIterator = true
} }
@ -448,13 +448,13 @@ func (s *memorySeries) firstTime() clientmodel.Timestamp {
return s.savedFirstTime return s.savedFirstTime
} }
// getChunksToPersist returns a slice of chunkDescs eligible for // chunksToPersist returns a slice of chunkDescs eligible for persistence. It's
// persistence. It's the caller's responsibility to actually persist the // the caller's responsibility to actually persist the returned chunks
// returned chunks afterwards. The method sets the persistWatermark and the // afterwards. The method sets the persistWatermark and the dirty flag
// dirty flag accordingly. // accordingly.
// //
// The caller must have locked the fingerprint of the series. // The caller must have locked the fingerprint of the series.
func (s *memorySeries) getChunksToPersist() []*chunkDesc { func (s *memorySeries) chunksToPersist() []*chunkDesc {
newWatermark := len(s.chunkDescs) newWatermark := len(s.chunkDescs)
if !s.headChunkClosed { if !s.headChunkClosed {
newWatermark-- newWatermark--
@ -470,16 +470,16 @@ func (s *memorySeries) getChunksToPersist() []*chunkDesc {
// memorySeriesIterator implements SeriesIterator. // memorySeriesIterator implements SeriesIterator.
type memorySeriesIterator struct { type memorySeriesIterator struct {
chunkIt chunkIterator // Last chunkIterator used by GetValueAtTime. chunkIt chunkIterator // Last chunkIterator used by ValueAtTime.
chunkIts []chunkIterator // Caches chunkIterators. chunkIts []chunkIterator // Caches chunkIterators.
chunks []chunk chunks []chunk
} }
// GetValueAtTime implements SeriesIterator. // ValueAtTime implements SeriesIterator.
func (it *memorySeriesIterator) GetValueAtTime(t clientmodel.Timestamp) metric.Values { func (it *memorySeriesIterator) ValueAtTime(t clientmodel.Timestamp) metric.Values {
// The most common case. We are iterating through a chunk. // The most common case. We are iterating through a chunk.
if it.chunkIt != nil && it.chunkIt.contains(t) { if it.chunkIt != nil && it.chunkIt.contains(t) {
return it.chunkIt.getValueAtTime(t) return it.chunkIt.valueAtTime(t)
} }
if len(it.chunks) == 0 { if len(it.chunks) == 0 {
@ -487,24 +487,24 @@ func (it *memorySeriesIterator) GetValueAtTime(t clientmodel.Timestamp) metric.V
} }
// Before or exactly on the first sample of the series. // Before or exactly on the first sample of the series.
it.chunkIt = it.getChunkIterator(0) it.chunkIt = it.chunkIterator(0)
ts := it.chunkIt.getTimestampAtIndex(0) ts := it.chunkIt.timestampAtIndex(0)
if !t.After(ts) { if !t.After(ts) {
// return first value of first chunk // return first value of first chunk
return metric.Values{metric.SamplePair{ return metric.Values{metric.SamplePair{
Timestamp: ts, Timestamp: ts,
Value: it.chunkIt.getSampleValueAtIndex(0), Value: it.chunkIt.sampleValueAtIndex(0),
}} }}
} }
// After or exactly on the last sample of the series. // After or exactly on the last sample of the series.
it.chunkIt = it.getChunkIterator(len(it.chunks) - 1) it.chunkIt = it.chunkIterator(len(it.chunks) - 1)
ts = it.chunkIt.getLastTimestamp() ts = it.chunkIt.lastTimestamp()
if !t.Before(ts) { if !t.Before(ts) {
// return last value of last chunk // return last value of last chunk
return metric.Values{metric.SamplePair{ return metric.Values{metric.SamplePair{
Timestamp: ts, Timestamp: ts,
Value: it.chunkIt.getSampleValueAtIndex(it.chunkIt.length() - 1), Value: it.chunkIt.sampleValueAtIndex(it.chunkIt.length() - 1),
}} }}
} }
@ -516,35 +516,35 @@ func (it *memorySeriesIterator) GetValueAtTime(t clientmodel.Timestamp) metric.V
if i == len(it.chunks) { if i == len(it.chunks) {
panic("out of bounds") panic("out of bounds")
} }
it.chunkIt = it.getChunkIterator(l - i) it.chunkIt = it.chunkIterator(l - i)
ts = it.chunkIt.getLastTimestamp() ts = it.chunkIt.lastTimestamp()
if t.After(ts) { if t.After(ts) {
// We ended up between two chunks. // We ended up between two chunks.
sp1 := metric.SamplePair{ sp1 := metric.SamplePair{
Timestamp: ts, Timestamp: ts,
Value: it.chunkIt.getSampleValueAtIndex(it.chunkIt.length() - 1), Value: it.chunkIt.sampleValueAtIndex(it.chunkIt.length() - 1),
} }
it.chunkIt = it.getChunkIterator(l - i + 1) it.chunkIt = it.chunkIterator(l - i + 1)
return metric.Values{ return metric.Values{
sp1, sp1,
metric.SamplePair{ metric.SamplePair{
Timestamp: it.chunkIt.getTimestampAtIndex(0), Timestamp: it.chunkIt.timestampAtIndex(0),
Value: it.chunkIt.getSampleValueAtIndex(0), Value: it.chunkIt.sampleValueAtIndex(0),
}, },
} }
} }
return it.chunkIt.getValueAtTime(t) return it.chunkIt.valueAtTime(t)
} }
// GetBoundaryValues implements SeriesIterator. // BoundaryValues implements SeriesIterator.
func (it *memorySeriesIterator) GetBoundaryValues(in metric.Interval) metric.Values { func (it *memorySeriesIterator) BoundaryValues(in metric.Interval) metric.Values {
// Find the first chunk for which the first sample is within the interval. // Find the first chunk for which the first sample is within the interval.
i := sort.Search(len(it.chunks), func(i int) bool { i := sort.Search(len(it.chunks), func(i int) bool {
return !it.chunks[i].firstTime().Before(in.OldestInclusive) return !it.chunks[i].firstTime().Before(in.OldestInclusive)
}) })
// Only now check the last timestamp of the previous chunk (which is // Only now check the last timestamp of the previous chunk (which is
// fairly expensive). // fairly expensive).
if i > 0 && !it.getChunkIterator(i-1).getLastTimestamp().Before(in.OldestInclusive) { if i > 0 && !it.chunkIterator(i-1).lastTimestamp().Before(in.OldestInclusive) {
i-- i--
} }
@ -556,37 +556,37 @@ func (it *memorySeriesIterator) GetBoundaryValues(in metric.Interval) metric.Val
// already past the last value. The value we // already past the last value. The value we
// want must be the last value of the previous // want must be the last value of the previous
// chunk. So backtrack... // chunk. So backtrack...
chunkIt := it.getChunkIterator(i + j - 1) chunkIt := it.chunkIterator(i + j - 1)
values = append(values, metric.SamplePair{ values = append(values, metric.SamplePair{
Timestamp: chunkIt.getLastTimestamp(), Timestamp: chunkIt.lastTimestamp(),
Value: chunkIt.getLastSampleValue(), Value: chunkIt.lastSampleValue(),
}) })
} }
break break
} }
chunkIt := it.getChunkIterator(i + j) chunkIt := it.chunkIterator(i + j)
if len(values) == 0 { if len(values) == 0 {
firstValues := chunkIt.getValueAtTime(in.OldestInclusive) firstValues := chunkIt.valueAtTime(in.OldestInclusive)
switch len(firstValues) { switch len(firstValues) {
case 2: case 2:
values = append(values, firstValues[1]) values = append(values, firstValues[1])
case 1: case 1:
values = firstValues values = firstValues
default: default:
panic("unexpected return from getValueAtTime") panic("unexpected return from valueAtTime")
} }
} }
if chunkIt.getLastTimestamp().After(in.NewestInclusive) { if chunkIt.lastTimestamp().After(in.NewestInclusive) {
values = append(values, chunkIt.getValueAtTime(in.NewestInclusive)[0]) values = append(values, chunkIt.valueAtTime(in.NewestInclusive)[0])
break break
} }
} }
if len(values) == 1 { if len(values) == 1 {
// We found exactly one value. In that case, add the most recent we know. // We found exactly one value. In that case, add the most recent we know.
chunkIt := it.getChunkIterator(len(it.chunks) - 1) chunkIt := it.chunkIterator(len(it.chunks) - 1)
values = append(values, metric.SamplePair{ values = append(values, metric.SamplePair{
Timestamp: chunkIt.getLastTimestamp(), Timestamp: chunkIt.lastTimestamp(),
Value: chunkIt.getLastSampleValue(), Value: chunkIt.lastSampleValue(),
}) })
} }
if len(values) == 2 && values[0].Equal(&values[1]) { if len(values) == 2 && values[0].Equal(&values[1]) {
@ -595,15 +595,15 @@ func (it *memorySeriesIterator) GetBoundaryValues(in metric.Interval) metric.Val
return values return values
} }
// GetRangeValues implements SeriesIterator. // RangeValues implements SeriesIterator.
func (it *memorySeriesIterator) GetRangeValues(in metric.Interval) metric.Values { func (it *memorySeriesIterator) RangeValues(in metric.Interval) metric.Values {
// Find the first chunk for which the first sample is within the interval. // Find the first chunk for which the first sample is within the interval.
i := sort.Search(len(it.chunks), func(i int) bool { i := sort.Search(len(it.chunks), func(i int) bool {
return !it.chunks[i].firstTime().Before(in.OldestInclusive) return !it.chunks[i].firstTime().Before(in.OldestInclusive)
}) })
// Only now check the last timestamp of the previous chunk (which is // Only now check the last timestamp of the previous chunk (which is
// fairly expensive). // fairly expensive).
if i > 0 && !it.getChunkIterator(i-1).getLastTimestamp().Before(in.OldestInclusive) { if i > 0 && !it.chunkIterator(i-1).lastTimestamp().Before(in.OldestInclusive) {
i-- i--
} }
@ -612,14 +612,14 @@ func (it *memorySeriesIterator) GetRangeValues(in metric.Interval) metric.Values
if c.firstTime().After(in.NewestInclusive) { if c.firstTime().After(in.NewestInclusive) {
break break
} }
values = append(values, it.getChunkIterator(i+j).getRangeValues(in)...) values = append(values, it.chunkIterator(i+j).rangeValues(in)...)
} }
return values return values
} }
// getChunkIterator returns the chunkIterator for the chunk at position i (and // chunkIterator returns the chunkIterator for the chunk at position i (and
// creates it if needed). // creates it if needed).
func (it *memorySeriesIterator) getChunkIterator(i int) chunkIterator { func (it *memorySeriesIterator) chunkIterator(i int) chunkIterator {
chunkIt := it.chunkIts[i] chunkIt := it.chunkIts[i]
if chunkIt == nil { if chunkIt == nil {
chunkIt = it.chunks[i].newIterator() chunkIt = it.chunks[i].newIterator()
@ -631,17 +631,17 @@ func (it *memorySeriesIterator) getChunkIterator(i int) chunkIterator {
// nopSeriesIterator implements Series Iterator. It never returns any values. // nopSeriesIterator implements Series Iterator. It never returns any values.
type nopSeriesIterator struct{} type nopSeriesIterator struct{}
// GetValueAtTime implements SeriesIterator. // ValueAtTime implements SeriesIterator.
func (_ nopSeriesIterator) GetValueAtTime(t clientmodel.Timestamp) metric.Values { func (_ nopSeriesIterator) ValueAtTime(t clientmodel.Timestamp) metric.Values {
return metric.Values{} return metric.Values{}
} }
// GetBoundaryValues implements SeriesIterator. // BoundaryValues implements SeriesIterator.
func (_ nopSeriesIterator) GetBoundaryValues(in metric.Interval) metric.Values { func (_ nopSeriesIterator) BoundaryValues(in metric.Interval) metric.Values {
return metric.Values{} return metric.Values{}
} }
// GetRangeValues implements SeriesIterator. // RangeValues implements SeriesIterator.
func (_ nopSeriesIterator) GetRangeValues(in metric.Interval) metric.Values { func (_ nopSeriesIterator) RangeValues(in metric.Interval) metric.Values {
return metric.Values{} return metric.Values{}
} }

View file

@ -287,14 +287,14 @@ func (s *memorySeriesStorage) NewPreloader() Preloader {
} }
} }
// GetFingerprintsForLabelMatchers implements Storage. // FingerprintsForLabelMatchers implements Storage.
func (s *memorySeriesStorage) GetFingerprintsForLabelMatchers(labelMatchers metric.LabelMatchers) clientmodel.Fingerprints { func (s *memorySeriesStorage) FingerprintsForLabelMatchers(labelMatchers metric.LabelMatchers) clientmodel.Fingerprints {
var result map[clientmodel.Fingerprint]struct{} var result map[clientmodel.Fingerprint]struct{}
for _, matcher := range labelMatchers { for _, matcher := range labelMatchers {
intersection := map[clientmodel.Fingerprint]struct{}{} intersection := map[clientmodel.Fingerprint]struct{}{}
switch matcher.Type { switch matcher.Type {
case metric.Equal: case metric.Equal:
fps, err := s.persistence.getFingerprintsForLabelPair( fps, err := s.persistence.fingerprintsForLabelPair(
metric.LabelPair{ metric.LabelPair{
Name: matcher.Name, Name: matcher.Name,
Value: matcher.Value, Value: matcher.Value,
@ -312,7 +312,7 @@ func (s *memorySeriesStorage) GetFingerprintsForLabelMatchers(labelMatchers metr
} }
} }
default: default:
values, err := s.persistence.getLabelValuesForLabelName(matcher.Name) values, err := s.persistence.labelValuesForLabelName(matcher.Name)
if err != nil { if err != nil {
glog.Errorf("Error getting label values for label name %q: %v", matcher.Name, err) glog.Errorf("Error getting label values for label name %q: %v", matcher.Name, err)
} }
@ -321,7 +321,7 @@ func (s *memorySeriesStorage) GetFingerprintsForLabelMatchers(labelMatchers metr
return nil return nil
} }
for _, v := range matches { for _, v := range matches {
fps, err := s.persistence.getFingerprintsForLabelPair( fps, err := s.persistence.fingerprintsForLabelPair(
metric.LabelPair{ metric.LabelPair{
Name: matcher.Name, Name: matcher.Name,
Value: v, Value: v,
@ -350,17 +350,17 @@ func (s *memorySeriesStorage) GetFingerprintsForLabelMatchers(labelMatchers metr
return fps return fps
} }
// GetLabelValuesForLabelName implements Storage. // LabelValuesForLabelName implements Storage.
func (s *memorySeriesStorage) GetLabelValuesForLabelName(labelName clientmodel.LabelName) clientmodel.LabelValues { func (s *memorySeriesStorage) LabelValuesForLabelName(labelName clientmodel.LabelName) clientmodel.LabelValues {
lvs, err := s.persistence.getLabelValuesForLabelName(labelName) lvs, err := s.persistence.labelValuesForLabelName(labelName)
if err != nil { if err != nil {
glog.Errorf("Error getting label values for label name %q: %v", labelName, err) glog.Errorf("Error getting label values for label name %q: %v", labelName, err)
} }
return lvs return lvs
} }
// GetMetricForFingerprint implements Storage. // MetricForFingerprint implements Storage.
func (s *memorySeriesStorage) GetMetricForFingerprint(fp clientmodel.Fingerprint) clientmodel.COWMetric { func (s *memorySeriesStorage) MetricForFingerprint(fp clientmodel.Fingerprint) clientmodel.COWMetric {
s.fpLocker.Lock(fp) s.fpLocker.Lock(fp)
defer s.fpLocker.Unlock(fp) defer s.fpLocker.Unlock(fp)
@ -372,7 +372,7 @@ func (s *memorySeriesStorage) GetMetricForFingerprint(fp clientmodel.Fingerprint
Metric: series.metric, Metric: series.metric,
} }
} }
metric, err := s.persistence.getArchivedMetric(fp) metric, err := s.persistence.archivedMetric(fp)
if err != nil { if err != nil {
glog.Errorf("Error retrieving archived metric for fingerprint %v: %v", fp, err) glog.Errorf("Error retrieving archived metric for fingerprint %v: %v", fp, err)
} }
@ -455,7 +455,7 @@ func (s *memorySeriesStorage) preloadChunksForRange(
return nil, nil return nil, nil
} }
if from.Add(-stalenessDelta).Before(last) && through.Add(stalenessDelta).After(first) { if from.Add(-stalenessDelta).Before(last) && through.Add(stalenessDelta).After(first) {
metric, err := s.persistence.getArchivedMetric(fp) metric, err := s.persistence.archivedMetric(fp)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -644,7 +644,7 @@ func (s *memorySeriesStorage) cycleThroughArchivedFingerprints() chan clientmode
defer close(archivedFingerprints) defer close(archivedFingerprints)
for { for {
archivedFPs, err := s.persistence.getFingerprintsModifiedBefore( archivedFPs, err := s.persistence.fingerprintsModifiedBefore(
clientmodel.TimestampFromTime(time.Now()).Add(-s.dropAfter), clientmodel.TimestampFromTime(time.Now()).Add(-s.dropAfter),
) )
if err != nil { if err != nil {
@ -840,20 +840,21 @@ func (s *memorySeriesStorage) maintainMemorySeries(
func (s *memorySeriesStorage) writeMemorySeries( func (s *memorySeriesStorage) writeMemorySeries(
fp clientmodel.Fingerprint, series *memorySeries, beforeTime clientmodel.Timestamp, fp clientmodel.Fingerprint, series *memorySeries, beforeTime clientmodel.Timestamp,
) bool { ) bool {
cds := series.getChunksToPersist() cds := series.chunksToPersist()
defer func() { defer func() {
for _, cd := range cds { for _, cd := range cds {
cd.unpin(s.evictRequests) cd.unpin(s.evictRequests)
} }
s.incNumChunksToPersist(-len(cds)) s.incNumChunksToPersist(-len(cds))
chunkOps.WithLabelValues(persistAndUnpin).Add(float64(len(cds))) chunkOps.WithLabelValues(persistAndUnpin).Add(float64(len(cds)))
series.modTime = s.persistence.getSeriesFileModTime(fp) series.modTime = s.persistence.seriesFileModTime(fp)
}() }()
// Get the actual chunks from underneath the chunkDescs. // Get the actual chunks from underneath the chunkDescs.
// No lock required as chunks still to persist cannot be evicted.
chunks := make([]chunk, len(cds)) chunks := make([]chunk, len(cds))
for i, cd := range cds { for i, cd := range cds {
chunks[i] = cd.chunk chunks[i] = cd.c
} }
if !series.firstTime().Before(beforeTime) { if !series.firstTime().Before(beforeTime) {

View file

@ -28,7 +28,7 @@ import (
"github.com/prometheus/prometheus/utility/test" "github.com/prometheus/prometheus/utility/test"
) )
func TestGetFingerprintsForLabelMatchers(t *testing.T) { func TestFingerprintsForLabelMatchers(t *testing.T) {
storage, closer := NewTestStorage(t, 1) storage, closer := NewTestStorage(t, 1)
defer closer.Close() defer closer.Close()
@ -121,7 +121,7 @@ func TestGetFingerprintsForLabelMatchers(t *testing.T) {
} }
for _, mt := range matcherTests { for _, mt := range matcherTests {
resfps := storage.GetFingerprintsForLabelMatchers(mt.matchers) resfps := storage.FingerprintsForLabelMatchers(mt.matchers)
if len(mt.expected) != len(resfps) { if len(mt.expected) != len(resfps) {
t.Fatalf("expected %d matches for %q, found %d", len(mt.expected), mt.matchers, len(resfps)) t.Fatalf("expected %d matches for %q, found %d", len(mt.expected), mt.matchers, len(resfps))
} }
@ -208,7 +208,7 @@ func testChunk(t *testing.T, encoding chunkEncoding) {
if cd.isEvicted() { if cd.isEvicted() {
continue continue
} }
for sample := range cd.chunk.newIterator().values() { for sample := range cd.c.newIterator().values() {
values = append(values, *sample) values = append(values, *sample)
} }
} }
@ -234,7 +234,7 @@ func TestChunkType1(t *testing.T) {
testChunk(t, 1) testChunk(t, 1)
} }
func testGetValueAtTime(t *testing.T, encoding chunkEncoding) { func testValueAtTime(t *testing.T, encoding chunkEncoding) {
samples := make(clientmodel.Samples, 10000) samples := make(clientmodel.Samples, 10000)
for i := range samples { for i := range samples {
samples[i] = &clientmodel.Sample{ samples[i] = &clientmodel.Sample{
@ -256,7 +256,7 @@ func testGetValueAtTime(t *testing.T, encoding chunkEncoding) {
// #1 Exactly on a sample. // #1 Exactly on a sample.
for i, expected := range samples { for i, expected := range samples {
actual := it.GetValueAtTime(expected.Timestamp) actual := it.ValueAtTime(expected.Timestamp)
if len(actual) != 1 { if len(actual) != 1 {
t.Fatalf("1.%d. Expected exactly one result, got %d.", i, len(actual)) t.Fatalf("1.%d. Expected exactly one result, got %d.", i, len(actual))
@ -275,7 +275,7 @@ func testGetValueAtTime(t *testing.T, encoding chunkEncoding) {
continue continue
} }
expected2 := samples[i+1] expected2 := samples[i+1]
actual := it.GetValueAtTime(expected1.Timestamp + 1) actual := it.ValueAtTime(expected1.Timestamp + 1)
if len(actual) != 2 { if len(actual) != 2 {
t.Fatalf("2.%d. Expected exactly 2 results, got %d.", i, len(actual)) t.Fatalf("2.%d. Expected exactly 2 results, got %d.", i, len(actual))
@ -296,7 +296,7 @@ func testGetValueAtTime(t *testing.T, encoding chunkEncoding) {
// #3 Corner cases: Just before the first sample, just after the last. // #3 Corner cases: Just before the first sample, just after the last.
expected := samples[0] expected := samples[0]
actual := it.GetValueAtTime(expected.Timestamp - 1) actual := it.ValueAtTime(expected.Timestamp - 1)
if len(actual) != 1 { if len(actual) != 1 {
t.Fatalf("3.1. Expected exactly one result, got %d.", len(actual)) t.Fatalf("3.1. Expected exactly one result, got %d.", len(actual))
} }
@ -307,7 +307,7 @@ func testGetValueAtTime(t *testing.T, encoding chunkEncoding) {
t.Errorf("3.1. Got %v; want %v", actual[0].Value, expected.Value) t.Errorf("3.1. Got %v; want %v", actual[0].Value, expected.Value)
} }
expected = samples[len(samples)-1] expected = samples[len(samples)-1]
actual = it.GetValueAtTime(expected.Timestamp + 1) actual = it.ValueAtTime(expected.Timestamp + 1)
if len(actual) != 1 { if len(actual) != 1 {
t.Fatalf("3.2. Expected exactly one result, got %d.", len(actual)) t.Fatalf("3.2. Expected exactly one result, got %d.", len(actual))
} }
@ -319,15 +319,15 @@ func testGetValueAtTime(t *testing.T, encoding chunkEncoding) {
} }
} }
func TestGetValueAtTimeChunkType0(t *testing.T) { func TestValueAtTimeChunkType0(t *testing.T) {
testGetValueAtTime(t, 0) testValueAtTime(t, 0)
} }
func TestGetValueAtTimeChunkType1(t *testing.T) { func TestValueAtTimeChunkType1(t *testing.T) {
testGetValueAtTime(t, 1) testValueAtTime(t, 1)
} }
func benchmarkGetValueAtTime(b *testing.B, encoding chunkEncoding) { func benchmarkValueAtTime(b *testing.B, encoding chunkEncoding) {
samples := make(clientmodel.Samples, 10000) samples := make(clientmodel.Samples, 10000)
for i := range samples { for i := range samples {
samples[i] = &clientmodel.Sample{ samples[i] = &clientmodel.Sample{
@ -352,7 +352,7 @@ func benchmarkGetValueAtTime(b *testing.B, encoding chunkEncoding) {
// #1 Exactly on a sample. // #1 Exactly on a sample.
for i, expected := range samples { for i, expected := range samples {
actual := it.GetValueAtTime(expected.Timestamp) actual := it.ValueAtTime(expected.Timestamp)
if len(actual) != 1 { if len(actual) != 1 {
b.Fatalf("1.%d. Expected exactly one result, got %d.", i, len(actual)) b.Fatalf("1.%d. Expected exactly one result, got %d.", i, len(actual))
@ -371,7 +371,7 @@ func benchmarkGetValueAtTime(b *testing.B, encoding chunkEncoding) {
continue continue
} }
expected2 := samples[i+1] expected2 := samples[i+1]
actual := it.GetValueAtTime(expected1.Timestamp + 1) actual := it.ValueAtTime(expected1.Timestamp + 1)
if len(actual) != 2 { if len(actual) != 2 {
b.Fatalf("2.%d. Expected exactly 2 results, got %d.", i, len(actual)) b.Fatalf("2.%d. Expected exactly 2 results, got %d.", i, len(actual))
@ -392,15 +392,15 @@ func benchmarkGetValueAtTime(b *testing.B, encoding chunkEncoding) {
} }
} }
func BenchmarkGetValueAtTimeChunkType0(b *testing.B) { func BenchmarkValueAtTimeChunkType0(b *testing.B) {
benchmarkGetValueAtTime(b, 0) benchmarkValueAtTime(b, 0)
} }
func BenchmarkGetValueAtTimeChunkType1(b *testing.B) { func BenchmarkValueAtTimeChunkType1(b *testing.B) {
benchmarkGetValueAtTime(b, 1) benchmarkValueAtTime(b, 1)
} }
func testGetRangeValues(t *testing.T, encoding chunkEncoding) { func testRangeValues(t *testing.T, encoding chunkEncoding) {
samples := make(clientmodel.Samples, 10000) samples := make(clientmodel.Samples, 10000)
for i := range samples { for i := range samples {
samples[i] = &clientmodel.Sample{ samples[i] = &clientmodel.Sample{
@ -422,7 +422,7 @@ func testGetRangeValues(t *testing.T, encoding chunkEncoding) {
// #1 Zero length interval at sample. // #1 Zero length interval at sample.
for i, expected := range samples { for i, expected := range samples {
actual := it.GetRangeValues(metric.Interval{ actual := it.RangeValues(metric.Interval{
OldestInclusive: expected.Timestamp, OldestInclusive: expected.Timestamp,
NewestInclusive: expected.Timestamp, NewestInclusive: expected.Timestamp,
}) })
@ -440,7 +440,7 @@ func testGetRangeValues(t *testing.T, encoding chunkEncoding) {
// #2 Zero length interval off sample. // #2 Zero length interval off sample.
for i, expected := range samples { for i, expected := range samples {
actual := it.GetRangeValues(metric.Interval{ actual := it.RangeValues(metric.Interval{
OldestInclusive: expected.Timestamp + 1, OldestInclusive: expected.Timestamp + 1,
NewestInclusive: expected.Timestamp + 1, NewestInclusive: expected.Timestamp + 1,
}) })
@ -452,7 +452,7 @@ func testGetRangeValues(t *testing.T, encoding chunkEncoding) {
// #3 2sec interval around sample. // #3 2sec interval around sample.
for i, expected := range samples { for i, expected := range samples {
actual := it.GetRangeValues(metric.Interval{ actual := it.RangeValues(metric.Interval{
OldestInclusive: expected.Timestamp - 1, OldestInclusive: expected.Timestamp - 1,
NewestInclusive: expected.Timestamp + 1, NewestInclusive: expected.Timestamp + 1,
}) })
@ -474,7 +474,7 @@ func testGetRangeValues(t *testing.T, encoding chunkEncoding) {
continue continue
} }
expected2 := samples[i+1] expected2 := samples[i+1]
actual := it.GetRangeValues(metric.Interval{ actual := it.RangeValues(metric.Interval{
OldestInclusive: expected1.Timestamp, OldestInclusive: expected1.Timestamp,
NewestInclusive: expected1.Timestamp + 2, NewestInclusive: expected1.Timestamp + 2,
}) })
@ -499,7 +499,7 @@ func testGetRangeValues(t *testing.T, encoding chunkEncoding) {
// #5 corner cases: Interval ends at first sample, interval starts // #5 corner cases: Interval ends at first sample, interval starts
// at last sample, interval entirely before/after samples. // at last sample, interval entirely before/after samples.
expected := samples[0] expected := samples[0]
actual := it.GetRangeValues(metric.Interval{ actual := it.RangeValues(metric.Interval{
OldestInclusive: expected.Timestamp - 2, OldestInclusive: expected.Timestamp - 2,
NewestInclusive: expected.Timestamp, NewestInclusive: expected.Timestamp,
}) })
@ -513,7 +513,7 @@ func testGetRangeValues(t *testing.T, encoding chunkEncoding) {
t.Errorf("5.1. Got %v; want %v.", actual[0].Value, expected.Value) t.Errorf("5.1. Got %v; want %v.", actual[0].Value, expected.Value)
} }
expected = samples[len(samples)-1] expected = samples[len(samples)-1]
actual = it.GetRangeValues(metric.Interval{ actual = it.RangeValues(metric.Interval{
OldestInclusive: expected.Timestamp, OldestInclusive: expected.Timestamp,
NewestInclusive: expected.Timestamp + 2, NewestInclusive: expected.Timestamp + 2,
}) })
@ -527,7 +527,7 @@ func testGetRangeValues(t *testing.T, encoding chunkEncoding) {
t.Errorf("5.2. Got %v; want %v.", actual[0].Value, expected.Value) t.Errorf("5.2. Got %v; want %v.", actual[0].Value, expected.Value)
} }
firstSample := samples[0] firstSample := samples[0]
actual = it.GetRangeValues(metric.Interval{ actual = it.RangeValues(metric.Interval{
OldestInclusive: firstSample.Timestamp - 4, OldestInclusive: firstSample.Timestamp - 4,
NewestInclusive: firstSample.Timestamp - 2, NewestInclusive: firstSample.Timestamp - 2,
}) })
@ -535,7 +535,7 @@ func testGetRangeValues(t *testing.T, encoding chunkEncoding) {
t.Fatalf("5.3. Expected no results, got %d.", len(actual)) t.Fatalf("5.3. Expected no results, got %d.", len(actual))
} }
lastSample := samples[len(samples)-1] lastSample := samples[len(samples)-1]
actual = it.GetRangeValues(metric.Interval{ actual = it.RangeValues(metric.Interval{
OldestInclusive: lastSample.Timestamp + 2, OldestInclusive: lastSample.Timestamp + 2,
NewestInclusive: lastSample.Timestamp + 4, NewestInclusive: lastSample.Timestamp + 4,
}) })
@ -544,15 +544,15 @@ func testGetRangeValues(t *testing.T, encoding chunkEncoding) {
} }
} }
func TestGetRangeValuesChunkType0(t *testing.T) { func TestRangeValuesChunkType0(t *testing.T) {
testGetRangeValues(t, 0) testRangeValues(t, 0)
} }
func TestGetRangeValuesChunkType1(t *testing.T) { func TestRangeValuesChunkType1(t *testing.T) {
testGetRangeValues(t, 1) testRangeValues(t, 1)
} }
func benchmarkGetRangeValues(b *testing.B, encoding chunkEncoding) { func benchmarkRangeValues(b *testing.B, encoding chunkEncoding) {
samples := make(clientmodel.Samples, 10000) samples := make(clientmodel.Samples, 10000)
for i := range samples { for i := range samples {
samples[i] = &clientmodel.Sample{ samples[i] = &clientmodel.Sample{
@ -577,7 +577,7 @@ func benchmarkGetRangeValues(b *testing.B, encoding chunkEncoding) {
it := s.NewIterator(fp) it := s.NewIterator(fp)
for _, sample := range samples { for _, sample := range samples {
actual := it.GetRangeValues(metric.Interval{ actual := it.RangeValues(metric.Interval{
OldestInclusive: sample.Timestamp - 20, OldestInclusive: sample.Timestamp - 20,
NewestInclusive: sample.Timestamp + 20, NewestInclusive: sample.Timestamp + 20,
}) })
@ -589,12 +589,12 @@ func benchmarkGetRangeValues(b *testing.B, encoding chunkEncoding) {
} }
} }
func BenchmarkGetRangeValuesChunkType0(b *testing.B) { func BenchmarkRangeValuesChunkType0(b *testing.B) {
benchmarkGetRangeValues(b, 0) benchmarkRangeValues(b, 0)
} }
func BenchmarkGetRangeValuesChunkType1(b *testing.B) { func BenchmarkRangeValuesChunkType1(b *testing.B) {
benchmarkGetRangeValues(b, 1) benchmarkRangeValues(b, 1)
} }
func testEvictAndPurgeSeries(t *testing.T, encoding chunkEncoding) { func testEvictAndPurgeSeries(t *testing.T, encoding chunkEncoding) {
@ -618,7 +618,7 @@ func testEvictAndPurgeSeries(t *testing.T, encoding chunkEncoding) {
// Drop ~half of the chunks. // Drop ~half of the chunks.
s.maintainMemorySeries(fp, 10000) s.maintainMemorySeries(fp, 10000)
it := s.NewIterator(fp) it := s.NewIterator(fp)
actual := it.GetBoundaryValues(metric.Interval{ actual := it.BoundaryValues(metric.Interval{
OldestInclusive: 0, OldestInclusive: 0,
NewestInclusive: 100000, NewestInclusive: 100000,
}) })
@ -636,7 +636,7 @@ func testEvictAndPurgeSeries(t *testing.T, encoding chunkEncoding) {
// Drop everything. // Drop everything.
s.maintainMemorySeries(fp, 100000) s.maintainMemorySeries(fp, 100000)
it = s.NewIterator(fp) it = s.NewIterator(fp)
actual = it.GetBoundaryValues(metric.Interval{ actual = it.BoundaryValues(metric.Interval{
OldestInclusive: 0, OldestInclusive: 0,
NewestInclusive: 100000, NewestInclusive: 100000,
}) })
@ -1035,7 +1035,7 @@ func verifyStorage(t testing.TB, s *memorySeriesStorage, samples clientmodel.Sam
} }
p := s.NewPreloader() p := s.NewPreloader()
p.PreloadRange(fp, sample.Timestamp, sample.Timestamp, time.Hour) p.PreloadRange(fp, sample.Timestamp, sample.Timestamp, time.Hour)
found := s.NewIterator(fp).GetValueAtTime(sample.Timestamp) found := s.NewIterator(fp).ValueAtTime(sample.Timestamp)
if len(found) != 1 { if len(found) != 1 {
t.Errorf("Sample %#v: Expected exactly one value, found %d.", sample, len(found)) t.Errorf("Sample %#v: Expected exactly one value, found %d.", sample, len(found))
result = false result = false

View file

@ -156,7 +156,7 @@ func (serv MetricsService) Metrics(w http.ResponseWriter, r *http.Request) {
setAccessControlHeaders(w) setAccessControlHeaders(w)
w.Header().Set("Content-Type", "application/json") w.Header().Set("Content-Type", "application/json")
metricNames := serv.Storage.GetLabelValuesForLabelName(clientmodel.MetricNameLabel) metricNames := serv.Storage.LabelValuesForLabelName(clientmodel.MetricNameLabel)
sort.Sort(metricNames) sort.Sort(metricNames)
resultBytes, err := json.Marshal(metricNames) resultBytes, err := json.Marshal(metricNames)
if err != nil { if err != nil {