Merge pull request #84 from prometheus/refactor/storage/batching

WIP — DO NOT MERGE
This commit is contained in:
Matt T. Proud 2013-03-21 10:13:27 -07:00
commit c11737c5fc
56 changed files with 7350 additions and 2636 deletions

View file

@ -36,7 +36,7 @@ format:
find . -iname '*.go' | egrep -v "generated|\.(l|y)\.go" | xargs -n1 gofmt -w -s=true
advice:
go tool vet -v .
go tool vet .
search_index:
godoc -index -write_index -index_files='search_index'

View file

@ -41,7 +41,7 @@ preparation-stamp: build-dependencies
build-dependencies: build-dependencies-stamp
build-dependencies-stamp: bison cc mercurial protoc goprotobuf gorest go instrumentation leveldb levigo skiplist vim-common
build-dependencies-stamp: bison cc mercurial protoc goprotobuf gorest goskiplist go instrumentation leveldb levigo
touch $@
overlay: overlay-stamp

View file

@ -21,8 +21,15 @@ type ProtocolBufferEncoder struct {
message proto.Message
}
func (p *ProtocolBufferEncoder) Encode() ([]byte, error) {
return proto.Marshal(p.message)
func (p *ProtocolBufferEncoder) Encode() (raw []byte, err error) {
raw, err = proto.Marshal(p.message)
// XXX: Adjust legacy users of this to not check for error.
if err != nil {
panic(err)
}
return
}
func NewProtocolBufferEncoder(message proto.Message) *ProtocolBufferEncoder {

57
main.go
View file

@ -15,23 +15,26 @@ package main
import (
"flag"
"fmt"
"github.com/prometheus/prometheus/appstate"
"github.com/prometheus/prometheus/config"
// "github.com/prometheus/prometheus/model"
"github.com/prometheus/prometheus/retrieval"
"github.com/prometheus/prometheus/retrieval/format"
"github.com/prometheus/prometheus/rules"
"github.com/prometheus/prometheus/rules/ast"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/metric/leveldb"
"github.com/prometheus/prometheus/storage/metric/memory"
"github.com/prometheus/prometheus/web"
"log"
"os"
"os/signal"
"time"
)
// Commandline flags.
var (
_ = fmt.Sprintf("")
configFile = flag.String("configFile", "prometheus.conf", "Prometheus configuration file name.")
metricsStoragePath = flag.String("metricsStoragePath", "/tmp/metrics", "Base path for metrics storage.")
scrapeResultsQueueCapacity = flag.Int("scrapeResultsQueueCapacity", 4096, "The size of the scrape results queue.")
@ -47,25 +50,32 @@ func main() {
log.Fatalf("Error loading configuration from %s: %v", *configFile, err)
}
var persistence metric.MetricPersistence
var (
persistence metric.MetricPersistence
ts metric.Storage
)
if *memoryArena {
persistence = memory.NewMemorySeriesStorage()
persistence = metric.NewMemorySeriesStorage()
} else {
persistence, err = leveldb.NewLevelDBMetricPersistence(*metricsStoragePath)
if err != nil {
log.Fatalf("Error opening storage: %v", err)
}
ts = metric.NewTieredStorage(5000, 5000, 100, time.Second*30, time.Second*1, time.Second*20, *metricsStoragePath)
go ts.Serve()
// persistence, err = metric.NewLevelDBMetricPersistence(*metricsStoragePath)
// if err != nil {
// log.Fatalf("Error opening storage: %v", err)
// }
}
go func() {
notifier := make(chan os.Signal)
signal.Notify(notifier, os.Interrupt)
<-notifier
persistence.Close()
// persistence.Close()
os.Exit(0)
}()
defer persistence.Close()
// defer persistence.Close()
// Queue depth will need to be exposed
scrapeResults := make(chan format.Result, *scrapeResultsQueueCapacity)
@ -91,16 +101,39 @@ func main() {
web.StartServing(appState)
// go func() {
// ticker := time.Tick(time.Second)
// for i := 0; i < 120; i++ {
// <-ticker
// if i%10 == 0 {
// fmt.Printf(".")
// }
// }
// fmt.Println()
// //f := model.NewFingerprintFromRowKey("9776005627788788740-g-131-0")
// f := model.NewFingerprintFromRowKey("09923616460706181007-g-131-0")
// v := metric.NewViewRequestBuilder()
// v.GetMetricAtTime(f, time.Now().Add(-120*time.Second))
// view, err := ts.MakeView(v, time.Minute)
// fmt.Println(view, err)
// }()
for {
select {
case scrapeResult := <-scrapeResults:
if scrapeResult.Err == nil {
persistence.AppendSample(scrapeResult.Sample)
// f := model.NewFingerprintFromMetric(scrapeResult.Sample.Metric)
// fmt.Println(f)
// persistence.AppendSample(scrapeResult.Sample)
ts.AppendSample(scrapeResult.Sample)
}
case ruleResult := <-ruleResults:
for _, sample := range ruleResult.Samples {
// XXX: Wart
persistence.AppendSample(*sample)
// persistence.AppendSample(*sample)
ts.AppendSample(*sample)
}
}
}

View file

@ -14,55 +14,48 @@
package dto;
message LabelPair {
optional int64 version = 1 [default = 1];
optional string name = 2;
optional string value = 3;
optional string name = 1;
optional string value = 2;
}
message LabelName {
optional int64 version = 1 [default = 1];
optional string name = 2;
optional string name = 1;
}
message Metric {
optional int64 version = 1 [default = 1];
repeated LabelPair label_pair = 2;
repeated LabelPair label_pair = 1;
}
message Fingerprint {
optional int64 version = 1 [default = 1];
optional string signature = 2;
optional string signature = 1;
}
message FingerprintCollection {
optional int64 version = 1 [default = 1];
repeated Fingerprint member = 2;
repeated Fingerprint member = 1;
}
message LabelSet {
optional int64 version = 1 [default = 1];
repeated LabelPair member = 2;
repeated LabelPair member = 1;
}
message SampleKey {
optional int64 version = 1 [default = 1];
optional Fingerprint fingerprint = 2;
optional bytes timestamp = 3;
optional Fingerprint fingerprint = 1;
optional bytes timestamp = 2;
optional int64 last_timestamp = 3;
optional uint32 sample_count = 4;
}
message SampleValue {
optional int64 version = 1 [default = 1];
optional float value = 2;
message SampleValueSeries {
message Value {
optional int64 timestamp = 1;
optional float value = 2;
}
repeated Value value = 1;
}
message MembershipIndexValue {
optional int64 version = 1 [default = 1];
}
message MetricHighWatermark {
optional int64 timestamp = 1;
}

View file

@ -15,8 +15,6 @@ package model
import (
"code.google.com/p/goprotobuf/proto"
"crypto/md5"
"encoding/hex"
dto "github.com/prometheus/prometheus/model/generated"
"sort"
"time"
@ -49,11 +47,11 @@ func SampleToMetricDTO(s *Sample) *dto.Metric {
}
}
func MetricToDTO(m *Metric) *dto.Metric {
metricLength := len(*m)
func MetricToDTO(m Metric) *dto.Metric {
metricLength := len(m)
labelNames := make([]string, 0, metricLength)
for labelName := range *m {
for labelName := range m {
labelNames = append(labelNames, string(labelName))
}
@ -63,7 +61,7 @@ func MetricToDTO(m *Metric) *dto.Metric {
for _, labelName := range labelNames {
l := LabelName(labelName)
labelValue := (*m)[l]
labelValue := m[l]
labelPair := &dto.LabelPair{
Name: proto.String(string(labelName)),
Value: proto.String(string(labelValue)),
@ -77,12 +75,6 @@ func MetricToDTO(m *Metric) *dto.Metric {
}
}
func BytesToFingerprint(v []byte) Fingerprint {
hash := md5.New()
hash.Write(v)
return Fingerprint(hex.EncodeToString(hash.Sum([]byte{})))
}
func LabelSetToDTOs(s *LabelSet) []*dto.LabelPair {
metricLength := len(*s)
labelNames := make([]string, 0, metricLength)
@ -121,15 +113,15 @@ func LabelNameToDTO(l *LabelName) *dto.LabelName {
}
}
func FingerprintToDTO(f *Fingerprint) *dto.Fingerprint {
func FingerprintToDTO(f Fingerprint) *dto.Fingerprint {
return &dto.Fingerprint{
Signature: proto.String(string(*f)),
Signature: proto.String(f.ToRowKey()),
}
}
func SampleFromDTO(m *Metric, t *time.Time, v *dto.SampleValue) *Sample {
func SampleFromDTO(m *Metric, t *time.Time, v *dto.SampleValueSeries) *Sample {
s := &Sample{
Value: SampleValue(*v.Value),
Value: SampleValue(*v.Value[0].Value),
Timestamp: *t,
}
@ -137,9 +129,3 @@ func SampleFromDTO(m *Metric, t *time.Time, v *dto.SampleValue) *Sample {
return s
}
func (f Fingerprint) ToDTO() *dto.Fingerprint {
return &dto.Fingerprint{
Signature: proto.String(string(f)),
}
}

201
model/fingerprinting.go Normal file
View file

@ -0,0 +1,201 @@
// Copyright 2013 Prometheus Team
// 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 model
import (
"code.google.com/p/goprotobuf/proto"
"encoding/binary"
"fmt"
dto "github.com/prometheus/prometheus/model/generated"
"hash/fnv"
"sort"
"strconv"
"strings"
)
const (
// rowKeyDelimiter is used to separate formatted versions of a metric's row
// key.
rowKeyDelimiter = "-"
)
// Provides a compact representation of a Metric.
type Fingerprint interface {
// Transforms the fingerprint into a database row key.
ToRowKey() string
Hash() uint64
FirstCharacterOfFirstLabelName() string
LabelMatterLength() uint
LastCharacterOfLastLabelValue() string
ToDTO() *dto.Fingerprint
Less(Fingerprint) bool
Equal(Fingerprint) bool
}
// Builds a Fingerprint from a row key.
func NewFingerprintFromRowKey(rowKey string) (f Fingerprint) {
components := strings.Split(rowKey, rowKeyDelimiter)
hash, err := strconv.ParseUint(components[0], 10, 64)
if err != nil {
panic(err)
}
labelMatterLength, err := strconv.ParseUint(components[2], 10, 0)
if err != nil {
panic(err)
}
return fingerprint{
hash: hash,
firstCharacterOfFirstLabelName: components[1],
labelMatterLength: uint(labelMatterLength),
lastCharacterOfLastLabelValue: components[3],
}
}
// Builds a Fingerprint from a datastore entry.
func NewFingerprintFromDTO(f *dto.Fingerprint) Fingerprint {
return NewFingerprintFromRowKey(*f.Signature)
}
// Decomposes a Metric into a Fingerprint.
func NewFingerprintFromMetric(metric Metric) (f Fingerprint) {
labelLength := len(metric)
labelNames := make([]string, 0, labelLength)
for labelName := range metric {
labelNames = append(labelNames, string(labelName))
}
sort.Strings(labelNames)
summer := fnv.New64a()
firstCharacterOfFirstLabelName := ""
lastCharacterOfLastLabelValue := ""
labelMatterLength := 0
for i, labelName := range labelNames {
labelValue := metric[LabelName(labelName)]
labelNameLength := len(labelName)
labelValueLength := len(labelValue)
labelMatterLength += labelNameLength + labelValueLength
if i == 0 {
firstCharacterOfFirstLabelName = labelName[0:1]
}
if i == labelLength-1 {
lastCharacterOfLastLabelValue = string(labelValue[labelValueLength-2 : labelValueLength-1])
}
summer.Write([]byte(labelName))
summer.Write([]byte(reservedDelimiter))
summer.Write([]byte(labelValue))
}
return fingerprint{
firstCharacterOfFirstLabelName: firstCharacterOfFirstLabelName,
hash: binary.LittleEndian.Uint64(summer.Sum(nil)),
labelMatterLength: uint(labelMatterLength),
lastCharacterOfLastLabelValue: lastCharacterOfLastLabelValue,
}
}
// A simplified representation of an entity.
type fingerprint struct {
// A hashed representation of the underyling entity. For our purposes, FNV-1A
// 64-bit is used.
hash uint64
firstCharacterOfFirstLabelName string
labelMatterLength uint
lastCharacterOfLastLabelValue string
}
func (f fingerprint) ToRowKey() string {
return strings.Join([]string{fmt.Sprintf("%020d", f.hash), f.firstCharacterOfFirstLabelName, fmt.Sprint(f.labelMatterLength), f.lastCharacterOfLastLabelValue}, rowKeyDelimiter)
}
func (f fingerprint) ToDTO() *dto.Fingerprint {
return &dto.Fingerprint{
Signature: proto.String(f.ToRowKey()),
}
}
func (f fingerprint) Hash() uint64 {
return f.hash
}
func (f fingerprint) FirstCharacterOfFirstLabelName() string {
return f.firstCharacterOfFirstLabelName
}
func (f fingerprint) LabelMatterLength() uint {
return f.labelMatterLength
}
func (f fingerprint) LastCharacterOfLastLabelValue() string {
return f.lastCharacterOfLastLabelValue
}
func (f fingerprint) Less(o Fingerprint) bool {
if f.Hash() < o.Hash() {
return true
}
if f.FirstCharacterOfFirstLabelName() < o.FirstCharacterOfFirstLabelName() {
return true
}
if f.LabelMatterLength() < o.LabelMatterLength() {
return true
}
if f.LastCharacterOfLastLabelValue() < o.LastCharacterOfLastLabelValue() {
return true
}
return false
}
func (f fingerprint) Equal(o Fingerprint) (equal bool) {
equal = f.Hash() == o.Hash()
if !equal {
return
}
equal = f.FirstCharacterOfFirstLabelName() == o.FirstCharacterOfFirstLabelName()
if !equal {
return
}
equal = f.LabelMatterLength() == o.LabelMatterLength()
if !equal {
return
}
equal = f.LastCharacterOfLastLabelValue() == o.LastCharacterOfLastLabelValue()
return
}
// Represents a collection of Fingerprint subject to a given natural sorting
// scheme.
type Fingerprints []Fingerprint
func (f Fingerprints) Len() int {
return len(f)
}
func (f Fingerprints) Less(i, j int) (less bool) {
return f[i].Less(f[j])
}
func (f Fingerprints) Swap(i, j int) {
f[i], f[j] = f[j], f[i]
}

32
model/labelname.go Normal file
View file

@ -0,0 +1,32 @@
// Copyright 2013 Prometheus Team
// 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 model
// A LabelName is a key for a LabelSet or Metric. It has a value associated
// therewith.
type LabelName string
type LabelNames []LabelName
func (l LabelNames) Len() int {
return len(l)
}
func (l LabelNames) Less(i, j int) bool {
return l[i] < l[j]
}
func (l LabelNames) Swap(i, j int) {
l[i], l[j] = l[j], l[i]
}

56
model/labelname_test.go Normal file
View file

@ -0,0 +1,56 @@
// Copyright 2013 Prometheus Team
// 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 model
import (
"github.com/prometheus/prometheus/utility/test"
"sort"
"testing"
)
func testLabelNames(t test.Tester) {
var scenarios = []struct {
in LabelNames
out LabelNames
}{
{
in: LabelNames{"ZZZ", "zzz"},
out: LabelNames{"ZZZ", "zzz"},
},
{
in: LabelNames{"aaa", "AAA"},
out: LabelNames{"AAA", "aaa"},
},
}
for i, scenario := range scenarios {
sort.Sort(scenario.in)
for j, expected := range scenario.out {
if expected != scenario.in[j] {
t.Errorf("%d.%d expected %s, got %s", i, j, expected, scenario.in[j])
}
}
}
}
func TestLabelNames(t *testing.T) {
testLabelNames(t)
}
func BenchmarkLabelNames(b *testing.B) {
for i := 0; i < b.N; i++ {
testLabelNames(b)
}
}

41
model/labelpair.go Normal file
View file

@ -0,0 +1,41 @@
// Copyright 2013 Prometheus Team
// 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 model
type LabelPair struct {
Name LabelName
Value LabelValue
}
type LabelPairs []LabelPair
func (l LabelPairs) Len() int {
return len(l)
}
func (l LabelPairs) Less(i, j int) (less bool) {
if l[i].Name < l[j].Name {
return true
}
if l[i].Value < l[j].Value {
return true
}
return false
}
func (l LabelPairs) Swap(i, j int) {
l[i], l[j] = l[j], l[i]
}

84
model/labelpair_test.go Normal file
View file

@ -0,0 +1,84 @@
// Copyright 2013 Prometheus Team
// 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 model
import (
"github.com/prometheus/prometheus/utility/test"
"sort"
"testing"
)
func testLabelPairs(t test.Tester) {
var scenarios = []struct {
in LabelPairs
out LabelPairs
}{
{
in: LabelPairs{
{
Name: "AAA",
Value: "aaa",
},
},
out: LabelPairs{
{
Name: "AAA",
Value: "aaa",
},
},
},
{
in: LabelPairs{
{
Name: "aaa",
Value: "aaa",
},
{
Name: "ZZZ",
Value: "aaa",
},
},
out: LabelPairs{
{
Name: "ZZZ",
Value: "aaa",
},
{
Name: "aaa",
Value: "aaa",
},
},
},
}
for i, scenario := range scenarios {
sort.Sort(scenario.in)
for j, expected := range scenario.out {
if expected != scenario.in[j] {
t.Errorf("%d.%d expected %s, got %s", i, j, expected, scenario.in[j])
}
}
}
}
func TestLabelPairs(t *testing.T) {
testLabelPairs(t)
}
func BenchmarkLabelPairs(b *testing.B) {
for i := 0; i < b.N; i++ {
testLabelPairs(b)
}
}

View file

@ -11,17 +11,25 @@
// See the License for the specific language governing permissions and
// limitations under the License.
package leveldb
package model
import (
index "github.com/prometheus/prometheus/storage/raw/index/leveldb"
storage "github.com/prometheus/prometheus/storage/raw/leveldb"
"sort"
)
type LevelDBMetricPersistence struct {
fingerprintToMetrics *storage.LevelDBPersistence
metricSamples *storage.LevelDBPersistence
labelNameToFingerprints *storage.LevelDBPersistence
labelSetToFingerprints *storage.LevelDBPersistence
metricMembershipIndex *index.LevelDBMembershipIndex
// A LabelValue is an associated value for a LabelName.
type LabelValue string
type LabelValues []LabelValue
func (l LabelValues) Len() int {
return len(l)
}
func (l LabelValues) Less(i, j int) bool {
return sort.StringsAreSorted([]string{string(l[i]), string(l[j])})
}
func (l LabelValues) Swap(i, j int) {
l[i], l[j] = l[j], l[i]
}

56
model/labelvalue_test.go Normal file
View file

@ -0,0 +1,56 @@
// Copyright 2013 Prometheus Team
// 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 model
import (
"github.com/prometheus/prometheus/utility/test"
"sort"
"testing"
)
func testLabelValues(t test.Tester) {
var scenarios = []struct {
in LabelValues
out LabelValues
}{
{
in: LabelValues{"ZZZ", "zzz"},
out: LabelValues{"ZZZ", "zzz"},
},
{
in: LabelValues{"aaa", "AAA"},
out: LabelValues{"AAA", "aaa"},
},
}
for i, scenario := range scenarios {
sort.Sort(scenario.in)
for j, expected := range scenario.out {
if expected != scenario.in[j] {
t.Errorf("%d.%d expected %s, got %s", i, j, expected, scenario.in[j])
}
}
}
}
func TestLabelValues(t *testing.T) {
testLabelValues(t)
}
func BenchmarkLabelValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testLabelValues(b)
}
}

View file

@ -15,8 +15,6 @@ package model
import (
"bytes"
"crypto/md5"
"encoding/hex"
"fmt"
"sort"
"time"
@ -27,17 +25,6 @@ const (
reservedDelimiter = `"`
)
// A Fingerprint is a simplified representation of an entity---e.g., a hash of
// an entire Metric.
type Fingerprint string
// A LabelName is a key for a LabelSet or Metric. It has a value associated
// therewith.
type LabelName string
// A LabelValue is an associated value for a LabelName.
type LabelValue string
// A LabelSet is a collection of LabelName and LabelValue pairs. The LabelSet
// may be fully-qualified down to the point where it may resolve to a single
// Metric in the data store or not. All operations that occur within the realm
@ -78,60 +65,10 @@ func (l LabelSet) String() string {
return buffer.String()
}
type LabelNames []LabelName
func (l LabelNames) Len() int {
return len(l)
}
func (l LabelNames) Less(i, j int) bool {
return l[i] < l[j]
}
func (l LabelNames) Swap(i, j int) {
l[i], l[j] = l[j], l[i]
}
// A Metric is similar to a LabelSet, but the key difference is that a Metric is
// a singleton and refers to one and only one stream of samples.
type Metric map[LabelName]LabelValue
type Fingerprints []Fingerprint
func (f Fingerprints) Len() int {
return len(f)
}
func (f Fingerprints) Less(i, j int) bool {
return sort.StringsAreSorted([]string{string(f[i]), string(f[j])})
}
func (f Fingerprints) Swap(i, j int) {
f[i], f[j] = f[j], f[i]
}
// Fingerprint generates a fingerprint for this given Metric.
func (m Metric) Fingerprint() Fingerprint {
labelLength := len(m)
labelNames := make([]string, 0, labelLength)
for labelName := range m {
labelNames = append(labelNames, string(labelName))
}
sort.Strings(labelNames)
summer := md5.New()
for _, labelName := range labelNames {
summer.Write([]byte(labelName))
summer.Write([]byte(reservedDelimiter))
summer.Write([]byte(m[LabelName(labelName)]))
}
return Fingerprint(hex.EncodeToString(summer.Sum(nil)))
}
// A SampleValue is a representation of a value for a given sample at a given
// time. It is presently float32 due to that being the representation that
// Protocol Buffers provide of floats in Go. This is a smell and should be
@ -146,12 +83,6 @@ func (s SamplePair) MarshalJSON() ([]byte, error) {
return []byte(fmt.Sprintf("{\"Value\": \"%f\", \"Timestamp\": %d}", s.Value, s.Timestamp.Unix())), nil
}
type Sample struct {
Metric Metric
Value SampleValue
Timestamp time.Time
}
type SamplePair struct {
Value SampleValue
Timestamp time.Time
@ -171,6 +102,24 @@ func (v Values) Swap(i, j int) {
v[i], v[j] = v[j], v[i]
}
// InsideInterval indicates whether a given range of sorted values could contain
// a value for a given time.
func (v Values) InsideInterval(t time.Time) (s bool) {
if v.Len() == 0 {
return
}
if t.Before(v[0].Timestamp) {
return
}
if !v[v.Len()-1].Timestamp.Before(t) {
return
}
return true
}
type SampleSet struct {
Metric Metric
Values Values
@ -180,10 +129,3 @@ type Interval struct {
OldestInclusive time.Time
NewestInclusive time.Time
}
// PENDING DELETION BELOW THIS LINE
type Samples struct {
Value SampleValue
Timestamp time.Time
}

View file

@ -21,11 +21,13 @@ import (
func testMetric(t test.Tester) {
var scenarios = []struct {
input map[string]string
output Fingerprint
hash uint64
rowkey string
}{
{
input: map[string]string{},
output: "d41d8cd98f00b204e9800998ecf8427e",
rowkey: "02676020557754725067--0-",
hash: 2676020557754725067,
},
{
input: map[string]string{
@ -33,7 +35,8 @@ func testMetric(t test.Tester) {
"occupation": "robot",
"manufacturer": "westinghouse",
},
output: "18596f03fce001153495d903b8b577c0",
rowkey: "04776841610193542734-f-56-o",
hash: 4776841610193542734,
},
}
@ -43,11 +46,17 @@ func testMetric(t test.Tester) {
metric[LabelName(key)] = LabelValue(value)
}
expected := scenario.output
actual := metric.Fingerprint()
expectedRowKey := scenario.rowkey
expectedHash := scenario.hash
fingerprint := NewFingerprintFromMetric(metric)
actualRowKey := fingerprint.ToRowKey()
actualHash := fingerprint.Hash()
if expected != actual {
t.Errorf("%d. expected %s, got %s", i, expected, actual)
if expectedRowKey != actualRowKey {
t.Errorf("%d. expected %s, got %s", i, expectedRowKey, actualRowKey)
}
if actualHash != expectedHash {
t.Errorf("%d. expected %d, got %d", i, expectedHash, actualHash)
}
}
}

View file

@ -11,21 +11,36 @@
// See the License for the specific language governing permissions and
// limitations under the License.
package memory
package model
import (
"github.com/prometheus/prometheus/storage/metric"
"testing"
"time"
)
var testGetFingerprintsForLabelSetUsesAndForLabelMatching = buildTestPersistence(metric.GetFingerprintsForLabelSetUsesAndForLabelMatchingTests)
func TestGetFingerprintsForLabelSetUsesAndForLabelMatching(t *testing.T) {
testGetFingerprintsForLabelSetUsesAndForLabelMatching(t)
type Sample struct {
Metric Metric
Value SampleValue
Timestamp time.Time
}
func BenchmarkGetFingerprintsForLabelSetUsesAndLabelMatching(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetFingerprintsForLabelSetUsesAndForLabelMatching(b)
type Samples []Sample
func (s Samples) Len() int {
return len(s)
}
func (s Samples) Less(i, j int) (less bool) {
if NewFingerprintFromMetric(s[i].Metric).Less(NewFingerprintFromMetric(s[j].Metric)) {
return true
}
if s[i].Timestamp.Before(s[j].Timestamp) {
return true
}
return false
}
func (s Samples) Swap(i, j int) {
s[i], s[j] = s[j], s[i]
}

View file

@ -30,55 +30,36 @@ type PersistenceAdapter struct {
// AST-global persistence to use.
var persistenceAdapter *PersistenceAdapter = nil
func (p *PersistenceAdapter) getMetricsWithLabels(labels model.LabelSet) (metrics []model.Metric, err error) {
func (p *PersistenceAdapter) GetValueAtTime(labels model.LabelSet, timestamp *time.Time) (samples []*model.Sample, err error) {
fingerprints, err := p.persistence.GetFingerprintsForLabelSet(labels)
if err != nil {
return
}
for _, fingerprint := range fingerprints {
metric, err := p.persistence.GetMetricForFingerprint(fingerprint)
var sample *model.Sample // Don't shadow err.
sample, err = p.persistence.GetValueAtTime(fingerprint, *timestamp, *p.stalenessPolicy)
if err != nil {
return metrics, err
}
if metric == nil {
continue
}
metrics = append(metrics, *metric)
}
return
}
func (p *PersistenceAdapter) GetValueAtTime(labels model.LabelSet, timestamp *time.Time) ([]*model.Sample, error) {
metrics, err := p.getMetricsWithLabels(labels)
if err != nil {
return nil, err
}
samples := []*model.Sample{}
for _, metric := range metrics {
sample, err := p.persistence.GetValueAtTime(metric, *timestamp, *p.stalenessPolicy)
if err != nil {
return nil, err
return
}
if sample == nil {
continue
}
samples = append(samples, sample)
}
return samples, nil
return
}
func (p *PersistenceAdapter) GetBoundaryValues(labels model.LabelSet, interval *model.Interval) ([]*model.SampleSet, error) {
metrics, err := p.getMetricsWithLabels(labels)
func (p *PersistenceAdapter) GetBoundaryValues(labels model.LabelSet, interval *model.Interval) (sampleSets []*model.SampleSet, err error) {
fingerprints, err := p.persistence.GetFingerprintsForLabelSet(labels)
if err != nil {
return nil, err
return
}
sampleSets := []*model.SampleSet{}
for _, metric := range metrics {
for _, fingerprint := range fingerprints {
var sampleSet *model.SampleSet // Don't shadow err.
// TODO: change to GetBoundaryValues() once it has the right return type.
sampleSet, err := p.persistence.GetRangeValues(metric, *interval)
sampleSet, err = p.persistence.GetRangeValues(fingerprint, *interval)
if err != nil {
return nil, err
}
@ -86,22 +67,20 @@ func (p *PersistenceAdapter) GetBoundaryValues(labels model.LabelSet, interval *
continue
}
// TODO remove when persistence return value is fixed.
sampleSet.Metric = metric
sampleSets = append(sampleSets, sampleSet)
}
return sampleSets, nil
}
func (p *PersistenceAdapter) GetRangeValues(labels model.LabelSet, interval *model.Interval) ([]*model.SampleSet, error) {
metrics, err := p.getMetricsWithLabels(labels)
func (p *PersistenceAdapter) GetRangeValues(labels model.LabelSet, interval *model.Interval) (sampleSets []*model.SampleSet, err error) {
fingerprints, err := p.persistence.GetFingerprintsForLabelSet(labels)
if err != nil {
return nil, err
return
}
sampleSets := []*model.SampleSet{}
for _, metric := range metrics {
sampleSet, err := p.persistence.GetRangeValues(metric, *interval)
for _, fingerprint := range fingerprints {
var sampleSet *model.SampleSet // Don't shadow err.
sampleSet, err = p.persistence.GetRangeValues(fingerprint, *interval)
if err != nil {
return nil, err
}
@ -109,8 +88,6 @@ func (p *PersistenceAdapter) GetRangeValues(labels model.LabelSet, interval *mod
continue
}
// TODO remove when persistence return value is fixed.
sampleSet.Metric = metric
sampleSets = append(sampleSets, sampleSet)
}
return sampleSets, nil

View file

@ -66,7 +66,7 @@ func exprTypeToString(exprType ExprType) string {
return exprTypeMap[exprType]
}
func (vector Vector) ToString() string {
func (vector Vector) String() string {
metricStrings := []string{}
for _, sample := range vector {
metricName, ok := sample.Metric["name"]
@ -91,7 +91,7 @@ func (vector Vector) ToString() string {
return strings.Join(metricStrings, "\n")
}
func (matrix Matrix) ToString() string {
func (matrix Matrix) String() string {
metricStrings := []string{}
for _, sampleSet := range matrix {
metricName, ok := sampleSet.Metric["name"]
@ -165,7 +165,7 @@ func EvalToString(node Node, timestamp *time.Time, format OutputFormat) string {
vector := node.(VectorNode).Eval(timestamp)
switch format {
case TEXT:
return vector.ToString()
return vector.String()
case JSON:
return TypedValueToJSON(vector, "vector")
}
@ -173,7 +173,7 @@ func EvalToString(node Node, timestamp *time.Time, format OutputFormat) string {
matrix := node.(MatrixNode).Eval(timestamp)
switch format {
case TEXT:
return matrix.ToString()
return matrix.String()
case JSON:
return TypedValueToJSON(matrix, "matrix")
}
@ -189,7 +189,7 @@ func EvalToString(node Node, timestamp *time.Time, format OutputFormat) string {
panic("Switch didn't cover all node types")
}
func (node *VectorLiteral) ToString() string {
func (node *VectorLiteral) String() string {
metricName, ok := node.labels["name"]
if !ok {
panic("Tried to print vector without metric name")
@ -204,8 +204,8 @@ func (node *VectorLiteral) ToString() string {
return fmt.Sprintf("%v{%v}", metricName, strings.Join(labelStrings, ","))
}
func (node *MatrixLiteral) ToString() string {
vectorString := (&VectorLiteral{labels: node.labels}).ToString()
func (node *MatrixLiteral) String() string {
vectorString := (&VectorLiteral{labels: node.labels}).String()
intervalString := fmt.Sprintf("['%v']", utility.DurationToString(node.interval))
return vectorString + intervalString
}
@ -241,7 +241,7 @@ func (node *ScalarArithExpr) NodeTreeToDotGraph() string {
}
func (node *VectorLiteral) NodeTreeToDotGraph() string {
return fmt.Sprintf("%#p[label=\"%v\"];\n", node, node.ToString())
return fmt.Sprintf("%#p[label=\"%v\"];\n", node, node.String())
}
func (node *VectorFunctionCall) NodeTreeToDotGraph() string {
@ -275,11 +275,11 @@ func (node *VectorArithExpr) NodeTreeToDotGraph() string {
}
func (node *MatrixLiteral) NodeTreeToDotGraph() string {
return fmt.Sprintf("%#p[label=\"%v\"];\n", node, node.ToString())
return fmt.Sprintf("%#p[label=\"%v\"];\n", node, node.String())
}
func (node *StringLiteral) NodeTreeToDotGraph() string {
return fmt.Sprintf("%#p[label=\"'%v'\"];\n", node.str)
return fmt.Sprintf("%#p[label=\"'%v'\"];\n", node, node.str)
}
func (node *StringFunctionCall) NodeTreeToDotGraph() string {

View file

@ -16,7 +16,7 @@ package rules
import (
"fmt"
"github.com/prometheus/prometheus/rules/ast"
"github.com/prometheus/prometheus/storage/metric/leveldb"
"github.com/prometheus/prometheus/storage/metric"
"io/ioutil"
"os"
"strings"
@ -172,7 +172,7 @@ func TestExpressions(t *testing.T) {
t.Errorf("Could not remove temporary directory: %q\n", err)
}
}()
persistence, err := leveldb.NewLevelDBMetricPersistence(temporaryDirectory)
persistence, err := metric.NewLevelDBMetricPersistence(temporaryDirectory)
if err != nil {
t.Errorf("Could not create LevelDB Metric Persistence: %q\n", err)
return

View file

@ -0,0 +1,462 @@
// Copyright 2013 Prometheus Team
// 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 metric
import (
"github.com/prometheus/prometheus/model"
"github.com/prometheus/prometheus/utility/test"
"testing"
"time"
)
func GetFingerprintsForLabelSetTests(p MetricPersistence, t test.Tester) {
testAppendSample(p, model.Sample{
Value: 0,
Timestamp: time.Time{},
Metric: model.Metric{
"name": "my_metric",
"request_type": "your_mom",
},
}, t)
testAppendSample(p, model.Sample{
Value: 0,
Timestamp: time.Time{},
Metric: model.Metric{
"name": "my_metric",
"request_type": "your_dad",
},
}, t)
result, err := p.GetFingerprintsForLabelSet(model.LabelSet{
model.LabelName("name"): model.LabelValue("my_metric"),
})
if err != nil {
t.Error(err)
}
if len(result) != 2 {
t.Errorf("Expected two elements.")
}
result, err = p.GetFingerprintsForLabelSet(model.LabelSet{
model.LabelName("request_type"): model.LabelValue("your_mom"),
})
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
result, err = p.GetFingerprintsForLabelSet(model.LabelSet{
model.LabelName("request_type"): model.LabelValue("your_dad"),
})
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
}
func GetFingerprintsForLabelNameTests(p MetricPersistence, t test.Tester) {
testAppendSample(p, model.Sample{
Value: 0,
Timestamp: time.Time{},
Metric: model.Metric{
"name": "my_metric",
"request_type": "your_mom",
"language": "english",
},
}, t)
testAppendSample(p, model.Sample{
Value: 0,
Timestamp: time.Time{},
Metric: model.Metric{
"name": "my_metric",
"request_type": "your_dad",
"sprache": "deutsch",
},
}, t)
b := model.LabelName("name")
result, err := p.GetFingerprintsForLabelName(b)
if err != nil {
t.Error(err)
}
if len(result) != 2 {
t.Errorf("Expected two elements.")
}
b = model.LabelName("request_type")
result, err = p.GetFingerprintsForLabelName(b)
if err != nil {
t.Error(err)
}
if len(result) != 2 {
t.Errorf("Expected two elements.")
}
b = model.LabelName("language")
result, err = p.GetFingerprintsForLabelName(b)
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
b = model.LabelName("sprache")
result, err = p.GetFingerprintsForLabelName(b)
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
}
func GetMetricForFingerprintTests(p MetricPersistence, t test.Tester) {
testAppendSample(p, model.Sample{
Value: 0,
Timestamp: time.Time{},
Metric: model.Metric{
"request_type": "your_mom",
},
}, t)
testAppendSample(p, model.Sample{
Value: 0,
Timestamp: time.Time{},
Metric: model.Metric{
"request_type": "your_dad",
"one-off": "value",
},
}, t)
result, err := p.GetFingerprintsForLabelSet(model.LabelSet{
model.LabelName("request_type"): model.LabelValue("your_mom"),
})
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
v, e := p.GetMetricForFingerprint(result[0])
if e != nil {
t.Error(e)
}
if v == nil {
t.Fatal("Did not expect nil.")
}
metric := *v
if len(metric) != 1 {
t.Errorf("Expected one-dimensional metric.")
}
if metric["request_type"] != "your_mom" {
t.Errorf("Expected metric to match.")
}
result, err = p.GetFingerprintsForLabelSet(model.LabelSet{
model.LabelName("request_type"): model.LabelValue("your_dad"),
})
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
v, e = p.GetMetricForFingerprint(result[0])
if v == nil {
t.Fatal("Did not expect nil.")
}
metric = *v
if e != nil {
t.Error(e)
}
if len(metric) != 2 {
t.Errorf("Expected one-dimensional metric.")
}
if metric["request_type"] != "your_dad" {
t.Errorf("Expected metric to match.")
}
if metric["one-off"] != "value" {
t.Errorf("Expected metric to match.")
}
}
func AppendRepeatingValuesTests(p MetricPersistence, t test.Tester) {
metric := model.Metric{
"controller": "foo",
"name": "errors_total",
"operation": "bar",
}
increments := 10
repetitions := 500
for i := 0; i < increments; i++ {
for j := 0; j < repetitions; j++ {
time := time.Time{}.Add(time.Duration(i) * time.Hour).Add(time.Duration(j) * time.Second)
testAppendSample(p, model.Sample{
Value: model.SampleValue(i),
Timestamp: time,
Metric: metric,
}, t)
}
}
if true {
// XXX: Purely a benchmark.
return
}
labelSet := model.LabelSet{
"controller": "foo",
"name": "errors_total",
"operation": "bar",
}
for i := 0; i < increments; i++ {
for j := 0; j < repetitions; j++ {
fingerprints, err := p.GetFingerprintsForLabelSet(labelSet)
if err != nil {
t.Fatal(err)
}
if len(fingerprints) != 1 {
t.Fatalf("expected %d fingerprints, got %d", 1, len(fingerprints))
}
time := time.Time{}.Add(time.Duration(i) * time.Hour).Add(time.Duration(j) * time.Second)
sample, err := p.GetValueAtTime(fingerprints[0], time, StalenessPolicy{})
if err != nil {
t.Fatal(err)
}
if sample == nil {
t.Fatal("expected non-nil sample.")
}
expected := model.SampleValue(i)
if sample.Value != expected {
t.Fatalf("expected %d value, got %d", expected, sample.Value)
}
}
}
}
func AppendsRepeatingValuesTests(p MetricPersistence, t test.Tester) {
metric := model.Metric{
"controller": "foo",
"name": "errors_total",
"operation": "bar",
}
increments := 10
repetitions := 500
s := model.Samples{}
for i := 0; i < increments; i++ {
for j := 0; j < repetitions; j++ {
time := time.Time{}.Add(time.Duration(i) * time.Hour).Add(time.Duration(j) * time.Second)
s = append(s, model.Sample{
Value: model.SampleValue(i),
Timestamp: time,
Metric: metric,
})
}
}
p.AppendSamples(s)
if true {
// XXX: Purely a benchmark.
return
}
labelSet := model.LabelSet{
"controller": "foo",
"name": "errors_total",
"operation": "bar",
}
for i := 0; i < increments; i++ {
for j := 0; j < repetitions; j++ {
fingerprints, err := p.GetFingerprintsForLabelSet(labelSet)
if err != nil {
t.Fatal(err)
}
if len(fingerprints) != 1 {
t.Fatalf("expected %d fingerprints, got %d", 1, len(fingerprints))
}
time := time.Time{}.Add(time.Duration(i) * time.Hour).Add(time.Duration(j) * time.Second)
sample, err := p.GetValueAtTime(fingerprints[0], time, StalenessPolicy{})
if err != nil {
t.Fatal(err)
}
if sample == nil {
t.Fatal("expected non-nil sample.")
}
expected := model.SampleValue(i)
if sample.Value != expected {
t.Fatalf("expected %d value, got %d", expected, sample.Value)
}
}
}
}
// Test Definitions Below
var testLevelDBGetFingerprintsForLabelSet = buildLevelDBTestPersistence("get_fingerprints_for_labelset", GetFingerprintsForLabelSetTests)
func TestLevelDBGetFingerprintsForLabelSet(t *testing.T) {
testLevelDBGetFingerprintsForLabelSet(t)
}
func BenchmarkLevelDBGetFingerprintsForLabelSet(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBGetFingerprintsForLabelSet(b)
}
}
var testLevelDBGetFingerprintsForLabelName = buildLevelDBTestPersistence("get_fingerprints_for_labelname", GetFingerprintsForLabelNameTests)
func TestLevelDBGetFingerprintsForLabelName(t *testing.T) {
testLevelDBGetFingerprintsForLabelName(t)
}
func BenchmarkLevelDBGetFingerprintsForLabelName(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBGetFingerprintsForLabelName(b)
}
}
var testLevelDBGetMetricForFingerprint = buildLevelDBTestPersistence("get_metric_for_fingerprint", GetMetricForFingerprintTests)
func TestLevelDBGetMetricForFingerprint(t *testing.T) {
testLevelDBGetMetricForFingerprint(t)
}
func BenchmarkLevelDBGetMetricForFingerprint(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBGetMetricForFingerprint(b)
}
}
var testLevelDBAppendRepeatingValues = buildLevelDBTestPersistence("append_repeating_values", AppendRepeatingValuesTests)
func TestLevelDBAppendRepeatingValues(t *testing.T) {
testLevelDBAppendRepeatingValues(t)
}
func BenchmarkLevelDBAppendRepeatingValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBAppendRepeatingValues(b)
}
}
var testLevelDBAppendsRepeatingValues = buildLevelDBTestPersistence("appends_repeating_values", AppendsRepeatingValuesTests)
func TestLevelDBAppendsRepeatingValues(t *testing.T) {
testLevelDBAppendsRepeatingValues(t)
}
func BenchmarkLevelDBAppendsRepeatingValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBAppendsRepeatingValues(b)
}
}
var testMemoryGetFingerprintsForLabelSet = buildMemoryTestPersistence(GetFingerprintsForLabelSetTests)
func TestMemoryGetFingerprintsForLabelSet(t *testing.T) {
testMemoryGetFingerprintsForLabelSet(t)
}
func BenchmarkMemoryGetFingerprintsForLabelSet(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryGetFingerprintsForLabelSet(b)
}
}
var testMemoryGetFingerprintsForLabelName = buildMemoryTestPersistence(GetFingerprintsForLabelNameTests)
func TestMemoryGetFingerprintsForLabelName(t *testing.T) {
testMemoryGetFingerprintsForLabelName(t)
}
func BenchmarkMemoryGetFingerprintsForLabelName(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryGetFingerprintsForLabelName(b)
}
}
var testMemoryGetMetricForFingerprint = buildMemoryTestPersistence(GetMetricForFingerprintTests)
func TestMemoryGetMetricForFingerprint(t *testing.T) {
testMemoryGetMetricForFingerprint(t)
}
func BenchmarkMemoryGetMetricForFingerprint(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryGetMetricForFingerprint(b)
}
}
var testMemoryAppendRepeatingValues = buildMemoryTestPersistence(AppendRepeatingValuesTests)
func TestMemoryAppendRepeatingValues(t *testing.T) {
testMemoryAppendRepeatingValues(t)
}
func BenchmarkMemoryAppendRepeatingValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryAppendRepeatingValues(b)
}
}

View file

@ -1,228 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 metric
import (
"github.com/prometheus/prometheus/model"
"github.com/prometheus/prometheus/utility/test"
"time"
)
func GetFingerprintsForLabelSetTests(p MetricPersistence, t test.Tester) {
appendSample(p, model.Sample{
Value: 0,
Timestamp: time.Time{},
Metric: model.Metric{
"name": "my_metric",
"request_type": "your_mom",
},
}, t)
appendSample(p, model.Sample{
Value: 0,
Timestamp: time.Time{},
Metric: model.Metric{
"name": "my_metric",
"request_type": "your_dad",
},
}, t)
result, err := p.GetFingerprintsForLabelSet(model.LabelSet{
model.LabelName("name"): model.LabelValue("my_metric"),
})
if err != nil {
t.Error(err)
}
if len(result) != 2 {
t.Errorf("Expected two elements.")
}
result, err = p.GetFingerprintsForLabelSet(model.LabelSet{
model.LabelName("request_type"): model.LabelValue("your_mom"),
})
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
result, err = p.GetFingerprintsForLabelSet(model.LabelSet{
model.LabelName("request_type"): model.LabelValue("your_dad"),
})
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
}
func GetFingerprintsForLabelNameTests(p MetricPersistence, t test.Tester) {
appendSample(p, model.Sample{
Value: 0,
Timestamp: time.Time{},
Metric: model.Metric{
"name": "my_metric",
"request_type": "your_mom",
"language": "english",
},
}, t)
appendSample(p, model.Sample{
Value: 0,
Timestamp: time.Time{},
Metric: model.Metric{
"name": "my_metric",
"request_type": "your_dad",
"sprache": "deutsch",
},
}, t)
b := model.LabelName("name")
result, err := p.GetFingerprintsForLabelName(b)
if err != nil {
t.Error(err)
}
if len(result) != 2 {
t.Errorf("Expected two elements.")
}
b = model.LabelName("request_type")
result, err = p.GetFingerprintsForLabelName(b)
if err != nil {
t.Error(err)
}
if len(result) != 2 {
t.Errorf("Expected two elements.")
}
b = model.LabelName("language")
result, err = p.GetFingerprintsForLabelName(b)
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
b = model.LabelName("sprache")
result, err = p.GetFingerprintsForLabelName(b)
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
}
func GetMetricForFingerprintTests(p MetricPersistence, t test.Tester) {
appendSample(p, model.Sample{
Value: 0,
Timestamp: time.Time{},
Metric: model.Metric{
"request_type": "your_mom",
},
}, t)
appendSample(p, model.Sample{
Value: 0,
Timestamp: time.Time{},
Metric: model.Metric{
"request_type": "your_dad",
"one-off": "value",
},
}, t)
result, err := p.GetFingerprintsForLabelSet(model.LabelSet{
model.LabelName("request_type"): model.LabelValue("your_mom"),
})
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
v, e := p.GetMetricForFingerprint(result[0])
if e != nil {
t.Error(e)
}
if v == nil {
t.Fatal("Did not expect nil.")
}
metric := *v
if len(metric) != 1 {
t.Errorf("Expected one-dimensional metric.")
}
if metric["request_type"] != "your_mom" {
t.Errorf("Expected metric to match.")
}
result, err = p.GetFingerprintsForLabelSet(model.LabelSet{
model.LabelName("request_type"): model.LabelValue("your_dad"),
})
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
v, e = p.GetMetricForFingerprint(result[0])
if v == nil {
t.Fatal("Did not expect nil.")
}
metric = *v
if e != nil {
t.Error(e)
}
if len(metric) != 2 {
t.Errorf("Expected one-dimensional metric.")
}
if metric["request_type"] != "your_dad" {
t.Errorf("Expected metric to match.")
}
if metric["one-off"] != "value" {
t.Errorf("Expected metric to match.")
}
}

170
storage/metric/frontier.go Normal file
View file

@ -0,0 +1,170 @@
// Copyright 2013 Prometheus Team
// 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 metric
import (
"fmt"
"github.com/prometheus/prometheus/coding"
"github.com/prometheus/prometheus/coding/indexable"
"github.com/prometheus/prometheus/model"
dto "github.com/prometheus/prometheus/model/generated"
"time"
)
// diskFrontier describes an on-disk store of series to provide a
// representation of the known keyspace and time series values available.
//
// This is used to reduce the burden associated with LevelDB iterator
// management.
type diskFrontier struct {
firstFingerprint model.Fingerprint
firstSupertime time.Time
lastFingerprint model.Fingerprint
lastSupertime time.Time
}
func (f *diskFrontier) String() string {
return fmt.Sprintf("diskFrontier from %s at %s to %s at %s", f.firstFingerprint.ToRowKey(), f.firstSupertime, f.lastFingerprint.ToRowKey(), f.lastSupertime)
}
func (f *diskFrontier) ContainsFingerprint(fingerprint model.Fingerprint) bool {
return !(fingerprint.Less(f.firstFingerprint) || f.lastFingerprint.Less(fingerprint))
}
func newDiskFrontier(i iterator) (d *diskFrontier, err error) {
i.SeekToLast()
if !i.Valid() || i.Key() == nil {
return
}
lastKey, err := extractSampleKey(i)
if err != nil {
panic(err)
}
i.SeekToFirst()
firstKey, err := extractSampleKey(i)
if i.Key() == nil {
return
}
if err != nil {
panic(err)
}
d = &diskFrontier{}
d.firstFingerprint = model.NewFingerprintFromRowKey(*firstKey.Fingerprint.Signature)
d.firstSupertime = indexable.DecodeTime(firstKey.Timestamp)
d.lastFingerprint = model.NewFingerprintFromRowKey(*lastKey.Fingerprint.Signature)
d.lastSupertime = indexable.DecodeTime(lastKey.Timestamp)
return
}
// seriesFrontier represents the valid seek frontier for a given series.
type seriesFrontier struct {
firstSupertime time.Time
lastSupertime time.Time
lastTime time.Time
}
func (f seriesFrontier) String() string {
return fmt.Sprintf("seriesFrontier from %s to %s at %s", f.firstSupertime, f.lastSupertime, f.lastTime)
}
// newSeriesFrontier furnishes a populated diskFrontier for a given
// fingerprint. A nil diskFrontier will be returned if the series cannot
// be found in the store.
func newSeriesFrontier(f model.Fingerprint, d diskFrontier, i iterator) (s *seriesFrontier, err error) {
var (
lowerSeek = firstSupertime
upperSeek = lastSupertime
)
// If we are either the first or the last key in the database, we need to use
// pessimistic boundary frontiers.
if f.Equal(d.firstFingerprint) {
lowerSeek = indexable.EncodeTime(d.firstSupertime)
}
if f.Equal(d.lastFingerprint) {
upperSeek = indexable.EncodeTime(d.lastSupertime)
}
key := &dto.SampleKey{
Fingerprint: f.ToDTO(),
Timestamp: upperSeek,
}
raw, err := coding.NewProtocolBufferEncoder(key).Encode()
if err != nil {
panic(err)
}
i.Seek(raw)
if i.Key() == nil {
return
}
retrievedKey, err := extractSampleKey(i)
if err != nil {
panic(err)
}
retrievedFingerprint := model.NewFingerprintFromRowKey(*retrievedKey.Fingerprint.Signature)
// The returned fingerprint may not match if the original seek key lives
// outside of a metric's frontier. This is probable, for we are seeking to
// to the maximum allowed time, which could advance us to the next
// fingerprint.
//
//
if !retrievedFingerprint.Equal(f) {
i.Prev()
retrievedKey, err = extractSampleKey(i)
if err != nil {
panic(err)
}
retrievedFingerprint := model.NewFingerprintFromRowKey(*retrievedKey.Fingerprint.Signature)
// If the previous key does not match, we know that the requested
// fingerprint does not live in the database.
if !retrievedFingerprint.Equal(f) {
return
}
}
s = &seriesFrontier{
lastSupertime: indexable.DecodeTime(retrievedKey.Timestamp),
lastTime: time.Unix(*retrievedKey.LastTimestamp, 0),
}
key.Timestamp = lowerSeek
raw, err = coding.NewProtocolBufferEncoder(key).Encode()
if err != nil {
panic(err)
}
i.Seek(raw)
retrievedKey, err = extractSampleKey(i)
if err != nil {
panic(err)
}
retrievedFingerprint = model.NewFingerprintFromRowKey(*retrievedKey.Fingerprint.Signature)
s.firstSupertime = indexable.DecodeTime(retrievedKey.Timestamp)
return
}

View file

@ -11,7 +11,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
package leveldb
package metric
import (
"github.com/prometheus/client_golang"
@ -30,6 +30,9 @@ const (
appendLabelNameFingerprint = "append_label_name_fingerprint"
appendLabelPairFingerprint = "append_label_pair_fingerprint"
appendSample = "append_sample"
appendSamples = "append_samples"
findUnindexedMetrics = "find_unindexed_metrics"
flushMemory = "flush_memory"
getBoundaryValues = "get_boundary_values"
getFingerprintsForLabelName = "get_fingerprints_for_label_name"
getFingerprintsForLabelSet = "get_fingerprints_for_labelset"
@ -40,9 +43,17 @@ const (
hasIndexMetric = "has_index_metric"
hasLabelName = "has_label_name"
hasLabelPair = "has_label_pair"
indexFingerprints = "index_fingerprints"
indexLabelNames = "index_label_names"
indexLabelPairs = "index_label_pairs"
indexMetric = "index_metric"
indexMetrics = "index_metrics"
rebuildDiskFrontier = "rebuild_disk_frontier"
refreshHighWatermarks = "refresh_high_watermarks"
renderView = "render_view"
setLabelNameFingerprints = "set_label_name_fingerprints"
setLabelPairFingerprints = "set_label_pair_fingerprints"
writeMemory = "write_memory"
)
var (
@ -52,21 +63,27 @@ var (
ReportablePercentiles: []float64{0.01, 0.05, 0.5, 0.90, 0.99},
}
storageOperations = metrics.NewCounter()
storageLatency = metrics.NewHistogram(diskLatencyHistogram)
storageOperations = metrics.NewCounter()
storageOperationDurations = metrics.NewCounter()
storageLatency = metrics.NewHistogram(diskLatencyHistogram)
queueSizes = metrics.NewGauge()
)
func recordOutcome(counter metrics.Counter, latency metrics.Histogram, duration time.Duration, err error, success, failure map[string]string) {
func recordOutcome(duration time.Duration, err error, success, failure map[string]string) {
labels := success
if err != nil {
labels = failure
}
counter.Increment(labels)
latency.Add(labels, float64(duration/time.Microsecond))
storageOperations.Increment(labels)
asFloat := float64(duration / time.Microsecond)
storageLatency.Add(labels, asFloat)
storageOperationDurations.IncrementBy(labels, asFloat)
}
func init() {
registry.Register("prometheus_metric_disk_operations_total", "Total number of metric-related disk operations.", registry.NilLabels, storageOperations)
registry.Register("prometheus_metric_disk_latency_microseconds", "Latency for metric disk operations in microseconds.", registry.NilLabels, storageLatency)
registry.Register("prometheus_storage_operation_time_total_microseconds", "The total time spent performing a given storage operation.", registry.NilLabels, storageOperationDurations)
registry.Register("prometheus_storage_queue_sizes_total", "The various sizes and capacities of the storage queues.", registry.NilLabels, queueSizes)
}

View file

@ -15,13 +15,10 @@ package metric
import (
"github.com/prometheus/prometheus/model"
"github.com/prometheus/prometheus/storage"
"time"
)
type StalenessPolicy struct {
DeltaAllowance time.Duration
}
// MetricPersistence is a system for storing metric samples in a persistence
// layer.
type MetricPersistence interface {
@ -29,9 +26,16 @@ type MetricPersistence interface {
// closed when finished.
Close() error
// Commit all pending operations, if any, since some of the storage components
// queue work on channels and operate on it in bulk.
// Flush() error
// Record a new sample in the storage layer.
AppendSample(model.Sample) error
// Record a new sample in the storage layer.
AppendSamples(model.Samples) error
// Get all of the metric fingerprints that are associated with the provided
// label set.
GetFingerprintsForLabelSet(model.LabelSet) (model.Fingerprints, error)
@ -42,15 +46,42 @@ type MetricPersistence interface {
GetMetricForFingerprint(model.Fingerprint) (*model.Metric, error)
GetValueAtTime(model.Metric, time.Time, StalenessPolicy) (*model.Sample, error)
GetBoundaryValues(model.Metric, model.Interval, StalenessPolicy) (*model.Sample, *model.Sample, error)
GetRangeValues(model.Metric, model.Interval) (*model.SampleSet, error)
GetValueAtTime(model.Fingerprint, time.Time, StalenessPolicy) (*model.Sample, error)
GetBoundaryValues(model.Fingerprint, model.Interval, StalenessPolicy) (*model.Sample, *model.Sample, error)
GetRangeValues(model.Fingerprint, model.Interval) (*model.SampleSet, error)
ForEachSample(IteratorsForFingerprintBuilder) (err error)
GetAllMetricNames() ([]string, error)
// DIAGNOSTIC FUNCTIONS PENDING DELETION BELOW HERE
GetAllLabelNames() ([]string, error)
GetAllLabelPairs() ([]model.LabelSet, error)
GetAllMetrics() ([]model.LabelSet, error)
// Requests the storage stack to build a materialized View of the values
// contained therein.
// MakeView(builder ViewRequestBuilder, deadline time.Duration) (View, error)
}
// Describes the lenience limits for querying the materialized View.
type StalenessPolicy struct {
// Describes the inclusive limit at which individual points if requested will
// be matched and subject to interpolation.
DeltaAllowance time.Duration
}
// View provides view of the values in the datastore subject to the request of a
// preloading operation.
type View interface {
GetValueAtTime(model.Fingerprint, time.Time) []model.SamplePair
GetBoundaryValues(model.Fingerprint, model.Interval) (first []model.SamplePair, second []model.SamplePair)
GetRangeValues(model.Fingerprint, model.Interval) []model.SamplePair
// Destroy this view.
Close()
}
type Series interface {
Fingerprint() model.Fingerprint
Metric() model.Metric
}
type IteratorsForFingerprintBuilder interface {
ForStream(stream stream) (storage.RecordDecoder, storage.RecordFilter, storage.RecordOperator)
}

View file

@ -11,13 +11,13 @@
// See the License for the specific language governing permissions and
// limitations under the License.
package memory
package metric
import (
"github.com/prometheus/prometheus/storage/metric"
"testing"
)
func TestInterfaceAdherence(t *testing.T) {
var _ metric.MetricPersistence = NewMemorySeriesStorage()
var _ MetricPersistence = &LevelDBMetricPersistence{}
var _ MetricPersistence = NewMemorySeriesStorage()
}

View file

@ -11,13 +11,16 @@
// See the License for the specific language governing permissions and
// limitations under the License.
package leveldb
package metric
import (
"github.com/prometheus/prometheus/storage/metric"
"testing"
)
func TestInterfaceAdherence(t *testing.T) {
var _ metric.MetricPersistence = &LevelDBMetricPersistence{}
type Iterator interface {
Seek(key interface{}) (ok bool)
Next() (ok bool)
Previous() (ok bool)
Key() interface{}
Value() interface{}
}
type IteratorManager interface {
Iterator() Iterator
}

1300
storage/metric/leveldb.go Normal file

File diff suppressed because it is too large Load diff

View file

@ -1,176 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 leveldb
import (
"code.google.com/p/goprotobuf/proto"
"errors"
"github.com/prometheus/prometheus/coding"
"github.com/prometheus/prometheus/coding/indexable"
"github.com/prometheus/prometheus/model"
dto "github.com/prometheus/prometheus/model/generated"
"github.com/prometheus/prometheus/utility"
"log"
)
func (l *LevelDBMetricPersistence) GetAllLabelNames() ([]string, error) {
if getAll, getAllError := l.labelNameToFingerprints.GetAll(); getAllError == nil {
result := make([]string, 0, len(getAll))
labelNameDTO := &dto.LabelName{}
for _, pair := range getAll {
if unmarshalError := proto.Unmarshal(pair.Left, labelNameDTO); unmarshalError == nil {
result = append(result, *labelNameDTO.Name)
} else {
return nil, unmarshalError
}
}
return result, nil
} else {
return nil, getAllError
}
return nil, errors.New("Unknown error encountered when querying label names.")
}
func (l *LevelDBMetricPersistence) GetAllLabelPairs() ([]model.LabelSet, error) {
if getAll, getAllError := l.labelSetToFingerprints.GetAll(); getAllError == nil {
result := make([]model.LabelSet, 0, len(getAll))
labelPairDTO := &dto.LabelPair{}
for _, pair := range getAll {
if unmarshalError := proto.Unmarshal(pair.Left, labelPairDTO); unmarshalError == nil {
n := model.LabelName(*labelPairDTO.Name)
v := model.LabelValue(*labelPairDTO.Value)
item := model.LabelSet{n: v}
result = append(result, item)
} else {
return nil, unmarshalError
}
}
return result, nil
} else {
return nil, getAllError
}
return nil, errors.New("Unknown error encountered when querying label pairs.")
}
func (l *LevelDBMetricPersistence) GetAllMetrics() ([]model.LabelSet, error) {
if getAll, getAllError := l.labelSetToFingerprints.GetAll(); getAllError == nil {
result := make([]model.LabelSet, 0)
fingerprintCollection := &dto.FingerprintCollection{}
fingerprints := make(utility.Set)
for _, pair := range getAll {
if unmarshalError := proto.Unmarshal(pair.Right, fingerprintCollection); unmarshalError == nil {
for _, member := range fingerprintCollection.Member {
if !fingerprints.Has(*member.Signature) {
fingerprints.Add(*member.Signature)
fingerprintEncoded := coding.NewProtocolBufferEncoder(member)
if labelPairCollectionRaw, labelPairCollectionRawError := l.fingerprintToMetrics.Get(fingerprintEncoded); labelPairCollectionRawError == nil {
labelPairCollectionDTO := &dto.LabelSet{}
if labelPairCollectionDTOMarshalError := proto.Unmarshal(labelPairCollectionRaw, labelPairCollectionDTO); labelPairCollectionDTOMarshalError == nil {
intermediate := make(model.LabelSet, 0)
for _, member := range labelPairCollectionDTO.Member {
n := model.LabelName(*member.Name)
v := model.LabelValue(*member.Value)
intermediate[n] = v
}
result = append(result, intermediate)
} else {
return nil, labelPairCollectionDTOMarshalError
}
} else {
return nil, labelPairCollectionRawError
}
}
}
} else {
return nil, unmarshalError
}
}
return result, nil
} else {
return nil, getAllError
}
return nil, errors.New("Unknown error encountered when querying metrics.")
}
func (l *LevelDBMetricPersistence) GetSamplesForMetric(metric model.Metric, interval model.Interval) ([]model.Samples, error) {
if iterator, closer, iteratorErr := l.metricSamples.GetIterator(); iteratorErr == nil {
defer closer.Close()
fingerprintDTO := metric.Fingerprint().ToDTO()
start := &dto.SampleKey{
Fingerprint: fingerprintDTO,
Timestamp: indexable.EncodeTime(interval.OldestInclusive),
}
emission := make([]model.Samples, 0)
if encode, encodeErr := coding.NewProtocolBufferEncoder(start).Encode(); encodeErr == nil {
iterator.Seek(encode)
predicate := keyIsAtMostOld(interval.NewestInclusive)
for iterator = iterator; iterator.Valid(); iterator.Next() {
key := &dto.SampleKey{}
value := &dto.SampleValue{}
if keyUnmarshalErr := proto.Unmarshal(iterator.Key(), key); keyUnmarshalErr == nil {
if valueUnmarshalErr := proto.Unmarshal(iterator.Value(), value); valueUnmarshalErr == nil {
if fingerprintsEqual(fingerprintDTO, key.Fingerprint) {
// Wart
if predicate(key) {
emission = append(emission, model.Samples{
Value: model.SampleValue(*value.Value),
Timestamp: indexable.DecodeTime(key.Timestamp),
})
} else {
break
}
} else {
break
}
} else {
return nil, valueUnmarshalErr
}
} else {
return nil, keyUnmarshalErr
}
}
return emission, nil
} else {
log.Printf("Could not encode the start key: %q\n", encodeErr)
return nil, encodeErr
}
} else {
log.Printf("Could not acquire iterator: %q\n", iteratorErr)
return nil, iteratorErr
}
panic("unreachable")
}

View file

@ -1,55 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 leveldb
import (
"github.com/prometheus/prometheus/storage/metric"
"testing"
)
var testGetFingerprintsForLabelSet = buildTestPersistence("get_fingerprints_for_labelset", metric.GetFingerprintsForLabelSetTests)
func TestGetFingerprintsForLabelSet(t *testing.T) {
testGetFingerprintsForLabelSet(t)
}
func BenchmarkGetFingerprintsForLabelSet(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetFingerprintsForLabelSet(b)
}
}
var testGetFingerprintsForLabelName = buildTestPersistence("get_fingerprints_for_labelname", metric.GetFingerprintsForLabelNameTests)
func TestGetFingerprintsForLabelName(t *testing.T) {
testGetFingerprintsForLabelName(t)
}
func BenchmarkGetFingerprintsForLabelName(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetFingerprintsForLabelName(b)
}
}
var testGetMetricForFingerprint = buildTestPersistence("get_metric_for_fingerprint", metric.GetMetricForFingerprintTests)
func TestGetMetricForFingerprint(t *testing.T) {
testGetMetricForFingerprint(t)
}
func BenchmarkGetMetricForFingerprint(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetMetricForFingerprint(b)
}
}

View file

@ -1,163 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 leveldb
import (
"flag"
index "github.com/prometheus/prometheus/storage/raw/index/leveldb"
storage "github.com/prometheus/prometheus/storage/raw/leveldb"
"io"
"log"
)
var (
// These flag values are back of the envelope, though they seem sensible.
// Please re-evaluate based on your own needs.
fingerprintsToLabelPairCacheSize = flag.Int("fingerprintsToLabelPairCacheSizeBytes", 100*1024*1024, "The size for the fingerprint to label pair index (bytes).")
samplesByFingerprintCacheSize = flag.Int("samplesByFingerprintCacheSizeBytes", 500*1024*1024, "The size for the samples database (bytes).")
labelNameToFingerprintsCacheSize = flag.Int("labelNameToFingerprintsCacheSizeBytes", 100*1024*1024, "The size for the label name to metric fingerprint index (bytes).")
labelPairToFingerprintsCacheSize = flag.Int("labelPairToFingerprintsCacheSizeBytes", 100*1024*1024, "The size for the label pair to metric fingerprint index (bytes).")
metricMembershipIndexCacheSize = flag.Int("metricMembershipCacheSizeBytes", 50*1024*1024, "The size for the metric membership index (bytes).")
)
type leveldbOpener func()
func (l *LevelDBMetricPersistence) Close() error {
var persistences = []struct {
name string
closer io.Closer
}{
{
"Fingerprint to Label Name and Value Pairs",
l.fingerprintToMetrics,
},
{
"Fingerprint Samples",
l.metricSamples,
},
{
"Label Name to Fingerprints",
l.labelNameToFingerprints,
},
{
"Label Name and Value Pairs to Fingerprints",
l.labelSetToFingerprints,
},
{
"Metric Membership Index",
l.metricMembershipIndex,
},
}
errorChannel := make(chan error, len(persistences))
for _, persistence := range persistences {
name := persistence.name
closer := persistence.closer
go func(name string, closer io.Closer) {
if closer != nil {
closingError := closer.Close()
if closingError != nil {
log.Printf("Could not close a LevelDBPersistence storage container; inconsistencies are possible: %q\n", closingError)
}
errorChannel <- closingError
} else {
errorChannel <- nil
}
}(name, closer)
}
for i := 0; i < cap(errorChannel); i++ {
closingError := <-errorChannel
if closingError != nil {
return closingError
}
}
return nil
}
func NewLevelDBMetricPersistence(baseDirectory string) (persistence *LevelDBMetricPersistence, err error) {
errorChannel := make(chan error, 5)
emission := &LevelDBMetricPersistence{}
var subsystemOpeners = []struct {
name string
opener leveldbOpener
}{
{
"Label Names and Value Pairs by Fingerprint",
func() {
var err error
emission.fingerprintToMetrics, err = storage.NewLevelDBPersistence(baseDirectory+"/label_name_and_value_pairs_by_fingerprint", *fingerprintsToLabelPairCacheSize, 10)
errorChannel <- err
},
},
{
"Samples by Fingerprint",
func() {
var err error
emission.metricSamples, err = storage.NewLevelDBPersistence(baseDirectory+"/samples_by_fingerprint", *samplesByFingerprintCacheSize, 10)
errorChannel <- err
},
},
{
"Fingerprints by Label Name",
func() {
var err error
emission.labelNameToFingerprints, err = storage.NewLevelDBPersistence(baseDirectory+"/fingerprints_by_label_name", *labelNameToFingerprintsCacheSize, 10)
errorChannel <- err
},
},
{
"Fingerprints by Label Name and Value Pair",
func() {
var err error
emission.labelSetToFingerprints, err = storage.NewLevelDBPersistence(baseDirectory+"/fingerprints_by_label_name_and_value_pair", *labelPairToFingerprintsCacheSize, 10)
errorChannel <- err
},
},
{
"Metric Membership Index",
func() {
var err error
emission.metricMembershipIndex, err = index.NewLevelDBMembershipIndex(baseDirectory+"/metric_membership_index", *metricMembershipIndexCacheSize, 10)
errorChannel <- err
},
},
}
for _, subsystem := range subsystemOpeners {
opener := subsystem.opener
go opener()
}
for i := 0; i < cap(errorChannel); i++ {
err = <-errorChannel
if err != nil {
log.Printf("Could not open a LevelDBPersistence storage container: %q\n", err)
return
}
}
persistence = emission
return
}

View file

@ -1,224 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 leveldb
import (
"code.google.com/p/goprotobuf/proto"
"github.com/prometheus/prometheus/coding"
"github.com/prometheus/prometheus/coding/indexable"
"github.com/prometheus/prometheus/model"
dto "github.com/prometheus/prometheus/model/generated"
"time"
)
func (l *LevelDBMetricPersistence) setLabelPairFingerprints(labelPair *dto.LabelPair, fingerprints *dto.FingerprintCollection) (err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: setLabelPairFingerprints, result: success}, map[string]string{operation: setLabelPairFingerprints, result: failure})
}()
labelPairEncoded := coding.NewProtocolBufferEncoder(labelPair)
fingerprintsEncoded := coding.NewProtocolBufferEncoder(fingerprints)
err = l.labelSetToFingerprints.Put(labelPairEncoded, fingerprintsEncoded)
return
}
func (l *LevelDBMetricPersistence) setLabelNameFingerprints(labelName *dto.LabelName, fingerprints *dto.FingerprintCollection) (err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: setLabelNameFingerprints, result: success}, map[string]string{operation: setLabelNameFingerprints, result: failure})
}()
labelNameEncoded := coding.NewProtocolBufferEncoder(labelName)
fingerprintsEncoded := coding.NewProtocolBufferEncoder(fingerprints)
err = l.labelNameToFingerprints.Put(labelNameEncoded, fingerprintsEncoded)
return
}
func (l *LevelDBMetricPersistence) appendLabelPairFingerprint(labelPair *dto.LabelPair, fingerprint *dto.Fingerprint) (err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: appendLabelPairFingerprint, result: success}, map[string]string{operation: appendLabelPairFingerprint, result: failure})
}()
has, err := l.HasLabelPair(labelPair)
if err != nil {
return
}
var fingerprints *dto.FingerprintCollection
if has {
fingerprints, err = l.getFingerprintsForLabelSet(labelPair)
if err != nil {
return
}
} else {
fingerprints = &dto.FingerprintCollection{}
}
fingerprints.Member = append(fingerprints.Member, fingerprint)
err = l.setLabelPairFingerprints(labelPair, fingerprints)
return
}
func (l *LevelDBMetricPersistence) appendLabelNameFingerprint(labelPair *dto.LabelPair, fingerprint *dto.Fingerprint) (err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: appendLabelNameFingerprint, result: success}, map[string]string{operation: appendLabelNameFingerprint, result: failure})
}()
labelName := &dto.LabelName{
Name: labelPair.Name,
}
has, err := l.HasLabelName(labelName)
if err != nil {
return
}
var fingerprints *dto.FingerprintCollection
if has {
fingerprints, err = l.GetLabelNameFingerprints(labelName)
if err != nil {
return
}
} else {
fingerprints = &dto.FingerprintCollection{}
}
fingerprints.Member = append(fingerprints.Member, fingerprint)
err = l.setLabelNameFingerprints(labelName, fingerprints)
return
}
func (l *LevelDBMetricPersistence) appendFingerprints(sample model.Sample) (err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: appendFingerprints, result: success}, map[string]string{operation: appendFingerprints, result: failure})
}()
fingerprintDTO := sample.Metric.Fingerprint().ToDTO()
fingerprintKey := coding.NewProtocolBufferEncoder(fingerprintDTO)
metricDTO := model.SampleToMetricDTO(&sample)
metricDTOEncoder := coding.NewProtocolBufferEncoder(metricDTO)
err = l.fingerprintToMetrics.Put(fingerprintKey, metricDTOEncoder)
if err != nil {
return
}
labelCount := len(metricDTO.LabelPair)
labelPairErrors := make(chan error, labelCount)
labelNameErrors := make(chan error, labelCount)
for _, labelPair := range metricDTO.LabelPair {
go func(labelPair *dto.LabelPair) {
labelNameErrors <- l.appendLabelNameFingerprint(labelPair, fingerprintDTO)
}(labelPair)
go func(labelPair *dto.LabelPair) {
labelPairErrors <- l.appendLabelPairFingerprint(labelPair, fingerprintDTO)
}(labelPair)
}
for i := 0; i < cap(labelPairErrors); i++ {
err = <-labelPairErrors
if err != nil {
return
}
}
for i := 0; i < cap(labelNameErrors); i++ {
err = <-labelNameErrors
if err != nil {
return
}
}
return
}
func (l *LevelDBMetricPersistence) AppendSample(sample model.Sample) (err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: appendSample, result: success}, map[string]string{operation: appendSample, result: failure})
}()
metricDTO := model.SampleToMetricDTO(&sample)
indexHas, err := l.hasIndexMetric(metricDTO)
if err != nil {
return
}
fingerprint := sample.Metric.Fingerprint()
if !indexHas {
err = l.indexMetric(metricDTO)
if err != nil {
return
}
err = l.appendFingerprints(sample)
if err != nil {
return
}
}
fingerprintDTO := fingerprint.ToDTO()
sampleKeyDTO := &dto.SampleKey{
Fingerprint: fingerprintDTO,
Timestamp: indexable.EncodeTime(sample.Timestamp),
}
sampleValueDTO := &dto.SampleValue{
Value: proto.Float32(float32(sample.Value)),
}
sampleKeyEncoded := coding.NewProtocolBufferEncoder(sampleKeyDTO)
sampleValueEncoded := coding.NewProtocolBufferEncoder(sampleValueDTO)
err = l.metricSamples.Put(sampleKeyEncoded, sampleValueEncoded)
if err != nil {
return
}
return
}

View file

@ -1,673 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 leveldb
import (
"code.google.com/p/goprotobuf/proto"
"github.com/prometheus/prometheus/coding"
"github.com/prometheus/prometheus/coding/indexable"
"github.com/prometheus/prometheus/model"
dto "github.com/prometheus/prometheus/model/generated"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility"
"sort"
"time"
)
func extractSampleKey(i iterator) (k *dto.SampleKey, err error) {
k = &dto.SampleKey{}
err = proto.Unmarshal(i.Key(), k)
return
}
func extractSampleValue(i iterator) (v *dto.SampleValue, err error) {
v = &dto.SampleValue{}
err = proto.Unmarshal(i.Value(), v)
return
}
func fingerprintsEqual(l *dto.Fingerprint, r *dto.Fingerprint) bool {
if l == r {
return true
}
if l == nil && r == nil {
return true
}
if r.Signature == l.Signature {
return true
}
if *r.Signature == *l.Signature {
return true
}
return false
}
type sampleKeyPredicate func(k *dto.SampleKey) bool
func keyIsOlderThan(t time.Time) sampleKeyPredicate {
return func(k *dto.SampleKey) bool {
return indexable.DecodeTime(k.Timestamp).After(t)
}
}
func keyIsAtMostOld(t time.Time) sampleKeyPredicate {
return func(k *dto.SampleKey) bool {
return !indexable.DecodeTime(k.Timestamp).After(t)
}
}
func (l *LevelDBMetricPersistence) hasIndexMetric(dto *dto.Metric) (value bool, err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: hasIndexMetric, result: success}, map[string]string{operation: hasIndexMetric, result: failure})
}()
dtoKey := coding.NewProtocolBufferEncoder(dto)
value, err = l.metricMembershipIndex.Has(dtoKey)
return
}
func (l *LevelDBMetricPersistence) indexMetric(dto *dto.Metric) (err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: indexMetric, result: success}, map[string]string{operation: indexMetric, result: failure})
}()
dtoKey := coding.NewProtocolBufferEncoder(dto)
err = l.metricMembershipIndex.Put(dtoKey)
return
}
func (l *LevelDBMetricPersistence) HasLabelPair(dto *dto.LabelPair) (value bool, err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: hasLabelPair, result: success}, map[string]string{operation: hasLabelPair, result: failure})
}()
dtoKey := coding.NewProtocolBufferEncoder(dto)
value, err = l.labelSetToFingerprints.Has(dtoKey)
return
}
func (l *LevelDBMetricPersistence) HasLabelName(dto *dto.LabelName) (value bool, err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: hasLabelName, result: success}, map[string]string{operation: hasLabelName, result: failure})
}()
dtoKey := coding.NewProtocolBufferEncoder(dto)
value, err = l.labelNameToFingerprints.Has(dtoKey)
return
}
func (l *LevelDBMetricPersistence) getFingerprintsForLabelSet(p *dto.LabelPair) (c *dto.FingerprintCollection, err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: getFingerprintsForLabelSet, result: success}, map[string]string{operation: getFingerprintsForLabelSet, result: failure})
}()
dtoKey := coding.NewProtocolBufferEncoder(p)
get, err := l.labelSetToFingerprints.Get(dtoKey)
if err != nil {
return
}
c = &dto.FingerprintCollection{}
err = proto.Unmarshal(get, c)
return
}
// XXX: Delete me and replace with GetFingerprintsForLabelName.
func (l *LevelDBMetricPersistence) GetLabelNameFingerprints(n *dto.LabelName) (c *dto.FingerprintCollection, err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: getLabelNameFingerprints, result: success}, map[string]string{operation: getLabelNameFingerprints, result: failure})
}()
dtoKey := coding.NewProtocolBufferEncoder(n)
get, err := l.labelNameToFingerprints.Get(dtoKey)
if err != nil {
return
}
c = &dto.FingerprintCollection{}
err = proto.Unmarshal(get, c)
return
}
func (l *LevelDBMetricPersistence) GetFingerprintsForLabelSet(labelSet model.LabelSet) (fps model.Fingerprints, err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: getFingerprintsForLabelSet, result: success}, map[string]string{operation: getFingerprintsForLabelSet, result: failure})
}()
sets := []utility.Set{}
for _, labelSetDTO := range model.LabelSetToDTOs(&labelSet) {
f, err := l.labelSetToFingerprints.Get(coding.NewProtocolBufferEncoder(labelSetDTO))
if err != nil {
return fps, err
}
unmarshaled := &dto.FingerprintCollection{}
err = proto.Unmarshal(f, unmarshaled)
if err != nil {
return fps, err
}
set := utility.Set{}
for _, m := range unmarshaled.Member {
fp := model.Fingerprint(*m.Signature)
set.Add(fp)
}
sets = append(sets, set)
}
numberOfSets := len(sets)
if numberOfSets == 0 {
return
}
base := sets[0]
for i := 1; i < numberOfSets; i++ {
base = base.Intersection(sets[i])
}
for _, e := range base.Elements() {
fingerprint := e.(model.Fingerprint)
fps = append(fps, fingerprint)
}
return
}
func (l *LevelDBMetricPersistence) GetFingerprintsForLabelName(labelName model.LabelName) (fps model.Fingerprints, err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: getFingerprintsForLabelName, result: success}, map[string]string{operation: getFingerprintsForLabelName, result: failure})
}()
raw, err := l.labelNameToFingerprints.Get(coding.NewProtocolBufferEncoder(model.LabelNameToDTO(&labelName)))
if err != nil {
return
}
unmarshaled := &dto.FingerprintCollection{}
err = proto.Unmarshal(raw, unmarshaled)
if err != nil {
return
}
for _, m := range unmarshaled.Member {
fp := model.Fingerprint(*m.Signature)
fps = append(fps, fp)
}
return
}
func (l *LevelDBMetricPersistence) GetMetricForFingerprint(f model.Fingerprint) (m *model.Metric, err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: getMetricForFingerprint, result: success}, map[string]string{operation: getMetricForFingerprint, result: failure})
}()
raw, err := l.fingerprintToMetrics.Get(coding.NewProtocolBufferEncoder(model.FingerprintToDTO(&f)))
if err != nil {
return
}
unmarshaled := &dto.Metric{}
err = proto.Unmarshal(raw, unmarshaled)
if err != nil {
return
}
metric := model.Metric{}
for _, v := range unmarshaled.LabelPair {
metric[model.LabelName(*v.Name)] = model.LabelValue(*v.Value)
}
// Explicit address passing here shaves immense amounts of time off of the
// code flow due to less tight-loop dereferencing.
m = &metric
return
}
func (l *LevelDBMetricPersistence) GetBoundaryValues(m model.Metric, i model.Interval, s metric.StalenessPolicy) (open *model.Sample, end *model.Sample, err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: getBoundaryValues, result: success}, map[string]string{operation: getBoundaryValues, result: failure})
}()
// XXX: Maybe we will want to emit incomplete sets?
open, err = l.GetValueAtTime(m, i.OldestInclusive, s)
if err != nil {
return
} else if open == nil {
return
}
end, err = l.GetValueAtTime(m, i.NewestInclusive, s)
if err != nil {
return
} else if end == nil {
open = nil
}
return
}
func interpolate(x1, x2 time.Time, y1, y2 float32, e time.Time) float32 {
yDelta := y2 - y1
xDelta := x2.Sub(x1)
dDt := yDelta / float32(xDelta)
offset := float32(e.Sub(x1))
return y1 + (offset * dDt)
}
type iterator interface {
Close()
Key() []byte
Next()
Prev()
Seek([]byte)
SeekToFirst()
SeekToLast()
Valid() bool
Value() []byte
}
func (l *LevelDBMetricPersistence) GetValueAtTime(m model.Metric, t time.Time, s metric.StalenessPolicy) (sample *model.Sample, err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: getValueAtTime, result: success}, map[string]string{operation: getValueAtTime, result: failure})
}()
f := m.Fingerprint().ToDTO()
// Candidate for Refactoring
k := &dto.SampleKey{
Fingerprint: f,
Timestamp: indexable.EncodeTime(t),
}
e, err := coding.NewProtocolBufferEncoder(k).Encode()
if err != nil {
return
}
iterator, closer, err := l.metricSamples.GetIterator()
if err != nil {
return
}
defer closer.Close()
iterator.Seek(e)
if !iterator.Valid() {
/*
* Two cases for this:
* 1.) Corruption in LevelDB.
* 2.) Key seek after AND outside known range.
*
* Once a LevelDB iterator goes invalid, it cannot be recovered; thusly,
* we need to create a new in order to check if the last value in the
* database is sufficient for our purposes. This is, in all reality, a
* corner case but one that could bring down the system.
*/
iterator, closer, err = l.metricSamples.GetIterator()
if err != nil {
return
}
defer closer.Close()
iterator.SeekToLast()
if !iterator.Valid() {
/*
* For whatever reason, the LevelDB cannot be recovered.
*/
return
}
}
var (
firstKey *dto.SampleKey
firstValue *dto.SampleValue
)
firstKey, err = extractSampleKey(iterator)
if err != nil {
return
}
peekAhead := false
if !fingerprintsEqual(firstKey.Fingerprint, k.Fingerprint) {
/*
* This allows us to grab values for metrics if our request time is after
* the last recorded time subject to the staleness policy due to the nuances
* of LevelDB storage:
*
* # Assumptions:
* - K0 < K1 in terms of sorting.
* - T0 < T1 in terms of sorting.
*
* # Data
*
* K0-T0
* K0-T1
* K0-T2
* K1-T0
* K1-T1
*
* # Scenario
* K0-T3, which does not exist, is requested. LevelDB will thusly seek to
* K1-T1, when K0-T2 exists as a perfectly good candidate to check subject
* to the provided staleness policy and such.
*/
peekAhead = true
}
firstTime := indexable.DecodeTime(firstKey.Timestamp)
if t.Before(firstTime) || peekAhead {
iterator.Prev()
if !iterator.Valid() {
/*
* Two cases for this:
* 1.) Corruption in LevelDB.
* 2.) Key seek before AND outside known range.
*
* This is an explicit validation to ensure that if no previous values for
* the series are found, the query aborts.
*/
return
}
var (
alternativeKey *dto.SampleKey
alternativeValue *dto.SampleValue
)
alternativeKey, err = extractSampleKey(iterator)
if err != nil {
return
}
if !fingerprintsEqual(alternativeKey.Fingerprint, k.Fingerprint) {
return
}
/*
* At this point, we found a previous value in the same series in the
* database. LevelDB originally seeked to the subsequent element given
* the key, but we need to consider this adjacency instead.
*/
alternativeTime := indexable.DecodeTime(alternativeKey.Timestamp)
firstKey = alternativeKey
firstValue = alternativeValue
firstTime = alternativeTime
}
firstDelta := firstTime.Sub(t)
if firstDelta < 0 {
firstDelta *= -1
}
if firstDelta > s.DeltaAllowance {
return
}
firstValue, err = extractSampleValue(iterator)
if err != nil {
return
}
sample = model.SampleFromDTO(&m, &t, firstValue)
if firstDelta == time.Duration(0) {
return
}
iterator.Next()
if !iterator.Valid() {
/*
* Two cases for this:
* 1.) Corruption in LevelDB.
* 2.) Key seek after AND outside known range.
*
* This means that there are no more values left in the storage; and if this
* point is reached, we know that the one that has been found is within the
* allowed staleness limits.
*/
return
}
var secondKey *dto.SampleKey
secondKey, err = extractSampleKey(iterator)
if err != nil {
return
}
if !fingerprintsEqual(secondKey.Fingerprint, k.Fingerprint) {
return
} else {
/*
* At this point, current entry in the database has the same key as the
* previous. For this reason, the validation logic will expect that the
* distance between the two points shall not exceed the staleness policy
* allowed limit to reduce interpolation errors.
*
* For this reason, the sample is reset in case of other subsequent
* validation behaviors.
*/
sample = nil
}
secondTime := indexable.DecodeTime(secondKey.Timestamp)
totalDelta := secondTime.Sub(firstTime)
if totalDelta > s.DeltaAllowance {
return
}
var secondValue *dto.SampleValue
secondValue, err = extractSampleValue(iterator)
if err != nil {
return
}
interpolated := interpolate(firstTime, secondTime, *firstValue.Value, *secondValue.Value, t)
sampleValue := &dto.SampleValue{
Value: &interpolated,
}
sample = model.SampleFromDTO(&m, &t, sampleValue)
return
}
func (l *LevelDBMetricPersistence) GetRangeValues(m model.Metric, i model.Interval) (v *model.SampleSet, err error) {
begin := time.Now()
defer func() {
duration := time.Now().Sub(begin)
recordOutcome(storageOperations, storageLatency, duration, err, map[string]string{operation: getRangeValues, result: success}, map[string]string{operation: getRangeValues, result: failure})
}()
f := m.Fingerprint().ToDTO()
k := &dto.SampleKey{
Fingerprint: f,
Timestamp: indexable.EncodeTime(i.OldestInclusive),
}
e, err := coding.NewProtocolBufferEncoder(k).Encode()
if err != nil {
return
}
iterator, closer, err := l.metricSamples.GetIterator()
if err != nil {
return
}
defer closer.Close()
iterator.Seek(e)
predicate := keyIsOlderThan(i.NewestInclusive)
for ; iterator.Valid(); iterator.Next() {
retrievedKey := &dto.SampleKey{}
retrievedKey, err = extractSampleKey(iterator)
if err != nil {
return
}
if predicate(retrievedKey) {
break
}
if !fingerprintsEqual(retrievedKey.Fingerprint, k.Fingerprint) {
break
}
retrievedValue, err := extractSampleValue(iterator)
if err != nil {
return nil, err
}
if v == nil {
v = &model.SampleSet{}
}
v.Values = append(v.Values, model.SamplePair{
Value: model.SampleValue(*retrievedValue.Value),
Timestamp: indexable.DecodeTime(retrievedKey.Timestamp),
})
}
// XXX: We should not explicitly sort here but rather rely on the datastore.
// This adds appreciable overhead.
if v != nil {
sort.Sort(v.Values)
}
return
}
type MetricKeyDecoder struct{}
func (d *MetricKeyDecoder) DecodeKey(in interface{}) (out interface{}, err error) {
unmarshaled := &dto.LabelPair{}
err = proto.Unmarshal(in.([]byte), unmarshaled)
if err != nil {
return
}
out = unmarshaled
return
}
func (d *MetricKeyDecoder) DecodeValue(in interface{}) (out interface{}, err error) {
return
}
type MetricNamesFilter struct{}
func (f *MetricNamesFilter) Filter(key, value interface{}) (filterResult storage.FilterResult) {
unmarshaled, ok := key.(*dto.LabelPair)
if ok && *unmarshaled.Name == "name" {
return storage.ACCEPT
}
return storage.SKIP
}
type CollectMetricNamesOp struct {
metricNames []string
}
func (op *CollectMetricNamesOp) Operate(key, value interface{}) (err *storage.OperatorError) {
unmarshaled := key.(*dto.LabelPair)
op.metricNames = append(op.metricNames, *unmarshaled.Value)
return
}
func (l *LevelDBMetricPersistence) GetAllMetricNames() (metricNames []string, err error) {
metricNamesOp := &CollectMetricNamesOp{}
_, err = l.labelSetToFingerprints.ForEach(&MetricKeyDecoder{}, &MetricNamesFilter{}, metricNamesOp)
if err != nil {
return
}
metricNames = metricNamesOp.metricNames
return
}

View file

@ -1,31 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 leveldb
import (
"github.com/prometheus/prometheus/storage/metric"
"testing"
)
var testGetFingerprintsForLabelSetUsesAndForLabelMatching = buildTestPersistence("get_fingerprints_for_labelset_uses_and_for_label_matching", metric.GetFingerprintsForLabelSetUsesAndForLabelMatchingTests)
func TestGetFingerprintsForLabelSetUsesAndForLabelMatching(t *testing.T) {
testGetFingerprintsForLabelSetUsesAndForLabelMatching(t)
}
func BenchmarkGetFingerprintsForLabelSetUsesAndForLabelMatching(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetFingerprintsForLabelSetUsesAndForLabelMatching(b)
}
}

View file

@ -1,67 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 leveldb
import (
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility/test"
"testing"
)
func testGetValueAtTime(t test.Tester) {
persistenceMaker := buildTestPersistencesMaker("get_value_at_time", t)
metric.GetValueAtTimeTests(persistenceMaker, t)
}
func TestGetValueAtTime(t *testing.T) {
testGetValueAtTime(t)
}
func BenchmarkGetValueAtTime(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetValueAtTime(b)
}
}
func testGetBoundaryValues(t test.Tester) {
persistenceMaker := buildTestPersistencesMaker("get_boundary_values", t)
metric.GetBoundaryValuesTests(persistenceMaker, t)
}
func TestGetBoundaryValues(t *testing.T) {
testGetBoundaryValues(t)
}
func BenchmarkGetBoundaryValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetBoundaryValues(b)
}
}
func testGetRangeValues(t test.Tester) {
persistenceMaker := buildTestPersistencesMaker("get_range_values", t)
metric.GetRangeValuesTests(persistenceMaker, t)
}
func TestGetRangeValues(t *testing.T) {
testGetRangeValues(t)
}
func BenchmarkGetRangeValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetRangeValues(b)
}
}

View file

@ -1,109 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 leveldb
import (
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility/test"
"io/ioutil"
"testing"
)
var testBasicLifecycle = buildTestPersistence("basic_lifecycle", metric.BasicLifecycleTests)
func TestBasicLifecycle(t *testing.T) {
testBasicLifecycle(t)
}
func BenchmarkBasicLifecycle(b *testing.B) {
for i := 0; i < b.N; i++ {
testBasicLifecycle(b)
}
}
var testReadEmpty = buildTestPersistence("read_empty", metric.ReadEmptyTests)
func TestReadEmpty(t *testing.T) {
testReadEmpty(t)
}
func BenchmarkReadEmpty(b *testing.B) {
for i := 0; i < b.N; i++ {
testReadEmpty(b)
}
}
var testAppendSampleAsPureSparseAppend = buildTestPersistence("append_sample_as_pure_sparse_append", metric.AppendSampleAsPureSparseAppendTests)
func TestAppendSampleAsPureSparseAppend(t *testing.T) {
testAppendSampleAsPureSparseAppend(t)
}
func BenchmarkAppendSampleAsPureSparseAppend(b *testing.B) {
for i := 0; i < b.N; i++ {
testAppendSampleAsPureSparseAppend(b)
}
}
var testAppendSampleAsSparseAppendWithReads = buildTestPersistence("append_sample_as_sparse_append_with_reads", metric.AppendSampleAsSparseAppendWithReadsTests)
func TestAppendSampleAsSparseAppendWithReads(t *testing.T) {
testAppendSampleAsSparseAppendWithReads(t)
}
func BenchmarkAppendSampleAsSparseAppendWithReads(b *testing.B) {
for i := 0; i < b.N; i++ {
testAppendSampleAsSparseAppendWithReads(b)
}
}
var testAppendSampleAsPureSingleEntityAppend = buildTestPersistence("append_sample_as_pure_single_entity_append", metric.AppendSampleAsPureSingleEntityAppendTests)
func TestAppendSampleAsPureSingleEntityAppend(t *testing.T) {
testAppendSampleAsPureSingleEntityAppend(t)
}
func BenchmarkAppendSampleAsPureSingleEntityAppend(b *testing.B) {
for i := 0; i < b.N; i++ {
testAppendSampleAsPureSingleEntityAppend(b)
}
}
func testStochastic(t test.Tester) {
persistenceMaker := func() metric.MetricPersistence {
temporaryDirectory, err := ioutil.TempDir("", "test_leveldb_stochastic")
if err != nil {
t.Errorf("Could not create test directory: %q\n", err)
}
p, err := NewLevelDBMetricPersistence(temporaryDirectory)
if err != nil {
t.Errorf("Could not start up LevelDB: %q\n", err)
}
return p
}
metric.StochasticTests(persistenceMaker, t)
}
func TestStochastic(t *testing.T) {
testStochastic(t)
}
func BenchmarkStochastic(b *testing.B) {
for i := 0; i < b.N; i++ {
testStochastic(b)
}
}

View file

@ -1,84 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 leveldb
import (
"fmt"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility/test"
"io"
"io/ioutil"
"os"
)
type purger struct {
path string
}
func (p purger) Close() error {
return os.RemoveAll(p.path)
}
func buildTestPersistencesMaker(name string, t test.Tester) func() (metric.MetricPersistence, io.Closer) {
return func() (metric.MetricPersistence, io.Closer) {
temporaryDirectory, err := ioutil.TempDir("", "get_value_at_time")
if err != nil {
t.Errorf("Could not create test directory: %q\n", err)
}
p, err := NewLevelDBMetricPersistence(temporaryDirectory)
if err != nil {
t.Errorf("Could not start up LevelDB: %q\n", err)
}
purger := purger{
path: temporaryDirectory,
}
return p, purger
}
}
func buildTestPersistence(name string, f func(p metric.MetricPersistence, t test.Tester)) func(t test.Tester) {
return func(t test.Tester) {
temporaryDirectory, err := ioutil.TempDir("", fmt.Sprintf("test_leveldb_%s", name))
if err != nil {
t.Errorf("Could not create test directory: %q\n", err)
return
}
defer func() {
err := os.RemoveAll(temporaryDirectory)
if err != nil {
t.Errorf("Could not remove temporary directory: %q\n", err)
}
}()
p, err := NewLevelDBMetricPersistence(temporaryDirectory)
if err != nil {
t.Errorf("Could not create LevelDB Metric Persistence: %q\n", err)
}
defer func() {
err := p.Close()
if err != nil {
t.Errorf("Anomaly while closing database: %q\n", err)
}
}()
f(p, t)
}
}

371
storage/metric/memory.go Normal file
View file

@ -0,0 +1,371 @@
// Copyright 2013 Prometheus Team
// 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 metric
import (
"fmt"
"github.com/prometheus/prometheus/model"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/utility"
"github.com/ryszard/goskiplist/skiplist"
"sort"
"time"
)
const (
// Used as a separator in the format string for generating the internal label
// value pair set fingerprints.
reservedDelimiter = `"`
)
// Models a given sample entry stored in the in-memory arena.
type value interface {
// Gets the given value.
get() model.SampleValue
}
// Models a single sample value. It presumes that there is either no subsequent
// value seen or that any subsequent values are of a different value.
type singletonValue model.SampleValue
func (v singletonValue) get() model.SampleValue {
return model.SampleValue(v)
}
type skipListTime time.Time
func (t skipListTime) LessThan(o skiplist.Ordered) bool {
return time.Time(o.(skipListTime)).Before(time.Time(t))
}
type stream struct {
metric model.Metric
values *skiplist.SkipList
}
func (s stream) add(timestamp time.Time, value model.SampleValue) {
s.values.Set(skipListTime(timestamp), singletonValue(value))
}
func (s stream) forEach(decoder storage.RecordDecoder, filter storage.RecordFilter, operator storage.RecordOperator) (scannedEntireCorpus bool, err error) {
if s.values.Len() == 0 {
return
}
iterator := s.values.SeekToLast()
defer iterator.Close()
for !(iterator.Key() == nil || iterator.Value() == nil) {
decodedKey, decodeErr := decoder.DecodeKey(iterator.Key())
if decodeErr != nil {
panic(decodeErr)
}
decodedValue, decodeErr := decoder.DecodeValue(iterator.Value())
if decodeErr != nil {
panic(decodeErr)
}
switch filter.Filter(decodedKey, decodedValue) {
case storage.STOP:
return
case storage.SKIP:
continue
case storage.ACCEPT:
opErr := operator.Operate(decodedKey, decodedValue)
if opErr != nil {
if opErr.Continuable {
continue
}
break
}
}
if !iterator.Previous() {
break
}
}
scannedEntireCorpus = true
return
}
func newStream(metric model.Metric) stream {
return stream{
values: skiplist.New(),
metric: metric,
}
}
type memorySeriesStorage struct {
fingerprintToSeries map[model.Fingerprint]stream
labelPairToFingerprints map[string]model.Fingerprints
labelNameToFingerprints map[model.LabelName]model.Fingerprints
}
func (s memorySeriesStorage) AppendSamples(samples model.Samples) (err error) {
for _, sample := range samples {
s.AppendSample(sample)
}
return
}
func (s memorySeriesStorage) AppendSample(sample model.Sample) (err error) {
var (
metric = sample.Metric
fingerprint = model.NewFingerprintFromMetric(metric)
series, ok = s.fingerprintToSeries[fingerprint]
)
if !ok {
series = newStream(metric)
s.fingerprintToSeries[fingerprint] = series
for k, v := range metric {
labelPair := fmt.Sprintf("%s%s%s", k, reservedDelimiter, v)
labelPairValues := s.labelPairToFingerprints[labelPair]
labelPairValues = append(labelPairValues, fingerprint)
s.labelPairToFingerprints[labelPair] = labelPairValues
labelNameValues := s.labelNameToFingerprints[k]
labelNameValues = append(labelNameValues, fingerprint)
s.labelNameToFingerprints[k] = labelNameValues
}
}
series.add(sample.Timestamp, sample.Value)
return
}
func (s memorySeriesStorage) GetFingerprintsForLabelSet(l model.LabelSet) (fingerprints model.Fingerprints, err error) {
sets := []utility.Set{}
for k, v := range l {
signature := fmt.Sprintf("%s%s%s", k, reservedDelimiter, v)
values := s.labelPairToFingerprints[signature]
set := utility.Set{}
for _, fingerprint := range values {
set.Add(fingerprint)
}
sets = append(sets, set)
}
setCount := len(sets)
if setCount == 0 {
return
}
base := sets[0]
for i := 1; i < setCount; i++ {
base = base.Intersection(sets[i])
}
for _, e := range base.Elements() {
fingerprint := e.(model.Fingerprint)
fingerprints = append(fingerprints, fingerprint)
}
return
}
func (s memorySeriesStorage) GetFingerprintsForLabelName(l model.LabelName) (fingerprints model.Fingerprints, err error) {
values := s.labelNameToFingerprints[l]
fingerprints = append(fingerprints, values...)
return
}
func (s memorySeriesStorage) GetMetricForFingerprint(f model.Fingerprint) (metric *model.Metric, err error) {
series, ok := s.fingerprintToSeries[f]
if !ok {
return
}
metric = &series.metric
return
}
// XXX: Terrible wart.
func interpolateSample(x1, x2 time.Time, y1, y2 float32, e time.Time) model.SampleValue {
yDelta := y2 - y1
xDelta := x2.Sub(x1)
dDt := yDelta / float32(xDelta)
offset := float32(e.Sub(x1))
return model.SampleValue(y1 + (offset * dDt))
}
func (s memorySeriesStorage) GetValueAtTime(fp model.Fingerprint, t time.Time, p StalenessPolicy) (sample *model.Sample, err error) {
series, ok := s.fingerprintToSeries[fp]
if !ok {
return
}
iterator := series.values.Seek(skipListTime(t))
if iterator == nil {
return
}
foundTime := time.Time(iterator.Key().(skipListTime))
if foundTime.Equal(t) {
value := iterator.Value().(value)
sample = &model.Sample{
Metric: series.metric,
Value: value.get(),
Timestamp: t,
}
return
}
if t.Sub(foundTime) > p.DeltaAllowance {
return
}
secondTime := foundTime
secondValue := iterator.Value().(value).get()
if !iterator.Previous() {
sample = &model.Sample{
Metric: series.metric,
Value: iterator.Value().(value).get(),
Timestamp: t,
}
return
}
firstTime := time.Time(iterator.Key().(skipListTime))
if t.Sub(firstTime) > p.DeltaAllowance {
return
}
if firstTime.Sub(secondTime) > p.DeltaAllowance {
return
}
firstValue := iterator.Value().(value).get()
sample = &model.Sample{
Metric: series.metric,
Value: interpolateSample(firstTime, secondTime, float32(firstValue), float32(secondValue), t),
Timestamp: t,
}
return
}
func (s memorySeriesStorage) GetBoundaryValues(fp model.Fingerprint, i model.Interval, p StalenessPolicy) (first *model.Sample, second *model.Sample, err error) {
first, err = s.GetValueAtTime(fp, i.OldestInclusive, p)
if err != nil {
return
} else if first == nil {
return
}
second, err = s.GetValueAtTime(fp, i.NewestInclusive, p)
if err != nil {
return
} else if second == nil {
first = nil
}
return
}
func (s memorySeriesStorage) GetRangeValues(fp model.Fingerprint, i model.Interval) (samples *model.SampleSet, err error) {
series, ok := s.fingerprintToSeries[fp]
if !ok {
return
}
samples = &model.SampleSet{
Metric: series.metric,
}
iterator := series.values.Seek(skipListTime(i.NewestInclusive))
if iterator == nil {
return
}
for {
timestamp := time.Time(iterator.Key().(skipListTime))
if timestamp.Before(i.OldestInclusive) {
break
}
samples.Values = append(samples.Values,
model.SamplePair{
Value: iterator.Value().(value).get(),
Timestamp: timestamp,
})
if !iterator.Next() {
break
}
}
// XXX: We should not explicitly sort here but rather rely on the datastore.
// This adds appreciable overhead.
if samples != nil {
sort.Sort(samples.Values)
}
return
}
func (s memorySeriesStorage) Close() (err error) {
// This can probably be simplified:
//
// s.fingerPrintToSeries = map[model.Fingerprint]*stream{}
// s.labelPairToFingerprints = map[string]model.Fingerprints{}
// s.labelNameToFingerprints = map[model.LabelName]model.Fingerprints{}
for fingerprint := range s.fingerprintToSeries {
delete(s.fingerprintToSeries, fingerprint)
}
for labelPair := range s.labelPairToFingerprints {
delete(s.labelPairToFingerprints, labelPair)
}
for labelName := range s.labelNameToFingerprints {
delete(s.labelNameToFingerprints, labelName)
}
return
}
func (s memorySeriesStorage) GetAllMetricNames() ([]string, error) {
panic("not implemented")
}
func (s memorySeriesStorage) ForEachSample(builder IteratorsForFingerprintBuilder) (err error) {
for _, stream := range s.fingerprintToSeries {
decoder, filter, operator := builder.ForStream(stream)
stream.forEach(decoder, filter, operator)
}
return
}
func NewMemorySeriesStorage() memorySeriesStorage {
return memorySeriesStorage{
fingerprintToSeries: make(map[model.Fingerprint]stream),
labelPairToFingerprints: make(map[string]model.Fingerprints),
labelNameToFingerprints: make(map[model.LabelName]model.Fingerprints),
}
}

View file

@ -1,55 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 memory
import (
"github.com/prometheus/prometheus/storage/metric"
"testing"
)
var testGetFingerprintsForLabelSet = buildTestPersistence(metric.GetFingerprintsForLabelSetTests)
func TestGetFingerprintsForLabelSet(t *testing.T) {
testGetFingerprintsForLabelSet(t)
}
func BenchmarkGetFingerprintsForLabelSet(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetFingerprintsForLabelSet(b)
}
}
var testGetFingerprintsForLabelName = buildTestPersistence(metric.GetFingerprintsForLabelNameTests)
func TestGetFingerprintsForLabelName(t *testing.T) {
testGetFingerprintsForLabelName(t)
}
func BenchmarkGetFingerprintsForLabelName(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetFingerprintsForLabelName(b)
}
}
var testGetMetricForFingerprint = buildTestPersistence(metric.GetMetricForFingerprintTests)
func TestGetMetricForFingerprint(t *testing.T) {
testGetMetricForFingerprint(t)
}
func BenchmarkGetMetricForFingerprint(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetMetricForFingerprint(b)
}
}

View file

@ -1,303 +0,0 @@
package memory
import (
"fmt"
"github.com/prometheus/prometheus/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility"
"github.com/ryszard/goskiplist/skiplist"
"sort"
"time"
)
const (
reservedDelimiter = `"`
)
type skipListTime time.Time
func (t skipListTime) LessThan(o skiplist.Ordered) bool {
// return time.Time(t).Before(time.Time(o.(skipListTime)))
return time.Time(o.(skipListTime)).Before(time.Time(t))
}
type stream struct {
metric model.Metric
values *skiplist.SkipList
}
func (s *stream) add(sample model.Sample) {
s.values.Set(skipListTime(sample.Timestamp), sample.Value)
}
func newStream(metric model.Metric) *stream {
return &stream{
values: skiplist.New(),
metric: metric,
}
}
type memorySeriesStorage struct {
fingerprintToSeries map[model.Fingerprint]*stream
labelPairToFingerprints map[string]model.Fingerprints
labelNameToFingerprints map[model.LabelName]model.Fingerprints
}
func (s *memorySeriesStorage) AppendSample(sample model.Sample) (err error) {
metric := sample.Metric
fingerprint := metric.Fingerprint()
series, ok := s.fingerprintToSeries[fingerprint]
if !ok {
series = newStream(metric)
s.fingerprintToSeries[fingerprint] = series
for k, v := range metric {
labelPair := fmt.Sprintf("%s%s%s", k, reservedDelimiter, v)
labelPairValues := s.labelPairToFingerprints[labelPair]
labelPairValues = append(labelPairValues, fingerprint)
s.labelPairToFingerprints[labelPair] = labelPairValues
labelNameValues := s.labelNameToFingerprints[k]
labelNameValues = append(labelNameValues, fingerprint)
s.labelNameToFingerprints[k] = labelNameValues
}
}
series.add(sample)
return
}
func (s *memorySeriesStorage) GetFingerprintsForLabelSet(l model.LabelSet) (fingerprints model.Fingerprints, err error) {
sets := []utility.Set{}
for k, v := range l {
signature := fmt.Sprintf("%s%s%s", k, reservedDelimiter, v)
values := s.labelPairToFingerprints[signature]
set := utility.Set{}
for _, fingerprint := range values {
set.Add(fingerprint)
}
sets = append(sets, set)
}
setCount := len(sets)
if setCount == 0 {
return
}
base := sets[0]
for i := 1; i < setCount; i++ {
base = base.Intersection(sets[i])
}
for _, e := range base.Elements() {
fingerprint := e.(model.Fingerprint)
fingerprints = append(fingerprints, fingerprint)
}
return
}
func (s *memorySeriesStorage) GetFingerprintsForLabelName(l model.LabelName) (fingerprints model.Fingerprints, err error) {
values := s.labelNameToFingerprints[l]
fingerprints = append(fingerprints, values...)
return
}
func (s *memorySeriesStorage) GetMetricForFingerprint(f model.Fingerprint) (metric *model.Metric, err error) {
series, ok := s.fingerprintToSeries[f]
if !ok {
return
}
metric = &series.metric
return
}
// XXX: Terrible wart.
func interpolate(x1, x2 time.Time, y1, y2 float32, e time.Time) model.SampleValue {
yDelta := y2 - y1
xDelta := x2.Sub(x1)
dDt := yDelta / float32(xDelta)
offset := float32(e.Sub(x1))
return model.SampleValue(y1 + (offset * dDt))
}
func (s *memorySeriesStorage) GetValueAtTime(m model.Metric, t time.Time, p metric.StalenessPolicy) (sample *model.Sample, err error) {
fingerprint := m.Fingerprint()
series, ok := s.fingerprintToSeries[fingerprint]
if !ok {
return
}
iterator := series.values.Seek(skipListTime(t))
if iterator == nil {
return
}
foundTime := time.Time(iterator.Key().(skipListTime))
if foundTime.Equal(t) {
sample = &model.Sample{
Metric: m,
Value: iterator.Value().(model.SampleValue),
Timestamp: t,
}
return
}
if t.Sub(foundTime) > p.DeltaAllowance {
return
}
secondTime := foundTime
secondValue := iterator.Value().(model.SampleValue)
if !iterator.Previous() {
sample = &model.Sample{
Metric: m,
Value: iterator.Value().(model.SampleValue),
Timestamp: t,
}
return
}
firstTime := time.Time(iterator.Key().(skipListTime))
if t.Sub(firstTime) > p.DeltaAllowance {
return
}
if firstTime.Sub(secondTime) > p.DeltaAllowance {
return
}
firstValue := iterator.Value().(model.SampleValue)
sample = &model.Sample{
Metric: m,
Value: interpolate(firstTime, secondTime, float32(firstValue), float32(secondValue), t),
Timestamp: t,
}
return
}
func (s *memorySeriesStorage) GetBoundaryValues(m model.Metric, i model.Interval, p metric.StalenessPolicy) (first *model.Sample, second *model.Sample, err error) {
first, err = s.GetValueAtTime(m, i.OldestInclusive, p)
if err != nil {
return
} else if first == nil {
return
}
second, err = s.GetValueAtTime(m, i.NewestInclusive, p)
if err != nil {
return
} else if second == nil {
first = nil
}
return
}
func (s *memorySeriesStorage) GetRangeValues(m model.Metric, i model.Interval) (samples *model.SampleSet, err error) {
fingerprint := m.Fingerprint()
series, ok := s.fingerprintToSeries[fingerprint]
if !ok {
return
}
samples = &model.SampleSet{
Metric: m,
}
iterator := series.values.Seek(skipListTime(i.NewestInclusive))
if iterator == nil {
return
}
for {
timestamp := time.Time(iterator.Key().(skipListTime))
if timestamp.Before(i.OldestInclusive) {
break
}
samples.Values = append(samples.Values,
model.SamplePair{
Value: model.SampleValue(iterator.Value().(model.SampleValue)),
Timestamp: timestamp,
})
if !iterator.Next() {
break
}
}
// XXX: We should not explicitly sort here but rather rely on the datastore.
// This adds appreciable overhead.
if samples != nil {
sort.Sort(samples.Values)
}
return
}
func (s *memorySeriesStorage) Close() (err error) {
for fingerprint := range s.fingerprintToSeries {
delete(s.fingerprintToSeries, fingerprint)
}
for labelPair := range s.labelPairToFingerprints {
delete(s.labelPairToFingerprints, labelPair)
}
for labelName := range s.labelNameToFingerprints {
delete(s.labelNameToFingerprints, labelName)
}
return
}
func (s *memorySeriesStorage) GetAllMetricNames() (names []string, err error) {
panic("not implemented")
return
}
func (s *memorySeriesStorage) GetAllLabelNames() (names []string, err error) {
panic("not implemented")
return
}
func (s *memorySeriesStorage) GetAllLabelPairs() (pairs []model.LabelSet, err error) {
panic("not implemented")
return
}
func (s *memorySeriesStorage) GetAllMetrics() (metrics []model.LabelSet, err error) {
panic("not implemented")
return
}
func NewMemorySeriesStorage() metric.MetricPersistence {
return newMemorySeriesStorage()
}
func newMemorySeriesStorage() *memorySeriesStorage {
return &memorySeriesStorage{
fingerprintToSeries: make(map[model.Fingerprint]*stream),
labelPairToFingerprints: make(map[string]model.Fingerprints),
labelNameToFingerprints: make(map[model.LabelName]model.Fingerprints),
}
}

View file

@ -1,76 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 memory
import (
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility/test"
"io"
"io/ioutil"
"testing"
)
func testGetValueAtTime(t test.Tester) {
persistenceMaker := func() (metric.MetricPersistence, io.Closer) {
return NewMemorySeriesStorage(), ioutil.NopCloser(nil)
}
metric.GetValueAtTimeTests(persistenceMaker, t)
}
func TestGetValueAtTime(t *testing.T) {
testGetValueAtTime(t)
}
func BenchmarkGetValueAtTime(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetValueAtTime(b)
}
}
func testGetBoundaryValues(t test.Tester) {
persistenceMaker := func() (metric.MetricPersistence, io.Closer) {
return NewMemorySeriesStorage(), ioutil.NopCloser(nil)
}
metric.GetBoundaryValuesTests(persistenceMaker, t)
}
func TestGetBoundaryValues(t *testing.T) {
testGetBoundaryValues(t)
}
func BenchmarkGetBoundaryValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetBoundaryValues(b)
}
}
func testGetRangeValues(t test.Tester) {
persistenceMaker := func() (metric.MetricPersistence, io.Closer) {
return NewMemorySeriesStorage(), ioutil.NopCloser(nil)
}
metric.GetRangeValuesTests(persistenceMaker, t)
}
func TestGetRangeValues(t *testing.T) {
testGetRangeValues(t)
}
func BenchmarkGetRangeValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testGetRangeValues(b)
}
}

View file

@ -1,114 +0,0 @@
// Copyright 2013 Prometheus Team
// 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 memory
import (
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility/test"
"testing"
)
func buildTestPersistence(f func(p metric.MetricPersistence, t test.Tester)) func(t test.Tester) {
return func(t test.Tester) {
p := NewMemorySeriesStorage()
defer func() {
err := p.Close()
if err != nil {
t.Errorf("Anomaly while closing database: %q\n", err)
}
}()
f(p, t)
}
}
var testBasicLifecycle = buildTestPersistence(metric.BasicLifecycleTests)
func TestBasicLifecycle(t *testing.T) {
testBasicLifecycle(t)
}
func BenchmarkBasicLifecycle(b *testing.B) {
for i := 0; i < b.N; i++ {
testBasicLifecycle(b)
}
}
var testReadEmpty = buildTestPersistence(metric.ReadEmptyTests)
func TestReadEmpty(t *testing.T) {
testReadEmpty(t)
}
func BenchmarkReadEmpty(b *testing.B) {
for i := 0; i < b.N; i++ {
testReadEmpty(b)
}
}
var testAppendSampleAsPureSparseAppend = buildTestPersistence(metric.AppendSampleAsPureSparseAppendTests)
func TestAppendSampleAsPureSparseAppend(t *testing.T) {
testAppendSampleAsPureSparseAppend(t)
}
func BenchmarkAppendSampleAsPureSparseAppend(b *testing.B) {
for i := 0; i < b.N; i++ {
testAppendSampleAsPureSparseAppend(b)
}
}
var testAppendSampleAsSparseAppendWithReads = buildTestPersistence(metric.AppendSampleAsSparseAppendWithReadsTests)
func TestAppendSampleAsSparseAppendWithReads(t *testing.T) {
testAppendSampleAsSparseAppendWithReads(t)
}
func BenchmarkAppendSampleAsSparseAppendWithReads(b *testing.B) {
for i := 0; i < b.N; i++ {
testAppendSampleAsSparseAppendWithReads(b)
}
}
var testAppendSampleAsPureSingleEntityAppend = buildTestPersistence(metric.AppendSampleAsPureSingleEntityAppendTests)
func TestAppendSampleAsPureSingleEntityAppend(t *testing.T) {
testAppendSampleAsPureSingleEntityAppend(t)
}
func BenchmarkAppendSampleAsPureSingleEntityAppend(b *testing.B) {
for i := 0; i < b.N; i++ {
testAppendSampleAsPureSingleEntityAppend(b)
}
}
func testStochastic(t test.Tester) {
persistenceMaker := func() metric.MetricPersistence {
return NewMemorySeriesStorage()
}
metric.StochasticTests(persistenceMaker, t)
}
func TestStochastic(t *testing.T) {
testStochastic(t)
}
func BenchmarkStochastic(b *testing.B) {
for i := 0; i < b.N; i++ {
testStochastic(b)
}
}

687
storage/metric/operation.go Normal file
View file

@ -0,0 +1,687 @@
// Copyright 2013 Prometheus Team
// 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 metric
import (
"fmt"
"github.com/prometheus/prometheus/model"
"math"
"sort"
"time"
)
// Encapsulates a primitive query operation.
type op interface {
// The time at which this operation starts.
StartsAt() time.Time
// Extract samples from stream of values and advance operation time.
ExtractSamples(in []model.SamplePair) (out []model.SamplePair)
// Get current operation time or nil if no subsequent work associated with
// this operator remains.
CurrentTime() *time.Time
// GreedierThan indicates whether this present operation should take
// precedence over the other operation due to greediness.
//
// A critical assumption is that this operator and the other occur at the
// same time: this.StartsAt().Equal(op.StartsAt()).
GreedierThan(op) bool
}
// Provides a sortable collection of operations.
type ops []op
func (o ops) Len() int {
return len(o)
}
// startsAtSort implements the sorting protocol and allows operator to be sorted
// in chronological order by when they start.
type startsAtSort struct {
ops
}
func (s startsAtSort) Less(i, j int) bool {
return s.ops[i].StartsAt().Before(s.ops[j].StartsAt())
}
func (o ops) Swap(i, j int) {
o[i], o[j] = o[j], o[i]
}
// Encapsulates getting values at or adjacent to a specific time.
type getValuesAtTimeOp struct {
time time.Time
consumed bool
}
func (o getValuesAtTimeOp) String() string {
return fmt.Sprintf("getValuesAtTimeOp at %s", o.time)
}
func (g getValuesAtTimeOp) StartsAt() time.Time {
return g.time
}
func (g *getValuesAtTimeOp) ExtractSamples(in []model.SamplePair) (out []model.SamplePair) {
if len(in) == 0 {
return
}
out = extractValuesAroundTime(g.time, in)
g.consumed = true
return
}
func (g getValuesAtTimeOp) GreedierThan(op op) (superior bool) {
switch op.(type) {
case *getValuesAtTimeOp:
superior = true
case durationOperator:
superior = false
default:
panic("unknown operation")
}
return
}
// extractValuesAroundTime searches for the provided time in the list of
// available samples and emits a slice containing the data points that
// are adjacent to it.
//
// An assumption of this is that the provided samples are already sorted!
func extractValuesAroundTime(t time.Time, in []model.SamplePair) (out []model.SamplePair) {
i := sort.Search(len(in), func(i int) bool {
return !in[i].Timestamp.Before(t)
})
if i == len(in) {
// Target time is past the end, return only the last sample.
out = in[len(in)-1:]
} else {
if in[i].Timestamp.Equal(t) && len(in) > i+1 {
// We hit exactly the current sample time. Very unlikely in practice.
// Return only the current sample.
out = append(out, in[i])
} else {
if i == 0 {
// We hit before the first sample time. Return only the first sample.
out = append(out, in[0:1]...)
} else {
// We hit between two samples. Return both surrounding samples.
out = append(out, in[i-1:i+1]...)
}
}
}
return
}
func (g getValuesAtTimeOp) CurrentTime() (currentTime *time.Time) {
if !g.consumed {
currentTime = &g.time
}
return
}
// Encapsulates getting values at a given interval over a duration.
type getValuesAtIntervalOp struct {
from time.Time
through time.Time
interval time.Duration
}
func (o getValuesAtIntervalOp) String() string {
return fmt.Sprintf("getValuesAtIntervalOp from %s each %s through %s", o.from, o.interval, o.through)
}
func (g getValuesAtIntervalOp) StartsAt() time.Time {
return g.from
}
func (g getValuesAtIntervalOp) Through() time.Time {
return g.through
}
func (g *getValuesAtIntervalOp) ExtractSamples(in []model.SamplePair) (out []model.SamplePair) {
if len(in) == 0 {
return
}
lastChunkTime := in[len(in)-1].Timestamp
for {
out = extractValuesAroundTime(g.from, in)
g.from = g.from.Add(g.interval)
if g.from.After(lastChunkTime) {
break
}
if g.from.After(g.through) {
break
}
}
return
}
func (g getValuesAtIntervalOp) CurrentTime() (currentTime *time.Time) {
if g.from.After(g.through) {
return
}
return &g.from
}
func (g getValuesAtIntervalOp) GreedierThan(op op) (superior bool) {
switch o := op.(type) {
case *getValuesAtTimeOp:
superior = true
case durationOperator:
superior = g.Through().After(o.Through())
default:
panic("unknown operation")
}
return
}
type getValuesAlongRangeOp struct {
from time.Time
through time.Time
}
func (o getValuesAlongRangeOp) String() string {
return fmt.Sprintf("getValuesAlongRangeOp from %s through %s", o.from, o.through)
}
func (g getValuesAlongRangeOp) StartsAt() time.Time {
return g.from
}
func (g getValuesAlongRangeOp) Through() time.Time {
return g.through
}
func (g *getValuesAlongRangeOp) ExtractSamples(in []model.SamplePair) (out []model.SamplePair) {
if len(in) == 0 {
return
}
// Find the first sample where time >= g.from.
firstIdx := sort.Search(len(in), func(i int) bool {
return !in[i].Timestamp.Before(g.from)
})
if firstIdx == len(in) {
// No samples at or after operator start time.
return
}
// Find the first sample where time > g.through.
lastIdx := sort.Search(len(in), func(i int) bool {
return in[i].Timestamp.After(g.through)
})
if lastIdx == firstIdx {
return
}
lastSampleTime := in[lastIdx-1].Timestamp
g.from = lastSampleTime.Add(time.Duration(1))
return in[firstIdx:lastIdx]
}
func (g getValuesAlongRangeOp) CurrentTime() (currentTime *time.Time) {
if g.from.After(g.through) {
return
}
return &g.from
}
func (g getValuesAlongRangeOp) GreedierThan(op op) (superior bool) {
switch o := op.(type) {
case *getValuesAtTimeOp:
superior = true
case durationOperator:
superior = g.Through().After(o.Through())
default:
panic("unknown operation")
}
return
}
// Provides a collection of getMetricRangeOperation.
type getMetricRangeOperations []*getValuesAlongRangeOp
func (s getMetricRangeOperations) Len() int {
return len(s)
}
func (s getMetricRangeOperations) Swap(i, j int) {
s[i], s[j] = s[j], s[i]
}
// Sorts getMetricRangeOperation according to duration in descending order.
type rangeDurationSorter struct {
getMetricRangeOperations
}
func (s rangeDurationSorter) Less(i, j int) bool {
l := s.getMetricRangeOperations[i]
r := s.getMetricRangeOperations[j]
return !l.through.Before(r.through)
}
// Encapsulates a general operation that occurs over a duration.
type durationOperator interface {
op
Through() time.Time
}
// greedinessSort sorts the operations in descending order by level of
// greediness.
type greedinessSort struct {
ops
}
func (g greedinessSort) Less(i, j int) bool {
return g.ops[i].GreedierThan(g.ops[j])
}
// Contains getValuesAtIntervalOp operations.
type getValuesAtIntervalOps []*getValuesAtIntervalOp
func (s getValuesAtIntervalOps) Len() int {
return len(s)
}
func (s getValuesAtIntervalOps) Swap(i, j int) {
s[i], s[j] = s[j], s[i]
}
// Sorts durationOperator by the operation's duration in descending order.
type intervalDurationSorter struct {
getValuesAtIntervalOps
}
func (s intervalDurationSorter) Less(i, j int) bool {
l := s.getValuesAtIntervalOps[i]
r := s.getValuesAtIntervalOps[j]
return !l.through.Before(r.through)
}
// Sorts getValuesAtIntervalOp operations in ascending order by their
// frequency.
type frequencySorter struct {
getValuesAtIntervalOps
}
func (s frequencySorter) Less(i, j int) bool {
l := s.getValuesAtIntervalOps[i]
r := s.getValuesAtIntervalOps[j]
return l.interval < r.interval
}
// Selects and returns all operations that are getValuesAtIntervalOp operations
// in a map whereby the operation interval is the key and the value are the
// operations sorted by respective level of greediness.
func collectIntervals(o ops) (intervals map[time.Duration]ops) {
intervals = make(map[time.Duration]ops)
for _, operation := range o {
switch t := operation.(type) {
case *getValuesAtIntervalOp:
operations, _ := intervals[t.interval]
operations = append(operations, t)
intervals[t.interval] = operations
}
}
return
}
// Selects and returns all operations that are getValuesAlongRangeOp operations.
func collectRanges(ops ops) (ranges ops) {
for _, operation := range ops {
switch t := operation.(type) {
case *getValuesAlongRangeOp:
ranges = append(ranges, t)
}
}
return
}
// optimizeForward iteratively scans operations and peeks ahead to subsequent
// ones to find candidates that can either be removed or truncated through
// simplification. For instance, if a range query happens to overlap a get-a-
// value-at-a-certain-point-request, the range query should flatten and subsume
// the other.
func optimizeForward(pending ops) (out ops) {
if len(pending) == 0 {
return
}
var (
head op = pending[0]
tail ops
)
pending = pending[1:len(pending)]
switch t := head.(type) {
case *getValuesAtTimeOp:
out = ops{head}
case *getValuesAtIntervalOp:
// If the last value was a scan at a given frequency along an interval,
// several optimizations may exist.
for _, peekOperation := range pending {
if peekOperation.StartsAt().After(t.Through()) {
break
}
// If the type is not a range request, we can't do anything.
switch next := peekOperation.(type) {
case *getValuesAlongRangeOp:
if !next.GreedierThan(t) {
var (
before = getValuesAtIntervalOp(*t)
after = getValuesAtIntervalOp(*t)
)
before.through = next.from
// Truncate the get value at interval request if a range request cuts
// it off somewhere.
var (
from = next.from
)
for !from.After(next.through) {
from = from.Add(t.interval)
}
after.from = from
pending = append(ops{&before, &after}, pending...)
sort.Sort(startsAtSort{pending})
return optimizeForward(pending)
}
}
}
case *getValuesAlongRangeOp:
for _, peekOperation := range pending {
if peekOperation.StartsAt().After(t.Through()) {
break
}
switch next := peekOperation.(type) {
// All values at a specific time may be elided into the range query.
case *getValuesAtTimeOp:
pending = pending[1:len(pending)]
continue
case *getValuesAlongRangeOp:
// Range queries should be concatenated if they overlap.
if next.GreedierThan(t) {
next.from = t.from
return optimizeForward(pending)
} else {
pending = pending[1:len(pending)]
}
case *getValuesAtIntervalOp:
pending = pending[1:len(pending)]
if next.GreedierThan(t) {
var (
nextStart = next.from
)
for !nextStart.After(next.through) {
nextStart = nextStart.Add(next.interval)
}
next.from = nextStart
tail = append(ops{next}, pending...)
}
default:
panic("unknown operation type")
}
}
default:
panic("unknown operation type")
}
// Strictly needed?
sort.Sort(startsAtSort{pending})
tail = optimizeForward(pending)
return append(ops{head}, tail...)
}
// selectQueriesForTime chooses all subsequent operations from the slice that
// have the same start time as the provided time and emits them.
func selectQueriesForTime(time time.Time, queries ops) (out ops) {
if len(queries) == 0 {
return
}
if !queries[0].StartsAt().Equal(time) {
return
}
out = append(out, queries[0])
tail := selectQueriesForTime(time, queries[1:len(queries)])
return append(out, tail...)
}
// selectGreediestRange scans through the various getValuesAlongRangeOp
// operations and emits the one that is the greediest.
func selectGreediestRange(in ops) (o durationOperator) {
if len(in) == 0 {
return
}
sort.Sort(greedinessSort{in})
o = in[0].(*getValuesAlongRangeOp)
return
}
// selectGreediestIntervals scans through the various getValuesAtIntervalOp
// operations and emits a map of the greediest operation keyed by its start
// time.
func selectGreediestIntervals(in map[time.Duration]ops) (out map[time.Duration]durationOperator) {
if len(in) == 0 {
return
}
out = make(map[time.Duration]durationOperator)
for i, ops := range in {
sort.Sort(greedinessSort{ops})
out[i] = ops[0].(*getValuesAtIntervalOp)
}
return
}
// rewriteForGreediestRange rewrites the current pending operation such that the
// greediest range operation takes precedence over all other operators in this
// time group.
//
// Between two range operations O1 and O2, they both start at the same time;
// however, O2 extends for a longer duration than O1. Thusly, O1 should be
// deleted with O2.
//
// O1------>|
// T1 T4
//
// O2------------>|
// T1 T7
//
// Thusly O1 can be squashed into O2 without having side-effects.
func rewriteForGreediestRange(greediestRange durationOperator) ops {
return ops{greediestRange}
}
// rewriteForGreediestInterval rewrites teh current pending interval operations
// such that the interval operation with the smallest collection period is
// invoked first, for it will skip around the soonest of any of the remaining
// other operators.
//
// Between two interval operations O1 and O2, they both start at the same time;
// however, O2's period is shorter than O1, meaning it will sample far more
// frequently from the underlying time series. Thusly, O2 should start before
// O1.
//
// O1---->|---->|
// T1 T5
//
// O2->|->|->|->|
// T1 T5
//
// The rewriter presently does not scan and compact for common divisors in the
// periods, though this may be nice to have. For instance, if O1 has a period
// of 2 and O2 has a period of 4, O2 would be dropped for O1 would implicitly
// cover its period.
func rewriteForGreediestInterval(greediestIntervals map[time.Duration]durationOperator) ops {
var (
memo getValuesAtIntervalOps
out ops
)
for _, o := range greediestIntervals {
memo = append(memo, o.(*getValuesAtIntervalOp))
}
sort.Sort(frequencySorter{memo})
for _, o := range memo {
out = append(out, o)
}
return out
}
// rewriteForRangeAndInterval examines the existence of a range operation and a
// set of interval operations that start at the same time and deletes all
// interval operations that start and finish before the range operation
// completes and rewrites all interval operations that continue longer than
// the range operation to start at the next best increment after the range.
//
// Assume that we have a range operator O1 and two interval operations O2 and
// O3. O2 and O3 have the same period (i.e., sampling interval), but O2
// terminates before O1 and O3 continue beyond O1.
//
// O1------------>|
// T1------------T7
//
// O2-->|-->|-->|
// T1----------T6
//
// O3-->|-->|-->|-->|-->|
// T1------------------T10
//
// This scenario will be rewritten such that O2 is deleted and O3 is truncated
// from T1 through T7, and O3's new starting time is at T7 and runs through T10:
//
// O1------------>|
// T1------------T7
//
// O2>|-->|
// T7---T10
//
// All rewritten interval operators will respect their original start time
// multipliers.
func rewriteForRangeAndInterval(greediestRange durationOperator, greediestIntervals map[time.Duration]durationOperator) (out ops) {
out = append(out, greediestRange)
for _, op := range greediestIntervals {
if !op.GreedierThan(greediestRange) {
continue
}
// The range operation does not exceed interval. Leave a snippet of
// interval.
var (
truncated = op.(*getValuesAtIntervalOp)
newIntervalOperation getValuesAtIntervalOp
// Refactor
remainingSlice = greediestRange.Through().Sub(greediestRange.StartsAt()) / time.Second
nextIntervalPoint = time.Duration(math.Ceil(float64(remainingSlice)/float64(truncated.interval)) * float64(truncated.interval/time.Second))
nextStart = greediestRange.Through().Add(nextIntervalPoint)
)
newIntervalOperation.from = nextStart
newIntervalOperation.interval = truncated.interval
newIntervalOperation.through = truncated.Through()
// Added back to the pending because additional curation could be
// necessary.
out = append(out, &newIntervalOperation)
}
return
}
// Flattens queries that occur at the same time according to duration and level
// of greed. Consult the various rewriter functions for their respective modes
// of operation.
func optimizeTimeGroup(group ops) (out ops) {
var (
greediestRange = selectGreediestRange(collectRanges(group))
greediestIntervals = selectGreediestIntervals(collectIntervals(group))
containsRange = greediestRange != nil
containsInterval = len(greediestIntervals) > 0
)
switch {
case containsRange && !containsInterval:
out = rewriteForGreediestRange(greediestRange)
case !containsRange && containsInterval:
out = rewriteForGreediestInterval(greediestIntervals)
case containsRange && containsInterval:
out = rewriteForRangeAndInterval(greediestRange, greediestIntervals)
default:
// Operation is OK as-is.
out = append(out, group[0])
}
return
}
// Flattens all groups of time according to greed.
func optimizeTimeGroups(pending ops) (out ops) {
if len(pending) == 0 {
return
}
sort.Sort(startsAtSort{pending})
var (
nextOperation = pending[0]
groupedQueries = selectQueriesForTime(nextOperation.StartsAt(), pending)
)
out = optimizeTimeGroup(groupedQueries)
pending = pending[len(groupedQueries):len(pending)]
tail := optimizeTimeGroups(pending)
return append(out, tail...)
}
func optimize(pending ops) (out ops) {
return optimizeForward(optimizeTimeGroups(pending))
}

File diff suppressed because it is too large Load diff

View file

@ -16,6 +16,7 @@ package metric
import (
"github.com/prometheus/prometheus/model"
"github.com/prometheus/prometheus/utility/test"
"testing"
"time"
)
@ -35,7 +36,7 @@ func GetFingerprintsForLabelSetUsesAndForLabelMatchingTests(p MetricPersistence,
m[model.LabelName(k)] = model.LabelValue(v)
}
appendSample(p, model.Sample{
testAppendSample(p, model.Sample{
Value: model.SampleValue(0.0),
Timestamp: time.Now(),
Metric: m,
@ -56,3 +57,29 @@ func GetFingerprintsForLabelSetUsesAndForLabelMatchingTests(p MetricPersistence,
t.Errorf("did not get a single metric as is expected, got %s", fingerprints)
}
}
// Test Definitions Below
var testLevelDBGetFingerprintsForLabelSetUsesAndForLabelMatching = buildLevelDBTestPersistence("get_fingerprints_for_labelset_uses_and_for_label_matching", GetFingerprintsForLabelSetUsesAndForLabelMatchingTests)
func TestLevelDBGetFingerprintsForLabelSetUsesAndForLabelMatching(t *testing.T) {
testLevelDBGetFingerprintsForLabelSetUsesAndForLabelMatching(t)
}
func BenchmarkLevelDBGetFingerprintsForLabelSetUsesAndForLabelMatching(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBGetFingerprintsForLabelSetUsesAndForLabelMatching(b)
}
}
var testMemoryGetFingerprintsForLabelSetUsesAndForLabelMatching = buildMemoryTestPersistence(GetFingerprintsForLabelSetUsesAndForLabelMatchingTests)
func TestMemoryGetFingerprintsForLabelSetUsesAndForLabelMatching(t *testing.T) {
testMemoryGetFingerprintsForLabelSetUsesAndForLabelMatching(t)
}
func BenchmarkMemoryGetFingerprintsForLabelSetUsesAndLabelMatching(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryGetFingerprintsForLabelSetUsesAndForLabelMatching(b)
}
}

View file

@ -17,6 +17,8 @@ import (
"github.com/prometheus/prometheus/model"
"github.com/prometheus/prometheus/utility/test"
"io"
"io/ioutil"
"testing"
"time"
)
@ -569,7 +571,7 @@ func GetValueAtTimeTests(persistenceMaker func() (MetricPersistence, io.Closer),
}
for _, value := range context.values {
appendSample(p, model.Sample{
testAppendSample(p, model.Sample{
Value: model.SampleValue(value.value),
Timestamp: time.Date(value.year, value.month, value.day, value.hour, 0, 0, 0, time.UTC),
Metric: m,
@ -583,7 +585,7 @@ func GetValueAtTimeTests(persistenceMaker func() (MetricPersistence, io.Closer),
DeltaAllowance: input.staleness,
}
actual, err := p.GetValueAtTime(m, time, sp)
actual, err := p.GetValueAtTime(model.NewFingerprintFromMetric(m), time, sp)
if err != nil {
t.Fatalf("%d.%d(%s). Could not query for value: %q\n", i, j, behavior.name, err)
}
@ -1014,7 +1016,7 @@ func GetBoundaryValuesTests(persistenceMaker func() (MetricPersistence, io.Close
}
for _, value := range context.values {
appendSample(p, model.Sample{
testAppendSample(p, model.Sample{
Value: model.SampleValue(value.value),
Timestamp: time.Date(value.year, value.month, value.day, value.hour, 0, 0, 0, time.UTC),
Metric: m,
@ -1033,7 +1035,7 @@ func GetBoundaryValuesTests(persistenceMaker func() (MetricPersistence, io.Close
DeltaAllowance: input.staleness,
}
openValue, endValue, err := p.GetBoundaryValues(m, interval, po)
openValue, endValue, err := p.GetBoundaryValues(model.NewFingerprintFromMetric(m), interval, po)
if err != nil {
t.Fatalf("%d.%d(%s). Could not query for value: %q\n", i, j, behavior.name, err)
}
@ -1371,7 +1373,7 @@ func GetRangeValuesTests(persistenceMaker func() (MetricPersistence, io.Closer),
}
for _, value := range context.values {
appendSample(p, model.Sample{
testAppendSample(p, model.Sample{
Value: model.SampleValue(value.value),
Timestamp: time.Date(value.year, value.month, value.day, value.hour, 0, 0, 0, time.UTC),
Metric: m,
@ -1387,7 +1389,7 @@ func GetRangeValuesTests(persistenceMaker func() (MetricPersistence, io.Closer),
NewestInclusive: end,
}
values, err := p.GetRangeValues(m, in)
values, err := p.GetRangeValues(model.NewFingerprintFromMetric(m), in)
if err != nil {
t.Fatalf("%d.%d(%s). Could not query for value: %q\n", i, j, behavior.name, err)
}
@ -1434,3 +1436,106 @@ func GetRangeValuesTests(persistenceMaker func() (MetricPersistence, io.Closer),
}()
}
}
// Test Definitions Follow
func testLevelDBGetValueAtTime(t test.Tester) {
persistenceMaker := buildLevelDBTestPersistencesMaker("get_value_at_time", t)
GetValueAtTimeTests(persistenceMaker, t)
}
func TestLevelDBGetValueAtTime(t *testing.T) {
testLevelDBGetValueAtTime(t)
}
func BenchmarkLevelDBGetValueAtTime(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBGetValueAtTime(b)
}
}
func testLevelDBGetBoundaryValues(t test.Tester) {
persistenceMaker := buildLevelDBTestPersistencesMaker("get_boundary_values", t)
GetBoundaryValuesTests(persistenceMaker, t)
}
func TestLevelDBGetBoundaryValues(t *testing.T) {
testLevelDBGetBoundaryValues(t)
}
func BenchmarkLevelDBGetBoundaryValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBGetBoundaryValues(b)
}
}
func testLevelDBGetRangeValues(t test.Tester) {
persistenceMaker := buildLevelDBTestPersistencesMaker("get_range_values", t)
GetRangeValuesTests(persistenceMaker, t)
}
func TestLevelDBGetRangeValues(t *testing.T) {
testLevelDBGetRangeValues(t)
}
func BenchmarkLevelDBGetRangeValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBGetRangeValues(b)
}
}
func testMemoryGetValueAtTime(t test.Tester) {
persistenceMaker := func() (MetricPersistence, io.Closer) {
return NewMemorySeriesStorage(), ioutil.NopCloser(nil)
}
GetValueAtTimeTests(persistenceMaker, t)
}
func TestMemoryGetValueAtTime(t *testing.T) {
testMemoryGetValueAtTime(t)
}
func BenchmarkMemoryGetValueAtTime(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryGetValueAtTime(b)
}
}
func testMemoryGetBoundaryValues(t test.Tester) {
persistenceMaker := func() (MetricPersistence, io.Closer) {
return NewMemorySeriesStorage(), ioutil.NopCloser(nil)
}
GetBoundaryValuesTests(persistenceMaker, t)
}
func TestMemoryGetBoundaryValues(t *testing.T) {
testMemoryGetBoundaryValues(t)
}
func BenchmarkMemoryGetBoundaryValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryGetBoundaryValues(b)
}
}
func testMemoryGetRangeValues(t test.Tester) {
persistenceMaker := func() (MetricPersistence, io.Closer) {
return NewMemorySeriesStorage(), ioutil.NopCloser(nil)
}
GetRangeValuesTests(persistenceMaker, t)
}
func TestMemoryGetRangeValues(t *testing.T) {
testMemoryGetRangeValues(t)
}
func BenchmarkMemoryGetRangeValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryGetRangeValues(b)
}
}

54
storage/metric/scanjob.go Normal file
View file

@ -0,0 +1,54 @@
// Copyright 2013 Prometheus Team
// 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 metric
import (
"bytes"
"fmt"
"github.com/prometheus/prometheus/model"
)
// scanJob models a range of queries.
type scanJob struct {
fingerprint model.Fingerprint
operations ops
}
func (s scanJob) String() string {
buffer := &bytes.Buffer{}
fmt.Fprintf(buffer, "Scan Job { fingerprint=%s ", s.fingerprint)
fmt.Fprintf(buffer, " with %d operations [", len(s.operations))
for _, operation := range s.operations {
fmt.Fprintf(buffer, "%s", operation)
}
fmt.Fprintf(buffer, "] }")
return buffer.String()
}
type scanJobs []scanJob
func (s scanJobs) Len() int {
return len(s)
}
func (s scanJobs) Less(i, j int) (less bool) {
less = s[i].fingerprint.Less(s[j].fingerprint)
return
}
func (s scanJobs) Swap(i, j int) {
s[i], s[j] = s[j], s[i]
}

View file

@ -17,8 +17,10 @@ import (
"fmt"
"github.com/prometheus/prometheus/model"
"github.com/prometheus/prometheus/utility/test"
"io/ioutil"
"math"
"math/rand"
"testing"
"testing/quick"
"time"
)
@ -154,7 +156,7 @@ func AppendSampleAsSparseAppendWithReadsTests(p MetricPersistence, t test.Tester
return
}
if len(fingerprints) != 1 {
t.Error("expected fingerprint count of %d, got %d", 1, len(fingerprints))
t.Errorf("expected fingerprint count of %d, got %d", 1, len(fingerprints))
return
}
@ -411,7 +413,7 @@ func StochasticTests(persistenceMaker func() MetricPersistence, t test.Tester) {
NewestInclusive: time.Unix(end, 0),
}
samples, err := p.GetRangeValues(metric, interval)
samples, err := p.GetRangeValues(model.NewFingerprintFromMetric(metric), interval)
if err != nil {
t.Error(err)
return
@ -431,3 +433,171 @@ func StochasticTests(persistenceMaker func() MetricPersistence, t test.Tester) {
t.Error(err)
}
}
// Test Definitions Follow
var testLevelDBBasicLifecycle = buildLevelDBTestPersistence("basic_lifecycle", BasicLifecycleTests)
func TestLevelDBBasicLifecycle(t *testing.T) {
testLevelDBBasicLifecycle(t)
}
func BenchmarkLevelDBBasicLifecycle(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBBasicLifecycle(b)
}
}
var testLevelDBReadEmpty = buildLevelDBTestPersistence("read_empty", ReadEmptyTests)
func TestLevelDBReadEmpty(t *testing.T) {
testLevelDBReadEmpty(t)
}
func BenchmarkLevelDBReadEmpty(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBReadEmpty(b)
}
}
var testLevelDBAppendSampleAsPureSparseAppend = buildLevelDBTestPersistence("append_sample_as_pure_sparse_append", AppendSampleAsPureSparseAppendTests)
func TestLevelDBAppendSampleAsPureSparseAppend(t *testing.T) {
testLevelDBAppendSampleAsPureSparseAppend(t)
}
func BenchmarkLevelDBAppendSampleAsPureSparseAppend(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBAppendSampleAsPureSparseAppend(b)
}
}
var testLevelDBAppendSampleAsSparseAppendWithReads = buildLevelDBTestPersistence("append_sample_as_sparse_append_with_reads", AppendSampleAsSparseAppendWithReadsTests)
func TestLevelDBAppendSampleAsSparseAppendWithReads(t *testing.T) {
testLevelDBAppendSampleAsSparseAppendWithReads(t)
}
func BenchmarkLevelDBAppendSampleAsSparseAppendWithReads(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBAppendSampleAsSparseAppendWithReads(b)
}
}
var testLevelDBAppendSampleAsPureSingleEntityAppend = buildLevelDBTestPersistence("append_sample_as_pure_single_entity_append", AppendSampleAsPureSingleEntityAppendTests)
func TestLevelDBAppendSampleAsPureSingleEntityAppend(t *testing.T) {
testLevelDBAppendSampleAsPureSingleEntityAppend(t)
}
func BenchmarkLevelDBAppendSampleAsPureSingleEntityAppend(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBAppendSampleAsPureSingleEntityAppend(b)
}
}
func testLevelDBStochastic(t test.Tester) {
persistenceMaker := func() MetricPersistence {
temporaryDirectory, err := ioutil.TempDir("", "test_leveldb_stochastic")
if err != nil {
t.Errorf("Could not create test directory: %q\n", err)
}
p, err := NewLevelDBMetricPersistence(temporaryDirectory)
if err != nil {
t.Errorf("Could not start up LevelDB: %q\n", err)
}
return p
}
StochasticTests(persistenceMaker, t)
}
func TestLevelDBStochastic(t *testing.T) {
testLevelDBStochastic(t)
}
func BenchmarkLevelDBStochastic(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBStochastic(b)
}
}
var testMemoryBasicLifecycle = buildMemoryTestPersistence(BasicLifecycleTests)
func TestMemoryBasicLifecycle(t *testing.T) {
testMemoryBasicLifecycle(t)
}
func BenchmarkMemoryBasicLifecycle(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryBasicLifecycle(b)
}
}
var testMemoryReadEmpty = buildMemoryTestPersistence(ReadEmptyTests)
func TestMemoryReadEmpty(t *testing.T) {
testMemoryReadEmpty(t)
}
func BenchmarkMemoryReadEmpty(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryReadEmpty(b)
}
}
var testMemoryAppendSampleAsPureSparseAppend = buildMemoryTestPersistence(AppendSampleAsPureSparseAppendTests)
func TestMemoryAppendSampleAsPureSparseAppend(t *testing.T) {
testMemoryAppendSampleAsPureSparseAppend(t)
}
func BenchmarkMemoryAppendSampleAsPureSparseAppend(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryAppendSampleAsPureSparseAppend(b)
}
}
var testMemoryAppendSampleAsSparseAppendWithReads = buildMemoryTestPersistence(AppendSampleAsSparseAppendWithReadsTests)
func TestMemoryAppendSampleAsSparseAppendWithReads(t *testing.T) {
testMemoryAppendSampleAsSparseAppendWithReads(t)
}
func BenchmarkMemoryAppendSampleAsSparseAppendWithReads(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryAppendSampleAsSparseAppendWithReads(b)
}
}
var testMemoryAppendSampleAsPureSingleEntityAppend = buildMemoryTestPersistence(AppendSampleAsPureSingleEntityAppendTests)
func TestMemoryAppendSampleAsPureSingleEntityAppend(t *testing.T) {
testMemoryAppendSampleAsPureSingleEntityAppend(t)
}
func BenchmarkMemoryAppendSampleAsPureSingleEntityAppend(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryAppendSampleAsPureSingleEntityAppend(b)
}
}
func testMemoryStochastic(t test.Tester) {
persistenceMaker := func() MetricPersistence {
return NewMemorySeriesStorage()
}
StochasticTests(persistenceMaker, t)
}
func TestMemoryStochastic(t *testing.T) {
testMemoryStochastic(t)
}
func BenchmarkMemoryStochastic(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryStochastic(b)
}
}

View file

@ -14,13 +14,99 @@
package metric
import (
"fmt"
"github.com/prometheus/prometheus/model"
"github.com/prometheus/prometheus/utility/test"
"io"
"io/ioutil"
"os"
"time"
)
func appendSample(p MetricPersistence, s model.Sample, t test.Tester) {
var (
testInstant = time.Time{}
)
func testAppendSample(p MetricPersistence, s model.Sample, t test.Tester) {
err := p.AppendSample(s)
if err != nil {
t.Fatal(err)
}
}
type purger struct {
path string
}
func (p purger) Close() error {
return os.RemoveAll(p.path)
}
func buildLevelDBTestPersistencesMaker(name string, t test.Tester) func() (MetricPersistence, io.Closer) {
return func() (MetricPersistence, io.Closer) {
temporaryDirectory, err := ioutil.TempDir("", "get_value_at_time")
if err != nil {
t.Errorf("Could not create test directory: %q\n", err)
}
p, err := NewLevelDBMetricPersistence(temporaryDirectory)
if err != nil {
t.Errorf("Could not start up LevelDB: %q\n", err)
}
purger := purger{
path: temporaryDirectory,
}
return p, purger
}
}
func buildLevelDBTestPersistence(name string, f func(p MetricPersistence, t test.Tester)) func(t test.Tester) {
return func(t test.Tester) {
temporaryDirectory, err := ioutil.TempDir("", fmt.Sprintf("test_leveldb_%s", name))
if err != nil {
t.Errorf("Could not create test directory: %q\n", err)
return
}
defer func() {
err := os.RemoveAll(temporaryDirectory)
if err != nil {
t.Errorf("Could not remove temporary directory: %q\n", err)
}
}()
p, err := NewLevelDBMetricPersistence(temporaryDirectory)
if err != nil {
t.Errorf("Could not create LevelDB Metric Persistence: %q\n", err)
}
defer func() {
err := p.Close()
if err != nil {
t.Errorf("Anomaly while closing database: %q\n", err)
}
}()
f(p, t)
}
}
func buildMemoryTestPersistence(f func(p MetricPersistence, t test.Tester)) func(t test.Tester) {
return func(t test.Tester) {
p := NewMemorySeriesStorage()
defer func() {
err := p.Close()
if err != nil {
t.Errorf("Anomaly while closing database: %q\n", err)
}
}()
f(p, t)
}
}

519
storage/metric/tiered.go Normal file
View file

@ -0,0 +1,519 @@
// Copyright 2013 Prometheus Team
// 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 metric
import (
"fmt"
"github.com/jmhodges/levigo"
"github.com/prometheus/prometheus/coding"
"github.com/prometheus/prometheus/coding/indexable"
"github.com/prometheus/prometheus/model"
dto "github.com/prometheus/prometheus/model/generated"
"github.com/prometheus/prometheus/storage"
"sort"
"sync"
"time"
)
// tieredStorage both persists samples and generates materialized views for
// queries.
type tieredStorage struct {
appendToDiskQueue chan model.Sample
appendToMemoryQueue chan model.Sample
diskFrontier *diskFrontier
diskStorage *LevelDBMetricPersistence
draining chan bool
flushMemoryInterval time.Duration
memoryArena memorySeriesStorage
memoryTTL time.Duration
mutex sync.Mutex
viewQueue chan viewJob
writeMemoryInterval time.Duration
}
// viewJob encapsulates a request to extract sample values from the datastore.
type viewJob struct {
builder ViewRequestBuilder
output chan View
err chan error
}
// Provides a unified means for batch appending values into the datastore along
// with querying for values in an efficient way.
type Storage interface {
// Enqueues a Sample for storage.
AppendSample(model.Sample) error
// Enqueus a ViewRequestBuilder for materialization, subject to a timeout.
MakeView(request ViewRequestBuilder, timeout time.Duration) (View, error)
// Starts serving requests.
Serve()
// Stops the storage subsystem, flushing all pending operations.
Drain()
Flush()
Close()
}
func NewTieredStorage(appendToMemoryQueueDepth, appendToDiskQueueDepth, viewQueueDepth uint, flushMemoryInterval, writeMemoryInterval, memoryTTL time.Duration, root string) Storage {
diskStorage, err := NewLevelDBMetricPersistence(root)
if err != nil {
panic(err)
}
return &tieredStorage{
appendToDiskQueue: make(chan model.Sample, appendToDiskQueueDepth),
appendToMemoryQueue: make(chan model.Sample, appendToMemoryQueueDepth),
diskStorage: diskStorage,
draining: make(chan bool),
flushMemoryInterval: flushMemoryInterval,
memoryArena: NewMemorySeriesStorage(),
memoryTTL: memoryTTL,
viewQueue: make(chan viewJob, viewQueueDepth),
writeMemoryInterval: writeMemoryInterval,
}
}
func (t *tieredStorage) AppendSample(s model.Sample) (err error) {
if len(t.draining) > 0 {
return fmt.Errorf("Storage is in the process of draining.")
}
t.appendToMemoryQueue <- s
return
}
func (t *tieredStorage) Drain() {
if len(t.draining) == 0 {
t.draining <- true
}
}
func (t *tieredStorage) MakeView(builder ViewRequestBuilder, deadline time.Duration) (view View, err error) {
if len(t.draining) > 0 {
err = fmt.Errorf("Storage is in the process of draining.")
return
}
result := make(chan View)
errChan := make(chan error)
t.viewQueue <- viewJob{
builder: builder,
output: result,
err: errChan,
}
select {
case value := <-result:
view = value
case err = <-errChan:
return
case <-time.After(deadline):
err = fmt.Errorf("MakeView timed out after %s.", deadline)
}
return
}
func (t *tieredStorage) rebuildDiskFrontier() (err error) {
begin := time.Now()
defer func() {
duration := time.Since(begin)
recordOutcome(duration, err, map[string]string{operation: appendSample, result: success}, map[string]string{operation: rebuildDiskFrontier, result: failure})
}()
i, closer, err := t.diskStorage.metricSamples.GetIterator()
if closer != nil {
defer closer.Close()
}
if err != nil {
panic(err)
}
t.diskFrontier, err = newDiskFrontier(i)
if err != nil {
panic(err)
}
return
}
func (t *tieredStorage) Serve() {
var (
flushMemoryTicker = time.Tick(t.flushMemoryInterval)
writeMemoryTicker = time.Tick(t.writeMemoryInterval)
)
for {
t.reportQueues()
select {
case <-writeMemoryTicker:
t.writeMemory()
case <-flushMemoryTicker:
t.flushMemory()
case viewRequest := <-t.viewQueue:
t.renderView(viewRequest)
case <-t.draining:
t.flush()
break
}
}
}
func (t *tieredStorage) reportQueues() {
queueSizes.Set(map[string]string{"queue": "append_to_disk", "facet": "occupancy"}, float64(len(t.appendToDiskQueue)))
queueSizes.Set(map[string]string{"queue": "append_to_disk", "facet": "capacity"}, float64(cap(t.appendToDiskQueue)))
queueSizes.Set(map[string]string{"queue": "append_to_memory", "facet": "occupancy"}, float64(len(t.appendToMemoryQueue)))
queueSizes.Set(map[string]string{"queue": "append_to_memory", "facet": "capacity"}, float64(cap(t.appendToMemoryQueue)))
queueSizes.Set(map[string]string{"queue": "view_generation", "facet": "occupancy"}, float64(len(t.viewQueue)))
queueSizes.Set(map[string]string{"queue": "view_generation", "facet": "capacity"}, float64(cap(t.viewQueue)))
}
func (t *tieredStorage) writeMemory() {
begin := time.Now()
defer func() {
duration := time.Since(begin)
recordOutcome(duration, nil, map[string]string{operation: appendSample, result: success}, map[string]string{operation: writeMemory, result: failure})
}()
t.mutex.Lock()
defer t.mutex.Unlock()
pendingLength := len(t.appendToMemoryQueue)
for i := 0; i < pendingLength; i++ {
t.memoryArena.AppendSample(<-t.appendToMemoryQueue)
}
}
func (t *tieredStorage) Flush() {
t.flush()
}
func (t *tieredStorage) Close() {
t.Drain()
t.diskStorage.Close()
}
// Write all pending appends.
func (t *tieredStorage) flush() (err error) {
// Trim any old values to reduce iterative write costs.
t.flushMemory()
t.writeMemory()
t.flushMemory()
return
}
type memoryToDiskFlusher struct {
toDiskQueue chan model.Sample
disk MetricPersistence
olderThan time.Time
valuesAccepted int
valuesRejected int
}
type memoryToDiskFlusherVisitor struct {
stream stream
flusher *memoryToDiskFlusher
}
func (f memoryToDiskFlusherVisitor) DecodeKey(in interface{}) (out interface{}, err error) {
out = time.Time(in.(skipListTime))
return
}
func (f memoryToDiskFlusherVisitor) DecodeValue(in interface{}) (out interface{}, err error) {
out = in.(value).get()
return
}
func (f memoryToDiskFlusherVisitor) Filter(key, value interface{}) (filterResult storage.FilterResult) {
var (
recordTime = key.(time.Time)
)
if recordTime.Before(f.flusher.olderThan) {
f.flusher.valuesAccepted++
return storage.ACCEPT
}
f.flusher.valuesRejected++
return storage.STOP
}
func (f memoryToDiskFlusherVisitor) Operate(key, value interface{}) (err *storage.OperatorError) {
var (
recordTime = key.(time.Time)
recordValue = value.(model.SampleValue)
)
if len(f.flusher.toDiskQueue) == cap(f.flusher.toDiskQueue) {
f.flusher.Flush()
}
f.flusher.toDiskQueue <- model.Sample{
Metric: f.stream.metric,
Timestamp: recordTime,
Value: recordValue,
}
f.stream.values.Delete(skipListTime(recordTime))
return
}
func (f *memoryToDiskFlusher) ForStream(stream stream) (decoder storage.RecordDecoder, filter storage.RecordFilter, operator storage.RecordOperator) {
visitor := memoryToDiskFlusherVisitor{
stream: stream,
flusher: f,
}
// fmt.Printf("fingerprint -> %s\n", model.NewFingerprintFromMetric(stream.metric).ToRowKey())
return visitor, visitor, visitor
}
func (f *memoryToDiskFlusher) Flush() {
length := len(f.toDiskQueue)
samples := model.Samples{}
for i := 0; i < length; i++ {
samples = append(samples, <-f.toDiskQueue)
}
start := time.Now()
f.disk.AppendSamples(samples)
if false {
fmt.Printf("Took %s to append...\n", time.Since(start))
}
}
func (f memoryToDiskFlusher) Close() {
f.Flush()
}
// Persist a whole bunch of samples to the datastore.
func (t *tieredStorage) flushMemory() {
begin := time.Now()
defer func() {
duration := time.Since(begin)
recordOutcome(duration, nil, map[string]string{operation: appendSample, result: success}, map[string]string{operation: flushMemory, result: failure})
}()
t.mutex.Lock()
defer t.mutex.Unlock()
flusher := &memoryToDiskFlusher{
disk: t.diskStorage,
olderThan: time.Now().Add(-1 * t.memoryTTL),
toDiskQueue: t.appendToDiskQueue,
}
defer flusher.Close()
t.memoryArena.ForEachSample(flusher)
return
}
func (t *tieredStorage) renderView(viewJob viewJob) {
// Telemetry.
var err error
begin := time.Now()
defer func() {
duration := time.Since(begin)
recordOutcome(duration, err, map[string]string{operation: renderView, result: success}, map[string]string{operation: renderView, result: failure})
}()
t.mutex.Lock()
defer t.mutex.Unlock()
var (
scans = viewJob.builder.ScanJobs()
view = newView()
)
// Rebuilding of the frontier should happen on a conditional basis if a
// (fingerprint, timestamp) tuple is outside of the current frontier.
err = t.rebuildDiskFrontier()
if err != nil {
panic(err)
}
if t.diskFrontier == nil {
// Storage still empty, return an empty view.
viewJob.output <- view
return
}
// Get a single iterator that will be used for all data extraction below.
iterator, closer, err := t.diskStorage.metricSamples.GetIterator()
if closer != nil {
defer closer.Close()
}
if err != nil {
panic(err)
}
for _, scanJob := range scans {
seriesFrontier, err := newSeriesFrontier(scanJob.fingerprint, *t.diskFrontier, iterator)
if err != nil {
panic(err)
}
if seriesFrontier == nil {
continue
}
standingOps := scanJob.operations
for len(standingOps) > 0 {
// Load data value chunk(s) around the first standing op's current time.
highWatermark := *standingOps[0].CurrentTime()
// XXX: For earnest performance gains analagous to the benchmarking we
// performed, chunk should only be reloaded if it no longer contains
// the values we're looking for.
//
// To better understand this, look at https://github.com/prometheus/prometheus/blob/benchmark/leveldb/iterator-seek-characteristics/leveldb.go#L239 and note the behavior around retrievedValue.
chunk := t.loadChunkAroundTime(iterator, seriesFrontier, scanJob.fingerprint, highWatermark)
lastChunkTime := chunk[len(chunk)-1].Timestamp
if lastChunkTime.After(highWatermark) {
highWatermark = lastChunkTime
}
// For each op, extract all needed data from the current chunk.
out := []model.SamplePair{}
for _, op := range standingOps {
if op.CurrentTime().After(highWatermark) {
break
}
for op.CurrentTime() != nil && !op.CurrentTime().After(highWatermark) {
out = op.ExtractSamples(chunk)
}
}
// Append the extracted samples to the materialized view.
for _, sample := range out {
view.appendSample(scanJob.fingerprint, sample.Timestamp, sample.Value)
}
// Throw away standing ops which are finished.
filteredOps := ops{}
for _, op := range standingOps {
if op.CurrentTime() != nil {
filteredOps = append(filteredOps, op)
}
}
standingOps = filteredOps
// Sort ops by start time again, since they might be slightly off now.
// For example, consider a current chunk of values and two interval ops
// with different interval lengths. Their states after the cycle above
// could be:
//
// (C = current op time)
//
// Chunk: [ X X X X X ]
// Op 1: [ X X C . . . ]
// Op 2: [ X X C . . .]
//
// Op 2 now has an earlier current time than Op 1.
sort.Sort(startsAtSort{standingOps})
}
}
viewJob.output <- view
return
}
func (t *tieredStorage) loadChunkAroundTime(iterator *levigo.Iterator, frontier *seriesFrontier, fingerprint model.Fingerprint, ts time.Time) (chunk []model.SamplePair) {
var (
targetKey = &dto.SampleKey{
Fingerprint: fingerprint.ToDTO(),
}
foundKey = &dto.SampleKey{}
foundValue *dto.SampleValueSeries
)
// Limit the target key to be within the series' keyspace.
if ts.After(frontier.lastSupertime) {
targetKey.Timestamp = indexable.EncodeTime(frontier.lastSupertime)
} else {
targetKey.Timestamp = indexable.EncodeTime(ts)
}
// Try seeking to target key.
rawKey, _ := coding.NewProtocolBufferEncoder(targetKey).Encode()
iterator.Seek(rawKey)
foundKey, err := extractSampleKey(iterator)
if err != nil {
panic(err)
}
// Figure out if we need to rewind by one block.
// Imagine the following supertime blocks with time ranges:
//
// Block 1: ft 1000 - lt 1009 <data>
// Block 1: ft 1010 - lt 1019 <data>
//
// If we are aiming to find time 1005, we would first seek to the block with
// supertime 1010, then need to rewind by one block by virtue of LevelDB
// iterator seek behavior.
//
// Only do the rewind if there is another chunk before this one.
rewound := false
firstTime := indexable.DecodeTime(foundKey.Timestamp)
if ts.Before(firstTime) && !frontier.firstSupertime.After(ts) {
iterator.Prev()
rewound = true
}
foundValue, err = extractSampleValues(iterator)
if err != nil {
panic(err)
}
// If we rewound, but the target time is still past the current block, return
// the last value of the current (rewound) block and the entire next block.
if rewound {
foundKey, err = extractSampleKey(iterator)
if err != nil {
panic(err)
}
currentChunkLastTime := time.Unix(*foundKey.LastTimestamp, 0)
if ts.After(currentChunkLastTime) {
sampleCount := len(foundValue.Value)
chunk = append(chunk, model.SamplePair{
Timestamp: time.Unix(*foundValue.Value[sampleCount-1].Timestamp, 0),
Value: model.SampleValue(*foundValue.Value[sampleCount-1].Value),
})
// We know there's a next block since we have rewound from it.
iterator.Next()
foundValue, err = extractSampleValues(iterator)
if err != nil {
panic(err)
}
}
}
// Now append all the samples of the currently seeked block to the output.
for _, sample := range foundValue.Value {
chunk = append(chunk, model.SamplePair{
Timestamp: time.Unix(*sample.Timestamp, 0),
Value: model.SampleValue(*sample.Value),
})
}
return
}

View file

@ -0,0 +1,419 @@
// Copyright 2013 Prometheus Team
// 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 metric
import (
"fmt"
"github.com/prometheus/prometheus/model"
"github.com/prometheus/prometheus/utility/test"
"io/ioutil"
"os"
"testing"
"time"
)
func buildSamples(from, to time.Time, interval time.Duration, m model.Metric) (v []model.Sample) {
i := model.SampleValue(0)
for from.Before(to) {
v = append(v, model.Sample{
Metric: m,
Value: i,
Timestamp: from,
})
from = from.Add(interval)
i++
}
return
}
func testMakeView(t test.Tester) {
type in struct {
atTime []getValuesAtTimeOp
atInterval []getValuesAtIntervalOp
alongRange []getValuesAlongRangeOp
}
type out struct {
atTime [][]model.SamplePair
atInterval [][]model.SamplePair
alongRange [][]model.SamplePair
}
var (
instant = time.Date(1984, 3, 30, 0, 0, 0, 0, time.Local)
metric = model.Metric{"name": "request_count"}
fingerprint = model.NewFingerprintFromMetric(metric)
scenarios = []struct {
data []model.Sample
in in
out out
}{
// No sample, but query asks for one.
{
in: in{
atTime: []getValuesAtTimeOp{
{
time: instant,
},
},
},
out: out{
atTime: [][]model.SamplePair{{}},
},
},
// Single sample, query asks for exact sample time.
{
data: []model.Sample{
{
Metric: metric,
Value: 0,
Timestamp: instant,
},
},
in: in{
atTime: []getValuesAtTimeOp{
{
time: instant,
},
},
},
out: out{
atTime: [][]model.SamplePair{
{
{
Timestamp: instant,
Value: 0,
},
},
},
},
},
// Single sample, query time before the sample.
{
data: []model.Sample{
{
Metric: metric,
Value: 0,
Timestamp: instant.Add(time.Second),
},
{
Metric: metric,
Value: 1,
Timestamp: instant.Add(time.Second * 2),
},
},
in: in{
atTime: []getValuesAtTimeOp{
{
time: instant,
},
},
},
out: out{
atTime: [][]model.SamplePair{
{
{
Timestamp: instant.Add(time.Second),
Value: 0,
},
},
},
},
},
// Single sample, query time after the sample.
{
data: []model.Sample{
{
Metric: metric,
Value: 0,
Timestamp: instant,
},
},
in: in{
atTime: []getValuesAtTimeOp{
{
time: instant.Add(time.Second),
},
},
},
out: out{
atTime: [][]model.SamplePair{
{
{
Timestamp: instant,
Value: 0,
},
},
},
},
},
// Two samples, query asks for first sample time.
{
data: []model.Sample{
{
Metric: metric,
Value: 0,
Timestamp: instant,
},
{
Metric: metric,
Value: 1,
Timestamp: instant.Add(time.Second),
},
},
in: in{
atTime: []getValuesAtTimeOp{
{
time: instant,
},
},
},
out: out{
atTime: [][]model.SamplePair{
{
{
Timestamp: instant,
Value: 0,
},
},
},
},
},
// Three samples, query asks for second sample time.
{
data: []model.Sample{
{
Metric: metric,
Value: 0,
Timestamp: instant,
},
{
Metric: metric,
Value: 1,
Timestamp: instant.Add(time.Second),
},
{
Metric: metric,
Value: 2,
Timestamp: instant.Add(time.Second * 2),
},
},
in: in{
atTime: []getValuesAtTimeOp{
{
time: instant.Add(time.Second),
},
},
},
out: out{
atTime: [][]model.SamplePair{
{
{
Timestamp: instant.Add(time.Second),
Value: 1,
},
},
},
},
},
// Three samples, query asks for time between first and second samples.
{
data: []model.Sample{
{
Metric: metric,
Value: 0,
Timestamp: instant,
},
{
Metric: metric,
Value: 1,
Timestamp: instant.Add(time.Second * 2),
},
{
Metric: metric,
Value: 2,
Timestamp: instant.Add(time.Second * 4),
},
},
in: in{
atTime: []getValuesAtTimeOp{
{
time: instant.Add(time.Second),
},
},
},
out: out{
atTime: [][]model.SamplePair{
{
{
Timestamp: instant,
Value: 0,
},
{
Timestamp: instant.Add(time.Second * 2),
Value: 1,
},
},
},
},
},
// Three samples, query asks for time between second and third samples.
{
data: []model.Sample{
{
Metric: metric,
Value: 0,
Timestamp: instant,
},
{
Metric: metric,
Value: 1,
Timestamp: instant.Add(time.Second * 2),
},
{
Metric: metric,
Value: 2,
Timestamp: instant.Add(time.Second * 4),
},
},
in: in{
atTime: []getValuesAtTimeOp{
{
time: instant.Add(time.Second * 3),
},
},
},
out: out{
atTime: [][]model.SamplePair{
{
{
Timestamp: instant.Add(time.Second * 2),
Value: 1,
},
{
Timestamp: instant.Add(time.Second * 4),
Value: 2,
},
},
},
},
},
// Two chunks of samples, query asks for values from first chunk.
{
data: buildSamples(instant, instant.Add(time.Duration(*leveldbChunkSize*2)*time.Second), time.Second, metric),
in: in{
atTime: []getValuesAtTimeOp{
{
time: instant.Add(time.Second*time.Duration(*leveldbChunkSize/2) + 1),
},
},
},
out: out{
atTime: [][]model.SamplePair{
{
{
Timestamp: instant.Add(time.Second * time.Duration(*leveldbChunkSize/2)),
Value: 100,
},
{
Timestamp: instant.Add(time.Second * (time.Duration(*leveldbChunkSize/2) + 1)),
Value: 101,
},
},
},
},
},
}
)
for i, scenario := range scenarios {
var (
temporary, _ = ioutil.TempDir("", "test_make_view")
tiered = NewTieredStorage(5000000, 2500, 1000, 5*time.Second, 15*time.Second, 0*time.Second, temporary)
)
if tiered == nil {
t.Fatalf("%d. tiered == nil", i)
}
go tiered.Serve()
defer tiered.Drain()
defer func() {
os.RemoveAll(temporary)
}()
for j, datum := range scenario.data {
err := tiered.AppendSample(datum)
if err != nil {
t.Fatalf("%d.%d. failed to add fixture data: %s", i, j, err)
}
}
start := time.Now()
tiered.Flush()
fmt.Printf("Took %s to flush %d items...\n", time.Since(start), len(scenario.data))
requestBuilder := NewViewRequestBuilder()
for _, atTime := range scenario.in.atTime {
requestBuilder.GetMetricAtTime(fingerprint, atTime.time)
}
for _, atInterval := range scenario.in.atInterval {
requestBuilder.GetMetricAtInterval(fingerprint, atInterval.from, atInterval.through, atInterval.interval)
}
for _, alongRange := range scenario.in.alongRange {
requestBuilder.GetMetricRange(fingerprint, alongRange.from, alongRange.through)
}
v, err := tiered.MakeView(requestBuilder, time.Second*5)
if err != nil {
t.Fatalf("%d. failed due to %s", i, err)
}
for j, atTime := range scenario.in.atTime {
actual := v.GetValueAtTime(fingerprint, atTime.time)
if len(actual) != len(scenario.out.atTime[j]) {
t.Fatalf("%d.%d. expected %d output, got %d", i, j, len(scenario.out.atTime[j]), len(actual))
}
for k, value := range scenario.out.atTime[j] {
if value.Value != actual[k].Value {
t.Fatalf("%d.%d.%d expected %v value, got %v", i, j, k, value.Value, actual[k].Value)
}
if !value.Timestamp.Equal(actual[k].Timestamp) {
t.Fatalf("%d.%d.%d expected %s timestamp, got %s", i, j, k, value.Timestamp, actual[k].Timestamp)
}
}
}
tiered.Drain()
}
}
func TestMakeView(t *testing.T) {
testMakeView(t)
}
func BenchmarkMakeView(b *testing.B) {
for i := 0; i < b.N; i++ {
testMakeView(b)
}
}

222
storage/metric/view.go Normal file
View file

@ -0,0 +1,222 @@
// Copyright 2013 Prometheus Team
// 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 metric
import (
"fmt"
"github.com/prometheus/prometheus/model"
"github.com/ryszard/goskiplist/skiplist"
"sort"
"time"
)
var (
_ = fmt.Sprintf("")
// firstSupertime is the smallest valid supertime that may be seeked to.
firstSupertime = []byte{0, 0, 0, 0, 0, 0, 0, 0}
// lastSupertime is the largest valid supertime that may be seeked to.
lastSupertime = []byte{127, 255, 255, 255, 255, 255, 255, 255}
)
// Represents the summation of all datastore queries that shall be performed to
// extract values. Each operation mutates the state of the builder.
type ViewRequestBuilder interface {
GetMetricAtTime(fingerprint model.Fingerprint, time time.Time)
GetMetricAtInterval(fingerprint model.Fingerprint, from, through time.Time, interval time.Duration)
GetMetricRange(fingerprint model.Fingerprint, from, through time.Time)
ScanJobs() scanJobs
}
// Contains the various unoptimized requests for data.
type viewRequestBuilder struct {
operations map[model.Fingerprint]ops
}
// Furnishes a ViewRequestBuilder for remarking what types of queries to perform.
func NewViewRequestBuilder() viewRequestBuilder {
return viewRequestBuilder{
operations: make(map[model.Fingerprint]ops),
}
}
// Gets for the given Fingerprint either the value at that time if there is an
// match or the one or two values adjacent thereto.
func (v viewRequestBuilder) GetMetricAtTime(fingerprint model.Fingerprint, time time.Time) {
ops := v.operations[fingerprint]
ops = append(ops, &getValuesAtTimeOp{
time: time,
})
v.operations[fingerprint] = ops
}
// Gets for the given Fingerprint either the value at that interval from From
// through Through if there is an match or the one or two values adjacent
// for each point.
func (v viewRequestBuilder) GetMetricAtInterval(fingerprint model.Fingerprint, from, through time.Time, interval time.Duration) {
ops := v.operations[fingerprint]
ops = append(ops, &getValuesAtIntervalOp{
from: from,
through: through,
interval: interval,
})
v.operations[fingerprint] = ops
}
// Gets for the given Fingerprint either the values that occur inclusively from
// From through Through.
func (v viewRequestBuilder) GetMetricRange(fingerprint model.Fingerprint, from, through time.Time) {
ops := v.operations[fingerprint]
ops = append(ops, &getValuesAlongRangeOp{
from: from,
through: through,
})
v.operations[fingerprint] = ops
}
// Emits the optimized scans that will occur in the data store. This
// effectively resets the ViewRequestBuilder back to a pristine state.
func (v viewRequestBuilder) ScanJobs() (j scanJobs) {
for fingerprint, operations := range v.operations {
sort.Sort(startsAtSort{operations})
j = append(j, scanJob{
fingerprint: fingerprint,
operations: optimize(operations),
})
delete(v.operations, fingerprint)
}
sort.Sort(j)
return
}
type view struct {
fingerprintToSeries map[model.Fingerprint]viewStream
}
func (v view) appendSample(fingerprint model.Fingerprint, timestamp time.Time, value model.SampleValue) {
var (
series, ok = v.fingerprintToSeries[fingerprint]
)
if !ok {
series = newViewStream()
v.fingerprintToSeries[fingerprint] = series
}
series.add(timestamp, value)
}
func (v view) Close() {
v.fingerprintToSeries = make(map[model.Fingerprint]viewStream)
}
func (v view) GetValueAtTime(f model.Fingerprint, t time.Time) (samples []model.SamplePair) {
series, ok := v.fingerprintToSeries[f]
if !ok {
return
}
iterator := series.values.Seek(skipListTime(t))
if iterator == nil {
// If the iterator is nil, it means we seeked past the end of the series,
// so we seek to the last value instead. Due to the reverse ordering
// defined on skipListTime, this corresponds to the sample with the
// earliest timestamp.
iterator = series.values.SeekToLast()
if iterator == nil {
// The list is empty.
return
}
}
defer iterator.Close()
if iterator.Key() == nil || iterator.Value() == nil {
return
}
samples = append(samples, model.SamplePair{
Timestamp: time.Time(iterator.Key().(skipListTime)),
Value: iterator.Value().(value).get(),
})
if iterator.Previous() {
samples = append(samples, model.SamplePair{
Timestamp: time.Time(iterator.Key().(skipListTime)),
Value: iterator.Value().(value).get(),
})
}
return
}
func (v view) GetBoundaryValues(f model.Fingerprint, i model.Interval) (first []model.SamplePair, second []model.SamplePair) {
first = v.GetValueAtTime(f, i.OldestInclusive)
second = v.GetValueAtTime(f, i.NewestInclusive)
return
}
func (v view) GetRangeValues(f model.Fingerprint, i model.Interval) (samples []model.SamplePair) {
series, ok := v.fingerprintToSeries[f]
if !ok {
return
}
iterator := series.values.Seek(skipListTime(i.NewestInclusive))
if iterator == nil {
return
}
for {
timestamp := time.Time(iterator.Key().(skipListTime))
if timestamp.Before(i.OldestInclusive) {
break
}
samples = append(samples, model.SamplePair{
Value: iterator.Value().(value).get(),
Timestamp: timestamp,
})
if !iterator.Next() {
break
}
}
return
}
func newView() view {
return view{
fingerprintToSeries: make(map[model.Fingerprint]viewStream),
}
}
type viewStream struct {
values *skiplist.SkipList
}
func (s viewStream) add(timestamp time.Time, value model.SampleValue) {
s.values.Set(skipListTime(timestamp), singletonValue(value))
}
func newViewStream() viewStream {
return viewStream{
values: skiplist.New(),
}
}

183
storage/metric/view_test.go Normal file
View file

@ -0,0 +1,183 @@
// Copyright 2013 Prometheus Team
// 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 metric
import (
"github.com/prometheus/prometheus/model"
"github.com/prometheus/prometheus/utility/test"
"testing"
"time"
)
func testBuilder(t test.Tester) {
type atTime struct {
fingerprint string
time time.Time
}
type atInterval struct {
fingerprint string
from time.Time
through time.Time
interval time.Duration
}
type atRange struct {
fingerprint string
from time.Time
through time.Time
}
type in struct {
atTimes []atTime
atIntervals []atInterval
atRanges []atRange
}
type out []struct {
fingerprint string
operations ops
}
var scenarios = []struct {
in in
out out
}{
// // Ensure that the fingerprint is sorted in proper order.
{
in: in{
atTimes: []atTime{
{
fingerprint: "0000000000000001111-a-4-a",
time: time.Unix(100, 0),
},
{
fingerprint: "0000000000000000000-a-4-a",
time: time.Unix(100, 0),
},
},
},
out: out{
{
fingerprint: "00000000000000000000-a-4-a",
},
{
fingerprint: "00000000000000001111-a-4-a",
},
},
},
// // Ensure that the fingerprint-timestamp pairs are sorted in proper order.
{
in: in{
atTimes: []atTime{
{
fingerprint: "1111-a-4-a",
time: time.Unix(100, 0),
},
{
fingerprint: "1111-a-4-a",
time: time.Unix(200, 0),
},
{
fingerprint: "0-a-4-a",
time: time.Unix(100, 0),
},
{
fingerprint: "0-a-4-a",
time: time.Unix(0, 0),
},
},
},
out: out{
{
fingerprint: "00000000000000000000-a-4-a",
},
{
fingerprint: "00000000000000001111-a-4-a",
},
},
},
// Ensure grouping of operations
{
in: in{
atTimes: []atTime{
{
fingerprint: "1111-a-4-a",
time: time.Unix(100, 0),
},
},
atRanges: []atRange{
{
fingerprint: "1111-a-4-a",
from: time.Unix(100, 0),
through: time.Unix(1000, 0),
},
{
fingerprint: "1111-a-4-a",
from: time.Unix(100, 0),
through: time.Unix(9000, 0),
},
},
},
out: out{
{
fingerprint: "00000000000000001111-a-4-a",
},
},
},
}
for i, scenario := range scenarios {
builder := viewRequestBuilder{
operations: map[model.Fingerprint]ops{},
}
for _, atTime := range scenario.in.atTimes {
fingerprint := model.NewFingerprintFromRowKey(atTime.fingerprint)
builder.GetMetricAtTime(fingerprint, atTime.time)
}
for _, atInterval := range scenario.in.atIntervals {
fingerprint := model.NewFingerprintFromRowKey(atInterval.fingerprint)
builder.GetMetricAtInterval(fingerprint, atInterval.from, atInterval.through, atInterval.interval)
}
for _, atRange := range scenario.in.atRanges {
fingerprint := model.NewFingerprintFromRowKey(atRange.fingerprint)
builder.GetMetricRange(fingerprint, atRange.from, atRange.through)
}
jobs := builder.ScanJobs()
if len(scenario.out) != len(jobs) {
t.Fatalf("%d. expected job length of %d, got %d\n", i, len(scenario.out), len(jobs))
}
for j, job := range scenario.out {
if jobs[j].fingerprint.ToRowKey() != job.fingerprint {
t.Fatalf("%d.%d. expected fingerprint %s, got %s\n", i, j, job.fingerprint, jobs[j].fingerprint.ToRowKey())
}
}
}
}
func TestBuilder(t *testing.T) {
testBuilder(t)
}
func BenchmarkBuilder(b *testing.B) {
for i := 0; i < b.N; i++ {
testBuilder(b)
}
}

View file

@ -56,3 +56,7 @@ func NewLevelDBMembershipIndex(storageRoot string, cacheCapacity, bitsPerBloomFi
return
}
func (l *LevelDBMembershipIndex) Commit(batch leveldb.Batch) error {
return l.persistence.Commit(batch)
}

View file

@ -23,11 +23,12 @@ import (
)
var (
leveldbFlushOnMutate = flag.Bool("leveldbFlushOnMutate", true, "Whether LevelDB should flush every operation to disk upon mutation before returning (bool).")
leveldbFlushOnMutate = flag.Bool("leveldbFlushOnMutate", false, "Whether LevelDB should flush every operation to disk upon mutation before returning (bool).")
leveldbUseSnappy = flag.Bool("leveldbUseSnappy", true, "Whether LevelDB attempts to use Snappy for compressing elements (bool).")
leveldbUseParanoidChecks = flag.Bool("leveldbUseParanoidChecks", true, "Whether LevelDB uses expensive checks (bool).")
)
// LevelDBPersistence is a disk-backed sorted key-value store.
type LevelDBPersistence struct {
cache *levigo.Cache
filterPolicy *levigo.FilterPolicy
@ -37,6 +38,8 @@ type LevelDBPersistence struct {
writeOptions *levigo.WriteOptions
}
// LevelDB iterators have a number of resources that need to be closed.
// iteratorCloser encapsulates the various ones.
type iteratorCloser struct {
iterator *levigo.Iterator
readOptions *levigo.ReadOptions
@ -169,6 +172,10 @@ func (l *LevelDBPersistence) Put(key, value coding.Encoder) (err error) {
return
}
func (l *LevelDBPersistence) Commit(b Batch) (err error) {
return l.storage.Write(l.writeOptions, b.(batch).batch)
}
func (l *LevelDBPersistence) GetAll() (pairs []raw.Pair, err error) {
snapshot := l.storage.NewSnapshot()
defer l.storage.ReleaseSnapshot(snapshot)
@ -272,3 +279,47 @@ func (l *LevelDBPersistence) ForEach(decoder storage.RecordDecoder, filter stora
scannedEntireCorpus = true
return
}
// Batch encapsulates a list of mutations to occur to the datastore. It must
// be closed once done.
type Batch interface {
Delete(coding.Encoder)
Put(coding.Encoder, coding.Encoder)
Close()
}
func NewBatch() Batch {
return batch{
batch: levigo.NewWriteBatch(),
}
}
type batch struct {
batch *levigo.WriteBatch
}
func (b batch) Delete(key coding.Encoder) {
keyEncoded, err := key.Encode()
if err != nil {
panic(err)
}
b.batch.Delete(keyEncoded)
}
func (b batch) Put(key, value coding.Encoder) {
keyEncoded, err := key.Encode()
if err != nil {
panic(err)
}
valueEncoded, err := value.Encode()
if err != nil {
panic(err)
}
b.batch.Put(keyEncoded, valueEncoded)
}
func (b batch) Close() {
b.batch.Close()
}

View file

@ -25,8 +25,8 @@ import (
"time"
)
func (serv MetricsService) Query(Expr string, Json string) (result string) {
exprNode, err := rules.LoadExprFromString(Expr)
func (serv MetricsService) Query(expr string, formatJson string) (result string) {
exprNode, err := rules.LoadExprFromString(expr)
if err != nil {
return ast.ErrorToJSON(err)
}
@ -35,7 +35,7 @@ func (serv MetricsService) Query(Expr string, Json string) (result string) {
rb := serv.ResponseBuilder()
var format ast.OutputFormat
if Json != "" {
if formatJson != "" {
format = ast.JSON
rb.SetContentType(gorest.Application_Json)
} else {
@ -46,8 +46,8 @@ func (serv MetricsService) Query(Expr string, Json string) (result string) {
return ast.EvalToString(exprNode, &timestamp, format)
}
func (serv MetricsService) QueryRange(Expr string, End int64, Range int64, Step int64) string {
exprNode, err := rules.LoadExprFromString(Expr)
func (serv MetricsService) QueryRange(expr string, end int64, duration int64, step int64) string {
exprNode, err := rules.LoadExprFromString(expr)
if err != nil {
return ast.ErrorToJSON(err)
}
@ -57,26 +57,26 @@ func (serv MetricsService) QueryRange(Expr string, End int64, Range int64, Step
rb := serv.ResponseBuilder()
rb.SetContentType(gorest.Application_Json)
if End == 0 {
End = serv.time.Now().Unix()
if end == 0 {
end = serv.time.Now().Unix()
}
if Step < 1 {
Step = 1
if step < 1 {
step = 1
}
if End-Range < 0 {
Range = End
if end-duration < 0 {
duration = end
}
// Align the start to step "tick" boundary.
End -= End % Step
end -= end % step
matrix := ast.EvalVectorRange(
exprNode.(ast.VectorNode),
time.Unix(End-Range, 0),
time.Unix(End, 0),
time.Duration(Step)*time.Second)
time.Unix(end-duration, 0),
time.Unix(end, 0),
time.Duration(step)*time.Second)
sort.Sort(matrix)
return ast.TypedValueToJSON(matrix, "matrix")

View file

@ -26,8 +26,8 @@ import (
// Commandline flags.
var (
listenAddress = flag.String("listenAddress", ":9090", "Address to listen on for web interface.")
useLocalAssets = flag.Bool("localAssets", false, "Read assets/templates from file instead of binary.")
listenAddress = flag.String("listenAddress", ":9090", "Address to listen on for web interface.")
useLocalAssets = flag.Bool("localAssets", false, "Read assets/templates from file instead of binary.")
)
func StartServing(appState *appstate.ApplicationState) {