Initial experimental snapshot of next-gen storage.

Change-Id: Ifb8709960dbedd1d9f5efd88cdd359ee9fa9d26d
This commit is contained in:
Julius Volz 2014-06-06 11:55:53 +02:00 committed by Bjoern Rabenstein
parent 134bd8fe34
commit e7ed39c9a6
81 changed files with 2829 additions and 13909 deletions

202
main.go
View file

@ -23,15 +23,15 @@ import (
"github.com/golang/glog"
"github.com/prometheus/client_golang/extraction"
registry "github.com/prometheus/client_golang/prometheus"
clientmodel "github.com/prometheus/client_golang/model"
registry "github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/notification"
"github.com/prometheus/prometheus/retrieval"
"github.com/prometheus/prometheus/rules/manager"
"github.com/prometheus/prometheus/storage/metric/tiered"
"github.com/prometheus/prometheus/storage/local"
"github.com/prometheus/prometheus/storage/remote"
"github.com/prometheus/prometheus/storage/remote/opentsdb"
"github.com/prometheus/prometheus/web"
@ -54,41 +54,30 @@ var (
diskAppendQueueCapacity = flag.Int("storage.queue.diskAppendCapacity", 1000000, "The size of the queue for items that are pending writing to disk.")
memoryAppendQueueCapacity = flag.Int("storage.queue.memoryAppendCapacity", 10000, "The size of the queue for items that are pending writing to memory.")
compactInterval = flag.Duration("compact.interval", 3*time.Hour, "The amount of time between compactions.")
compactGroupSize = flag.Int("compact.groupSize", 500, "The minimum group size for compacted samples.")
compactAgeInclusiveness = flag.Duration("compact.ageInclusiveness", 5*time.Minute, "The age beyond which samples should be compacted.")
deleteInterval = flag.Duration("delete.interval", 11*time.Hour, "The amount of time between deletion of old values.")
deleteAge = flag.Duration("delete.ageMaximum", 15*24*time.Hour, "The relative maximum age for values before they are deleted.")
arenaFlushInterval = flag.Duration("arena.flushInterval", 15*time.Minute, "The period at which the in-memory arena is flushed to disk.")
arenaTTL = flag.Duration("arena.ttl", 10*time.Minute, "The relative age of values to purge to disk from memory.")
memoryEvictionInterval = flag.Duration("storage.memory.evictionInterval", 15*time.Minute, "The period at which old data is evicted from memory.")
memoryRetentionPeriod = flag.Duration("storage.memory.retentionPeriod", time.Hour, "The period of time to retain in memory during evictions.")
storagePurgeInterval = flag.Duration("storage.purgeInterval", time.Hour, "How frequently to purge old data from the storage.")
storageRetentionPeriod = flag.Duration("storage.retentionPeriod", 15*24*time.Hour, "The period of time to retain in storage.")
notificationQueueCapacity = flag.Int("alertmanager.notificationQueueCapacity", 100, "The size of the queue for pending alert manager notifications.")
printVersion = flag.Bool("version", false, "print version information")
shutdownTimeout = flag.Duration("shutdownGracePeriod", 0*time.Second, "The amount of time Prometheus gives background services to finish running when shutdown is requested.")
)
type prometheus struct {
compactionTimer *time.Ticker
deletionTimer *time.Ticker
curationSema chan struct{}
stopBackgroundOperations chan struct{}
unwrittenSamples chan *extraction.Result
ruleManager manager.RuleManager
targetManager retrieval.TargetManager
notifications chan notification.NotificationReqs
storage *tiered.TieredStorage
storage storage_ng.Storage
remoteTSDBQueue *remote.TSDBQueueManager
curationState tiered.CurationStateUpdater
closeOnce sync.Once
}
@ -105,73 +94,6 @@ func (p *prometheus) interruptHandler() {
os.Exit(0)
}
func (p *prometheus) compact(olderThan time.Duration, groupSize int) error {
select {
case s, ok := <-p.curationSema:
if !ok {
glog.Warning("Prometheus is shutting down; no more curation runs are allowed.")
return nil
}
defer func() {
p.curationSema <- s
}()
default:
glog.Warningf("Deferred compaction for %s and %s due to existing operation.", olderThan, groupSize)
return nil
}
processor := tiered.NewCompactionProcessor(&tiered.CompactionProcessorOptions{
MaximumMutationPoolBatch: groupSize * 3,
MinimumGroupSize: groupSize,
})
defer processor.Close()
curator := tiered.NewCurator(&tiered.CuratorOptions{
Stop: p.stopBackgroundOperations,
ViewQueue: p.storage.ViewQueue,
})
defer curator.Close()
return curator.Run(olderThan, clientmodel.Now(), processor, p.storage.DiskStorage.CurationRemarks, p.storage.DiskStorage.MetricSamples, p.storage.DiskStorage.MetricHighWatermarks, p.curationState)
}
func (p *prometheus) delete(olderThan time.Duration, batchSize int) error {
select {
case s, ok := <-p.curationSema:
if !ok {
glog.Warning("Prometheus is shutting down; no more curation runs are allowed.")
return nil
}
defer func() {
p.curationSema <- s
}()
default:
glog.Warningf("Deferred deletion for %s due to existing operation.", olderThan)
return nil
}
processor := tiered.NewDeletionProcessor(&tiered.DeletionProcessorOptions{
MaximumMutationPoolBatch: batchSize,
})
defer processor.Close()
curator := tiered.NewCurator(&tiered.CuratorOptions{
Stop: p.stopBackgroundOperations,
ViewQueue: p.storage.ViewQueue,
})
defer curator.Close()
return curator.Run(olderThan, clientmodel.Now(), processor, p.storage.DiskStorage.CurationRemarks, p.storage.DiskStorage.MetricSamples, p.storage.DiskStorage.MetricHighWatermarks, p.curationState)
}
func (p *prometheus) Close() {
p.closeOnce.Do(p.close)
}
@ -185,31 +107,11 @@ func (p *prometheus) close() {
p.ruleManager.Stop()
glog.Info("Rule Executor: Done")
// Stop any currently active curation (deletion or compaction).
close(p.stopBackgroundOperations)
glog.Info("Current Curation Workers: Requested")
// Disallow further curation work.
close(p.curationSema)
// Stop curation timers.
if p.compactionTimer != nil {
p.compactionTimer.Stop()
}
if p.deletionTimer != nil {
p.deletionTimer.Stop()
}
glog.Info("Future Curation Workers: Done")
glog.Infof("Waiting %s for background systems to exit and flush before finalizing (DO NOT INTERRUPT THE PROCESS) ...", *shutdownTimeout)
// Wart: We should have a concrete form of synchronization for this, not a
// hokey sleep statement.
time.Sleep(*shutdownTimeout)
close(p.unwrittenSamples)
p.storage.Close()
if err := p.storage.Close(); err != nil {
glog.Error("Error closing local storage: ", err)
}
glog.Info("Local Storage: Done")
if p.remoteTSDBQueue != nil {
@ -239,11 +141,23 @@ func main() {
glog.Fatalf("Error loading configuration from %s: %v", *configFile, err)
}
ts, err := tiered.NewTieredStorage(uint(*diskAppendQueueCapacity), 100, *arenaFlushInterval, *arenaTTL, *metricsStoragePath)
persistence, err := storage_ng.NewDiskPersistence(*metricsStoragePath, 1024)
if err != nil {
glog.Fatal("Error opening storage: ", err)
glog.Fatal("Error opening disk persistence: ", err)
}
registry.MustRegister(ts)
o := &storage_ng.MemorySeriesStorageOptions{
Persistence: persistence,
MemoryEvictionInterval: *memoryEvictionInterval,
MemoryRetentionPeriod: *memoryRetentionPeriod,
PersistencePurgeInterval: *storagePurgeInterval,
PersistenceRetentionPeriod: *storageRetentionPeriod,
}
memStorage, err := storage_ng.NewMemorySeriesStorage(o)
if err != nil {
glog.Fatal("Error opening memory series storage: ", err)
}
//registry.MustRegister(memStorage)
var remoteTSDBQueue *remote.TSDBQueueManager
if *remoteTSDBUrl == "" {
@ -263,9 +177,6 @@ func main() {
Ingester: retrieval.ChannelIngester(unwrittenSamples),
}
compactionTimer := time.NewTicker(*compactInterval)
deletionTimer := time.NewTicker(*deleteInterval)
// Queue depth will need to be exposed
targetManager := retrieval.NewTargetManager(ingester)
targetManager.AddTargetsFromConfig(conf)
@ -277,7 +188,7 @@ func main() {
Results: unwrittenSamples,
Notifications: notifications,
EvaluationInterval: conf.EvaluationInterval(),
Storage: ts,
Storage: memStorage,
PrometheusUrl: web.MustBuildServerUrl(),
})
if err := ruleManager.AddRulesFromConfig(conf); err != nil {
@ -309,82 +220,41 @@ func main() {
}
consolesHandler := &web.ConsolesHandler{
Storage: ts,
}
databasesHandler := &web.DatabasesHandler{
Provider: ts.DiskStorage,
RefreshInterval: 5 * time.Minute,
Storage: memStorage,
}
metricsService := &api.MetricsService{
Config: &conf,
TargetManager: targetManager,
Storage: ts,
Storage: memStorage,
}
prometheus := &prometheus{
compactionTimer: compactionTimer,
deletionTimer: deletionTimer,
curationState: prometheusStatus,
curationSema: make(chan struct{}, 1),
unwrittenSamples: unwrittenSamples,
stopBackgroundOperations: make(chan struct{}),
ruleManager: ruleManager,
targetManager: targetManager,
notifications: notifications,
storage: ts,
storage: memStorage,
remoteTSDBQueue: remoteTSDBQueue,
}
defer prometheus.Close()
webService := &web.WebService{
StatusHandler: prometheusStatus,
MetricsHandler: metricsService,
DatabasesHandler: databasesHandler,
ConsolesHandler: consolesHandler,
AlertsHandler: alertsHandler,
StatusHandler: prometheusStatus,
MetricsHandler: metricsService,
ConsolesHandler: consolesHandler,
AlertsHandler: alertsHandler,
QuitDelegate: prometheus.Close,
}
prometheus.curationSema <- struct{}{}
storageStarted := make(chan bool)
go ts.Serve(storageStarted)
go memStorage.Serve(storageStarted)
<-storageStarted
go prometheus.interruptHandler()
go func() {
for _ = range prometheus.compactionTimer.C {
glog.Info("Starting compaction...")
err := prometheus.compact(*compactAgeInclusiveness, *compactGroupSize)
if err != nil {
glog.Error("could not compact: ", err)
}
glog.Info("Done")
}
}()
go func() {
for _ = range prometheus.deletionTimer.C {
glog.Info("Starting deletion of stale values...")
err := prometheus.delete(*deleteAge, deletionBatchSize)
if err != nil {
glog.Error("could not delete: ", err)
}
glog.Info("Done")
}
}()
go func() {
err := webService.ServeForever()
if err != nil {
@ -395,7 +265,7 @@ func main() {
// TODO(all): Migrate this into prometheus.serve().
for block := range unwrittenSamples {
if block.Err == nil && len(block.Samples) > 0 {
ts.AppendSamples(block.Samples)
memStorage.AppendSamples(block.Samples)
if remoteTSDBQueue != nil {
remoteTSDBQueue.Queue(block.Samples)
}

View file

@ -23,7 +23,7 @@ import (
"github.com/prometheus/prometheus/rules/ast"
"github.com/prometheus/prometheus/stats"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/local"
"github.com/prometheus/prometheus/utility"
)
@ -118,11 +118,11 @@ func (rule *AlertingRule) Name() string {
return rule.name
}
func (rule *AlertingRule) EvalRaw(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error) {
func (rule *AlertingRule) EvalRaw(timestamp clientmodel.Timestamp, storage storage_ng.Storage) (ast.Vector, error) {
return ast.EvalVectorInstant(rule.Vector, timestamp, storage, stats.NewTimerGroup())
}
func (rule *AlertingRule) Eval(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error) {
func (rule *AlertingRule) Eval(timestamp clientmodel.Timestamp, storage storage_ng.Storage) (ast.Vector, error) {
// Get the raw value of the rule expression.
exprResult, err := rule.EvalRaw(timestamp, storage)
if err != nil {

View file

@ -15,20 +15,22 @@ package ast
import (
"errors"
"flag"
"fmt"
"hash/fnv"
"math"
"sort"
"time"
"github.com/golang/glog"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/stats"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/local"
)
var defaultStalenessDelta = flag.Duration("defaultStalenessDelta", 300*time.Second, "Default staleness delta allowance in seconds during expression evaluations.")
// ----------------------------------------------------------------------------
// Raw data value types.
@ -114,7 +116,7 @@ type Node interface {
type ScalarNode interface {
Node
// Eval evaluates and returns the value of the scalar represented by this node.
Eval(timestamp clientmodel.Timestamp, view *viewAdapter) clientmodel.SampleValue
Eval(timestamp clientmodel.Timestamp) clientmodel.SampleValue
}
// VectorNode is a Node for vector values.
@ -123,17 +125,17 @@ type VectorNode interface {
// Eval evaluates the node recursively and returns the result
// as a Vector (i.e. a slice of Samples all at the given
// Timestamp).
Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Vector
Eval(timestamp clientmodel.Timestamp) Vector
}
// MatrixNode is a Node for matrix values.
type MatrixNode interface {
Node
// Eval evaluates the node recursively and returns the result as a Matrix.
Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Matrix
Eval(timestamp clientmodel.Timestamp) Matrix
// Eval evaluates the node recursively and returns the result
// as a Matrix that only contains the boundary values.
EvalBoundaries(timestamp clientmodel.Timestamp, view *viewAdapter) Matrix
EvalBoundaries(timestamp clientmodel.Timestamp) Matrix
}
// StringNode is a Node for string values.
@ -141,7 +143,7 @@ type StringNode interface {
Node
// Eval evaluates and returns the value of the string
// represented by this node.
Eval(timestamp clientmodel.Timestamp, view *viewAdapter) string
Eval(timestamp clientmodel.Timestamp) string
}
// ----------------------------------------------------------------------------
@ -176,7 +178,11 @@ type (
// A VectorSelector represents a metric name plus labelset.
VectorSelector struct {
labelMatchers metric.LabelMatchers
// The series iterators are populated at query analysis time.
iterators map[clientmodel.Fingerprint]storage_ng.SeriesIterator
metrics map[clientmodel.Fingerprint]clientmodel.Metric
// Fingerprints are populated from label matchers at query analysis time.
// TODO: do we still need these?
fingerprints clientmodel.Fingerprints
}
@ -213,8 +219,11 @@ type (
// timerange.
MatrixSelector struct {
labelMatchers metric.LabelMatchers
// Fingerprints are populated from label matchers at query
// analysis time.
// The series iterators are populated at query analysis time.
iterators map[clientmodel.Fingerprint]storage_ng.SeriesIterator
metrics map[clientmodel.Fingerprint]clientmodel.Metric
// Fingerprints are populated from label matchers at query analysis time.
// TODO: do we still need these?
fingerprints clientmodel.Fingerprints
interval time.Duration
}
@ -308,22 +317,22 @@ func (node StringFunctionCall) Children() Nodes { return node.args }
// Eval implements the ScalarNode interface and returns the selector
// value.
func (node *ScalarLiteral) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) clientmodel.SampleValue {
func (node *ScalarLiteral) Eval(timestamp clientmodel.Timestamp) clientmodel.SampleValue {
return node.value
}
// Eval implements the ScalarNode interface and returns the result of
// the expression.
func (node *ScalarArithExpr) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) clientmodel.SampleValue {
lhs := node.lhs.Eval(timestamp, view)
rhs := node.rhs.Eval(timestamp, view)
func (node *ScalarArithExpr) Eval(timestamp clientmodel.Timestamp) clientmodel.SampleValue {
lhs := node.lhs.Eval(timestamp)
rhs := node.rhs.Eval(timestamp)
return evalScalarBinop(node.opType, lhs, rhs)
}
// Eval implements the ScalarNode interface and returns the result of
// the function call.
func (node *ScalarFunctionCall) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) clientmodel.SampleValue {
return node.function.callFn(timestamp, view, node.args).(clientmodel.SampleValue)
func (node *ScalarFunctionCall) Eval(timestamp clientmodel.Timestamp) clientmodel.SampleValue {
return node.function.callFn(timestamp, node.args).(clientmodel.SampleValue)
}
func (node *VectorAggregation) labelsToGroupingKey(labels clientmodel.Metric) uint64 {
@ -357,33 +366,34 @@ func labelsToKey(labels clientmodel.Metric) uint64 {
}
// EvalVectorInstant evaluates a VectorNode with an instant query.
func EvalVectorInstant(node VectorNode, timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) (vector Vector, err error) {
viewAdapter, err := viewAdapterForInstantQuery(node, timestamp, storage, queryStats)
func EvalVectorInstant(node VectorNode, timestamp clientmodel.Timestamp, storage storage_ng.Storage, queryStats *stats.TimerGroup) (Vector, error) {
closer, err := prepareInstantQuery(node, timestamp, storage, queryStats)
if err != nil {
return
return nil, err
}
vector = node.Eval(timestamp, viewAdapter)
return
defer closer.Close()
return node.Eval(timestamp), nil
}
// EvalVectorRange evaluates a VectorNode with a range query.
func EvalVectorRange(node VectorNode, start clientmodel.Timestamp, end clientmodel.Timestamp, interval time.Duration, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) (Matrix, error) {
func EvalVectorRange(node VectorNode, start clientmodel.Timestamp, end clientmodel.Timestamp, interval time.Duration, storage storage_ng.Storage, queryStats *stats.TimerGroup) (Matrix, error) {
// Explicitly initialize to an empty matrix since a nil Matrix encodes to
// null in JSON.
matrix := Matrix{}
viewTimer := queryStats.GetTimer(stats.TotalViewBuildingTime).Start()
viewAdapter, err := viewAdapterForRangeQuery(node, start, end, interval, storage, queryStats)
viewTimer.Stop()
prepareTimer := queryStats.GetTimer(stats.TotalQueryPreparationTime).Start()
closer, err := prepareRangeQuery(node, start, end, interval, storage, queryStats)
prepareTimer.Stop()
if err != nil {
return nil, err
}
defer closer.Close()
// TODO implement watchdog timer for long-running queries.
evalTimer := queryStats.GetTimer(stats.InnerEvalTime).Start()
sampleSets := map[uint64]*metric.SampleSet{}
for t := start; t.Before(end); t = t.Add(interval) {
vector := node.Eval(t, viewAdapter)
vector := node.Eval(t)
for _, sample := range vector {
samplePair := metric.SamplePair{
Value: sample.Value,
@ -444,8 +454,8 @@ func (node *VectorAggregation) groupedAggregationsToVector(aggregations map[uint
// Eval implements the VectorNode interface and returns the aggregated
// Vector.
func (node *VectorAggregation) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Vector {
vector := node.vector.Eval(timestamp, view)
func (node *VectorAggregation) Eval(timestamp clientmodel.Timestamp) Vector {
vector := node.vector.Eval(timestamp)
result := map[uint64]*groupedAggregation{}
for _, sample := range vector {
groupingKey := node.labelsToGroupingKey(sample.Metric)
@ -498,19 +508,91 @@ func (node *VectorAggregation) Eval(timestamp clientmodel.Timestamp, view *viewA
// Eval implements the VectorNode interface and returns the value of
// the selector.
func (node *VectorSelector) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Vector {
values, err := view.GetValueAtTime(node.fingerprints, timestamp)
if err != nil {
glog.Error("Unable to get vector values: ", err)
return Vector{}
func (node *VectorSelector) Eval(timestamp clientmodel.Timestamp) Vector {
//// timer := v.stats.GetTimer(stats.GetValueAtTimeTime).Start()
samples := Vector{}
for fp, it := range node.iterators {
sampleCandidates := it.GetValueAtTime(timestamp)
samplePair := chooseClosestSample(sampleCandidates, timestamp)
if samplePair != nil {
samples = append(samples, &clientmodel.Sample{
Metric: node.metrics[fp], // TODO: need copy here because downstream can modify!
Value: samplePair.Value,
Timestamp: timestamp,
})
}
}
//// timer.Stop()
return samples
}
// chooseClosestSample chooses the closest sample of a list of samples
// surrounding a given target time. If samples are found both before and after
// the target time, the sample value is interpolated between these. Otherwise,
// the single closest sample is returned verbatim.
func chooseClosestSample(samples metric.Values, timestamp clientmodel.Timestamp) *metric.SamplePair {
var closestBefore *metric.SamplePair
var closestAfter *metric.SamplePair
for _, candidate := range samples {
delta := candidate.Timestamp.Sub(timestamp)
// Samples before target time.
if delta < 0 {
// Ignore samples outside of staleness policy window.
if -delta > *defaultStalenessDelta {
continue
}
// Ignore samples that are farther away than what we've seen before.
if closestBefore != nil && candidate.Timestamp.Before(closestBefore.Timestamp) {
continue
}
sample := candidate
closestBefore = &sample
}
// Samples after target time.
if delta >= 0 {
// Ignore samples outside of staleness policy window.
if delta > *defaultStalenessDelta {
continue
}
// Ignore samples that are farther away than samples we've seen before.
if closestAfter != nil && candidate.Timestamp.After(closestAfter.Timestamp) {
continue
}
sample := candidate
closestAfter = &sample
}
}
switch {
case closestBefore != nil && closestAfter != nil:
return interpolateSamples(closestBefore, closestAfter, timestamp)
case closestBefore != nil:
return closestBefore
default:
return closestAfter
}
}
// interpolateSamples interpolates a value at a target time between two
// provided sample pairs.
func interpolateSamples(first, second *metric.SamplePair, timestamp clientmodel.Timestamp) *metric.SamplePair {
dv := second.Value - first.Value
dt := second.Timestamp.Sub(first.Timestamp)
dDt := dv / clientmodel.SampleValue(dt)
offset := clientmodel.SampleValue(timestamp.Sub(first.Timestamp))
return &metric.SamplePair{
Value: first.Value + (offset * dDt),
Timestamp: timestamp,
}
return values
}
// Eval implements the VectorNode interface and returns the result of
// the function call.
func (node *VectorFunctionCall) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Vector {
return node.function.callFn(timestamp, view, node.args).(Vector)
func (node *VectorFunctionCall) Eval(timestamp clientmodel.Timestamp) Vector {
return node.function.callFn(timestamp, node.args).(Vector)
}
func evalScalarBinop(opType BinOpType,
@ -639,11 +721,11 @@ func labelsEqual(labels1, labels2 clientmodel.Metric) bool {
// Eval implements the VectorNode interface and returns the result of
// the expression.
func (node *VectorArithExpr) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Vector {
func (node *VectorArithExpr) Eval(timestamp clientmodel.Timestamp) Vector {
result := Vector{}
if node.lhs.Type() == SCALAR && node.rhs.Type() == VECTOR {
lhs := node.lhs.(ScalarNode).Eval(timestamp, view)
rhs := node.rhs.(VectorNode).Eval(timestamp, view)
lhs := node.lhs.(ScalarNode).Eval(timestamp)
rhs := node.rhs.(VectorNode).Eval(timestamp)
for _, rhsSample := range rhs {
value, keep := evalVectorBinop(node.opType, lhs, rhsSample.Value)
if keep {
@ -653,8 +735,8 @@ func (node *VectorArithExpr) Eval(timestamp clientmodel.Timestamp, view *viewAda
}
return result
} else if node.lhs.Type() == VECTOR && node.rhs.Type() == SCALAR {
lhs := node.lhs.(VectorNode).Eval(timestamp, view)
rhs := node.rhs.(ScalarNode).Eval(timestamp, view)
lhs := node.lhs.(VectorNode).Eval(timestamp)
rhs := node.rhs.(ScalarNode).Eval(timestamp)
for _, lhsSample := range lhs {
value, keep := evalVectorBinop(node.opType, lhsSample.Value, rhs)
if keep {
@ -664,8 +746,8 @@ func (node *VectorArithExpr) Eval(timestamp clientmodel.Timestamp, view *viewAda
}
return result
} else if node.lhs.Type() == VECTOR && node.rhs.Type() == VECTOR {
lhs := node.lhs.(VectorNode).Eval(timestamp, view)
rhs := node.rhs.(VectorNode).Eval(timestamp, view)
lhs := node.lhs.(VectorNode).Eval(timestamp)
rhs := node.rhs.(VectorNode).Eval(timestamp)
for _, lhsSample := range lhs {
for _, rhsSample := range rhs {
if labelsEqual(lhsSample.Metric, rhsSample.Metric) {
@ -684,32 +766,54 @@ func (node *VectorArithExpr) Eval(timestamp clientmodel.Timestamp, view *viewAda
// Eval implements the MatrixNode interface and returns the value of
// the selector.
func (node *MatrixSelector) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Matrix {
func (node *MatrixSelector) Eval(timestamp clientmodel.Timestamp) Matrix {
interval := &metric.Interval{
OldestInclusive: timestamp.Add(-node.interval),
NewestInclusive: timestamp,
}
values, err := view.GetRangeValues(node.fingerprints, interval)
if err != nil {
glog.Error("Unable to get values for vector interval: ", err)
return Matrix{}
//// timer := v.stats.GetTimer(stats.GetRangeValuesTime).Start()
sampleSets := []metric.SampleSet{}
for fp, it := range node.iterators {
samplePairs := it.GetRangeValues(*interval)
if len(samplePairs) == 0 {
continue
}
sampleSet := metric.SampleSet{
Metric: node.metrics[fp], // TODO: need copy here because downstream can modify!
Values: samplePairs,
}
sampleSets = append(sampleSets, sampleSet)
}
return values
//// timer.Stop()
return sampleSets
}
// EvalBoundaries implements the MatrixNode interface and returns the
// boundary values of the selector.
func (node *MatrixSelector) EvalBoundaries(timestamp clientmodel.Timestamp, view *viewAdapter) Matrix {
func (node *MatrixSelector) EvalBoundaries(timestamp clientmodel.Timestamp) Matrix {
interval := &metric.Interval{
OldestInclusive: timestamp.Add(-node.interval),
NewestInclusive: timestamp,
}
values, err := view.GetBoundaryValues(node.fingerprints, interval)
if err != nil {
glog.Error("Unable to get boundary values for vector interval: ", err)
return Matrix{}
//// timer := v.stats.GetTimer(stats.GetBoundaryValuesTime).Start()
sampleSets := []metric.SampleSet{}
for fp, it := range node.iterators {
samplePairs := it.GetBoundaryValues(*interval)
if len(samplePairs) == 0 {
continue
}
sampleSet := metric.SampleSet{
Metric: node.metrics[fp], // TODO: make copy of metric.
Values: samplePairs,
}
sampleSets = append(sampleSets, sampleSet)
}
return values
//// timer.Stop()
return sampleSets
}
// Len implements sort.Interface.
@ -729,14 +833,14 @@ func (matrix Matrix) Swap(i, j int) {
// Eval implements the StringNode interface and returns the value of
// the selector.
func (node *StringLiteral) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) string {
func (node *StringLiteral) Eval(timestamp clientmodel.Timestamp) string {
return node.str
}
// Eval implements the StringNode interface and returns the result of
// the function call.
func (node *StringFunctionCall) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) string {
return node.function.callFn(timestamp, view, node.args).(string)
func (node *StringFunctionCall) Eval(timestamp clientmodel.Timestamp) string {
return node.function.callFn(timestamp, node.args).(string)
}
// ----------------------------------------------------------------------------
@ -754,6 +858,8 @@ func NewScalarLiteral(value clientmodel.SampleValue) *ScalarLiteral {
func NewVectorSelector(m metric.LabelMatchers) *VectorSelector {
return &VectorSelector{
labelMatchers: m,
iterators: map[clientmodel.Fingerprint]storage_ng.SeriesIterator{},
metrics: map[clientmodel.Fingerprint]clientmodel.Metric{},
}
}
@ -845,6 +951,8 @@ func NewMatrixSelector(vector *VectorSelector, interval time.Duration) *MatrixSe
return &MatrixSelector{
labelMatchers: vector.labelMatchers,
interval: interval,
iterators: map[clientmodel.Fingerprint]storage_ng.SeriesIterator{},
metrics: map[clientmodel.Fingerprint]clientmodel.Metric{},
}
}

View file

@ -31,7 +31,7 @@ type Function struct {
name string
argTypes []ExprType
returnType ExprType
callFn func(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{}
callFn func(timestamp clientmodel.Timestamp, args []Node) interface{}
}
// CheckArgTypes returns a non-nil error if the number or types of
@ -74,14 +74,14 @@ func (function *Function) CheckArgTypes(args []Node) error {
}
// === time() clientmodel.SampleValue ===
func timeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
func timeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
return clientmodel.SampleValue(timestamp.Unix())
}
// === delta(matrix MatrixNode, isCounter ScalarNode) Vector ===
func deltaImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
func deltaImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
matrixNode := args[0].(MatrixNode)
isCounter := args[1].(ScalarNode).Eval(timestamp, view) > 0
isCounter := args[1].(ScalarNode).Eval(timestamp) > 0
resultVector := Vector{}
// If we treat these metrics as counters, we need to fetch all values
@ -89,9 +89,9 @@ func deltaImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node)
// I.e. if a counter resets, we want to ignore that reset.
var matrixValue Matrix
if isCounter {
matrixValue = matrixNode.Eval(timestamp, view)
matrixValue = matrixNode.Eval(timestamp)
} else {
matrixValue = matrixNode.EvalBoundaries(timestamp, view)
matrixValue = matrixNode.EvalBoundaries(timestamp)
}
for _, samples := range matrixValue {
// No sense in trying to compute a delta without at least two points. Drop
@ -139,9 +139,9 @@ func deltaImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node)
}
// === rate(node MatrixNode) Vector ===
func rateImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
func rateImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
args = append(args, &ScalarLiteral{value: 1})
vector := deltaImpl(timestamp, view, args).(Vector)
vector := deltaImpl(timestamp, args).(Vector)
// TODO: could be other type of MatrixNode in the future (right now, only
// MatrixSelector exists). Find a better way of getting the duration of a
@ -188,28 +188,28 @@ func (s reverseHeap) Less(i, j int) bool {
}
// === sort(node VectorNode) Vector ===
func sortImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
byValueSorter := vectorByValueHeap(args[0].(VectorNode).Eval(timestamp, view))
func sortImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
byValueSorter := vectorByValueHeap(args[0].(VectorNode).Eval(timestamp))
sort.Sort(byValueSorter)
return Vector(byValueSorter)
}
// === sortDesc(node VectorNode) Vector ===
func sortDescImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
byValueSorter := vectorByValueHeap(args[0].(VectorNode).Eval(timestamp, view))
func sortDescImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
byValueSorter := vectorByValueHeap(args[0].(VectorNode).Eval(timestamp))
sort.Sort(sort.Reverse(byValueSorter))
return Vector(byValueSorter)
}
// === topk(k ScalarNode, node VectorNode) Vector ===
func topkImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
k := int(args[0].(ScalarNode).Eval(timestamp, view))
func topkImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
k := int(args[0].(ScalarNode).Eval(timestamp))
if k < 1 {
return Vector{}
}
topk := make(vectorByValueHeap, 0, k)
vector := args[1].(VectorNode).Eval(timestamp, view)
vector := args[1].(VectorNode).Eval(timestamp)
for _, el := range vector {
if len(topk) < k || topk[0].Value < el.Value {
@ -224,15 +224,15 @@ func topkImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) i
}
// === bottomk(k ScalarNode, node VectorNode) Vector ===
func bottomkImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
k := int(args[0].(ScalarNode).Eval(timestamp, view))
func bottomkImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
k := int(args[0].(ScalarNode).Eval(timestamp))
if k < 1 {
return Vector{}
}
bottomk := make(vectorByValueHeap, 0, k)
bkHeap := reverseHeap{Interface: &bottomk}
vector := args[1].(VectorNode).Eval(timestamp, view)
vector := args[1].(VectorNode).Eval(timestamp)
for _, el := range vector {
if len(bottomk) < k || bottomk[0].Value > el.Value {
@ -247,8 +247,8 @@ func bottomkImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node
}
// === drop_common_labels(node VectorNode) Vector ===
func dropCommonLabelsImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
vector := args[0].(VectorNode).Eval(timestamp, view)
func dropCommonLabelsImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
vector := args[0].(VectorNode).Eval(timestamp)
if len(vector) < 1 {
return Vector{}
}
@ -285,7 +285,7 @@ func dropCommonLabelsImpl(timestamp clientmodel.Timestamp, view *viewAdapter, ar
}
// === sampleVectorImpl() Vector ===
func sampleVectorImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
func sampleVectorImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
return Vector{
&clientmodel.Sample{
Metric: clientmodel.Metric{
@ -358,8 +358,8 @@ func sampleVectorImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args [
}
// === scalar(node VectorNode) Scalar ===
func scalarImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
v := args[0].(VectorNode).Eval(timestamp, view)
func scalarImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
v := args[0].(VectorNode).Eval(timestamp)
if len(v) != 1 {
return clientmodel.SampleValue(math.NaN())
}
@ -367,13 +367,13 @@ func scalarImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node)
}
// === count_scalar(vector VectorNode) model.SampleValue ===
func countScalarImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
return clientmodel.SampleValue(len(args[0].(VectorNode).Eval(timestamp, view)))
func countScalarImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
return clientmodel.SampleValue(len(args[0].(VectorNode).Eval(timestamp)))
}
func aggrOverTime(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node, aggrFn func(metric.Values) clientmodel.SampleValue) interface{} {
func aggrOverTime(timestamp clientmodel.Timestamp, args []Node, aggrFn func(metric.Values) clientmodel.SampleValue) interface{} {
n := args[0].(MatrixNode)
matrixVal := n.Eval(timestamp, view)
matrixVal := n.Eval(timestamp)
resultVector := Vector{}
for _, el := range matrixVal {
@ -391,8 +391,8 @@ func aggrOverTime(timestamp clientmodel.Timestamp, view *viewAdapter, args []Nod
}
// === avg_over_time(matrix MatrixNode) Vector ===
func avgOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
return aggrOverTime(timestamp, view, args, func(values metric.Values) clientmodel.SampleValue {
func avgOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
return aggrOverTime(timestamp, args, func(values metric.Values) clientmodel.SampleValue {
var sum clientmodel.SampleValue
for _, v := range values {
sum += v.Value
@ -402,15 +402,15 @@ func avgOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []
}
// === count_over_time(matrix MatrixNode) Vector ===
func countOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
return aggrOverTime(timestamp, view, args, func(values metric.Values) clientmodel.SampleValue {
func countOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
return aggrOverTime(timestamp, args, func(values metric.Values) clientmodel.SampleValue {
return clientmodel.SampleValue(len(values))
})
}
// === max_over_time(matrix MatrixNode) Vector ===
func maxOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
return aggrOverTime(timestamp, view, args, func(values metric.Values) clientmodel.SampleValue {
func maxOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
return aggrOverTime(timestamp, args, func(values metric.Values) clientmodel.SampleValue {
max := math.Inf(-1)
for _, v := range values {
max = math.Max(max, float64(v.Value))
@ -420,8 +420,8 @@ func maxOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []
}
// === min_over_time(matrix MatrixNode) Vector ===
func minOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
return aggrOverTime(timestamp, view, args, func(values metric.Values) clientmodel.SampleValue {
func minOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
return aggrOverTime(timestamp, args, func(values metric.Values) clientmodel.SampleValue {
min := math.Inf(1)
for _, v := range values {
min = math.Min(min, float64(v.Value))
@ -431,8 +431,8 @@ func minOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []
}
// === sum_over_time(matrix MatrixNode) Vector ===
func sumOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
return aggrOverTime(timestamp, view, args, func(values metric.Values) clientmodel.SampleValue {
func sumOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
return aggrOverTime(timestamp, args, func(values metric.Values) clientmodel.SampleValue {
var sum clientmodel.SampleValue
for _, v := range values {
sum += v.Value
@ -442,9 +442,9 @@ func sumOverTimeImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []
}
// === abs(vector VectorNode) Vector ===
func absImpl(timestamp clientmodel.Timestamp, view *viewAdapter, args []Node) interface{} {
func absImpl(timestamp clientmodel.Timestamp, args []Node) interface{} {
n := args[0].(VectorNode)
vector := n.Eval(timestamp, view)
vector := n.Eval(timestamp)
for _, el := range vector {
el.Value = clientmodel.SampleValue(math.Abs(float64(el.Value)))
}

View file

@ -28,7 +28,7 @@ func (node emptyRangeNode) NodeTreeToDotGraph() string { return "" }
func (node emptyRangeNode) String() string { return "" }
func (node emptyRangeNode) Children() Nodes { return Nodes{} }
func (node emptyRangeNode) Eval(timestamp clientmodel.Timestamp, view *viewAdapter) Matrix {
func (node emptyRangeNode) Eval(timestamp clientmodel.Timestamp) Matrix {
return Matrix{
metric.SampleSet{
Metric: clientmodel.Metric{clientmodel.MetricNameLabel: "empty_metric"},
@ -37,7 +37,7 @@ func (node emptyRangeNode) Eval(timestamp clientmodel.Timestamp, view *viewAdapt
}
}
func (node emptyRangeNode) EvalBoundaries(timestamp clientmodel.Timestamp, view *viewAdapter) Matrix {
func (node emptyRangeNode) EvalBoundaries(timestamp clientmodel.Timestamp) Matrix {
return Matrix{
metric.SampleSet{
Metric: clientmodel.Metric{clientmodel.MetricNameLabel: "empty_metric"},
@ -48,11 +48,11 @@ func (node emptyRangeNode) EvalBoundaries(timestamp clientmodel.Timestamp, view
func TestDeltaWithEmptyElementDoesNotCrash(t *testing.T) {
now := clientmodel.Now()
vector := deltaImpl(now, nil, []Node{emptyRangeNode{}, &ScalarLiteral{value: 0}}).(Vector)
vector := deltaImpl(now, []Node{emptyRangeNode{}, &ScalarLiteral{value: 0}}).(Vector)
if len(vector) != 0 {
t.Fatalf("Expected empty result vector, got: %v", vector)
}
vector = deltaImpl(now, nil, []Node{emptyRangeNode{}, &ScalarLiteral{value: 1}}).(Vector)
vector = deltaImpl(now, []Node{emptyRangeNode{}, &ScalarLiteral{value: 1}}).(Vector)
if len(vector) != 0 {
t.Fatalf("Expected empty result vector, got: %v", vector)
}

View file

@ -23,7 +23,7 @@ import (
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/stats"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/local"
"github.com/prometheus/prometheus/utility"
)
@ -151,18 +151,19 @@ func TypedValueToJSON(data interface{}, typeStr string) string {
}
// EvalToString evaluates the given node into a string of the given format.
func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputFormat, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) string {
viewTimer := queryStats.GetTimer(stats.TotalViewBuildingTime).Start()
viewAdapter, err := viewAdapterForInstantQuery(node, timestamp, storage, queryStats)
viewTimer.Stop()
func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputFormat, storage storage_ng.Storage, queryStats *stats.TimerGroup) string {
prepareTimer := queryStats.GetTimer(stats.TotalQueryPreparationTime).Start()
closer, err := prepareInstantQuery(node, timestamp, storage, queryStats)
prepareTimer.Stop()
if err != nil {
panic(err)
}
defer closer.Close()
evalTimer := queryStats.GetTimer(stats.InnerEvalTime).Start()
switch node.Type() {
case SCALAR:
scalar := node.(ScalarNode).Eval(timestamp, viewAdapter)
scalar := node.(ScalarNode).Eval(timestamp)
evalTimer.Stop()
switch format {
case TEXT:
@ -171,7 +172,7 @@ func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputForma
return TypedValueToJSON(scalar, "scalar")
}
case VECTOR:
vector := node.(VectorNode).Eval(timestamp, viewAdapter)
vector := node.(VectorNode).Eval(timestamp)
evalTimer.Stop()
switch format {
case TEXT:
@ -180,7 +181,7 @@ func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputForma
return TypedValueToJSON(vector, "vector")
}
case MATRIX:
matrix := node.(MatrixNode).Eval(timestamp, viewAdapter)
matrix := node.(MatrixNode).Eval(timestamp)
evalTimer.Stop()
switch format {
case TEXT:
@ -189,7 +190,7 @@ func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputForma
return TypedValueToJSON(matrix, "matrix")
}
case STRING:
str := node.(StringNode).Eval(timestamp, viewAdapter)
str := node.(StringNode).Eval(timestamp)
evalTimer.Stop()
switch format {
case TEXT:
@ -202,28 +203,29 @@ func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputForma
}
// EvalToVector evaluates the given node into a Vector. Matrices aren't supported.
func EvalToVector(node Node, timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) (Vector, error) {
viewTimer := queryStats.GetTimer(stats.TotalViewBuildingTime).Start()
viewAdapter, err := viewAdapterForInstantQuery(node, timestamp, storage, queryStats)
viewTimer.Stop()
func EvalToVector(node Node, timestamp clientmodel.Timestamp, storage storage_ng.Storage, queryStats *stats.TimerGroup) (Vector, error) {
prepareTimer := queryStats.GetTimer(stats.TotalQueryPreparationTime).Start()
closer, err := prepareInstantQuery(node, timestamp, storage, queryStats)
prepareTimer.Stop()
if err != nil {
panic(err)
}
defer closer.Close()
evalTimer := queryStats.GetTimer(stats.InnerEvalTime).Start()
switch node.Type() {
case SCALAR:
scalar := node.(ScalarNode).Eval(timestamp, viewAdapter)
scalar := node.(ScalarNode).Eval(timestamp)
evalTimer.Stop()
return Vector{&clientmodel.Sample{Value: scalar}}, nil
case VECTOR:
vector := node.(VectorNode).Eval(timestamp, viewAdapter)
vector := node.(VectorNode).Eval(timestamp)
evalTimer.Stop()
return vector, nil
case MATRIX:
return nil, errors.New("Matrices not supported by EvalToVector")
case STRING:
str := node.(StringNode).Eval(timestamp, viewAdapter)
str := node.(StringNode).Eval(timestamp)
evalTimer.Stop()
return Vector{&clientmodel.Sample{
Metric: clientmodel.Metric{"__value__": clientmodel.LabelValue(str)}}}, nil

View file

@ -16,12 +16,10 @@ package ast
import (
"time"
"github.com/golang/glog"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/stats"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/local"
)
// FullRangeMap maps the fingerprint of a full range to the duration
@ -48,13 +46,13 @@ type QueryAnalyzer struct {
IntervalRanges IntervalRangeMap
// The underlying storage to which the query will be applied. Needed for
// extracting timeseries fingerprint information during query analysis.
storage metric.Persistence
storage storage_ng.Storage
}
// NewQueryAnalyzer returns a pointer to a newly instantiated
// QueryAnalyzer. The storage is needed to extract timeseries
// fingerprint information during query analysis.
func NewQueryAnalyzer(storage metric.Persistence) *QueryAnalyzer {
func NewQueryAnalyzer(storage storage_ng.Storage) *QueryAnalyzer {
return &QueryAnalyzer{
FullRanges: FullRangeMap{},
IntervalRanges: IntervalRangeMap{},
@ -66,94 +64,122 @@ func NewQueryAnalyzer(storage metric.Persistence) *QueryAnalyzer {
func (analyzer *QueryAnalyzer) Visit(node Node) {
switch n := node.(type) {
case *VectorSelector:
fingerprints, err := analyzer.storage.GetFingerprintsForLabelMatchers(n.labelMatchers)
if err != nil {
glog.Errorf("Error getting fingerprints for label matchers %v: %v", n.labelMatchers, err)
return
}
fingerprints := analyzer.storage.GetFingerprintsForLabelMatchers(n.labelMatchers)
n.fingerprints = fingerprints
for _, fingerprint := range fingerprints {
for _, fp := range fingerprints {
// Only add the fingerprint to IntervalRanges if not yet present in FullRanges.
// Full ranges always contain more points and span more time than interval ranges.
if _, alreadyInFullRanges := analyzer.FullRanges[*fingerprint]; !alreadyInFullRanges {
analyzer.IntervalRanges[*fingerprint] = true
if _, alreadyInFullRanges := analyzer.FullRanges[fp]; !alreadyInFullRanges {
analyzer.IntervalRanges[fp] = true
}
n.metrics[fp] = analyzer.storage.GetMetricForFingerprint(fp)
}
case *MatrixSelector:
fingerprints, err := analyzer.storage.GetFingerprintsForLabelMatchers(n.labelMatchers)
if err != nil {
glog.Errorf("Error getting fingerprints for label matchers %v: %v", n.labelMatchers, err)
return
}
fingerprints := analyzer.storage.GetFingerprintsForLabelMatchers(n.labelMatchers)
n.fingerprints = fingerprints
for _, fingerprint := range fingerprints {
if analyzer.FullRanges[*fingerprint] < n.interval {
analyzer.FullRanges[*fingerprint] = n.interval
for _, fp := range fingerprints {
if analyzer.FullRanges[fp] < n.interval {
analyzer.FullRanges[fp] = n.interval
// Delete the fingerprint from IntervalRanges. Full ranges always contain
// more points and span more time than interval ranges, so we don't need
// an interval range for the same fingerprint, should we have one.
delete(analyzer.IntervalRanges, *fingerprint)
delete(analyzer.IntervalRanges, fp)
}
n.metrics[fp] = analyzer.storage.GetMetricForFingerprint(fp)
}
}
}
// AnalyzeQueries walks the AST, starting at node, calling Visit on
// each node to collect fingerprints.
func (analyzer *QueryAnalyzer) AnalyzeQueries(node Node) {
type iteratorInitializer struct {
storage storage_ng.Storage
}
func (i *iteratorInitializer) Visit(node Node) {
switch n := node.(type) {
case *VectorSelector:
for _, fp := range n.fingerprints {
n.iterators[fp] = i.storage.NewIterator(fp)
}
case *MatrixSelector:
for _, fp := range n.fingerprints {
n.iterators[fp] = i.storage.NewIterator(fp)
}
}
}
func prepareInstantQuery(node Node, timestamp clientmodel.Timestamp, storage storage_ng.Storage, queryStats *stats.TimerGroup) (storage_ng.Closer, error) {
analyzeTimer := queryStats.GetTimer(stats.QueryAnalysisTime).Start()
analyzer := NewQueryAnalyzer(storage)
Walk(analyzer, node)
}
func viewAdapterForInstantQuery(node Node, timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) (*viewAdapter, error) {
analyzeTimer := queryStats.GetTimer(stats.QueryAnalysisTime).Start()
analyzer := NewQueryAnalyzer(storage)
analyzer.AnalyzeQueries(node)
analyzeTimer.Stop()
requestBuildTimer := queryStats.GetTimer(stats.ViewRequestBuildTime).Start()
viewBuilder := storage.NewViewRequestBuilder()
for fingerprint, rangeDuration := range analyzer.FullRanges {
viewBuilder.GetMetricRange(&fingerprint, timestamp.Add(-rangeDuration), timestamp)
}
for fingerprint := range analyzer.IntervalRanges {
viewBuilder.GetMetricAtTime(&fingerprint, timestamp)
}
requestBuildTimer.Stop()
buildTimer := queryStats.GetTimer(stats.InnerViewBuildingTime).Start()
view, err := viewBuilder.Execute(60*time.Second, queryStats)
buildTimer.Stop()
if err != nil {
return nil, err
}
return NewViewAdapter(view, storage, queryStats), nil
}
func viewAdapterForRangeQuery(node Node, start clientmodel.Timestamp, end clientmodel.Timestamp, interval time.Duration, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) (*viewAdapter, error) {
analyzeTimer := queryStats.GetTimer(stats.QueryAnalysisTime).Start()
analyzer := NewQueryAnalyzer(storage)
analyzer.AnalyzeQueries(node)
analyzeTimer.Stop()
requestBuildTimer := queryStats.GetTimer(stats.ViewRequestBuildTime).Start()
viewBuilder := storage.NewViewRequestBuilder()
for fingerprint, rangeDuration := range analyzer.FullRanges {
if interval < rangeDuration {
viewBuilder.GetMetricRange(&fingerprint, start.Add(-rangeDuration), end)
} else {
viewBuilder.GetMetricRangeAtInterval(&fingerprint, start.Add(-rangeDuration), end, interval, rangeDuration)
// TODO: Preloading should time out after a given duration.
preloadTimer := queryStats.GetTimer(stats.PreloadTime).Start()
p := storage.NewPreloader()
for fp, rangeDuration := range analyzer.FullRanges {
if err := p.PreloadRange(fp, timestamp.Add(-rangeDuration), timestamp); err != nil {
p.Close()
return nil, err
}
}
for fingerprint := range analyzer.IntervalRanges {
viewBuilder.GetMetricAtInterval(&fingerprint, start, end, interval)
for fp := range analyzer.IntervalRanges {
if err := p.PreloadRange(fp, timestamp, timestamp); err != nil {
p.Close()
return nil, err
}
}
requestBuildTimer.Stop()
preloadTimer.Stop()
buildTimer := queryStats.GetTimer(stats.InnerViewBuildingTime).Start()
view, err := viewBuilder.Execute(time.Duration(60)*time.Second, queryStats)
buildTimer.Stop()
if err != nil {
return nil, err
ii := &iteratorInitializer{
storage: storage,
}
return NewViewAdapter(view, storage, queryStats), nil
Walk(ii, node)
return p, nil
}
func prepareRangeQuery(node Node, start clientmodel.Timestamp, end clientmodel.Timestamp, interval time.Duration, storage storage_ng.Storage, queryStats *stats.TimerGroup) (storage_ng.Closer, error) {
analyzeTimer := queryStats.GetTimer(stats.QueryAnalysisTime).Start()
analyzer := NewQueryAnalyzer(storage)
Walk(analyzer, node)
analyzeTimer.Stop()
// TODO: Preloading should time out after a given duration.
preloadTimer := queryStats.GetTimer(stats.PreloadTime).Start()
p := storage.NewPreloader()
for fp, rangeDuration := range analyzer.FullRanges {
if err := p.PreloadRange(fp, start.Add(-rangeDuration), end); err != nil {
p.Close()
return nil, err
}
/*
if interval < rangeDuration {
if err := p.GetMetricRange(fp, end, end.Sub(start)+rangeDuration); err != nil {
p.Close()
return nil, err
}
} else {
if err := p.GetMetricRangeAtInterval(fp, start, end, interval, rangeDuration); err != nil {
p.Close()
return nil, err
}
}
*/
}
for fp := range analyzer.IntervalRanges {
if err := p.PreloadRange(fp, start, end); err != nil {
p.Close()
return nil, err
}
}
preloadTimer.Stop()
ii := &iteratorInitializer{
storage: storage,
}
Walk(ii, node)
return p, nil
}

View file

@ -13,186 +13,4 @@
package ast
import (
"flag"
"time"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/stats"
"github.com/prometheus/prometheus/storage/metric"
)
var defaultStalenessDelta = flag.Int("defaultStalenessDelta", 300, "Default staleness delta allowance in seconds during expression evaluations.")
// StalenessPolicy describes the lenience limits to apply to values
// from 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
}
type viewAdapter struct {
// Policy that dictates when sample values around an evaluation time are to
// be interpreted as stale.
stalenessPolicy StalenessPolicy
// AST-global storage to use for operations that are not supported by views
// (i.e. fingerprint->metric lookups).
storage metric.Persistence
// The materialized view which contains all timeseries data required for
// executing a query.
view metric.View
// The TimerGroup object in which to capture query timing statistics.
stats *stats.TimerGroup
}
// interpolateSamples interpolates a value at a target time between two
// provided sample pairs.
func interpolateSamples(first, second *metric.SamplePair, timestamp clientmodel.Timestamp) *metric.SamplePair {
dv := second.Value - first.Value
dt := second.Timestamp.Sub(first.Timestamp)
dDt := dv / clientmodel.SampleValue(dt)
offset := clientmodel.SampleValue(timestamp.Sub(first.Timestamp))
return &metric.SamplePair{
Value: first.Value + (offset * dDt),
Timestamp: timestamp,
}
}
// chooseClosestSample chooses the closest sample of a list of samples
// surrounding a given target time. If samples are found both before and after
// the target time, the sample value is interpolated between these. Otherwise,
// the single closest sample is returned verbatim.
func (v *viewAdapter) chooseClosestSample(samples metric.Values, timestamp clientmodel.Timestamp) *metric.SamplePair {
var closestBefore *metric.SamplePair
var closestAfter *metric.SamplePair
for _, candidate := range samples {
delta := candidate.Timestamp.Sub(timestamp)
// Samples before target time.
if delta < 0 {
// Ignore samples outside of staleness policy window.
if -delta > v.stalenessPolicy.DeltaAllowance {
continue
}
// Ignore samples that are farther away than what we've seen before.
if closestBefore != nil && candidate.Timestamp.Before(closestBefore.Timestamp) {
continue
}
sample := candidate
closestBefore = &sample
}
// Samples after target time.
if delta >= 0 {
// Ignore samples outside of staleness policy window.
if delta > v.stalenessPolicy.DeltaAllowance {
continue
}
// Ignore samples that are farther away than samples we've seen before.
if closestAfter != nil && candidate.Timestamp.After(closestAfter.Timestamp) {
continue
}
sample := candidate
closestAfter = &sample
}
}
switch {
case closestBefore != nil && closestAfter != nil:
return interpolateSamples(closestBefore, closestAfter, timestamp)
case closestBefore != nil:
return closestBefore
default:
return closestAfter
}
}
func (v *viewAdapter) GetValueAtTime(fingerprints clientmodel.Fingerprints, timestamp clientmodel.Timestamp) (Vector, error) {
timer := v.stats.GetTimer(stats.GetValueAtTimeTime).Start()
samples := Vector{}
for _, fingerprint := range fingerprints {
sampleCandidates := v.view.GetValueAtTime(fingerprint, timestamp)
samplePair := v.chooseClosestSample(sampleCandidates, timestamp)
m, err := v.storage.GetMetricForFingerprint(fingerprint)
if err != nil {
return nil, err
}
if samplePair != nil {
samples = append(samples, &clientmodel.Sample{
Metric: m,
Value: samplePair.Value,
Timestamp: timestamp,
})
}
}
timer.Stop()
return samples, nil
}
func (v *viewAdapter) GetBoundaryValues(fingerprints clientmodel.Fingerprints, interval *metric.Interval) ([]metric.SampleSet, error) {
timer := v.stats.GetTimer(stats.GetBoundaryValuesTime).Start()
sampleSets := []metric.SampleSet{}
for _, fingerprint := range fingerprints {
samplePairs := v.view.GetBoundaryValues(fingerprint, *interval)
if len(samplePairs) == 0 {
continue
}
// TODO: memoize/cache this.
m, err := v.storage.GetMetricForFingerprint(fingerprint)
if err != nil {
return nil, err
}
sampleSet := metric.SampleSet{
Metric: m,
Values: samplePairs,
}
sampleSets = append(sampleSets, sampleSet)
}
timer.Stop()
return sampleSets, nil
}
func (v *viewAdapter) GetRangeValues(fingerprints clientmodel.Fingerprints, interval *metric.Interval) ([]metric.SampleSet, error) {
timer := v.stats.GetTimer(stats.GetRangeValuesTime).Start()
sampleSets := []metric.SampleSet{}
for _, fingerprint := range fingerprints {
samplePairs := v.view.GetRangeValues(fingerprint, *interval)
if len(samplePairs) == 0 {
continue
}
// TODO: memoize/cache this.
m, err := v.storage.GetMetricForFingerprint(fingerprint)
if err != nil {
return nil, err
}
sampleSet := metric.SampleSet{
Metric: m,
Values: samplePairs,
}
sampleSets = append(sampleSets, sampleSet)
}
timer.Stop()
return sampleSets, nil
}
// NewViewAdapter returns an initialized view adapter with a default
// staleness policy (based on the --defaultStalenessDelta flag).
func NewViewAdapter(view metric.View, storage metric.Persistence, queryStats *stats.TimerGroup) *viewAdapter {
stalenessPolicy := StalenessPolicy{
DeltaAllowance: time.Duration(*defaultStalenessDelta) * time.Second,
}
return &viewAdapter{
stalenessPolicy: stalenessPolicy,
storage: storage,
view: view,
stats: queryStats,
}
}
// TODO: remove file.

View file

@ -19,6 +19,7 @@ import (
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/rules/ast"
"github.com/prometheus/prometheus/storage/local"
"github.com/prometheus/prometheus/storage/metric"
)
@ -51,7 +52,7 @@ func getTestVectorFromTestMatrix(matrix ast.Matrix) ast.Vector {
return vector
}
func storeMatrix(storage metric.Persistence, matrix ast.Matrix) (err error) {
func storeMatrix(storage storage_ng.Storage, matrix ast.Matrix) {
pendingSamples := clientmodel.Samples{}
for _, sampleSet := range matrix {
for _, sample := range sampleSet.Values {
@ -62,8 +63,7 @@ func storeMatrix(storage metric.Persistence, matrix ast.Matrix) (err error) {
})
}
}
err = storage.AppendSamples(pendingSamples)
return
storage.AppendSamples(pendingSamples)
}
var testMatrix = ast.Matrix{

View file

@ -27,7 +27,7 @@ import (
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/notification"
"github.com/prometheus/prometheus/rules"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/local"
"github.com/prometheus/prometheus/templates"
)
@ -83,7 +83,7 @@ type ruleManager struct {
done chan bool
interval time.Duration
storage metric.PreloadingPersistence
storage storage_ng.Storage
results chan<- *extraction.Result
notifications chan<- notification.NotificationReqs
@ -93,7 +93,7 @@ type ruleManager struct {
type RuleManagerOptions struct {
EvaluationInterval time.Duration
Storage metric.PreloadingPersistence
Storage storage_ng.Storage
Notifications chan<- notification.NotificationReqs
Results chan<- *extraction.Result
@ -126,17 +126,14 @@ func (m *ruleManager) Run() {
m.runIteration(m.results)
iterationDuration.Observe(float64(time.Since(start) / time.Millisecond))
case <-m.done:
glog.Info("rules.Rule manager exiting...")
glog.Info("Rule manager exiting...")
return
}
}
}
func (m *ruleManager) Stop() {
select {
case m.done <- true:
default:
}
m.done <- true
}
func (m *ruleManager) queueAlertNotifications(rule *rules.AlertingRule, timestamp clientmodel.Timestamp) {

View file

@ -21,7 +21,7 @@ import (
"github.com/prometheus/prometheus/rules/ast"
"github.com/prometheus/prometheus/stats"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/local"
)
// A RecordingRule records its vector expression into new timeseries.
@ -34,11 +34,11 @@ type RecordingRule struct {
func (rule RecordingRule) Name() string { return rule.name }
func (rule RecordingRule) EvalRaw(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error) {
func (rule RecordingRule) EvalRaw(timestamp clientmodel.Timestamp, storage storage_ng.Storage) (ast.Vector, error) {
return ast.EvalVectorInstant(rule.vector, timestamp, storage, stats.NewTimerGroup())
}
func (rule RecordingRule) Eval(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error) {
func (rule RecordingRule) Eval(timestamp clientmodel.Timestamp, storage storage_ng.Storage) (ast.Vector, error) {
// Get the raw value of the rule expression.
vector, err := rule.EvalRaw(timestamp, storage)
if err != nil {

View file

@ -19,7 +19,7 @@ import (
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/rules/ast"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/local"
)
// A Rule encapsulates a vector expression which is evaluated at a specified
@ -29,9 +29,9 @@ type Rule interface {
Name() string
// EvalRaw evaluates the rule's vector expression without triggering any
// other actions, like recording or alerting.
EvalRaw(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error)
EvalRaw(timestamp clientmodel.Timestamp, storage storage_ng.Storage) (ast.Vector, error)
// Eval evaluates the rule, including any associated recording or alerting actions.
Eval(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error)
Eval(timestamp clientmodel.Timestamp, storage storage_ng.Storage) (ast.Vector, error)
// ToDotGraph returns a Graphviz dot graph of the rule.
ToDotGraph() string
// String returns a human-readable string representation of the rule.

View file

@ -24,7 +24,7 @@ import (
"github.com/prometheus/prometheus/rules/ast"
"github.com/prometheus/prometheus/stats"
"github.com/prometheus/prometheus/storage/metric/tiered"
"github.com/prometheus/prometheus/storage/local"
"github.com/prometheus/prometheus/utility/test"
)
@ -52,23 +52,10 @@ func vectorComparisonString(expected []string, actual []string) string {
separator)
}
type testTieredStorageCloser struct {
storage *tiered.TieredStorage
directory test.Closer
}
func (t testTieredStorageCloser) Close() {
t.storage.Close()
t.directory.Close()
}
func newTestStorage(t testing.TB) (storage *tiered.TieredStorage, closer test.Closer) {
storage, closer = tiered.NewTestTieredStorage(t)
if storage == nil {
t.Fatal("storage == nil")
}
func newTestStorage(t testing.TB) (storage storage_ng.Storage, closer test.Closer) {
storage, closer = storage_ng.NewTestStorage(t)
storeMatrix(storage, testMatrix)
return
return storage, closer
}
func TestExpressions(t *testing.T) {
@ -551,9 +538,8 @@ func TestExpressions(t *testing.T) {
},
}
tieredStorage, closer := newTestStorage(t)
storage, closer := newTestStorage(t)
defer closer.Close()
tieredStorage.Flush()
for i, exprTest := range expressionTests {
expectedLines := annotateWithTime(exprTest.output, testEvalTime)
@ -571,7 +557,7 @@ func TestExpressions(t *testing.T) {
t.Errorf("%d. Test should fail, but didn't", i)
}
failed := false
resultStr := ast.EvalToString(testExpr, testEvalTime, ast.TEXT, tieredStorage, stats.NewTimerGroup())
resultStr := ast.EvalToString(testExpr, testEvalTime, ast.TEXT, storage, stats.NewTimerGroup())
resultLines := strings.Split(resultStr, "\n")
if len(exprTest.output) != len(resultLines) {
@ -601,8 +587,8 @@ func TestExpressions(t *testing.T) {
}
}
analyzer := ast.NewQueryAnalyzer(tieredStorage)
analyzer.AnalyzeQueries(testExpr)
analyzer := ast.NewQueryAnalyzer(storage)
ast.Walk(analyzer, testExpr)
if exprTest.fullRanges != len(analyzer.FullRanges) {
t.Errorf("%d. Count of full ranges didn't match: %v vs %v", i, exprTest.fullRanges, len(analyzer.FullRanges))
failed = true
@ -711,9 +697,8 @@ func TestAlertingRule(t *testing.T) {
},
}
tieredStorage, closer := newTestStorage(t)
storage, closer := newTestStorage(t)
defer closer.Close()
tieredStorage.Flush()
alertExpr, err := LoadExprFromString(`http_requests{group="canary", job="app-server"} < 100`)
if err != nil {
@ -727,7 +712,7 @@ func TestAlertingRule(t *testing.T) {
for i, expected := range evalOutputs {
evalTime := testStartTime.Add(testSampleInterval * time.Duration(i))
actual, err := rule.Eval(evalTime, tieredStorage)
actual, err := rule.Eval(evalTime, storage)
if err != nil {
t.Fatalf("Error during alerting rule evaluation: %s", err)
}

View file

@ -22,8 +22,8 @@ const (
TotalEvalTime QueryTiming = iota
ResultSortTime
JsonEncodeTime
TotalViewBuildingTime
ViewRequestBuildTime
PreloadTime
TotalQueryPreparationTime
InnerViewBuildingTime
InnerEvalTime
ResultAppendTime
@ -46,10 +46,10 @@ func (s QueryTiming) String() string {
return "Result sorting time"
case JsonEncodeTime:
return "JSON encoding time"
case TotalViewBuildingTime:
return "Total view building time"
case ViewRequestBuildTime:
return "View request building time"
case PreloadTime:
return "Query preloading time"
case TotalQueryPreparationTime:
return "Total query preparation time"
case InnerViewBuildingTime:
return "Inner view building time"
case InnerEvalTime:

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 storage
// RecordDecoder decodes each key-value pair in the database. The protocol
// around it makes the assumption that the underlying implementation is
// concurrency safe.
type RecordDecoder interface {
DecodeKey(in interface{}) (out interface{}, err error)
DecodeValue(in interface{}) (out interface{}, err error)
}
// FilterResult describes the record matching and scanning behavior for the
// database.
type FilterResult int
const (
// Stop scanning the database.
Stop FilterResult = iota
// Skip this record but continue scanning.
Skip
// Accept this record for the Operator.
Accept
)
func (f FilterResult) String() string {
switch f {
case Stop:
return "STOP"
case Skip:
return "SKIP"
case Accept:
return "ACCEPT"
}
panic("unknown")
}
// OperatorError is used for storage operations upon errors that may or may not
// be continuable.
type OperatorError struct {
Error error
Continuable bool
}
// RecordFilter is responsible for controlling the behavior of the database scan
// process and determines the disposition of various records.
//
// The protocol around it makes the assumption that the underlying
// implementation is concurrency safe.
type RecordFilter interface {
// Filter receives the key and value as decoded from the RecordDecoder type.
Filter(key, value interface{}) (filterResult FilterResult)
}
// RecordOperator is responsible for taking action upon each entity that is
// passed to it.
//
// The protocol around it makes the assumption that the underlying
// implementation is concurrency safe.
type RecordOperator interface {
// Take action on a given record. If the action returns an error, the entire
// scan process stops.
Operate(key, value interface{}) (err *OperatorError)
}

67
storage/local/chunk.go Normal file
View file

@ -0,0 +1,67 @@
package storage_ng
import (
"io"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
)
type chunks []chunk
type chunk interface {
add(*metric.SamplePair) chunks
clone() chunk
firstTime() clientmodel.Timestamp
lastTime() clientmodel.Timestamp
newIterator() chunkIterator
marshal(io.Writer) error
unmarshal(io.Reader) error
close()
// TODO: remove?
values() <-chan *metric.SamplePair
}
type chunkIterator interface {
getValueAtTime(clientmodel.Timestamp) metric.Values
getBoundaryValues(metric.Interval) metric.Values
getRangeValues(metric.Interval) metric.Values
contains(clientmodel.Timestamp) bool
}
func transcodeAndAdd(dst chunk, src chunk, s *metric.SamplePair) chunks {
numTranscodes.Inc()
defer src.close()
head := dst
body := chunks{}
for v := range src.values() {
newChunks := head.add(v)
body = append(body, newChunks[:len(newChunks)-1]...)
head = newChunks[len(newChunks)-1]
}
newChunks := head.add(s)
body = append(body, newChunks[:len(newChunks)-1]...)
head = newChunks[len(newChunks)-1]
return append(body, head)
}
func chunkType(c chunk) byte {
switch c.(type) {
case *deltaEncodedChunk:
return 0
default:
panic("unknown chunk type")
}
}
func chunkForType(chunkType byte) chunk {
switch chunkType {
case 0:
return newDeltaEncodedChunk(1, 1, false)
default:
panic("unknown chunk type")
}
}

399
storage/local/delta.go Normal file
View file

@ -0,0 +1,399 @@
package storage_ng
import (
"encoding/binary"
"fmt"
"io"
"math"
"sort"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
)
type deltaBytes int
const (
d0 deltaBytes = 0
d1 = 1
d2 = 2
d4 = 4
d8 = 8
)
// The 21-byte header of a delta-encoded chunk looks like:
//
// - time delta bytes: 1 bytes
// - value delta bytes: 1 bytes
// - is integer: 1 byte
// - base time: 8 bytes
// - base value: 8 bytes
// - used buf bytes: 2 bytes
const (
deltaHeaderBytes = 21
deltaHeaderTimeBytesOffset = 0
deltaHeaderValueBytesOffset = 1
deltaHeaderIsIntOffset = 2
deltaHeaderBaseTimeOffset = 3
deltaHeaderBaseValueOffset = 11
deltaHeaderBufLenOffset = 19
)
type deltaEncodedChunk struct {
buf []byte
}
func newDeltaEncodedChunk(tb, vb deltaBytes, isInt bool) *deltaEncodedChunk {
buf := chunkBufs.Get()
buf = buf[:deltaHeaderIsIntOffset+1]
buf[deltaHeaderTimeBytesOffset] = byte(tb)
buf[deltaHeaderValueBytesOffset] = byte(vb)
if isInt {
buf[deltaHeaderIsIntOffset] = 1
} else {
buf[deltaHeaderIsIntOffset] = 0
}
return &deltaEncodedChunk{
buf: buf,
}
}
func (c *deltaEncodedChunk) newFollowupChunk() chunk {
return newDeltaEncodedChunk(d1, d1, true)
//return newDeltaEncodedChunk(c.timeBytes(), c.valueBytes(), c.isInt())
}
func (c *deltaEncodedChunk) clone() chunk {
buf := chunkBufs.Get()
buf = buf[:len(c.buf)]
copy(buf, c.buf)
return &deltaEncodedChunk{
buf: buf,
}
}
func neededDeltaBytes(deltaT clientmodel.Timestamp, deltaV clientmodel.SampleValue, isInt bool) (dtb, dvb deltaBytes) {
dtb = 1
if deltaT >= 256 {
dtb = 2
}
if deltaT >= 256*256 {
dtb = 4
}
if deltaT >= 256*256*256*256 {
dtb = 8
}
if isInt {
dvb = 0
if deltaV != 0 {
dvb = 1
}
if deltaV < -(256/2) || deltaV > (256/2)-1 {
dvb = 2
}
if deltaV < -(256*256/2) || deltaV > (256*256/2)-1 {
dvb = 4
}
if deltaV < -(256*256*256*256/2) || deltaV > (256*256*256*256/2)-1 {
dvb = 8
}
} else {
dvb = 4
if clientmodel.SampleValue(float32(deltaV)) != deltaV {
dvb = 8
}
}
return dtb, dvb
}
func max(a, b deltaBytes) deltaBytes {
if a > b {
return a
}
return b
}
func (c *deltaEncodedChunk) timeBytes() deltaBytes {
return deltaBytes(c.buf[deltaHeaderTimeBytesOffset])
}
func (c *deltaEncodedChunk) valueBytes() deltaBytes {
return deltaBytes(c.buf[deltaHeaderValueBytesOffset])
}
func (c *deltaEncodedChunk) isInt() bool {
return c.buf[deltaHeaderIsIntOffset] == 1
}
func (c *deltaEncodedChunk) baseTime() clientmodel.Timestamp {
return clientmodel.Timestamp(binary.LittleEndian.Uint64(c.buf[deltaHeaderBaseTimeOffset:]))
}
func (c *deltaEncodedChunk) baseValue() clientmodel.SampleValue {
return clientmodel.SampleValue(math.Float64frombits(binary.LittleEndian.Uint64(c.buf[deltaHeaderBaseValueOffset:])))
}
func (c *deltaEncodedChunk) add(s *metric.SamplePair) chunks {
if len(c.buf) < deltaHeaderBytes {
c.buf = c.buf[:deltaHeaderBytes]
binary.LittleEndian.PutUint64(c.buf[deltaHeaderBaseTimeOffset:], uint64(s.Timestamp))
binary.LittleEndian.PutUint64(c.buf[deltaHeaderBaseValueOffset:], math.Float64bits(float64(s.Value)))
}
remainingBytes := cap(c.buf) - len(c.buf)
sampleSize := c.sampleSize()
// Do we generally have space for another sample in this chunk? If not,
// overflow into a new one. We assume that if we have seen floating point
// values once, the series will most likely contain floats in the future.
if remainingBytes < sampleSize {
//fmt.Println("overflow")
overflowChunks := c.newFollowupChunk().add(s)
return chunks{c, overflowChunks[0]}
}
dt := s.Timestamp - c.baseTime()
dv := s.Value - c.baseValue()
// If the new sample is incompatible with the current encoding, reencode the
// existing chunk data into new chunk(s).
//
// int->float.
// TODO: compare speed with Math.Modf.
if c.isInt() && clientmodel.SampleValue(int64(dv)) != dv {
//fmt.Println("int->float", len(c.buf), cap(c.buf))
return transcodeAndAdd(newDeltaEncodedChunk(c.timeBytes(), d4, false), c, s)
}
// float32->float64.
if !c.isInt() && c.valueBytes() == d4 && clientmodel.SampleValue(float32(dv)) != dv {
//fmt.Println("float32->float64", float32(dv), dv, len(c.buf), cap(c.buf))
return transcodeAndAdd(newDeltaEncodedChunk(c.timeBytes(), d8, false), c, s)
}
// More bytes per sample.
if dtb, dvb := neededDeltaBytes(dt, dv, c.isInt()); dtb > c.timeBytes() || dvb > c.valueBytes() {
//fmt.Printf("transcoding T: %v->%v, V: %v->%v, I: %v; len %v, cap %v\n", c.timeBytes(), dtb, c.valueBytes(), dvb, c.isInt(), len(c.buf), cap(c.buf))
dtb = max(dtb, c.timeBytes())
dvb = max(dvb, c.valueBytes())
return transcodeAndAdd(newDeltaEncodedChunk(dtb, dvb, c.isInt()), c, s)
}
offset := len(c.buf)
c.buf = c.buf[:offset+sampleSize]
switch c.timeBytes() {
case 1:
c.buf[offset] = byte(dt)
case 2:
binary.LittleEndian.PutUint16(c.buf[offset:], uint16(dt))
case 4:
binary.LittleEndian.PutUint32(c.buf[offset:], uint32(dt))
case 8:
binary.LittleEndian.PutUint64(c.buf[offset:], uint64(dt))
}
offset += int(c.timeBytes())
if c.isInt() {
switch c.valueBytes() {
case 0:
// No-op. Constant value is stored as base value.
case 1:
c.buf[offset] = byte(dv)
case 2:
binary.LittleEndian.PutUint16(c.buf[offset:], uint16(dv))
case 4:
binary.LittleEndian.PutUint32(c.buf[offset:], uint32(dv))
case 8:
binary.LittleEndian.PutUint64(c.buf[offset:], uint64(dv))
default:
panic("Invalid number of bytes for integer delta")
}
} else {
switch c.valueBytes() {
case 4:
binary.LittleEndian.PutUint32(c.buf[offset:], math.Float32bits(float32(dv)))
case 8:
binary.LittleEndian.PutUint64(c.buf[offset:], math.Float64bits(float64(dv)))
default:
panic("Invalid number of bytes for floating point delta")
}
}
return chunks{c}
}
func (c *deltaEncodedChunk) close() {
//fmt.Println("returning chunk")
chunkBufs.Give(c.buf)
}
func (c *deltaEncodedChunk) sampleSize() int {
return int(c.timeBytes() + c.valueBytes())
}
func (c *deltaEncodedChunk) len() int {
if len(c.buf) < deltaHeaderBytes {
return 0
}
return (len(c.buf) - deltaHeaderBytes) / c.sampleSize()
}
// TODO: remove?
func (c *deltaEncodedChunk) values() <-chan *metric.SamplePair {
n := c.len()
valuesChan := make(chan *metric.SamplePair)
go func() {
for i := 0; i < n; i++ {
valuesChan <- c.valueAtIndex(i)
}
close(valuesChan)
}()
return valuesChan
}
func (c *deltaEncodedChunk) valueAtIndex(idx int) *metric.SamplePair {
offset := deltaHeaderBytes + idx*c.sampleSize()
var dt uint64
switch c.timeBytes() {
case 1:
dt = uint64(uint8(c.buf[offset]))
case 2:
dt = uint64(binary.LittleEndian.Uint16(c.buf[offset:]))
case 4:
dt = uint64(binary.LittleEndian.Uint32(c.buf[offset:]))
case 8:
dt = uint64(binary.LittleEndian.Uint64(c.buf[offset:]))
}
offset += int(c.timeBytes())
var dv clientmodel.SampleValue
if c.isInt() {
switch c.valueBytes() {
case 0:
dv = clientmodel.SampleValue(0)
case 1:
dv = clientmodel.SampleValue(int8(c.buf[offset]))
case 2:
dv = clientmodel.SampleValue(int16(binary.LittleEndian.Uint16(c.buf[offset:])))
case 4:
dv = clientmodel.SampleValue(int32(binary.LittleEndian.Uint32(c.buf[offset:])))
case 8:
dv = clientmodel.SampleValue(int64(binary.LittleEndian.Uint64(c.buf[offset:])))
default:
panic("Invalid number of bytes for integer delta")
}
} else {
switch c.valueBytes() {
case 4:
dv = clientmodel.SampleValue(math.Float32frombits(binary.LittleEndian.Uint32(c.buf[offset:])))
case 8:
dv = clientmodel.SampleValue(math.Float64frombits(binary.LittleEndian.Uint64(c.buf[offset:])))
default:
panic("Invalid number of bytes for floating point delta")
}
}
return &metric.SamplePair{
Timestamp: c.baseTime() + clientmodel.Timestamp(dt),
Value: c.baseValue() + dv,
}
}
func (c *deltaEncodedChunk) firstTime() clientmodel.Timestamp {
return c.valueAtIndex(0).Timestamp
}
func (c *deltaEncodedChunk) lastTime() clientmodel.Timestamp {
return c.valueAtIndex(c.len() - 1).Timestamp
}
func (c *deltaEncodedChunk) marshal(w io.Writer) error {
// TODO: check somewhere that configured buf len cannot overflow 16 bit.
binary.LittleEndian.PutUint16(c.buf[deltaHeaderBufLenOffset:], uint16(len(c.buf)))
n, err := w.Write(c.buf[:cap(c.buf)])
if err != nil {
return err
}
if n != cap(c.buf) {
return fmt.Errorf("wanted to write %d bytes, wrote %d", len(c.buf), n)
}
return nil
}
func (c *deltaEncodedChunk) unmarshal(r io.Reader) error {
c.buf = c.buf[:cap(c.buf)]
readBytes := 0
for readBytes < len(c.buf) {
n, err := r.Read(c.buf[readBytes:])
if err != nil {
return err
}
readBytes += n
}
c.buf = c.buf[:binary.LittleEndian.Uint16(c.buf[deltaHeaderBufLenOffset:])]
return nil
}
type deltaEncodedChunkIterator struct {
chunk *deltaEncodedChunk
// TODO: add more fields here to keep track of last position.
}
func (c *deltaEncodedChunk) newIterator() chunkIterator {
return &deltaEncodedChunkIterator{
chunk: c,
}
}
func (it *deltaEncodedChunkIterator) getValueAtTime(t clientmodel.Timestamp) metric.Values {
i := sort.Search(it.chunk.len(), func(i int) bool {
return !it.chunk.valueAtIndex(i).Timestamp.Before(t)
})
switch i {
case 0:
return metric.Values{*it.chunk.valueAtIndex(0)}
case it.chunk.len():
return metric.Values{*it.chunk.valueAtIndex(it.chunk.len() - 1)}
default:
if v := it.chunk.valueAtIndex(i); v.Timestamp.Equal(t) {
return metric.Values{*v}
} else {
return metric.Values{*it.chunk.valueAtIndex(i - 1), *v}
}
}
}
func (it *deltaEncodedChunkIterator) getBoundaryValues(in metric.Interval) metric.Values {
return nil
}
func (it *deltaEncodedChunkIterator) getRangeValues(in metric.Interval) metric.Values {
oldest := sort.Search(it.chunk.len(), func(i int) bool {
return !it.chunk.valueAtIndex(i).Timestamp.Before(in.OldestInclusive)
})
newest := sort.Search(it.chunk.len(), func(i int) bool {
return it.chunk.valueAtIndex(i).Timestamp.After(in.NewestInclusive)
})
if oldest == it.chunk.len() {
return nil
}
result := make(metric.Values, 0, newest-oldest)
for i := oldest; i < newest; i++ {
result = append(result, *it.chunk.valueAtIndex(i))
}
return result
}
func (it *deltaEncodedChunkIterator) contains(t clientmodel.Timestamp) bool {
return !t.Before(it.chunk.firstTime()) && !t.After(it.chunk.lastTime())
}

44
storage/local/freelist.go Normal file
View file

@ -0,0 +1,44 @@
package storage_ng
import (
"github.com/prometheus/prometheus/utility"
)
var chunkBufs = newChunkBufList(10000, 10000)
type chunkBufList struct {
l utility.FreeList
}
func newChunkBuf() []byte {
return make([]byte, 0, 1024) // TODO: This value somehow needs to be set in coordination with the one passed into the disk persistence.
}
func newChunkBufList(length, capacity int) *chunkBufList {
l := &chunkBufList{
l: utility.NewFreeList(capacity),
}
for i := 0; i < length; i++ {
l.l.Give(newChunkBuf())
}
return l
}
func (l *chunkBufList) Get() []byte {
numChunkGets.Inc()
if v, ok := l.l.Get(); ok {
return v.([]byte)
}
return newChunkBuf()
}
func (l *chunkBufList) Give(v []byte) bool {
numChunkGives.Inc()
v = v[:0]
return l.l.Give(v)
}
func (l *chunkBufList) Close() {
l.l.Close()
}

View file

@ -0,0 +1,88 @@
// 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 storage_ng
import (
"github.com/prometheus/client_golang/prometheus"
)
const (
address = "instance"
alive = "alive"
failure = "failure"
outcome = "outcome"
state = "state"
success = "success"
unreachable = "unreachable"
)
var (
numSeries = prometheus.NewGauge(prometheus.GaugeOpts{
Name: "prometheus_stored_series_count",
Help: "The number of currently stored series.",
})
numSamples = prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_stored_samples_total",
Help: "The total number of stored samples.",
})
numChunks = prometheus.NewGauge(prometheus.GaugeOpts{
Name: "prometheus_used_chunks_count",
Help: "The current number of chunks in use.",
})
numChunkGives = prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_chunk_gives_total",
Help: "The total number of returned (freed) chunks.",
})
numChunkGets = prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_chunk_gets_total",
Help: "The total number of retrieved chunks.",
})
numTranscodes = prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_chunk_transcodes_total",
Help: "The total number of chunk transcodes.",
})
numPinnedChunks = prometheus.NewGauge(prometheus.GaugeOpts{
Name: "prometheus_pinned_chunks_count",
Help: "The current number of pinned chunks.",
})
persistLatencies = prometheus.NewSummaryVec(prometheus.SummaryOpts{
Name: "prometheus_persist_latency_ms",
Help: "A summary of latencies for persisting each chunk.",
}, []string{outcome})
persistQueueLength = prometheus.NewGauge(prometheus.GaugeOpts{
Name: "prometheus_persist_queue_length",
Help: "The current number of chunks waiting in the persist queue.",
})
persistQueueCapacity = prometheus.NewGauge(prometheus.GaugeOpts{
Name: "prometheus_persist_queue_capacity",
Help: "The total capacity of the persist queue.",
})
)
func init() {
prometheus.MustRegister(numSeries)
prometheus.MustRegister(numSamples)
prometheus.MustRegister(numChunks)
prometheus.MustRegister(numChunkGives)
prometheus.MustRegister(numChunkGets)
prometheus.MustRegister(numTranscodes)
prometheus.MustRegister(numPinnedChunks)
prometheus.MustRegister(persistLatencies)
prometheus.MustRegister(persistQueueLength)
prometheus.MustRegister(persistQueueCapacity)
persistQueueCapacity.Set(float64(persistQueueCap))
}

View file

@ -0,0 +1,98 @@
package storage_ng
import (
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility"
)
type Storage interface {
// AppendSamples stores a group of new samples. Multiple samples for the same
// fingerprint need to be submitted in chronological order, from oldest to
// newest (both in the same call to AppendSamples and across multiple calls).
AppendSamples(clientmodel.Samples)
// NewPreloader returns a new Preloader which allows preloading and pinning
// series data into memory for use within a query.
NewPreloader() Preloader
// Get all of the metric fingerprints that are associated with the
// provided label matchers.
GetFingerprintsForLabelMatchers(metric.LabelMatchers) clientmodel.Fingerprints
// Get all of the label values that are associated with a given label name.
GetLabelValuesForLabelName(clientmodel.LabelName) clientmodel.LabelValues
// Get the metric associated with the provided fingerprint.
GetMetricForFingerprint(clientmodel.Fingerprint) clientmodel.Metric
// Get all label values that are associated with a given label name.
GetAllValuesForLabel(clientmodel.LabelName) clientmodel.LabelValues
// Construct an iterator for a given fingerprint.
NewIterator(clientmodel.Fingerprint) SeriesIterator
// Run the request-serving and maintenance loop.
Serve(started chan<- bool)
// Close the MetricsStorage and releases all resources.
Close() error
}
type SeriesIterator interface {
// Get the two values that are immediately adjacent to a given time.
GetValueAtTime(clientmodel.Timestamp) metric.Values
// Get the boundary values of an interval: the first value older than
// the interval start, and the first value younger than the interval
// end.
GetBoundaryValues(metric.Interval) metric.Values
// Get all values contained within a provided interval.
GetRangeValues(metric.Interval) metric.Values
}
// A Persistence stores samples persistently across restarts.
type Persistence interface {
// PersistChunk persists a single chunk of a series.
PersistChunk(clientmodel.Fingerprint, chunk) error
// PersistIndexes persists a Prometheus server's timeseries indexes.
PersistIndexes(i *Indexes) error
// PersistHeads persists all open (non-full) head chunks.
PersistHeads(map[clientmodel.Fingerprint]*memorySeries) error
// DropChunks deletes all chunks from a timeseries whose last sample time is
// before beforeTime.
DropChunks(fp clientmodel.Fingerprint, beforeTime clientmodel.Timestamp) error
// LoadChunks loads a group of chunks of a timeseries by their index. The
// chunk with the earliest time will have index 0, the following ones will
// have incrementally larger indexes.
LoadChunks(fp clientmodel.Fingerprint, indexes []int) (chunks, error)
// LoadChunkDescs loads chunkDescs for a series up until a given time.
LoadChunkDescs(fp clientmodel.Fingerprint, beforeTime clientmodel.Timestamp) (chunkDescs, error)
// LoadHeads loads all open (non-full) head chunks.
LoadHeads(map[clientmodel.Fingerprint]*memorySeries) error
// LoadIndexes loads and returns all timeseries indexes.
LoadIndexes() (*Indexes, error)
}
// A Preloader preloads series data necessary for a query into memory and pins
// them until released via Close().
type Preloader interface {
PreloadRange(fp clientmodel.Fingerprint, from clientmodel.Timestamp, through clientmodel.Timestamp) error
/*
// GetMetricAtTime loads and pins samples around a given time.
GetMetricAtTime(clientmodel.Fingerprint, clientmodel.Timestamp) error
// GetMetricAtInterval loads and pins samples at intervals.
GetMetricAtInterval(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval time.Duration) error
// GetMetricRange loads and pins a given range of samples.
GetMetricRange(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp) error
// GetMetricRangeAtInterval loads and pins sample ranges at intervals.
GetMetricRangeAtInterval(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval, rangeDuration time.Duration) error
*/
// Close unpins any previously requested series data from memory.
Close()
}
type Closer interface {
// Close cleans up any used resources.
Close()
}
type Indexes struct {
FingerprintToSeries map[clientmodel.Fingerprint]*memorySeries
LabelPairToFingerprints map[metric.LabelPair]utility.Set
LabelNameToLabelValues map[clientmodel.LabelName]utility.Set
}

View file

@ -0,0 +1,388 @@
package storage_ng
import (
"bufio"
"encoding/binary"
"encoding/gob"
"fmt"
"io"
"os"
"path"
"github.com/golang/glog"
//"github.com/prometheus/prometheus/storage/metric"
//"github.com/prometheus/client_golang/prometheus"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility"
)
const (
seriesFileName = "series.db"
seriesTempFileName = "series.db.tmp"
indexFileName = "index.db"
headsFileName = "heads.db"
chunkHeaderLen = 17
chunkHeaderTypeOffset = 0
chunkHeaderFirstTimeOffset = 1
chunkHeaderLastTimeOffset = 9
headsHeaderLen = 9
headsHeaderFingerprintOffset = 0
headsHeaderTypeOffset = 8
)
type diskPersistence struct {
basePath string
chunkLen int
}
func NewDiskPersistence(basePath string, chunkLen int) (Persistence, error) {
gob.Register(clientmodel.Fingerprint(0))
gob.Register(clientmodel.LabelValue(""))
err := os.MkdirAll(basePath, 0700)
if err != nil {
return nil, err
}
return &diskPersistence{
basePath: basePath,
chunkLen: chunkLen,
}, nil
}
func (p *diskPersistence) dirForFingerprint(fp clientmodel.Fingerprint) string {
fpStr := fp.String()
return fmt.Sprintf("%s/%c%c/%s", p.basePath, fpStr[0], fpStr[1], fpStr[2:])
}
// exists returns true when the given file or directory exists.
func exists(path string) (bool, error) {
_, err := os.Stat(path)
if err == nil {
return true, nil
}
if os.IsNotExist(err) {
return false, nil
}
return false, err
}
func (p *diskPersistence) openChunkFileForWriting(fp clientmodel.Fingerprint) (*os.File, error) {
dirname := p.dirForFingerprint(fp)
ex, err := exists(dirname)
if err != nil {
return nil, err
}
if !ex {
if err := os.MkdirAll(dirname, 0700); err != nil {
return nil, err
}
}
return os.OpenFile(path.Join(dirname, seriesFileName), os.O_WRONLY|os.O_APPEND|os.O_CREATE, 0640)
}
func (p *diskPersistence) openChunkFileForReading(fp clientmodel.Fingerprint) (*os.File, error) {
dirname := p.dirForFingerprint(fp)
return os.Open(path.Join(dirname, seriesFileName))
}
func writeChunkHeader(w io.Writer, c chunk) error {
header := make([]byte, chunkHeaderLen)
header[chunkHeaderTypeOffset] = chunkType(c)
binary.LittleEndian.PutUint64(header[chunkHeaderFirstTimeOffset:], uint64(c.firstTime()))
binary.LittleEndian.PutUint64(header[chunkHeaderLastTimeOffset:], uint64(c.lastTime()))
_, err := w.Write(header)
return err
}
func (p *diskPersistence) PersistChunk(fp clientmodel.Fingerprint, c chunk) error {
// 1. Open chunk file.
f, err := p.openChunkFileForWriting(fp)
if err != nil {
return err
}
defer f.Close()
b := bufio.NewWriterSize(f, chunkHeaderLen+p.chunkLen)
defer b.Flush()
// 2. Write the header (chunk type and first/last times).
err = writeChunkHeader(b, c)
if err != nil {
return err
}
// 3. Write chunk into file.
return c.marshal(b)
}
func (p *diskPersistence) offsetForChunkIndex(i int) int64 {
return int64(i * (chunkHeaderLen + p.chunkLen))
}
func (p *diskPersistence) LoadChunks(fp clientmodel.Fingerprint, indexes []int) (chunks, error) {
// TODO: we need to verify at some point that file length is a multiple of
// the chunk size. When is the best time to do this, and where to remember
// it? Right now, we only do it when loading chunkDescs.
f, err := p.openChunkFileForReading(fp)
if err != nil {
return nil, err
}
defer f.Close()
chunks := make(chunks, 0, len(indexes))
defer func() {
if err == nil {
return
}
for _, c := range chunks {
c.close()
}
}()
typeBuf := make([]byte, 1)
for _, idx := range indexes {
_, err := f.Seek(p.offsetForChunkIndex(idx), os.SEEK_SET)
if err != nil {
return nil, err
}
// TODO: check seek offset too?
n, err := f.Read(typeBuf)
if err != nil {
return nil, err
}
if n != 1 {
// Shouldn't happen?
panic("read returned != 1 bytes")
}
_, err = f.Seek(chunkHeaderLen-1, os.SEEK_CUR)
if err != nil {
return nil, err
}
chunk := chunkForType(typeBuf[0])
chunk.unmarshal(f)
chunks = append(chunks, chunk)
}
return chunks, nil
}
func (p *diskPersistence) LoadChunkDescs(fp clientmodel.Fingerprint, beforeTime clientmodel.Timestamp) (chunkDescs, error) {
f, err := p.openChunkFileForReading(fp)
if os.IsNotExist(err) {
return nil, nil
}
if err != nil {
return nil, err
}
defer f.Close()
fi, err := f.Stat()
if err != nil {
return nil, err
}
totalChunkLen := chunkHeaderLen + p.chunkLen
if fi.Size()%int64(totalChunkLen) != 0 {
// TODO: record number of encountered corrupt series files in a metric?
// Truncate the file size to the nearest multiple of chunkLen.
truncateTo := fi.Size() - fi.Size()%int64(totalChunkLen)
glog.Infof("Bad series file size for %s: %d bytes (no multiple of %d). Truncating to %d bytes.", fp, fi.Size(), totalChunkLen, truncateTo)
// TODO: this doesn't work, as this is a read-only file handle.
if err := f.Truncate(truncateTo); err != nil {
return nil, err
}
}
numChunks := int(fi.Size()) / totalChunkLen
cds := make(chunkDescs, 0, numChunks)
for i := 0; i < numChunks; i++ {
_, err := f.Seek(p.offsetForChunkIndex(i)+chunkHeaderFirstTimeOffset, os.SEEK_SET)
if err != nil {
return nil, err
}
chunkTimesBuf := make([]byte, 16)
_, err = io.ReadAtLeast(f, chunkTimesBuf, 16)
if err != nil {
return nil, err
}
cd := &chunkDesc{
firstTimeField: clientmodel.Timestamp(binary.LittleEndian.Uint64(chunkTimesBuf)),
lastTimeField: clientmodel.Timestamp(binary.LittleEndian.Uint64(chunkTimesBuf[8:])),
}
if !cd.firstTime().Before(beforeTime) {
// From here on, we have chunkDescs in memory already.
break
}
cds = append(cds, cd)
}
return cds, nil
}
func (p *diskPersistence) indexPath() string {
return path.Join(p.basePath, indexFileName)
}
func (p *diskPersistence) PersistIndexes(i *Indexes) error {
f, err := os.OpenFile(p.indexPath(), os.O_WRONLY|os.O_TRUNC|os.O_CREATE, 0640)
if err != nil {
return err
}
defer f.Close()
enc := gob.NewEncoder(f)
if err := enc.Encode(i); err != nil {
return err
}
return nil
}
func (p *diskPersistence) LoadIndexes() (*Indexes, error) {
f, err := os.Open(p.indexPath())
if os.IsNotExist(err) {
return &Indexes{
FingerprintToSeries: make(map[clientmodel.Fingerprint]*memorySeries),
LabelPairToFingerprints: make(map[metric.LabelPair]utility.Set),
LabelNameToLabelValues: make(map[clientmodel.LabelName]utility.Set),
}, nil
}
if err != nil {
return nil, err
}
defer f.Close()
dec := gob.NewDecoder(f)
var i Indexes
if err := dec.Decode(&i); err != nil {
return nil, err
}
return &i, nil
}
func (p *diskPersistence) headsPath() string {
return path.Join(p.basePath, headsFileName)
}
func (p *diskPersistence) PersistHeads(fpToSeries map[clientmodel.Fingerprint]*memorySeries) error {
f, err := os.OpenFile(p.headsPath(), os.O_WRONLY|os.O_TRUNC|os.O_CREATE, 0640)
if err != nil {
return err
}
header := make([]byte, 9)
for fp, series := range fpToSeries {
head := series.head().chunk
binary.LittleEndian.PutUint64(header[headsHeaderFingerprintOffset:], uint64(fp))
header[headsHeaderTypeOffset] = chunkType(head)
_, err := f.Write(header)
if err != nil {
return err
}
err = head.marshal(f)
if err != nil {
return err
}
}
return nil
}
func (p *diskPersistence) DropChunks(fp clientmodel.Fingerprint, beforeTime clientmodel.Timestamp) error {
f, err := p.openChunkFileForReading(fp)
if os.IsNotExist(err) {
return nil
}
if err != nil {
return err
}
defer f.Close()
// Find the first chunk that should be kept.
for i := 0; ; i++ {
_, err := f.Seek(p.offsetForChunkIndex(i)+chunkHeaderLastTimeOffset, os.SEEK_SET)
if err != nil {
return err
}
lastTimeBuf := make([]byte, 8)
_, err = io.ReadAtLeast(f, lastTimeBuf, 8)
if err == io.EOF {
// We ran into the end of the file without finding any chunks that should
// be kept. Remove the whole file.
if err := os.Remove(f.Name()); err != nil {
return err
}
return nil
}
if err != nil {
return err
}
lastTime := clientmodel.Timestamp(binary.LittleEndian.Uint64(lastTimeBuf))
if !lastTime.Before(beforeTime) {
break
}
}
// We've found the first chunk that should be kept. Seek backwards to the
// beginning of its header and start copying everything from there into a new
// file.
_, err = f.Seek(-(chunkHeaderLastTimeOffset + 8), os.SEEK_CUR)
if err != nil {
return err
}
dirname := p.dirForFingerprint(fp)
temp, err := os.OpenFile(path.Join(dirname, seriesTempFileName), os.O_WRONLY|os.O_CREATE, 0640)
if err != nil {
return err
}
defer temp.Close()
if _, err := io.Copy(temp, f); err != nil {
return err
}
os.Rename(path.Join(dirname, seriesTempFileName), path.Join(dirname, seriesFileName))
return nil
}
func (p *diskPersistence) LoadHeads(fpToSeries map[clientmodel.Fingerprint]*memorySeries) error {
f, err := os.Open(p.headsPath())
if os.IsNotExist(err) {
// TODO: this should only happen if there never was a shutdown before. In
// that case, all heads should be in order already, since the series got
// created during this process' runtime.
// Still, make this more robust.
return nil
}
header := make([]byte, headsHeaderLen)
for {
_, err := io.ReadAtLeast(f, header, headsHeaderLen)
if err == io.ErrUnexpectedEOF {
// TODO: this should only be ok if n is 0.
break
}
if err != nil {
return nil
}
// TODO: this relies on the implementation (uint64) of Fingerprint.
fp := clientmodel.Fingerprint(binary.LittleEndian.Uint64(header[headsHeaderFingerprintOffset:]))
chunk := chunkForType(header[headsHeaderTypeOffset])
chunk.unmarshal(f)
fpToSeries[fp].chunkDescs = append(fpToSeries[fp].chunkDescs, &chunkDesc{
chunk: chunk,
refCount: 1,
})
}
return nil
}

View file

@ -0,0 +1,107 @@
package storage_ng
import (
"io/ioutil"
"testing"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility"
)
func TestIndexPersistence(t *testing.T) {
expected := Indexes{
FingerprintToSeries: map[clientmodel.Fingerprint]*memorySeries{
0: {
metric: clientmodel.Metric{
clientmodel.MetricNameLabel: "metric_0",
"label_1": "value_1",
},
},
1: {
metric: clientmodel.Metric{
clientmodel.MetricNameLabel: "metric_0",
"label_2": "value_2",
},
},
},
LabelPairToFingerprints: map[metric.LabelPair]utility.Set{
metric.LabelPair{
Name: clientmodel.MetricNameLabel,
Value: "metric_0",
}: {
0: struct{}{},
1: struct{}{},
},
metric.LabelPair{
Name: "label_1",
Value: "value_1",
}: {
0: struct{}{},
},
metric.LabelPair{
Name: "label_2",
Value: "value_2",
}: {
1: struct{}{},
},
},
LabelNameToLabelValues: map[clientmodel.LabelName]utility.Set{
clientmodel.MetricNameLabel: {
clientmodel.LabelValue("metric_0"): struct{}{},
},
"label_1": {
clientmodel.LabelValue("value_1"): struct{}{},
},
"label_2": {
clientmodel.LabelValue("value_2"): struct{}{},
},
},
}
basePath, err := ioutil.TempDir("", "test_index_persistence")
if err != nil {
t.Fatal(err)
}
p, err := NewDiskPersistence(basePath, 1024)
if err != nil {
t.Fatal(err)
}
p.PersistIndexes(&expected)
actual, err := p.LoadIndexes()
if err != nil {
t.Fatal(err)
}
if len(actual.FingerprintToSeries) != len(expected.FingerprintToSeries) {
t.Fatalf("Count mismatch: Got %d; want %d", len(actual.FingerprintToSeries), len(expected.FingerprintToSeries))
}
for fp, actualSeries := range actual.FingerprintToSeries {
expectedSeries := expected.FingerprintToSeries[fp]
if !expectedSeries.metric.Equal(actualSeries.metric) {
t.Fatalf("%s: Got %s; want %s", fp, actualSeries.metric, expectedSeries.metric)
}
}
if len(actual.LabelPairToFingerprints) != len(expected.LabelPairToFingerprints) {
t.Fatalf("Count mismatch: Got %d; want %d", len(actual.LabelPairToFingerprints), len(expected.LabelPairToFingerprints))
}
for lp, actualFps := range actual.LabelPairToFingerprints {
expectedFps := expected.LabelPairToFingerprints[lp]
if len(actualFps) != len(actualFps.Intersection(expectedFps)) {
t.Fatalf("%s: Got %s; want %s", lp, actualFps, expectedFps)
}
}
if len(actual.LabelNameToLabelValues) != len(expected.LabelNameToLabelValues) {
t.Fatalf("Count mismatch: Got %d; want %d", len(actual.LabelNameToLabelValues), len(expected.LabelNameToLabelValues))
}
for name, actualVals := range actual.LabelNameToLabelValues {
expectedVals := expected.LabelNameToLabelValues[name]
if len(actualVals) != len(actualVals.Intersection(expectedVals)) {
t.Fatalf("%s: Got %s; want %s", name, actualVals, expectedVals)
}
}
}

102
storage/local/preload.go Normal file
View file

@ -0,0 +1,102 @@
// 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 storage_ng
import (
clientmodel "github.com/prometheus/client_golang/model"
)
// memorySeriesPreloader is a Preloader for the memorySeriesStorage.
type memorySeriesPreloader struct {
storage *memorySeriesStorage
pinnedChunkDescs chunkDescs
}
func (p *memorySeriesPreloader) PreloadRange(fp clientmodel.Fingerprint, from clientmodel.Timestamp, through clientmodel.Timestamp) error {
cds, err := p.storage.preloadChunksForRange(fp, from, through)
if err != nil {
return err
}
p.pinnedChunkDescs = append(p.pinnedChunkDescs, cds...)
return nil
}
/*
// GetMetricAtTime implements Preloader.
func (p *memorySeriesPreloader) GetMetricAtTime(fp clientmodel.Fingerprint, t clientmodel.Timestamp) error {
cds, err := p.storage.preloadChunks(fp, &timeSelector{
from: t,
through: t,
})
if err != nil {
return err
}
p.pinnedChunkDescs = append(p.pinnedChunkDescs, cds...)
return nil
}
// GetMetricAtInterval implements Preloader.
func (p *memorySeriesPreloader) GetMetricAtInterval(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval time.Duration) error {
cds, err := p.storage.preloadChunks(fp, &timeSelector{
from: from,
through: through,
interval: interval,
})
if err != nil {
return err
}
p.pinnedChunkDescs = append(p.pinnedChunkDescs, cds...)
return
}
// GetMetricRange implements Preloader.
func (p *memorySeriesPreloader) GetMetricRange(fp clientmodel.Fingerprint, t clientmodel.Timestamp, rangeDuration time.Duration) error {
cds, err := p.storage.preloadChunks(fp, &timeSelector{
from: t,
through: t,
rangeDuration: through.Sub(from),
})
if err != nil {
return err
}
p.pinnedChunkDescs = append(p.pinnedChunkDescs, cds...)
return
}
// GetMetricRangeAtInterval implements Preloader.
func (p *memorySeriesPreloader) GetMetricRangeAtInterval(fp clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval, rangeDuration time.Duration) error {
cds, err := p.storage.preloadChunks(fp, &timeSelector{
from: from,
through: through,
interval: interval,
rangeDuration: rangeDuration,
})
if err != nil {
return err
}
p.pinnedChunkDescs = append(p.pinnedChunkDescs, cds...)
return
}
*/
// Close implements Preloader.
func (p *memorySeriesPreloader) Close() {
for _, cd := range p.pinnedChunkDescs {
// TODO: unpinning may synchronously cause closing of chunks if they have
// been marked to be evicted. This could interfere with other parts of the
// storage that check whether a chunk is swapped in or not. Is it a good
// idea / sufficient to take the storage lock here?
cd.unpin()
}
}

453
storage/local/series.go Normal file
View file

@ -0,0 +1,453 @@
package storage_ng
import (
"bytes"
"encoding/gob"
"sort"
"sync"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
)
type chunkDescs []*chunkDesc
type chunkDesc struct {
sync.Mutex
chunk chunk
refCount int
evict bool
firstTimeField clientmodel.Timestamp // TODO: stupid name, reorganize.
lastTimeField clientmodel.Timestamp
}
func (cd *chunkDesc) add(s *metric.SamplePair) chunks {
cd.Lock()
defer cd.Unlock()
return cd.chunk.add(s)
}
func (cd *chunkDesc) pin() {
cd.Lock()
defer cd.Unlock()
numPinnedChunks.Inc()
cd.refCount++
}
func (cd *chunkDesc) unpin() {
cd.Lock()
defer cd.Unlock()
if cd.refCount == 0 {
panic("cannot unpin already unpinned chunk")
}
numPinnedChunks.Dec()
cd.refCount--
if cd.refCount == 0 && cd.evict {
cd.evictNow()
}
}
func (cd *chunkDesc) firstTime() clientmodel.Timestamp {
if cd.chunk == nil {
return cd.firstTimeField
}
return cd.chunk.firstTime()
}
func (cd *chunkDesc) lastTime() clientmodel.Timestamp {
if cd.chunk == nil {
return cd.lastTimeField
}
return cd.chunk.lastTime()
}
func (cd *chunkDesc) contains(t clientmodel.Timestamp) bool {
return !t.Before(cd.firstTime()) && !t.After(cd.lastTime())
}
func (cd *chunkDesc) open(c chunk) {
cd.Lock()
defer cd.Unlock()
if cd.refCount != 0 || cd.chunk != nil {
panic("cannot open already pinned chunk")
}
cd.evict = false
cd.chunk = c
numPinnedChunks.Inc()
cd.refCount++
}
func (cd *chunkDesc) evictOnUnpin() {
cd.Lock()
defer cd.Unlock()
if cd.refCount == 0 {
cd.evictNow()
}
cd.evict = true
}
func (cd *chunkDesc) evictNow() {
cd.firstTimeField = cd.chunk.firstTime()
cd.lastTimeField = cd.chunk.lastTime()
cd.chunk.close()
cd.chunk = nil
}
type memorySeries struct {
mtx sync.Mutex
metric clientmodel.Metric
// Sorted by start time, no overlapping chunk ranges allowed.
chunkDescs chunkDescs
chunkDescsLoaded bool
}
func newMemorySeries(m clientmodel.Metric) *memorySeries {
return &memorySeries{
metric: m,
// TODO: should we set this to nil initially and only create a chunk when
// adding? But right now, we also only call newMemorySeries when adding, so
// it turns out to be the same.
chunkDescs: chunkDescs{
// TODO: should there be a newChunkDesc() function?
&chunkDesc{
chunk: newDeltaEncodedChunk(d1, d0, true),
refCount: 1,
},
},
chunkDescsLoaded: true,
}
}
func (s *memorySeries) add(v *metric.SamplePair, persistQueue chan *persistRequest) {
s.mtx.Lock()
defer s.mtx.Unlock()
chunks := s.head().add(v)
s.head().chunk = chunks[0]
if len(chunks) > 1 {
fp := s.metric.Fingerprint()
queuePersist := func(cd *chunkDesc) {
persistQueue <- &persistRequest{
fingerprint: fp,
chunkDesc: cd,
}
}
queuePersist(s.head())
for i, c := range chunks[1:] {
cd := &chunkDesc{
chunk: c,
refCount: 1,
}
s.chunkDescs = append(s.chunkDescs, cd)
// The last chunk is still growing.
if i < len(chunks[1:])-1 {
queuePersist(cd)
}
}
}
}
func (s *memorySeries) evictOlderThan(t clientmodel.Timestamp) {
s.mtx.Lock()
defer s.mtx.Unlock()
// For now, always drop the entire range from oldest to t.
for _, cd := range s.chunkDescs {
if !cd.lastTime().Before(t) {
break
}
if cd.chunk == nil {
continue
}
cd.evictOnUnpin()
}
}
func (s *memorySeries) purgeOlderThan(t clientmodel.Timestamp, p Persistence) (dropSeries bool, err error) {
s.mtx.Lock()
defer s.mtx.Unlock()
if err := p.DropChunks(s.metric.Fingerprint(), t); err != nil {
return false, err
}
keepIdx := len(s.chunkDescs)
for i, cd := range s.chunkDescs {
if !cd.lastTime().Before(t) {
keepIdx = i
break
}
}
for i := 0; i < keepIdx; i++ {
if s.chunkDescs[i].chunk != nil {
s.chunkDescs[i].evictOnUnpin()
}
}
s.chunkDescs = s.chunkDescs[keepIdx:]
return len(s.chunkDescs) == 0, nil
}
func (s *memorySeries) close() {
for _, cd := range s.chunkDescs {
if cd.chunk != nil {
cd.evictNow()
}
// TODO: need to handle unwritten heads here.
}
}
// TODO: in this method (and other places), we just fudge around with chunkDesc
// internals without grabbing the chunkDesc lock. Study how this needs to be
// protected against other accesses that don't hold the series lock.
func (s *memorySeries) preloadChunks(indexes []int, p Persistence) (chunkDescs, error) {
loadIndexes := []int{}
pinnedChunkDescs := make(chunkDescs, 0, len(indexes))
for _, idx := range indexes {
pinnedChunkDescs = append(pinnedChunkDescs, s.chunkDescs[idx])
if s.chunkDescs[idx].chunk == nil {
loadIndexes = append(loadIndexes, idx)
} else {
s.chunkDescs[idx].pin()
}
}
if len(loadIndexes) > 0 {
fp := s.metric.Fingerprint()
chunks, err := p.LoadChunks(fp, loadIndexes)
if err != nil {
// TODO: unpin all chunks that were already loaded before.
return nil, err
}
for i, c := range chunks {
cd := s.chunkDescs[loadIndexes[i]]
cd.open(c)
}
}
return pinnedChunkDescs, nil
}
/*
func (s *memorySeries) preloadChunksAtTime(t clientmodel.Timestamp, p Persistence) (chunkDescs, error) {
s.mtx.Lock()
defer s.mtx.Unlock()
if len(s.chunkDescs) == 0 {
return nil, nil
}
var pinIndexes []int
// Find first chunk where lastTime() is after or equal to t.
i := sort.Search(len(s.chunkDescs), func(i int) bool {
return !s.chunkDescs[i].lastTime().Before(t)
})
switch i {
case 0:
pinIndexes = []int{0}
case len(s.chunkDescs):
pinIndexes = []int{i - 1}
default:
if s.chunkDescs[i].contains(t) {
pinIndexes = []int{i}
} else {
pinIndexes = []int{i - 1, i}
}
}
return s.preloadChunks(pinIndexes, p)
}
*/
func (s *memorySeries) loadChunkDescs(p Persistence) error {
cds, err := p.LoadChunkDescs(s.metric.Fingerprint(), s.chunkDescs[0].firstTime())
if err != nil {
return err
}
s.chunkDescs = append(cds, s.chunkDescs...)
s.chunkDescsLoaded = true
return nil
}
func (s *memorySeries) preloadChunksForRange(from clientmodel.Timestamp, through clientmodel.Timestamp, p Persistence) (chunkDescs, error) {
s.mtx.Lock()
defer s.mtx.Unlock()
if !s.chunkDescsLoaded && from.Before(s.chunkDescs[0].firstTime()) {
if err := s.loadChunkDescs(p); err != nil {
return nil, err
}
}
if len(s.chunkDescs) == 0 {
return nil, nil
}
// Find first chunk with start time after "from".
fromIdx := sort.Search(len(s.chunkDescs), func(i int) bool {
return s.chunkDescs[i].firstTime().After(from)
})
// Find first chunk with start time after "through".
throughIdx := sort.Search(len(s.chunkDescs), func(i int) bool {
return s.chunkDescs[i].firstTime().After(through)
})
if fromIdx > 0 {
fromIdx--
}
if throughIdx == len(s.chunkDescs) {
throughIdx--
}
pinIndexes := make([]int, 0, throughIdx-fromIdx+1)
for i := fromIdx; i <= throughIdx; i++ {
pinIndexes = append(pinIndexes, i)
}
return s.preloadChunks(pinIndexes, p)
}
type memorySeriesIterator struct {
mtx *sync.Mutex
chunkIt chunkIterator
chunks chunks
}
func (s *memorySeries) newIterator() SeriesIterator {
chunks := make(chunks, 0, len(s.chunkDescs))
for i, cd := range s.chunkDescs {
if cd.chunk != nil {
if i == len(s.chunkDescs)-1 {
chunks = append(chunks, cd.chunk.clone())
} else {
chunks = append(chunks, cd.chunk)
}
}
}
return &memorySeriesIterator{
mtx: &s.mtx,
chunks: chunks,
}
}
func (s *memorySeries) head() *chunkDesc {
return s.chunkDescs[len(s.chunkDescs)-1]
}
func (s *memorySeries) values() metric.Values {
var values metric.Values
for _, cd := range s.chunkDescs {
for sample := range cd.chunk.values() {
values = append(values, *sample)
}
}
return values
}
var gobWriter bytes.Buffer
var seriesEncoder *gob.Encoder
func (s *memorySeries) GobEncode() ([]byte, error) {
gobWriter.Reset()
if seriesEncoder == nil {
seriesEncoder = gob.NewEncoder(&gobWriter)
}
err := seriesEncoder.Encode(s.metric)
return gobWriter.Bytes(), err
}
var gobReader bytes.Reader
var seriesDecoder *gob.Decoder
func (s *memorySeries) GobDecode(buf []byte) error {
gobReader = *bytes.NewReader(buf)
if seriesDecoder == nil {
seriesDecoder = gob.NewDecoder(&gobReader)
}
err := seriesDecoder.Decode(&s.metric)
return err
}
func (it *memorySeriesIterator) GetValueAtTime(t clientmodel.Timestamp) metric.Values {
it.mtx.Lock()
defer it.mtx.Unlock()
// The most common case. We are iterating through a chunk.
if it.chunkIt != nil && it.chunkIt.contains(t) {
return it.chunkIt.getValueAtTime(t)
}
it.chunkIt = nil
if len(it.chunks) == 0 {
return nil
}
// Before or exactly on the first sample of the series.
if !t.After(it.chunks[0].firstTime()) {
// return first value of first chunk
return it.chunks[0].newIterator().getValueAtTime(t)
}
// After or exactly on the last sample of the series.
if !t.Before(it.chunks[len(it.chunks)-1].lastTime()) {
// return last value of last chunk
return it.chunks[len(it.chunks)-1].newIterator().getValueAtTime(t)
}
// Find first chunk where lastTime() is after or equal to t.
i := sort.Search(len(it.chunks), func(i int) bool {
return !it.chunks[i].lastTime().Before(t)
})
if i == len(it.chunks) {
panic("out of bounds")
}
if t.Before(it.chunks[i].firstTime()) {
// We ended up between two chunks.
return metric.Values{
it.chunks[i-1].newIterator().getValueAtTime(t)[0],
it.chunks[i].newIterator().getValueAtTime(t)[0],
}
} else {
// We ended up in the middle of a chunk. We might stay there for a while,
// so save it as the current chunk iterator.
it.chunkIt = it.chunks[i].newIterator()
return it.chunkIt.getValueAtTime(t)
}
}
func (it *memorySeriesIterator) GetBoundaryValues(in metric.Interval) metric.Values {
// TODO: implement real GetBoundaryValues here.
return it.GetRangeValues(in)
}
func (it *memorySeriesIterator) GetRangeValues(in metric.Interval) metric.Values {
it.mtx.Lock()
defer it.mtx.Unlock()
// Find the first relevant chunk.
i := sort.Search(len(it.chunks), func(i int) bool {
return !it.chunks[i].lastTime().Before(in.OldestInclusive)
})
values := metric.Values{}
for _, c := range it.chunks[i:] {
if c.firstTime().After(in.NewestInclusive) {
break
}
// TODO: actually reuse an iterator between calls if we get multiple ranges
// from the same chunk.
values = append(values, c.newIterator().getRangeValues(in)...)
}
return values
}

503
storage/local/storage.go Normal file
View file

@ -0,0 +1,503 @@
package storage_ng
import (
"sync"
"time"
"github.com/golang/glog"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility"
)
const persistQueueCap = 1024
type storageState uint
const (
storageStarting storageState = iota
storageServing
storageStopping
)
type memorySeriesStorage struct {
mtx sync.RWMutex
state storageState
persistDone chan bool
stopServing chan chan<- bool
fingerprintToSeries map[clientmodel.Fingerprint]*memorySeries
labelPairToFingerprints map[metric.LabelPair]utility.Set
labelNameToLabelValues map[clientmodel.LabelName]utility.Set
memoryEvictionInterval time.Duration
memoryRetentionPeriod time.Duration
persistencePurgeInterval time.Duration
persistenceRetentionPeriod time.Duration
persistQueue chan *persistRequest
persistence Persistence
}
type MemorySeriesStorageOptions struct {
Persistence Persistence
MemoryEvictionInterval time.Duration
MemoryRetentionPeriod time.Duration
PersistencePurgeInterval time.Duration
PersistenceRetentionPeriod time.Duration
}
func NewMemorySeriesStorage(o *MemorySeriesStorageOptions) (*memorySeriesStorage, error) { // TODO: change to return Storage?
glog.Info("Loading indexes...")
i, err := o.Persistence.LoadIndexes()
if err != nil {
return nil, err
}
glog.Info("Loading series head chunks...")
if err := o.Persistence.LoadHeads(i.FingerprintToSeries); err != nil {
return nil, err
}
numSeries.Set(float64(len(i.FingerprintToSeries)))
return &memorySeriesStorage{
fingerprintToSeries: i.FingerprintToSeries,
labelPairToFingerprints: i.LabelPairToFingerprints,
labelNameToLabelValues: i.LabelNameToLabelValues,
persistDone: make(chan bool),
stopServing: make(chan chan<- bool),
memoryEvictionInterval: o.MemoryEvictionInterval,
memoryRetentionPeriod: o.MemoryRetentionPeriod,
persistencePurgeInterval: o.PersistencePurgeInterval,
persistenceRetentionPeriod: o.PersistenceRetentionPeriod,
persistQueue: make(chan *persistRequest, persistQueueCap),
persistence: o.Persistence,
}, nil
}
type persistRequest struct {
fingerprint clientmodel.Fingerprint
chunkDesc *chunkDesc
}
func (s *memorySeriesStorage) AppendSamples(samples clientmodel.Samples) {
/*
s.mtx.Lock()
defer s.mtx.Unlock()
if s.state != storageServing {
panic("storage is not serving")
}
s.mtx.Unlock()
*/
for _, sample := range samples {
s.appendSample(sample)
}
numSamples.Add(float64(len(samples)))
}
func (s *memorySeriesStorage) appendSample(sample *clientmodel.Sample) {
s.mtx.Lock()
defer s.mtx.Unlock()
series := s.getOrCreateSeries(sample.Metric)
series.add(&metric.SamplePair{
Value: sample.Value,
Timestamp: sample.Timestamp,
}, s.persistQueue)
}
func (s *memorySeriesStorage) getOrCreateSeries(m clientmodel.Metric) *memorySeries {
fp := m.Fingerprint()
series, ok := s.fingerprintToSeries[fp]
if !ok {
series = newMemorySeries(m)
s.fingerprintToSeries[fp] = series
numSeries.Set(float64(len(s.fingerprintToSeries)))
for k, v := range m {
labelPair := metric.LabelPair{
Name: k,
Value: v,
}
fps, ok := s.labelPairToFingerprints[labelPair]
if !ok {
fps = utility.Set{}
s.labelPairToFingerprints[labelPair] = fps
}
fps.Add(fp)
values, ok := s.labelNameToLabelValues[k]
if !ok {
values = utility.Set{}
s.labelNameToLabelValues[k] = values
}
values.Add(v)
}
}
return series
}
/*
func (s *memorySeriesStorage) preloadChunksAtTime(fp clientmodel.Fingerprint, ts clientmodel.Timestamp) (chunkDescs, error) {
series, ok := s.fingerprintToSeries[fp]
if !ok {
panic("requested preload for non-existent series")
}
return series.preloadChunksAtTime(ts, s.persistence)
}
*/
func (s *memorySeriesStorage) preloadChunksForRange(fp clientmodel.Fingerprint, from clientmodel.Timestamp, through clientmodel.Timestamp) (chunkDescs, error) {
s.mtx.RLock()
series, ok := s.fingerprintToSeries[fp]
s.mtx.RUnlock()
if !ok {
panic("requested preload for non-existent series")
}
return series.preloadChunksForRange(from, through, s.persistence)
}
func (s *memorySeriesStorage) NewIterator(fp clientmodel.Fingerprint) SeriesIterator {
s.mtx.RLock()
series, ok := s.fingerprintToSeries[fp]
s.mtx.RUnlock()
if !ok {
panic("requested iterator for non-existent series")
}
return series.newIterator()
}
func (s *memorySeriesStorage) evictMemoryChunks(ttl time.Duration) {
s.mtx.RLock()
defer s.mtx.RUnlock()
for _, series := range s.fingerprintToSeries {
series.evictOlderThan(clientmodel.TimestampFromTime(time.Now()).Add(-1 * ttl))
}
}
func recordPersist(start time.Time, err error) {
outcome := success
if err != nil {
outcome = failure
}
persistLatencies.WithLabelValues(outcome).Observe(float64(time.Since(start) / time.Millisecond))
}
func (s *memorySeriesStorage) handlePersistQueue() {
for req := range s.persistQueue {
// TODO: Make this thread-safe?
persistQueueLength.Set(float64(len(s.persistQueue)))
//glog.Info("Persist request: ", *req.fingerprint)
start := time.Now()
err := s.persistence.PersistChunk(req.fingerprint, req.chunkDesc.chunk)
recordPersist(start, err)
if err != nil {
glog.Error("Error persisting chunk, requeuing: ", err)
s.persistQueue <- req
continue
}
req.chunkDesc.unpin()
}
s.persistDone <- true
}
// Close stops serving, flushes all pending operations, and frees all resources.
func (s *memorySeriesStorage) Close() error {
s.mtx.Lock()
defer s.mtx.Unlock()
if s.state == storageStopping {
panic("Illegal State: Attempted to restop memorySeriesStorage.")
}
stopped := make(chan bool)
glog.Info("Waiting for storage to stop serving...")
s.stopServing <- (stopped)
glog.Info("Serving stopped.")
<-stopped
glog.Info("Stopping persist loop...")
close(s.persistQueue)
<-s.persistDone
glog.Info("Persist loop stopped.")
glog.Info("Persisting head chunks...")
if err := s.persistHeads(); err != nil {
return err
}
glog.Info("Done persisting head chunks.")
glog.Info("Persisting indexes...")
if err := s.persistIndexes(); err != nil {
return err
}
glog.Info("Done persisting indexes.")
for _, series := range s.fingerprintToSeries {
series.close()
}
s.fingerprintToSeries = nil
s.state = storageStopping
return nil
}
func (s *memorySeriesStorage) persistHeads() error {
return s.persistence.PersistHeads(s.fingerprintToSeries)
}
func (s *memorySeriesStorage) persistIndexes() error {
err := s.persistence.PersistIndexes(&Indexes{
FingerprintToSeries: s.fingerprintToSeries,
LabelPairToFingerprints: s.labelPairToFingerprints,
LabelNameToLabelValues: s.labelNameToLabelValues,
})
return err
}
func (s *memorySeriesStorage) purgePeriodically(stop <-chan bool) {
purgeTicker := time.NewTicker(s.persistencePurgeInterval)
defer purgeTicker.Stop()
for {
select {
case <-stop:
return
case <-purgeTicker.C:
glog.Info("Purging old series data...")
s.mtx.RLock()
fps := make([]clientmodel.Fingerprint, 0, len(s.fingerprintToSeries))
for fp, _ := range s.fingerprintToSeries {
fps = append(fps, fp)
}
s.mtx.RUnlock()
for _, fp := range fps {
select {
case <-stop:
glog.Info("Interrupted running series purge.")
return
default:
s.purgeSeries(&fp)
}
}
glog.Info("Done purging old series data.")
}
}
}
func (s *memorySeriesStorage) purgeSeries(fp *clientmodel.Fingerprint) {
s.mtx.RLock()
series, ok := s.fingerprintToSeries[*fp]
if !ok {
return
}
s.mtx.RUnlock()
drop, err := series.purgeOlderThan(clientmodel.TimestampFromTime(time.Now()).Add(-1*s.persistenceRetentionPeriod), s.persistence)
if err != nil {
glog.Error("Error purging series data: ", err)
}
if drop {
s.dropSeries(fp)
}
}
// Drop a label value from the label names to label values index.
func (s *memorySeriesStorage) dropLabelValue(l clientmodel.LabelName, v clientmodel.LabelValue) {
if set, ok := s.labelNameToLabelValues[l]; ok {
set.Remove(v)
if len(set) == 0 {
delete(s.labelNameToLabelValues, l)
}
}
}
// Drop all references to a series, including any samples.
func (s *memorySeriesStorage) dropSeries(fp *clientmodel.Fingerprint) {
s.mtx.Lock()
defer s.mtx.Unlock()
series, ok := s.fingerprintToSeries[*fp]
if !ok {
return
}
for k, v := range series.metric {
labelPair := metric.LabelPair{
Name: k,
Value: v,
}
if set, ok := s.labelPairToFingerprints[labelPair]; ok {
set.Remove(*fp)
if len(set) == 0 {
delete(s.labelPairToFingerprints, labelPair)
s.dropLabelValue(k, v)
}
}
}
delete(s.fingerprintToSeries, *fp)
}
func (s *memorySeriesStorage) Serve(started chan<- bool) {
s.mtx.Lock()
if s.state != storageStarting {
panic("Illegal State: Attempted to restart memorySeriesStorage.")
}
s.state = storageServing
s.mtx.Unlock()
evictMemoryTicker := time.NewTicker(s.memoryEvictionInterval)
defer evictMemoryTicker.Stop()
go s.handlePersistQueue()
stopPurge := make(chan bool)
go s.purgePeriodically(stopPurge)
started <- true
for {
select {
case <-evictMemoryTicker.C:
s.evictMemoryChunks(s.memoryRetentionPeriod)
case stopped := <-s.stopServing:
stopPurge <- true
stopped <- true
return
}
}
}
func (s *memorySeriesStorage) NewPreloader() Preloader {
return &memorySeriesPreloader{
storage: s,
}
}
func (s *memorySeriesStorage) GetFingerprintsForLabelMatchers(labelMatchers metric.LabelMatchers) clientmodel.Fingerprints {
s.mtx.RLock()
defer s.mtx.RUnlock()
sets := []utility.Set{}
for _, matcher := range labelMatchers {
switch matcher.Type {
case metric.Equal:
set, ok := s.labelPairToFingerprints[metric.LabelPair{
Name: matcher.Name,
Value: matcher.Value,
}]
if !ok {
return nil
}
sets = append(sets, set)
default:
values := s.getLabelValuesForLabelName(matcher.Name)
matches := matcher.Filter(values)
if len(matches) == 0 {
return nil
}
set := utility.Set{}
for _, v := range matches {
subset, ok := s.labelPairToFingerprints[metric.LabelPair{
Name: matcher.Name,
Value: v,
}]
if !ok {
return nil
}
for fp := range subset {
set.Add(fp)
}
}
sets = append(sets, set)
}
}
setCount := len(sets)
if setCount == 0 {
return nil
}
base := sets[0]
for i := 1; i < setCount; i++ {
base = base.Intersection(sets[i])
}
fingerprints := clientmodel.Fingerprints{}
for _, e := range base.Elements() {
fingerprints = append(fingerprints, e.(clientmodel.Fingerprint))
}
return fingerprints
}
func (s *memorySeriesStorage) GetLabelValuesForLabelName(labelName clientmodel.LabelName) clientmodel.LabelValues {
s.mtx.RLock()
defer s.mtx.RUnlock()
return s.getLabelValuesForLabelName(labelName)
}
func (s *memorySeriesStorage) getLabelValuesForLabelName(labelName clientmodel.LabelName) clientmodel.LabelValues {
set, ok := s.labelNameToLabelValues[labelName]
if !ok {
return nil
}
values := make(clientmodel.LabelValues, 0, len(set))
for e := range set {
val := e.(clientmodel.LabelValue)
values = append(values, val)
}
return values
}
func (s *memorySeriesStorage) GetMetricForFingerprint(fp clientmodel.Fingerprint) clientmodel.Metric {
s.mtx.RLock()
defer s.mtx.RUnlock()
series, ok := s.fingerprintToSeries[fp]
if !ok {
return nil
}
metric := clientmodel.Metric{}
for label, value := range series.metric {
metric[label] = value
}
return metric
}
func (s *memorySeriesStorage) GetAllValuesForLabel(labelName clientmodel.LabelName) clientmodel.LabelValues {
s.mtx.RLock()
defer s.mtx.RUnlock()
var values clientmodel.LabelValues
valueSet := map[clientmodel.LabelValue]struct{}{}
for _, series := range s.fingerprintToSeries {
if value, ok := series.metric[labelName]; ok {
if _, ok := valueSet[value]; !ok {
values = append(values, value)
valueSet[value] = struct{}{}
}
}
}
return values
}

View file

@ -0,0 +1,112 @@
package storage_ng
import (
"fmt"
"testing"
"time"
clientmodel "github.com/prometheus/client_golang/model"
)
func TestChunk(t *testing.T) {
samples := make(clientmodel.Samples, 500000)
for i := range samples {
samples[i] = &clientmodel.Sample{
Timestamp: clientmodel.Timestamp(i),
Value: clientmodel.SampleValue(float64(i) * 0.2),
}
}
s, closer := NewTestStorage(t)
defer closer.Close()
s.AppendSamples(samples)
for _, s := range s.(*memorySeriesStorage).fingerprintToSeries {
for i, v := range s.values() {
if samples[i].Timestamp != v.Timestamp {
t.Fatalf("%d. Got %v; want %v", i, v.Timestamp, samples[i].Timestamp)
}
if samples[i].Value != v.Value {
t.Fatalf("%d. Got %v; want %v", i, v.Value, samples[i].Value)
}
}
}
}
func TestGetValueAtTime(t *testing.T) {
samples := make(clientmodel.Samples, 50000)
for i := range samples {
samples[i] = &clientmodel.Sample{
Timestamp: clientmodel.Timestamp(time.Duration(i) * time.Second),
Value: clientmodel.SampleValue(float64(i) * 0.2),
}
}
s, closer := NewTestStorage(t)
defer closer.Close()
s.AppendSamples(samples)
fp := clientmodel.Metric{}.Fingerprint()
it := s.NewIterator(fp)
for i, expected := range samples {
actual := it.GetValueAtTime(samples[i].Timestamp)
if expected.Timestamp != actual[0].Timestamp {
t.Fatalf("%d. Got %v; want %v", i, actual[0].Timestamp, expected.Timestamp)
}
if expected.Value != actual[0].Value {
t.Fatalf("%d. Got %v; want %v", i, actual[0].Value, expected.Value)
}
}
}
func TestGetRangeValues(t *testing.T) {
samples := make(clientmodel.Samples, 50000)
for i := range samples {
samples[i] = &clientmodel.Sample{
Timestamp: clientmodel.Timestamp(time.Duration(i) * time.Second),
Value: clientmodel.SampleValue(float64(i) * 0.2),
}
}
s, closer := NewTestStorage(t)
defer closer.Close()
s.AppendSamples(samples)
fp := clientmodel.Metric{}.Fingerprint()
it := s.NewIterator(fp)
for i, expected := range samples {
actual := it.GetValueAtTime(samples[i].Timestamp)
if expected.Timestamp != actual[0].Timestamp {
t.Fatalf("%d. Got %v; want %v", i, actual[0].Timestamp, expected.Timestamp)
}
if expected.Value != actual[0].Value {
t.Fatalf("%d. Got %v; want %v", i, actual[0].Value, expected.Value)
}
}
}
func BenchmarkAppend(b *testing.B) {
samples := make(clientmodel.Samples, b.N)
for i := range samples {
samples[i] = &clientmodel.Sample{
Metric: clientmodel.Metric{
clientmodel.MetricNameLabel: clientmodel.LabelValue(fmt.Sprintf("test_metric_%d", i%10)),
"label1": clientmodel.LabelValue(fmt.Sprintf("test_metric_%d", i%10)),
"label2": clientmodel.LabelValue(fmt.Sprintf("test_metric_%d", i%10)),
},
Timestamp: clientmodel.Timestamp(i),
Value: clientmodel.SampleValue(i),
}
}
b.ResetTimer()
s, closer := NewTestStorage(b)
defer closer.Close()
s.AppendSamples(samples)
}

View file

@ -0,0 +1,47 @@
package storage_ng
import (
"testing"
"time"
"github.com/prometheus/prometheus/utility/test"
)
type testStorageCloser struct {
storage Storage
directory test.Closer
}
func (t *testStorageCloser) Close() {
t.storage.Close()
t.directory.Close()
}
func NewTestStorage(t testing.TB) (Storage, test.Closer) {
directory := test.NewTemporaryDirectory("test_storage", t)
persistence, err := NewDiskPersistence(directory.Path(), 1024)
if err != nil {
t.Fatal("Error opening disk persistence: ", err)
}
o := &MemorySeriesStorageOptions{
Persistence: persistence,
MemoryEvictionInterval: time.Minute,
MemoryRetentionPeriod: time.Hour,
}
storage, err := NewMemorySeriesStorage(o)
if err != nil {
directory.Close()
t.Fatalf("Error creating storage: %s", err)
}
storageStarted := make(chan bool)
go storage.Serve(storageStarted)
<-storageStarted
closer := &testStorageCloser{
storage: storage,
directory: directory,
}
return storage, closer
}

View file

@ -1,142 +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 (
"time"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/stats"
)
// Persistence is a system for storing metric samples in a persistence
// layer.
type Persistence interface {
// A storage system may rely on external resources and thusly should be
// closed when finished.
Close()
// Record a group of new samples in the storage layer. Multiple samples for
// the same fingerprint need to be submitted in chronological order, from
// oldest to newest (both in the same call to AppendSamples and across
// multiple calls).
AppendSamples(clientmodel.Samples) error
// Get all of the metric fingerprints that are associated with the
// provided label matchers.
GetFingerprintsForLabelMatchers(LabelMatchers) (clientmodel.Fingerprints, error)
// Get all of the label values that are associated with a given label name.
GetLabelValuesForLabelName(clientmodel.LabelName) (clientmodel.LabelValues, error)
// Get the metric associated with the provided fingerprint.
GetMetricForFingerprint(*clientmodel.Fingerprint) (clientmodel.Metric, error)
// Get all label values that are associated with a given label name.
GetAllValuesForLabel(clientmodel.LabelName) (clientmodel.LabelValues, error)
}
// PreloadingPersistence is a Persistence which supports building
// preloaded views.
type PreloadingPersistence interface {
Persistence
// NewViewRequestBuilder furnishes a ViewRequestBuilder for remarking what
// types of queries to perform.
NewViewRequestBuilder() ViewRequestBuilder
}
// View provides a view of the values in the datastore subject to the request
// of a preloading operation.
type View interface {
// Get the two values that are immediately adjacent to a given time.
GetValueAtTime(*clientmodel.Fingerprint, clientmodel.Timestamp) Values
// Get the boundary values of an interval: the first value older than
// the interval start, and the first value younger than the interval
// end.
GetBoundaryValues(*clientmodel.Fingerprint, Interval) Values
// Get all values contained within a provided interval.
GetRangeValues(*clientmodel.Fingerprint, Interval) Values
}
// ViewablePersistence is a Persistence that is able to present the
// samples it has stored as a View.
type ViewablePersistence interface {
Persistence
View
}
// ViewRequestBuilder represents the summation of all datastore queries that
// shall be performed to extract values. Call the Get... methods to record the
// queries. Once done, use HasOp and PopOp to retrieve the resulting
// operations. The operations are sorted by their fingerprint (and, for equal
// fingerprints, by the StartsAt timestamp of their operation).
type ViewRequestBuilder interface {
// GetMetricAtTime records a query to get, for the given Fingerprint,
// either the value at that time if there is a match or the one or two
// values adjacent thereto.
GetMetricAtTime(fingerprint *clientmodel.Fingerprint, time clientmodel.Timestamp)
// GetMetricAtInterval records a query to get, for the given
// Fingerprint, either the value at that interval from From through
// Through if there is a match or the one or two values adjacent for
// each point.
GetMetricAtInterval(fingerprint *clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval time.Duration)
// GetMetricRange records a query to get, for the given Fingerprint, the
// values that occur inclusively from From through Through.
GetMetricRange(fingerprint *clientmodel.Fingerprint, from, through clientmodel.Timestamp)
// GetMetricRangeAtInterval records a query to get value ranges at
// intervals for the given Fingerprint:
//
// |----| |----| |----| |----|
// ^ ^ ^ ^ ^ ^
// | \------------/ \----/ |
// from interval rangeDuration through
GetMetricRangeAtInterval(fp *clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval, rangeDuration time.Duration)
// Execute materializes a View, subject to a timeout.
Execute(deadline time.Duration, queryStats *stats.TimerGroup) (View, error)
// PopOp emits the next operation in the queue (sorted by
// fingerprint). If called while HasOps returns false, the
// behavior is undefined.
PopOp() Op
// HasOp returns true if there is at least one more operation in the
// queue.
HasOp() bool
}
// Op encapsulates a primitive query operation.
type Op interface {
// Fingerprint returns the fingerprint of the metric this operation
// operates on.
Fingerprint() *clientmodel.Fingerprint
// ExtractSamples extracts samples from a stream of values and advances
// the operation time.
ExtractSamples(Values) Values
// Consumed returns whether the operator has consumed all data it needs.
Consumed() bool
// CurrentTime gets the current operation time. In a newly created op,
// this is the starting time of the operation. During ongoing execution
// of the op, the current time is advanced accordingly. Once no
// subsequent work associated with the operation remains, nil is
// returned.
CurrentTime() clientmodel.Timestamp
}
// CurationState contains high-level curation state information for the
// heads-up-display.
type CurationState struct {
Active bool
Name string
Limit time.Duration
Fingerprint *clientmodel.Fingerprint
}

View file

@ -14,9 +14,7 @@
package metric
import (
"bytes"
"fmt"
"sort"
clientmodel "github.com/prometheus/client_golang/model"
)
@ -32,6 +30,7 @@ type SamplePair struct {
Value clientmodel.SampleValue
}
// TODO: can this method be deleted, or is it used in tests?
// Equal returns true if this SamplePair and o have equal Values and equal
// Timestamps.
func (s *SamplePair) Equal(o *SamplePair) bool {
@ -46,96 +45,9 @@ func (s *SamplePair) String() string {
return fmt.Sprintf("SamplePair at %s of %s", s.Timestamp, s.Value)
}
// Values is a sortable slice of SamplePairs (as in: it implements
// sort.Interface). Sorting happens by Timestamp.
// Values is a slice of SamplePairs.
type Values []SamplePair
// Len implements sort.Interface.
func (v Values) Len() int {
return len(v)
}
// Less implements sort.Interface.
func (v Values) Less(i, j int) bool {
return v[i].Timestamp.Before(v[j].Timestamp)
}
// Swap implements sort.Interface.
func (v Values) Swap(i, j int) {
v[i], v[j] = v[j], v[i]
}
// Equal returns true if these Values are of the same length as o, and each
// value is equal to the corresponding value in o (i.e. at the same index).
func (v Values) Equal(o Values) bool {
if len(v) != len(o) {
return false
}
for i, expected := range v {
if !expected.Equal(&o[i]) {
return false
}
}
return true
}
// FirstTimeAfter indicates whether the first sample of a set is after a given
// timestamp.
func (v Values) FirstTimeAfter(t clientmodel.Timestamp) bool {
return v[0].Timestamp.After(t)
}
// LastTimeBefore indicates whether the last sample of a set is before a given
// timestamp.
func (v Values) LastTimeBefore(t clientmodel.Timestamp) bool {
return v[len(v)-1].Timestamp.Before(t)
}
// InsideInterval indicates whether a given range of sorted values could contain
// a value for a given time.
func (v Values) InsideInterval(t clientmodel.Timestamp) bool {
switch {
case v.Len() == 0:
return false
case t.Before(v[0].Timestamp):
return false
case !v[v.Len()-1].Timestamp.Before(t):
return false
default:
return true
}
}
// TruncateBefore returns a subslice of the original such that extraneous
// samples in the collection that occur before the provided time are
// dropped. The original slice is not mutated.
func (v Values) TruncateBefore(t clientmodel.Timestamp) Values {
index := sort.Search(len(v), func(i int) bool {
timestamp := v[i].Timestamp
return !timestamp.Before(t)
})
return v[index:]
}
func (v Values) String() string {
buffer := bytes.Buffer{}
fmt.Fprintf(&buffer, "[")
for i, value := range v {
fmt.Fprintf(&buffer, "%d. %s", i, value)
if i != len(v)-1 {
fmt.Fprintf(&buffer, "\n")
}
}
fmt.Fprintf(&buffer, "]")
return buffer.String()
}
// SampleSet is Values with a Metric attached.
type SampleSet struct {
Metric clientmodel.Metric

View file

@ -1,267 +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 tiered
import (
"flag"
"fmt"
"sort"
"testing"
"time"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/storage/metric"
clientmodel "github.com/prometheus/client_golang/model"
)
type nopCurationStateUpdater struct{}
func (n *nopCurationStateUpdater) UpdateCurationState(*metric.CurationState) {}
func generateTestSamples(endTime clientmodel.Timestamp, numTs int, samplesPerTs int, interval time.Duration) clientmodel.Samples {
samples := make(clientmodel.Samples, 0, numTs*samplesPerTs)
startTime := endTime.Add(-interval * time.Duration(samplesPerTs-1))
for ts := 0; ts < numTs; ts++ {
metric := clientmodel.Metric{}
metric[clientmodel.MetricNameLabel] = clientmodel.LabelValue(fmt.Sprintf("metric_%d", ts))
for i := 0; i < samplesPerTs; i++ {
sample := &clientmodel.Sample{
Metric: metric,
Value: clientmodel.SampleValue(ts + 1000*i),
Timestamp: startTime.Add(interval * time.Duration(i)),
}
samples = append(samples, sample)
}
}
sort.Sort(samples)
return samples
}
type compactionChecker struct {
t *testing.T
sampleIdx int
numChunks int
expectedSamples clientmodel.Samples
}
func (c *compactionChecker) Operate(key, value interface{}) *storage.OperatorError {
c.numChunks++
sampleKey := key.(*SampleKey)
if sampleKey.FirstTimestamp.After(sampleKey.LastTimestamp) {
c.t.Fatalf("Chunk FirstTimestamp (%v) is after LastTimestamp (%v): %v", sampleKey.FirstTimestamp.Unix(), sampleKey.LastTimestamp.Unix(), sampleKey)
}
fp := &clientmodel.Fingerprint{}
for _, sample := range value.(metric.Values) {
if sample.Timestamp.Before(sampleKey.FirstTimestamp) || sample.Timestamp.After(sampleKey.LastTimestamp) {
c.t.Fatalf("Sample not within chunk boundaries: chunk FirstTimestamp (%v), chunk LastTimestamp (%v) vs. sample Timestamp (%v)", sampleKey.FirstTimestamp.Unix(), sampleKey.LastTimestamp.Unix(), sample.Timestamp)
}
expected := c.expectedSamples[c.sampleIdx]
fp.LoadFromMetric(expected.Metric)
if !sampleKey.Fingerprint.Equal(fp) {
c.t.Fatalf("%d. Expected fingerprint %s, got %s", c.sampleIdx, fp, sampleKey.Fingerprint)
}
sp := &metric.SamplePair{
Value: expected.Value,
Timestamp: expected.Timestamp,
}
if !sample.Equal(sp) {
c.t.Fatalf("%d. Expected sample %s, got %s", c.sampleIdx, sp, sample)
}
c.sampleIdx++
}
return nil
}
func checkStorageSaneAndEquivalent(t *testing.T, name string, ts *TieredStorage, samples clientmodel.Samples, expectedNumChunks int) {
cc := &compactionChecker{
expectedSamples: samples,
t: t,
}
entire, err := ts.DiskStorage.MetricSamples.ForEach(&MetricSamplesDecoder{}, &AcceptAllFilter{}, cc)
if err != nil {
t.Fatalf("%s: Error checking samples: %s", name, err)
}
if !entire {
t.Fatalf("%s: Didn't scan entire corpus", name)
}
if cc.numChunks != expectedNumChunks {
t.Fatalf("%s: Expected %d chunks, got %d", name, expectedNumChunks, cc.numChunks)
}
}
type compactionTestScenario struct {
leveldbChunkSize int
numTimeseries int
samplesPerTs int
ignoreYoungerThan time.Duration
maximumMutationPoolBatch int
minimumGroupSize int
uncompactedChunks int
compactedChunks int
}
func (s compactionTestScenario) test(t *testing.T) {
defer flag.Set("leveldbChunkSize", flag.Lookup("leveldbChunkSize").Value.String())
flag.Set("leveldbChunkSize", fmt.Sprintf("%d", s.leveldbChunkSize))
ts, closer := NewTestTieredStorage(t)
defer closer.Close()
// 1. Store test values.
samples := generateTestSamples(testInstant, s.numTimeseries, s.samplesPerTs, time.Minute)
ts.AppendSamples(samples)
ts.Flush()
// 2. Check sanity of uncompacted values.
checkStorageSaneAndEquivalent(t, "Before compaction", ts, samples, s.uncompactedChunks)
// 3. Compact test storage.
processor := NewCompactionProcessor(&CompactionProcessorOptions{
MaximumMutationPoolBatch: s.maximumMutationPoolBatch,
MinimumGroupSize: s.minimumGroupSize,
})
defer processor.Close()
curator := NewCurator(&CuratorOptions{
Stop: make(chan struct{}),
ViewQueue: ts.ViewQueue,
})
defer curator.Close()
err := curator.Run(s.ignoreYoungerThan, testInstant, processor, ts.DiskStorage.CurationRemarks, ts.DiskStorage.MetricSamples, ts.DiskStorage.MetricHighWatermarks, &nopCurationStateUpdater{})
if err != nil {
t.Fatalf("Failed to run curator: %s", err)
}
// 4. Check sanity of compacted values.
checkStorageSaneAndEquivalent(t, "After compaction", ts, samples, s.compactedChunks)
}
func TestCompaction(t *testing.T) {
scenarios := []compactionTestScenario{
// BEFORE COMPACTION:
//
// Chunk size | Fingerprint | Samples
// 5 | A | 1 .. 5
// 5 | A | 6 .. 10
// 5 | A | 11 .. 15
// 5 | B | 1 .. 5
// 5 | B | 6 .. 10
// 5 | B | 11 .. 15
// 5 | C | 1 .. 5
// 5 | C | 6 .. 10
// 5 | C | 11 .. 15
//
// AFTER COMPACTION:
//
// Chunk size | Fingerprint | Samples
// 10 | A | 1 .. 10
// 5 | A | 11 .. 15
// 10 | B | 1 .. 10
// 5 | B | 11 .. 15
// 10 | C | 1 .. 10
// 5 | C | 11 .. 15
{
leveldbChunkSize: 5,
numTimeseries: 3,
samplesPerTs: 15,
ignoreYoungerThan: time.Minute,
maximumMutationPoolBatch: 30,
minimumGroupSize: 10,
uncompactedChunks: 9,
compactedChunks: 6,
},
// BEFORE COMPACTION:
//
// Chunk size | Fingerprint | Samples
// 5 | A | 1 .. 5
// 5 | A | 6 .. 10
// 5 | A | 11 .. 15
// 5 | B | 1 .. 5
// 5 | B | 6 .. 10
// 5 | B | 11 .. 15
// 5 | C | 1 .. 5
// 5 | C | 6 .. 10
// 5 | C | 11 .. 15
//
// AFTER COMPACTION:
//
// Chunk size | Fingerprint | Samples
// 10 | A | 1 .. 15
// 10 | B | 1 .. 15
// 10 | C | 1 .. 15
{
leveldbChunkSize: 5,
numTimeseries: 3,
samplesPerTs: 15,
ignoreYoungerThan: time.Minute,
maximumMutationPoolBatch: 30,
minimumGroupSize: 30,
uncompactedChunks: 9,
compactedChunks: 3,
},
// BEFORE COMPACTION:
//
// Chunk size | Fingerprint | Samples
// 5 | A | 1 .. 5
// 5 | A | 6 .. 10
// 5 | A | 11 .. 15
// 5 | A | 16 .. 20
// 5 | B | 1 .. 5
// 5 | B | 6 .. 10
// 5 | B | 11 .. 15
// 5 | B | 16 .. 20
// 5 | C | 1 .. 5
// 5 | C | 6 .. 10
// 5 | C | 11 .. 15
// 5 | C | 16 .. 20
//
// AFTER COMPACTION:
//
// Chunk size | Fingerprint | Samples
// 10 | A | 1 .. 15
// 10 | A | 16 .. 20
// 10 | B | 1 .. 15
// 10 | B | 16 .. 20
// 10 | C | 1 .. 15
// 10 | C | 16 .. 20
{
leveldbChunkSize: 5,
numTimeseries: 3,
samplesPerTs: 20,
ignoreYoungerThan: time.Minute,
maximumMutationPoolBatch: 30,
minimumGroupSize: 10,
uncompactedChunks: 12,
compactedChunks: 6,
},
}
for _, s := range scenarios {
s.test(t)
}
}

View file

@ -1,509 +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 tiered
import (
"bytes"
"errors"
"fmt"
"strings"
"time"
"code.google.com/p/goprotobuf/proto"
"github.com/golang/glog"
"github.com/prometheus/client_golang/prometheus"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/raw"
"github.com/prometheus/prometheus/storage/raw/leveldb"
dto "github.com/prometheus/prometheus/model/generated"
)
const curationYieldPeriod = 250 * time.Millisecond
var errIllegalIterator = errors.New("iterator invalid")
// Constants for instrumentation.
const (
cutOff = "recency_threshold"
processorName = "processor"
)
var (
curationDurations = prometheus.NewSummaryVec(
prometheus.SummaryOpts{
Namespace: namespace,
Name: "curation_durations_milliseconds",
Help: "Histogram of time spent in curation.",
Objectives: []float64{0.01, 0.05, 0.5, 0.90, 0.99},
},
[]string{cutOff, processorName, result},
)
curationFilterOperations = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: namespace,
Name: "curation_filter_operations_total",
Help: "The number of curation filter operations completed.",
},
[]string{cutOff, processorName, result},
)
)
func init() {
prometheus.MustRegister(curationDurations)
prometheus.MustRegister(curationFilterOperations)
}
// CurationStateUpdater receives updates about the curation state.
type CurationStateUpdater interface {
UpdateCurationState(*metric.CurationState)
}
// CuratorOptions bundles the parameters needed to create a Curator.
type CuratorOptions struct {
Stop chan struct{}
ViewQueue chan viewJob
}
// Curator is responsible for effectuating a given curation policy across the
// stored samples on-disk. This is useful to compact sparse sample values into
// single sample entities to reduce keyspace load on the datastore.
type Curator struct {
stop chan struct{}
viewQueue chan viewJob
dtoSampleKeys *dtoSampleKeyList
sampleKeys *sampleKeyList
}
// NewCurator returns an initialized Curator.
func NewCurator(o *CuratorOptions) *Curator {
return &Curator{
stop: o.Stop,
viewQueue: o.ViewQueue,
dtoSampleKeys: newDtoSampleKeyList(10),
sampleKeys: newSampleKeyList(10),
}
}
// watermarkScanner converts (dto.Fingerprint, dto.MetricHighWatermark) doubles
// into (model.Fingerprint, model.Watermark) doubles.
//
// watermarkScanner determines whether to include or exclude candidate
// values from the curation process by virtue of how old the high watermark is.
//
// watermarkScanner scans over the curator.samples table for metrics whose
// high watermark has been determined to be allowable for curation. This type
// is individually responsible for compaction.
//
// The scanning starts from CurationRemark.LastCompletionTimestamp and goes
// forward until the stop point or end of the series is reached.
type watermarkScanner struct {
// curationState is the data store for curation remarks.
curationState CurationRemarker
// ignoreYoungerThan is passed into the curation remark for the given series.
ignoreYoungerThan time.Duration
// processor is responsible for executing a given stategy on the
// to-be-operated-on series.
processor Processor
// sampleIterator is a snapshotted iterator for the time series.
sampleIterator leveldb.Iterator
// samples
samples raw.Persistence
// stopAt is a cue for when to stop mutating a given series.
stopAt clientmodel.Timestamp
// stop functions as the global stop channel for all future operations.
stop chan struct{}
// status is the outbound channel for notifying the status page of its state.
status CurationStateUpdater
firstBlock, lastBlock *SampleKey
ViewQueue chan viewJob
dtoSampleKeys *dtoSampleKeyList
sampleKeys *sampleKeyList
}
// Run facilitates the curation lifecycle.
//
// recencyThreshold represents the most recent time up to which values will be
// curated.
// curationState is the on-disk store where the curation remarks are made for
// how much progress has been made.
func (c *Curator) Run(ignoreYoungerThan time.Duration, instant clientmodel.Timestamp, processor Processor, curationState CurationRemarker, samples *leveldb.LevelDBPersistence, watermarks HighWatermarker, status CurationStateUpdater) (err error) {
defer func(t time.Time) {
duration := float64(time.Since(t) / time.Millisecond)
labels := prometheus.Labels{
cutOff: fmt.Sprint(ignoreYoungerThan),
processorName: processor.Name(),
result: success,
}
if err != nil {
labels[result] = failure
}
curationDurations.With(labels).Observe(duration)
}(time.Now())
defer status.UpdateCurationState(&metric.CurationState{Active: false})
iterator, err := samples.NewIterator(true)
if err != nil {
return err
}
defer iterator.Close()
if !iterator.SeekToLast() {
glog.Info("Empty database; skipping curation.")
return
}
keyDto, _ := c.dtoSampleKeys.Get()
defer c.dtoSampleKeys.Give(keyDto)
lastBlock, _ := c.sampleKeys.Get()
defer c.sampleKeys.Give(lastBlock)
if err := iterator.Key(keyDto); err != nil {
panic(err)
}
lastBlock.Load(keyDto)
if !iterator.SeekToFirst() {
glog.Info("Empty database; skipping curation.")
return
}
firstBlock, _ := c.sampleKeys.Get()
defer c.sampleKeys.Give(firstBlock)
if err := iterator.Key(keyDto); err != nil {
panic(err)
}
firstBlock.Load(keyDto)
scanner := &watermarkScanner{
curationState: curationState,
ignoreYoungerThan: ignoreYoungerThan,
processor: processor,
status: status,
stop: c.stop,
stopAt: instant.Add(-1 * ignoreYoungerThan),
sampleIterator: iterator,
samples: samples,
firstBlock: firstBlock,
lastBlock: lastBlock,
ViewQueue: c.viewQueue,
dtoSampleKeys: c.dtoSampleKeys,
sampleKeys: c.sampleKeys,
}
// Right now, the ability to stop a curation is limited to the beginning of
// each fingerprint cycle. It is impractical to cease the work once it has
// begun for a given series.
_, err = watermarks.ForEach(scanner, scanner, scanner)
return
}
// Close needs to be called to cleanly dispose of a curator.
func (c *Curator) Close() {
c.dtoSampleKeys.Close()
c.sampleKeys.Close()
}
func (w *watermarkScanner) DecodeKey(in interface{}) (interface{}, error) {
key := &dto.Fingerprint{}
bytes := in.([]byte)
if err := proto.Unmarshal(bytes, key); err != nil {
return nil, err
}
fingerprint := &clientmodel.Fingerprint{}
loadFingerprint(fingerprint, key)
return fingerprint, nil
}
func (w *watermarkScanner) DecodeValue(in interface{}) (interface{}, error) {
value := &dto.MetricHighWatermark{}
bytes := in.([]byte)
if err := proto.Unmarshal(bytes, value); err != nil {
return nil, err
}
watermark := &watermarks{}
watermark.load(value)
return watermark, nil
}
func (w *watermarkScanner) shouldStop() bool {
select {
case _, ok := <-w.stop:
if ok {
panic("channel should be closed only")
}
return true
default:
return false
}
}
func (w *watermarkScanner) Filter(key, value interface{}) (r storage.FilterResult) {
fingerprint := key.(*clientmodel.Fingerprint)
defer func() {
labels := prometheus.Labels{
cutOff: fmt.Sprint(w.ignoreYoungerThan),
result: strings.ToLower(r.String()),
processorName: w.processor.Name(),
}
curationFilterOperations.With(labels).Inc()
w.status.UpdateCurationState(&metric.CurationState{
Active: true,
Name: w.processor.Name(),
Limit: w.ignoreYoungerThan,
Fingerprint: fingerprint,
})
}()
if w.shouldStop() {
return storage.Stop
}
k := &curationKey{
Fingerprint: fingerprint,
ProcessorMessageRaw: w.processor.Signature(),
ProcessorMessageTypeName: w.processor.Name(),
IgnoreYoungerThan: w.ignoreYoungerThan,
}
curationRemark, present, err := w.curationState.Get(k)
if err != nil {
return
}
if !present {
return storage.Accept
}
if !curationRemark.Before(w.stopAt) {
return storage.Skip
}
watermark := value.(*watermarks)
if !curationRemark.Before(watermark.High) {
return storage.Skip
}
curationConsistent, err := w.curationConsistent(fingerprint, watermark)
if err != nil {
return
}
if curationConsistent {
return storage.Skip
}
return storage.Accept
}
// curationConsistent determines whether the given metric is in a dirty state
// and needs curation.
func (w *watermarkScanner) curationConsistent(f *clientmodel.Fingerprint, watermark *watermarks) (bool, error) {
k := &curationKey{
Fingerprint: f,
ProcessorMessageRaw: w.processor.Signature(),
ProcessorMessageTypeName: w.processor.Name(),
IgnoreYoungerThan: w.ignoreYoungerThan,
}
curationRemark, present, err := w.curationState.Get(k)
if err != nil {
return false, err
}
if !present {
return false, nil
}
if !curationRemark.Before(watermark.High) {
return true, nil
}
return false, nil
}
func (w *watermarkScanner) Operate(key, _ interface{}) (oErr *storage.OperatorError) {
fingerprint := key.(*clientmodel.Fingerprint)
glog.Infof("Curating %s...", fingerprint)
if len(w.ViewQueue) > 0 {
glog.Warning("Deferred due to view queue.")
time.Sleep(curationYieldPeriod)
}
if fingerprint.Less(w.firstBlock.Fingerprint) {
glog.Warning("Skipped since before keyspace.")
return nil
}
if w.lastBlock.Fingerprint.Less(fingerprint) {
glog.Warning("Skipped since after keyspace.")
return nil
}
curationState, _, err := w.curationState.Get(&curationKey{
Fingerprint: fingerprint,
ProcessorMessageRaw: w.processor.Signature(),
ProcessorMessageTypeName: w.processor.Name(),
IgnoreYoungerThan: w.ignoreYoungerThan,
})
if err != nil {
glog.Warning("Unable to get curation state: %s", err)
// An anomaly with the curation remark is likely not fatal in the sense that
// there was a decoding error with the entity and shouldn't be cause to stop
// work. The process will simply start from a pessimistic work time and
// work forward. With an idempotent processor, this is safe.
return &storage.OperatorError{Error: err, Continuable: true}
}
keySet, _ := w.sampleKeys.Get()
defer w.sampleKeys.Give(keySet)
keySet.Fingerprint = fingerprint
keySet.FirstTimestamp = curationState
// Invariant: The fingerprint tests above ensure that we have the same
// fingerprint.
keySet.Constrain(w.firstBlock, w.lastBlock)
seeker := &iteratorSeekerState{
i: w.sampleIterator,
obj: keySet,
first: w.firstBlock,
last: w.lastBlock,
dtoSampleKeys: w.dtoSampleKeys,
sampleKeys: w.sampleKeys,
}
for state := seeker.initialize; state != nil; state = state() {
}
if seeker.err != nil {
glog.Warningf("Got error in state machine: %s", seeker.err)
return &storage.OperatorError{Error: seeker.err, Continuable: !seeker.iteratorInvalid}
}
if seeker.iteratorInvalid {
glog.Warningf("Got illegal iterator in state machine: %s", err)
return &storage.OperatorError{Error: errIllegalIterator, Continuable: false}
}
if !seeker.seriesOperable {
return
}
lastTime, err := w.processor.Apply(w.sampleIterator, w.samples, w.stopAt, fingerprint)
if err != nil {
// We can't divine the severity of a processor error without refactoring the
// interface.
return &storage.OperatorError{Error: err, Continuable: false}
}
if err = w.curationState.Update(&curationKey{
Fingerprint: fingerprint,
ProcessorMessageRaw: w.processor.Signature(),
ProcessorMessageTypeName: w.processor.Name(),
IgnoreYoungerThan: w.ignoreYoungerThan,
}, lastTime); err != nil {
// Under the assumption that the processors are idempotent, they can be
// re-run; thusly, the commitment of the curation remark is no cause
// to cease further progress.
return &storage.OperatorError{Error: err, Continuable: true}
}
return nil
}
// curationKey provides a representation of dto.CurationKey with associated
// business logic methods attached to it to enhance code readability.
type curationKey struct {
Fingerprint *clientmodel.Fingerprint
ProcessorMessageRaw []byte
ProcessorMessageTypeName string
IgnoreYoungerThan time.Duration
}
// Equal answers whether the two curationKeys are equivalent.
func (c *curationKey) Equal(o *curationKey) bool {
switch {
case !c.Fingerprint.Equal(o.Fingerprint):
return false
case bytes.Compare(c.ProcessorMessageRaw, o.ProcessorMessageRaw) != 0:
return false
case c.ProcessorMessageTypeName != o.ProcessorMessageTypeName:
return false
case c.IgnoreYoungerThan != o.IgnoreYoungerThan:
return false
}
return true
}
func (c *curationKey) dump(d *dto.CurationKey) {
d.Reset()
// BUG(matt): Avenue for simplification.
fingerprintDTO := &dto.Fingerprint{}
dumpFingerprint(fingerprintDTO, c.Fingerprint)
d.Fingerprint = fingerprintDTO
d.ProcessorMessageRaw = c.ProcessorMessageRaw
d.ProcessorMessageTypeName = proto.String(c.ProcessorMessageTypeName)
d.IgnoreYoungerThan = proto.Int64(int64(c.IgnoreYoungerThan))
}
func (c *curationKey) load(d *dto.CurationKey) {
// BUG(matt): Avenue for simplification.
c.Fingerprint = &clientmodel.Fingerprint{}
loadFingerprint(c.Fingerprint, d.Fingerprint)
c.ProcessorMessageRaw = d.ProcessorMessageRaw
c.ProcessorMessageTypeName = d.GetProcessorMessageTypeName()
c.IgnoreYoungerThan = time.Duration(d.GetIgnoreYoungerThan())
}

View file

@ -1,68 +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 tiered
import (
"sort"
"code.google.com/p/goprotobuf/proto"
clientmodel "github.com/prometheus/client_golang/model"
dto "github.com/prometheus/prometheus/model/generated"
)
func dumpFingerprint(d *dto.Fingerprint, f *clientmodel.Fingerprint) {
d.Reset()
d.Signature = proto.String(f.String())
}
func loadFingerprint(f *clientmodel.Fingerprint, d *dto.Fingerprint) {
f.LoadFromString(d.GetSignature())
}
func dumpMetric(d *dto.Metric, m clientmodel.Metric) {
d.Reset()
metricLength := len(m)
labelNames := make([]string, 0, metricLength)
for labelName := range m {
labelNames = append(labelNames, string(labelName))
}
sort.Strings(labelNames)
pairs := make([]*dto.LabelPair, 0, metricLength)
for _, labelName := range labelNames {
l := clientmodel.LabelName(labelName)
labelValue := m[l]
labelPair := &dto.LabelPair{
Name: proto.String(string(labelName)),
Value: proto.String(string(labelValue)),
}
pairs = append(pairs, labelPair)
}
d.LabelPair = pairs
}
func dumpLabelName(d *dto.LabelName, l clientmodel.LabelName) {
d.Reset()
d.Name = proto.String(string(l))
}

View file

@ -1,548 +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 tiered
import (
"sort"
"testing"
"time"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
)
func GetFingerprintsForLabelSetTests(p metric.Persistence, t testing.TB) {
metrics := []clientmodel.Metric{
{
clientmodel.MetricNameLabel: "test_metric",
"method": "get",
"result": "success",
},
{
clientmodel.MetricNameLabel: "test_metric",
"method": "get",
"result": "failure",
},
{
clientmodel.MetricNameLabel: "test_metric",
"method": "post",
"result": "success",
},
{
clientmodel.MetricNameLabel: "test_metric",
"method": "post",
"result": "failure",
},
}
newTestLabelMatcher := func(matchType metric.MatchType, name clientmodel.LabelName, value clientmodel.LabelValue) *metric.LabelMatcher {
m, err := metric.NewLabelMatcher(matchType, name, value)
if err != nil {
t.Fatalf("Couldn't create label matcher: %v", err)
}
return m
}
scenarios := []struct {
in metric.LabelMatchers
outIndexes []int
}{
{
in: metric.LabelMatchers{
newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "test_metric"),
},
outIndexes: []int{0, 1, 2, 3},
},
{
in: metric.LabelMatchers{
newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "non_existent_metric"),
},
outIndexes: []int{},
},
{
in: metric.LabelMatchers{
newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "non_existent_metric"),
newTestLabelMatcher(metric.Equal, "result", "success"),
},
outIndexes: []int{},
},
{
in: metric.LabelMatchers{
newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "test_metric"),
newTestLabelMatcher(metric.Equal, "result", "success"),
},
outIndexes: []int{0, 2},
},
{
in: metric.LabelMatchers{
newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "test_metric"),
newTestLabelMatcher(metric.NotEqual, "result", "success"),
},
outIndexes: []int{1, 3},
},
{
in: metric.LabelMatchers{
newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "test_metric"),
newTestLabelMatcher(metric.RegexMatch, "result", "foo|success|bar"),
},
outIndexes: []int{0, 2},
},
{
in: metric.LabelMatchers{
newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "test_metric"),
newTestLabelMatcher(metric.RegexNoMatch, "result", "foo|success|bar"),
},
outIndexes: []int{1, 3},
},
{
in: metric.LabelMatchers{
newTestLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, "test_metric"),
newTestLabelMatcher(metric.RegexNoMatch, "result", "foo|success|bar"),
newTestLabelMatcher(metric.RegexMatch, "method", "os"),
},
outIndexes: []int{3},
},
}
for _, m := range metrics {
testAppendSamples(p, &clientmodel.Sample{
Value: 0,
Timestamp: 0,
Metric: m,
}, t)
}
for i, s := range scenarios {
actualFps, err := p.GetFingerprintsForLabelMatchers(s.in)
if err != nil {
t.Fatalf("%d. Couldn't get fingerprints for label matchers: %v", i, err)
}
expectedFps := clientmodel.Fingerprints{}
for _, i := range s.outIndexes {
fp := &clientmodel.Fingerprint{}
fp.LoadFromMetric(metrics[i])
expectedFps = append(expectedFps, fp)
}
sort.Sort(actualFps)
sort.Sort(expectedFps)
if len(actualFps) != len(expectedFps) {
t.Fatalf("%d. Got %d fingerprints; want %d", i, len(actualFps), len(expectedFps))
}
for j, actualFp := range actualFps {
if !actualFp.Equal(expectedFps[j]) {
t.Fatalf("%d.%d. Got fingerprint %v; want %v", i, j, actualFp, expectedFps[j])
}
}
}
}
func GetLabelValuesForLabelNameTests(p metric.Persistence, t testing.TB) {
testAppendSamples(p, &clientmodel.Sample{
Value: 0,
Timestamp: 0,
Metric: clientmodel.Metric{
clientmodel.MetricNameLabel: "my_metric",
"request_type": "create",
"result": "success",
},
}, t)
testAppendSamples(p, &clientmodel.Sample{
Value: 0,
Timestamp: 0,
Metric: clientmodel.Metric{
clientmodel.MetricNameLabel: "my_metric",
"request_type": "delete",
"outcome": "failure",
},
}, t)
expectedIndex := map[clientmodel.LabelName]clientmodel.LabelValues{
clientmodel.MetricNameLabel: {"my_metric"},
"request_type": {"create", "delete"},
"result": {"success"},
"outcome": {"failure"},
}
for name, expected := range expectedIndex {
actual, err := p.GetLabelValuesForLabelName(name)
if err != nil {
t.Fatalf("Error getting values for label %s: %v", name, err)
}
if len(actual) != len(expected) {
t.Fatalf("Number of values don't match for label %s: got %d; want %d", name, len(actual), len(expected))
}
for i := range expected {
inActual := false
for _, a := range actual {
if expected[i] == a {
inActual = true
break
}
}
if !inActual {
t.Fatalf("%d. Expected label value %s not in output", i, expected[i])
}
}
}
}
func GetMetricForFingerprintTests(p metric.Persistence, t testing.TB) {
testAppendSamples(p, &clientmodel.Sample{
Value: 0,
Timestamp: 0,
Metric: clientmodel.Metric{
"request_type": "your_mom",
},
}, t)
testAppendSamples(p, &clientmodel.Sample{
Value: 0,
Timestamp: 0,
Metric: clientmodel.Metric{
"request_type": "your_dad",
"one-off": "value",
},
}, t)
result, err := p.GetFingerprintsForLabelMatchers(metric.LabelMatchers{{
Type: metric.Equal,
Name: "request_type",
Value: "your_mom",
}})
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
m, err := p.GetMetricForFingerprint(result[0])
if err != nil {
t.Error(err)
}
if m == nil {
t.Fatal("Did not expect nil.")
}
if len(m) != 1 {
t.Errorf("Expected one-dimensional metric.")
}
if m["request_type"] != "your_mom" {
t.Errorf("Expected metric to match.")
}
result, err = p.GetFingerprintsForLabelMatchers(metric.LabelMatchers{{
Type: metric.Equal,
Name: "request_type",
Value: "your_dad",
}})
if err != nil {
t.Error(err)
}
if len(result) != 1 {
t.Errorf("Expected one element.")
}
m, err = p.GetMetricForFingerprint(result[0])
if m == nil {
t.Fatal("Did not expect nil.")
}
if err != nil {
t.Error(err)
}
if len(m) != 2 {
t.Errorf("Expected two-dimensional metric.")
}
if m["request_type"] != "your_dad" {
t.Errorf("Expected metric to match.")
}
if m["one-off"] != "value" {
t.Errorf("Expected metric to match.")
}
// Verify that mutating a returned metric does not result in the mutated
// metric to be returned at the next GetMetricForFingerprint() call.
m["one-off"] = "new value"
m, err = p.GetMetricForFingerprint(result[0])
if m == nil {
t.Fatal("Did not expect nil.")
}
if err != nil {
t.Error(err)
}
if len(m) != 2 {
t.Errorf("Expected two-dimensional metric.")
}
if m["request_type"] != "your_dad" {
t.Errorf("Expected metric to match.")
}
if m["one-off"] != "value" {
t.Errorf("Expected metric to match.")
}
}
func AppendRepeatingValuesTests(p metric.Persistence, t testing.TB) {
m := clientmodel.Metric{
clientmodel.MetricNameLabel: "errors_total",
"controller": "foo",
"operation": "bar",
}
increments := 10
repetitions := 500
for i := 0; i < increments; i++ {
for j := 0; j < repetitions; j++ {
time := clientmodel.Timestamp(0).Add(time.Duration(i) * time.Hour).Add(time.Duration(j) * time.Second)
testAppendSamples(p, &clientmodel.Sample{
Value: clientmodel.SampleValue(i),
Timestamp: time,
Metric: m,
}, t)
}
}
v, ok := p.(metric.View)
if !ok {
// It's purely a benchmark for a Persistence that is not viewable.
return
}
matchers := labelMatchersFromLabelSet(clientmodel.LabelSet{
clientmodel.MetricNameLabel: "errors_total",
"controller": "foo",
"operation": "bar",
})
for i := 0; i < increments; i++ {
for j := 0; j < repetitions; j++ {
fingerprints, err := p.GetFingerprintsForLabelMatchers(matchers)
if err != nil {
t.Fatal(err)
}
if len(fingerprints) != 1 {
t.Fatalf("expected %d fingerprints, got %d", 1, len(fingerprints))
}
time := clientmodel.Timestamp(0).Add(time.Duration(i) * time.Hour).Add(time.Duration(j) * time.Second)
samples := v.GetValueAtTime(fingerprints[0], time)
if len(samples) == 0 {
t.Fatal("expected at least one sample.")
}
expected := clientmodel.SampleValue(i)
for _, sample := range samples {
if sample.Value != expected {
t.Fatalf("expected %v value, got %v", expected, sample.Value)
}
}
}
}
}
func AppendsRepeatingValuesTests(p metric.Persistence, t testing.TB) {
m := clientmodel.Metric{
clientmodel.MetricNameLabel: "errors_total",
"controller": "foo",
"operation": "bar",
}
increments := 10
repetitions := 500
s := clientmodel.Samples{}
for i := 0; i < increments; i++ {
for j := 0; j < repetitions; j++ {
time := clientmodel.Timestamp(0).Add(time.Duration(i) * time.Hour).Add(time.Duration(j) * time.Second)
s = append(s, &clientmodel.Sample{
Value: clientmodel.SampleValue(i),
Timestamp: time,
Metric: m,
})
}
}
p.AppendSamples(s)
v, ok := p.(metric.View)
if !ok {
// It's purely a benchmark for a MetricPersistance that is not viewable.
return
}
matchers := labelMatchersFromLabelSet(clientmodel.LabelSet{
clientmodel.MetricNameLabel: "errors_total",
"controller": "foo",
"operation": "bar",
})
for i := 0; i < increments; i++ {
for j := 0; j < repetitions; j++ {
fingerprints, err := p.GetFingerprintsForLabelMatchers(matchers)
if err != nil {
t.Fatal(err)
}
if len(fingerprints) != 1 {
t.Fatalf("expected %d fingerprints, got %d", 1, len(fingerprints))
}
time := clientmodel.Timestamp(0).Add(time.Duration(i) * time.Hour).Add(time.Duration(j) * time.Second)
samples := v.GetValueAtTime(fingerprints[0], time)
if len(samples) == 0 {
t.Fatal("expected at least one sample.")
}
expected := clientmodel.SampleValue(i)
for _, sample := range samples {
if sample.Value != expected {
t.Fatalf("expected %v value, got %v", 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 testLevelDBGetLabelValuesForLabelName = buildLevelDBTestPersistence("get_label_values_for_labelname", GetLabelValuesForLabelNameTests)
func TestLevelDBGetFingerprintsForLabelName(t *testing.T) {
testLevelDBGetLabelValuesForLabelName(t)
}
func BenchmarkLevelDBGetLabelValuesForLabelName(b *testing.B) {
for i := 0; i < b.N; i++ {
testLevelDBGetLabelValuesForLabelName(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 testMemoryGetLabelValuesForLabelName = buildMemoryTestPersistence(GetLabelValuesForLabelNameTests)
func TestMemoryGetLabelValuesForLabelName(t *testing.T) {
testMemoryGetLabelValuesForLabelName(t)
}
func BenchmarkMemoryGetLabelValuesForLabelName(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryGetLabelValuesForLabelName(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,212 +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 tiered
import (
"time"
"github.com/prometheus/prometheus/utility"
clientmodel "github.com/prometheus/client_golang/model"
dto "github.com/prometheus/prometheus/model/generated"
)
type dtoSampleKeyList struct {
l utility.FreeList
}
func newDtoSampleKeyList(cap int) *dtoSampleKeyList {
return &dtoSampleKeyList{
l: utility.NewFreeList(cap),
}
}
func (l *dtoSampleKeyList) Get() (*dto.SampleKey, bool) {
if v, ok := l.l.Get(); ok {
return v.(*dto.SampleKey), ok
}
return &dto.SampleKey{}, false
}
func (l *dtoSampleKeyList) Give(v *dto.SampleKey) bool {
v.Reset()
return l.l.Give(v)
}
func (l *dtoSampleKeyList) Close() {
l.l.Close()
}
type sampleKeyList struct {
l utility.FreeList
}
var defaultSampleKey = &SampleKey{}
func newSampleKeyList(cap int) *sampleKeyList {
return &sampleKeyList{
l: utility.NewFreeList(cap),
}
}
func (l *sampleKeyList) Get() (*SampleKey, bool) {
if v, ok := l.l.Get(); ok {
return v.(*SampleKey), ok
}
return &SampleKey{}, false
}
func (l *sampleKeyList) Give(v *SampleKey) bool {
*v = *defaultSampleKey
return l.l.Give(v)
}
func (l *sampleKeyList) Close() {
l.l.Close()
}
type valueAtTimeList struct {
l utility.FreeList
}
func (l *valueAtTimeList) Get(fp *clientmodel.Fingerprint, time clientmodel.Timestamp) *getValuesAtTimeOp {
var op *getValuesAtTimeOp
v, ok := l.l.Get()
if ok {
op = v.(*getValuesAtTimeOp)
} else {
op = &getValuesAtTimeOp{}
}
op.fp = *fp
op.current = time
return op
}
var pGetValuesAtTimeOp = &getValuesAtTimeOp{}
func (l *valueAtTimeList) Give(v *getValuesAtTimeOp) bool {
*v = *pGetValuesAtTimeOp
return l.l.Give(v)
}
func newValueAtTimeList(cap int) *valueAtTimeList {
return &valueAtTimeList{
l: utility.NewFreeList(cap),
}
}
type valueAtIntervalList struct {
l utility.FreeList
}
func (l *valueAtIntervalList) Get(fp *clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval time.Duration) *getValuesAtIntervalOp {
var op *getValuesAtIntervalOp
v, ok := l.l.Get()
if ok {
op = v.(*getValuesAtIntervalOp)
} else {
op = &getValuesAtIntervalOp{}
}
op.fp = *fp
op.current = from
op.through = through
op.interval = interval
return op
}
var pGetValuesAtIntervalOp = &getValuesAtIntervalOp{}
func (l *valueAtIntervalList) Give(v *getValuesAtIntervalOp) bool {
*v = *pGetValuesAtIntervalOp
return l.l.Give(v)
}
func newValueAtIntervalList(cap int) *valueAtIntervalList {
return &valueAtIntervalList{
l: utility.NewFreeList(cap),
}
}
type valueAlongRangeList struct {
l utility.FreeList
}
func (l *valueAlongRangeList) Get(fp *clientmodel.Fingerprint, from, through clientmodel.Timestamp) *getValuesAlongRangeOp {
var op *getValuesAlongRangeOp
v, ok := l.l.Get()
if ok {
op = v.(*getValuesAlongRangeOp)
} else {
op = &getValuesAlongRangeOp{}
}
op.fp = *fp
op.current = from
op.through = through
return op
}
var pGetValuesAlongRangeOp = &getValuesAlongRangeOp{}
func (l *valueAlongRangeList) Give(v *getValuesAlongRangeOp) bool {
*v = *pGetValuesAlongRangeOp
return l.l.Give(v)
}
func newValueAlongRangeList(cap int) *valueAlongRangeList {
return &valueAlongRangeList{
l: utility.NewFreeList(cap),
}
}
type valueAtIntervalAlongRangeList struct {
l utility.FreeList
}
func (l *valueAtIntervalAlongRangeList) Get(fp *clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval, rangeDuration time.Duration) *getValueRangeAtIntervalOp {
var op *getValueRangeAtIntervalOp
v, ok := l.l.Get()
if ok {
op = v.(*getValueRangeAtIntervalOp)
} else {
op = &getValueRangeAtIntervalOp{}
}
op.fp = *fp
op.current = from
op.rangeThrough = from.Add(rangeDuration)
op.rangeDuration = rangeDuration
op.interval = interval
op.through = through
return op
}
var pGetValueRangeAtIntervalOp = &getValueRangeAtIntervalOp{}
func (l *valueAtIntervalAlongRangeList) Give(v *getValueRangeAtIntervalOp) bool {
*v = *pGetValueRangeAtIntervalOp
return l.l.Give(v)
}
func newValueAtIntervalAlongRangeList(cap int) *valueAtIntervalAlongRangeList {
return &valueAtIntervalAlongRangeList{
l: utility.NewFreeList(cap),
}
}

View file

@ -1,37 +0,0 @@
// Copyright 2014 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 tiered
import (
"testing"
"github.com/prometheus/client_golang/model"
)
// TestValueAtTimeListGet tests if the timestamp is set properly in the op
// retrieved from the free list and if the 'consumed' member is zeroed properly.
func TestValueAtTimeListGet(t *testing.T) {
l := newValueAtTimeList(1)
op := l.Get(&model.Fingerprint{}, 42)
op.consumed = true
l.Give(op)
op2 := l.Get(&model.Fingerprint{}, 4711)
if op2.Consumed() {
t.Error("Op retrieved from freelist is already consumed.")
}
if got, expected := op2.CurrentTime(), model.Timestamp(4711); got != expected {
t.Errorf("op2.CurrentTime() = %d; want %d.", got, expected)
}
}

View file

@ -1,689 +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 tiered
import (
"io"
"sort"
"sync"
"code.google.com/p/goprotobuf/proto"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/raw"
"github.com/prometheus/prometheus/storage/raw/leveldb"
"github.com/prometheus/prometheus/utility"
dto "github.com/prometheus/prometheus/model/generated"
)
// FingerprintMetricMapping is an in-memory map of Fingerprints to Metrics.
type FingerprintMetricMapping map[clientmodel.Fingerprint]clientmodel.Metric
// FingerprintMetricIndex models a database mapping Fingerprints to Metrics.
type FingerprintMetricIndex interface {
raw.Database
raw.Pruner
IndexBatch(FingerprintMetricMapping) error
Lookup(*clientmodel.Fingerprint) (m clientmodel.Metric, ok bool, err error)
}
// LevelDBFingerprintMetricIndex implements FingerprintMetricIndex using
// leveldb.
type LevelDBFingerprintMetricIndex struct {
*leveldb.LevelDBPersistence
}
// IndexBatch implements FingerprintMetricIndex.
func (i *LevelDBFingerprintMetricIndex) IndexBatch(mapping FingerprintMetricMapping) error {
b := leveldb.NewBatch()
defer b.Close()
for f, m := range mapping {
k := &dto.Fingerprint{}
dumpFingerprint(k, &f)
v := &dto.Metric{}
dumpMetric(v, m)
b.Put(k, v)
}
return i.LevelDBPersistence.Commit(b)
}
// Lookup implements FingerprintMetricIndex.
func (i *LevelDBFingerprintMetricIndex) Lookup(f *clientmodel.Fingerprint) (m clientmodel.Metric, ok bool, err error) {
k := &dto.Fingerprint{}
dumpFingerprint(k, f)
v := &dto.Metric{}
if ok, err := i.LevelDBPersistence.Get(k, v); !ok {
return nil, false, nil
} else if err != nil {
return nil, false, err
}
m = clientmodel.Metric{}
for _, pair := range v.LabelPair {
m[clientmodel.LabelName(pair.GetName())] = clientmodel.LabelValue(pair.GetValue())
}
return m, true, nil
}
// NewLevelDBFingerprintMetricIndex returns a LevelDBFingerprintMetricIndex
// object ready to use.
func NewLevelDBFingerprintMetricIndex(o leveldb.LevelDBOptions) (*LevelDBFingerprintMetricIndex, error) {
s, err := leveldb.NewLevelDBPersistence(o)
if err != nil {
return nil, err
}
return &LevelDBFingerprintMetricIndex{
LevelDBPersistence: s,
}, nil
}
// LabelNameLabelValuesMapping is an in-memory map of LabelNames to
// LabelValues.
type LabelNameLabelValuesMapping map[clientmodel.LabelName]clientmodel.LabelValues
// LabelNameLabelValuesIndex models a database mapping LabelNames to
// LabelValues.
type LabelNameLabelValuesIndex interface {
raw.Database
raw.Pruner
IndexBatch(LabelNameLabelValuesMapping) error
Lookup(clientmodel.LabelName) (values clientmodel.LabelValues, ok bool, err error)
Has(clientmodel.LabelName) (ok bool, err error)
}
// LevelDBLabelNameLabelValuesIndex implements LabelNameLabelValuesIndex using
// leveldb.
type LevelDBLabelNameLabelValuesIndex struct {
*leveldb.LevelDBPersistence
}
// IndexBatch implements LabelNameLabelValuesIndex.
func (i *LevelDBLabelNameLabelValuesIndex) IndexBatch(b LabelNameLabelValuesMapping) error {
batch := leveldb.NewBatch()
defer batch.Close()
for labelName, labelValues := range b {
sort.Sort(labelValues)
key := &dto.LabelName{
Name: proto.String(string(labelName)),
}
value := &dto.LabelValueCollection{}
value.Member = make([]string, 0, len(labelValues))
for _, labelValue := range labelValues {
value.Member = append(value.Member, string(labelValue))
}
batch.Put(key, value)
}
return i.LevelDBPersistence.Commit(batch)
}
// Lookup implements LabelNameLabelValuesIndex.
func (i *LevelDBLabelNameLabelValuesIndex) Lookup(l clientmodel.LabelName) (values clientmodel.LabelValues, ok bool, err error) {
k := &dto.LabelName{}
dumpLabelName(k, l)
v := &dto.LabelValueCollection{}
ok, err = i.LevelDBPersistence.Get(k, v)
if err != nil {
return nil, false, err
}
if !ok {
return nil, false, nil
}
for _, m := range v.Member {
values = append(values, clientmodel.LabelValue(m))
}
return values, true, nil
}
// Has implements LabelNameLabelValuesIndex.
func (i *LevelDBLabelNameLabelValuesIndex) Has(l clientmodel.LabelName) (ok bool, err error) {
return i.LevelDBPersistence.Has(&dto.LabelName{
Name: proto.String(string(l)),
})
}
// NewLevelDBLabelNameLabelValuesIndex returns a LevelDBLabelNameLabelValuesIndex
// ready to use.
func NewLevelDBLabelNameLabelValuesIndex(o leveldb.LevelDBOptions) (*LevelDBLabelNameLabelValuesIndex, error) {
s, err := leveldb.NewLevelDBPersistence(o)
if err != nil {
return nil, err
}
return &LevelDBLabelNameLabelValuesIndex{
LevelDBPersistence: s,
}, nil
}
// LabelPairFingerprintMapping is an in-memory map of LabelPairs to
// Fingerprints.
type LabelPairFingerprintMapping map[metric.LabelPair]clientmodel.Fingerprints
// LabelPairFingerprintIndex models a database mapping LabelPairs to
// Fingerprints.
type LabelPairFingerprintIndex interface {
raw.Database
raw.ForEacher
raw.Pruner
IndexBatch(LabelPairFingerprintMapping) error
Lookup(*metric.LabelPair) (m clientmodel.Fingerprints, ok bool, err error)
Has(*metric.LabelPair) (ok bool, err error)
}
// LevelDBLabelPairFingerprintIndex implements LabelPairFingerprintIndex using
// leveldb.
type LevelDBLabelPairFingerprintIndex struct {
*leveldb.LevelDBPersistence
}
// IndexBatch implements LabelPairFingerprintMapping.
func (i *LevelDBLabelPairFingerprintIndex) IndexBatch(m LabelPairFingerprintMapping) error {
batch := leveldb.NewBatch()
defer batch.Close()
for pair, fps := range m {
sort.Sort(fps)
key := &dto.LabelPair{
Name: proto.String(string(pair.Name)),
Value: proto.String(string(pair.Value)),
}
value := &dto.FingerprintCollection{}
for _, fp := range fps {
f := &dto.Fingerprint{}
dumpFingerprint(f, fp)
value.Member = append(value.Member, f)
}
batch.Put(key, value)
}
return i.LevelDBPersistence.Commit(batch)
}
// Lookup implements LabelPairFingerprintMapping.
func (i *LevelDBLabelPairFingerprintIndex) Lookup(p *metric.LabelPair) (m clientmodel.Fingerprints, ok bool, err error) {
k := &dto.LabelPair{
Name: proto.String(string(p.Name)),
Value: proto.String(string(p.Value)),
}
v := &dto.FingerprintCollection{}
ok, err = i.LevelDBPersistence.Get(k, v)
if !ok {
return nil, false, nil
}
if err != nil {
return nil, false, err
}
for _, pair := range v.Member {
fp := &clientmodel.Fingerprint{}
loadFingerprint(fp, pair)
m = append(m, fp)
}
return m, true, nil
}
// Has implements LabelPairFingerprintMapping.
func (i *LevelDBLabelPairFingerprintIndex) Has(p *metric.LabelPair) (ok bool, err error) {
k := &dto.LabelPair{
Name: proto.String(string(p.Name)),
Value: proto.String(string(p.Value)),
}
return i.LevelDBPersistence.Has(k)
}
// NewLevelDBLabelSetFingerprintIndex returns a LevelDBLabelPairFingerprintIndex
// object ready to use.
func NewLevelDBLabelSetFingerprintIndex(o leveldb.LevelDBOptions) (*LevelDBLabelPairFingerprintIndex, error) {
s, err := leveldb.NewLevelDBPersistence(o)
if err != nil {
return nil, err
}
return &LevelDBLabelPairFingerprintIndex{
LevelDBPersistence: s,
}, nil
}
// MetricMembershipIndex models a database tracking the existence of Metrics.
type MetricMembershipIndex interface {
raw.Database
raw.Pruner
IndexBatch(FingerprintMetricMapping) error
Has(clientmodel.Metric) (ok bool, err error)
}
// LevelDBMetricMembershipIndex implements MetricMembershipIndex using leveldb.
type LevelDBMetricMembershipIndex struct {
*leveldb.LevelDBPersistence
}
var existenceIdentity = &dto.MembershipIndexValue{}
// IndexBatch implements MetricMembershipIndex.
func (i *LevelDBMetricMembershipIndex) IndexBatch(b FingerprintMetricMapping) error {
batch := leveldb.NewBatch()
defer batch.Close()
for _, m := range b {
k := &dto.Metric{}
dumpMetric(k, m)
batch.Put(k, existenceIdentity)
}
return i.LevelDBPersistence.Commit(batch)
}
// Has implements MetricMembershipIndex.
func (i *LevelDBMetricMembershipIndex) Has(m clientmodel.Metric) (ok bool, err error) {
k := &dto.Metric{}
dumpMetric(k, m)
return i.LevelDBPersistence.Has(k)
}
// NewLevelDBMetricMembershipIndex returns a LevelDBMetricMembershipIndex object
// ready to use.
func NewLevelDBMetricMembershipIndex(o leveldb.LevelDBOptions) (*LevelDBMetricMembershipIndex, error) {
s, err := leveldb.NewLevelDBPersistence(o)
if err != nil {
return nil, err
}
return &LevelDBMetricMembershipIndex{
LevelDBPersistence: s,
}, nil
}
// MetricIndexer indexes facets of a clientmodel.Metric.
type MetricIndexer interface {
// IndexMetric makes no assumptions about the concurrency safety of the
// underlying implementer.
IndexMetrics(FingerprintMetricMapping) error
}
// IndexerObserver listens and receives changes to a given
// FingerprintMetricMapping.
type IndexerObserver interface {
Observe(FingerprintMetricMapping) error
}
// IndexerProxy receives IndexMetric requests and proxies them to the underlying
// MetricIndexer. Upon success of the underlying receiver, the registered
// IndexObservers are called serially.
//
// If an error occurs in the underlying MetricIndexer or any of the observers,
// this proxy will not work any further and return the offending error in this
// call or any subsequent ones.
type IndexerProxy struct {
err error
i MetricIndexer
observers []IndexerObserver
}
// IndexMetrics proxies the given FingerprintMetricMapping to the underlying
// MetricIndexer and calls all registered observers with it.
func (p *IndexerProxy) IndexMetrics(b FingerprintMetricMapping) error {
if p.err != nil {
return p.err
}
if p.err = p.i.IndexMetrics(b); p.err != nil {
return p.err
}
for _, o := range p.observers {
if p.err = o.Observe(b); p.err != nil {
return p.err
}
}
return nil
}
// Close closes the underlying indexer.
func (p *IndexerProxy) Close() error {
if p.err != nil {
return p.err
}
if closer, ok := p.i.(io.Closer); ok {
p.err = closer.Close()
return p.err
}
return nil
}
// Flush flushes the underlying index requests before closing.
func (p *IndexerProxy) Flush() error {
if p.err != nil {
return p.err
}
if flusher, ok := p.i.(flusher); ok {
p.err = flusher.Flush()
return p.err
}
return nil
}
// NewIndexerProxy builds an IndexerProxy for the given configuration.
func NewIndexerProxy(i MetricIndexer, o ...IndexerObserver) *IndexerProxy {
return &IndexerProxy{
i: i,
observers: o,
}
}
// SynchronizedIndexer provides naive locking for any MetricIndexer.
type SynchronizedIndexer struct {
mu sync.Mutex
i MetricIndexer
}
// IndexMetrics calls IndexMetrics of the wrapped MetricIndexer after acquiring
// a lock.
func (i *SynchronizedIndexer) IndexMetrics(b FingerprintMetricMapping) error {
i.mu.Lock()
defer i.mu.Unlock()
return i.i.IndexMetrics(b)
}
type flusher interface {
Flush() error
}
// Flush calls Flush of the wrapped MetricIndexer after acquiring a lock. If the
// wrapped MetricIndexer has no Flush method, this is a no-op.
func (i *SynchronizedIndexer) Flush() error {
if flusher, ok := i.i.(flusher); ok {
i.mu.Lock()
defer i.mu.Unlock()
return flusher.Flush()
}
return nil
}
// Close calls Close of the wrapped MetricIndexer after acquiring a lock. If the
// wrapped MetricIndexer has no Close method, this is a no-op.
func (i *SynchronizedIndexer) Close() error {
if closer, ok := i.i.(io.Closer); ok {
i.mu.Lock()
defer i.mu.Unlock()
return closer.Close()
}
return nil
}
// NewSynchronizedIndexer returns a SynchronizedIndexer wrapping the given
// MetricIndexer.
func NewSynchronizedIndexer(i MetricIndexer) *SynchronizedIndexer {
return &SynchronizedIndexer{
i: i,
}
}
// BufferedIndexer provides unsynchronized index buffering.
//
// If an error occurs in the underlying MetricIndexer or any of the observers,
// this proxy will not work any further and return the offending error.
type BufferedIndexer struct {
i MetricIndexer
limit int
buf []FingerprintMetricMapping
err error
}
// IndexMetrics writes the entries in the given FingerprintMetricMapping to the
// index.
func (i *BufferedIndexer) IndexMetrics(b FingerprintMetricMapping) error {
if i.err != nil {
return i.err
}
if len(i.buf) < i.limit {
i.buf = append(i.buf, b)
return nil
}
i.err = i.Flush()
return i.err
}
// Flush writes all pending entries to the index.
func (i *BufferedIndexer) Flush() error {
if i.err != nil {
return i.err
}
if len(i.buf) == 0 {
return nil
}
superset := FingerprintMetricMapping{}
for _, b := range i.buf {
for fp, m := range b {
if _, ok := superset[fp]; ok {
continue
}
superset[fp] = m
}
}
i.buf = make([]FingerprintMetricMapping, 0, i.limit)
i.err = i.i.IndexMetrics(superset)
return i.err
}
// Close flushes and closes the underlying buffer.
func (i *BufferedIndexer) Close() error {
if err := i.Flush(); err != nil {
return err
}
if closer, ok := i.i.(io.Closer); ok {
return closer.Close()
}
return nil
}
// NewBufferedIndexer returns a BufferedIndexer ready to use.
func NewBufferedIndexer(i MetricIndexer, limit int) *BufferedIndexer {
return &BufferedIndexer{
i: i,
limit: limit,
buf: make([]FingerprintMetricMapping, 0, limit),
}
}
// TotalIndexer is a MetricIndexer that indexes all standard facets of a metric
// that a user or the Prometheus subsystem would want to query against:
//
// "<Label Name>" -> {Fingerprint, ...}
// "<Label Name> <Label Value>" -> {Fingerprint, ...}
//
// "<Fingerprint>" -> Metric
//
// "<Metric>" -> Existence Value
//
// This type supports concrete queries but only single writes, and it has no
// locking semantics to enforce this.
type TotalIndexer struct {
FingerprintToMetric FingerprintMetricIndex
LabelNameToLabelValues LabelNameLabelValuesIndex
LabelPairToFingerprint LabelPairFingerprintIndex
MetricMembership MetricMembershipIndex
}
func findUnindexed(i MetricMembershipIndex, b FingerprintMetricMapping) (FingerprintMetricMapping, error) {
out := FingerprintMetricMapping{}
for fp, m := range b {
has, err := i.Has(m)
if err != nil {
return nil, err
}
if !has {
out[fp] = m
}
}
return out, nil
}
func extendLabelNameToLabelValuesIndex(i LabelNameLabelValuesIndex, b FingerprintMetricMapping) (LabelNameLabelValuesMapping, error) {
collection := map[clientmodel.LabelName]utility.Set{}
for _, m := range b {
for l, v := range m {
set, ok := collection[l]
if !ok {
baseValues, _, err := i.Lookup(l)
if err != nil {
return nil, err
}
set = utility.Set{}
for _, baseValue := range baseValues {
set.Add(baseValue)
}
collection[l] = set
}
set.Add(v)
}
}
batch := LabelNameLabelValuesMapping{}
for l, set := range collection {
values := make(clientmodel.LabelValues, 0, len(set))
for e := range set {
val := e.(clientmodel.LabelValue)
values = append(values, val)
}
batch[l] = values
}
return batch, nil
}
func extendLabelPairIndex(i LabelPairFingerprintIndex, b FingerprintMetricMapping) (LabelPairFingerprintMapping, error) {
collection := map[metric.LabelPair]utility.Set{}
for fp, m := range b {
for n, v := range m {
pair := metric.LabelPair{
Name: n,
Value: v,
}
set, ok := collection[pair]
if !ok {
baseFps, _, err := i.Lookup(&pair)
if err != nil {
return nil, err
}
set = utility.Set{}
for _, baseFp := range baseFps {
set.Add(*baseFp)
}
collection[pair] = set
}
set.Add(fp)
}
}
batch := LabelPairFingerprintMapping{}
for pair, set := range collection {
fps := batch[pair]
for element := range set {
fp := element.(clientmodel.Fingerprint)
fps = append(fps, &fp)
}
batch[pair] = fps
}
return batch, nil
}
// IndexMetrics adds the facets of all unindexed metrics found in the given
// FingerprintMetricMapping to the corresponding indices.
func (i *TotalIndexer) IndexMetrics(b FingerprintMetricMapping) error {
unindexed, err := findUnindexed(i.MetricMembership, b)
if err != nil {
return err
}
labelNames, err := extendLabelNameToLabelValuesIndex(i.LabelNameToLabelValues, unindexed)
if err != nil {
return err
}
if err := i.LabelNameToLabelValues.IndexBatch(labelNames); err != nil {
return err
}
labelPairs, err := extendLabelPairIndex(i.LabelPairToFingerprint, unindexed)
if err != nil {
return err
}
if err := i.LabelPairToFingerprint.IndexBatch(labelPairs); err != nil {
return err
}
if err := i.FingerprintToMetric.IndexBatch(unindexed); err != nil {
return err
}
return i.MetricMembership.IndexBatch(unindexed)
}

View file

@ -1,25 +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 tiered
import (
"testing"
"github.com/prometheus/prometheus/storage/metric"
)
func TestInterfaceAdherence(t *testing.T) {
var _ metric.Persistence = &LevelDBPersistence{}
var _ metric.Persistence = NewMemorySeriesStorage(MemorySeriesOptions{})
}

View file

@ -1,677 +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 tiered
import (
"flag"
"fmt"
"sync"
"time"
"code.google.com/p/goprotobuf/proto"
"github.com/golang/glog"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/raw"
"github.com/prometheus/prometheus/storage/raw/leveldb"
"github.com/prometheus/prometheus/utility"
dto "github.com/prometheus/prometheus/model/generated"
)
const sortConcurrency = 2
// LevelDBPersistence is a leveldb-backed persistence layer for metrics.
type LevelDBPersistence struct {
CurationRemarks CurationRemarker
FingerprintToMetrics FingerprintMetricIndex
LabelNameToLabelValues LabelNameLabelValuesIndex
LabelPairToFingerprints LabelPairFingerprintIndex
MetricHighWatermarks HighWatermarker
MetricMembershipIndex MetricMembershipIndex
Indexer MetricIndexer
MetricSamples *leveldb.LevelDBPersistence
// The remaining indices will be replaced with generalized interface resolvers:
//
// type FingerprintResolver interface {
// GetFingerprintForMetric(clientmodel.Metric) (*clientmodel.Fingerprint, bool, error)
// GetFingerprintsForLabelMatchers(metric.LabelPair) (clientmodel.Fingerprints, bool, error)
// }
// type MetricResolver interface {
// GetMetricsForFingerprint(clientmodel.Fingerprints) (FingerprintMetricMapping, bool, error)
// }
}
var (
leveldbChunkSize = flag.Int("leveldbChunkSize", 200, "Maximum number of samples stored under one key.")
// These flag values are back of the envelope, though they seem
// sensible. Please re-evaluate based on your own needs.
curationRemarksCacheSize = flag.Int("curationRemarksCacheSize", 5*1024*1024, "The size for the curation remarks cache (bytes).")
fingerprintsToLabelPairCacheSize = flag.Int("fingerprintsToLabelPairCacheSizeBytes", 25*1024*1024, "The size for the fingerprint to label pair index (bytes).")
highWatermarkCacheSize = flag.Int("highWatermarksByFingerprintSizeBytes", 5*1024*1024, "The size for the metric high watermarks (bytes).")
labelNameToLabelValuesCacheSize = flag.Int("labelNameToLabelValuesCacheSizeBytes", 25*1024*1024, "The size for the label name to label values index (bytes).")
labelPairToFingerprintsCacheSize = flag.Int("labelPairToFingerprintsCacheSizeBytes", 25*1024*1024, "The size for the label pair to metric fingerprint index (bytes).")
metricMembershipIndexCacheSize = flag.Int("metricMembershipCacheSizeBytes", 5*1024*1024, "The size for the metric membership index (bytes).")
samplesByFingerprintCacheSize = flag.Int("samplesByFingerprintCacheSizeBytes", 50*1024*1024, "The size for the samples database (bytes).")
)
type leveldbOpener func()
// Close closes all the underlying persistence layers. It implements the
// Persistence interface.
func (l *LevelDBPersistence) Close() {
var persistences = []raw.Database{
l.CurationRemarks,
l.FingerprintToMetrics,
l.LabelNameToLabelValues,
l.LabelPairToFingerprints,
l.MetricHighWatermarks,
l.MetricMembershipIndex,
l.MetricSamples,
}
closerGroup := sync.WaitGroup{}
for _, c := range persistences {
closerGroup.Add(1)
go func(c raw.Database) {
if c != nil {
if err := c.Close(); err != nil {
glog.Error("Error closing persistence: ", err)
}
}
closerGroup.Done()
}(c)
}
closerGroup.Wait()
}
// NewLevelDBPersistence returns a LevelDBPersistence object ready
// to use.
func NewLevelDBPersistence(baseDirectory string) (*LevelDBPersistence, error) {
workers := utility.NewUncertaintyGroup(7)
emission := &LevelDBPersistence{}
var subsystemOpeners = []struct {
name string
opener leveldbOpener
}{
{
"Label Names and Value Pairs by Fingerprint",
func() {
var err error
emission.FingerprintToMetrics, err = NewLevelDBFingerprintMetricIndex(
leveldb.LevelDBOptions{
Name: "Metrics by Fingerprint",
Purpose: "Index",
Path: baseDirectory + "/label_name_and_value_pairs_by_fingerprint",
CacheSizeBytes: *fingerprintsToLabelPairCacheSize,
},
)
workers.MayFail(err)
},
},
{
"Samples by Fingerprint",
func() {
var err error
emission.MetricSamples, err = leveldb.NewLevelDBPersistence(leveldb.LevelDBOptions{
Name: "Samples",
Purpose: "Timeseries",
Path: baseDirectory + "/samples_by_fingerprint",
CacheSizeBytes: *fingerprintsToLabelPairCacheSize,
})
workers.MayFail(err)
},
},
{
"High Watermarks by Fingerprint",
func() {
var err error
emission.MetricHighWatermarks, err = NewLevelDBHighWatermarker(
leveldb.LevelDBOptions{
Name: "High Watermarks",
Purpose: "The youngest sample in the database per metric.",
Path: baseDirectory + "/high_watermarks_by_fingerprint",
CacheSizeBytes: *highWatermarkCacheSize,
},
)
workers.MayFail(err)
},
},
{
"Fingerprints by Label Name",
func() {
var err error
emission.LabelNameToLabelValues, err = NewLevelDBLabelNameLabelValuesIndex(
leveldb.LevelDBOptions{
Name: "Label Values by Label Name",
Purpose: "Index",
Path: baseDirectory + "/label_values_by_label_name",
CacheSizeBytes: *labelNameToLabelValuesCacheSize,
},
)
workers.MayFail(err)
},
},
{
"Fingerprints by Label Name and Value Pair",
func() {
var err error
emission.LabelPairToFingerprints, err = NewLevelDBLabelSetFingerprintIndex(
leveldb.LevelDBOptions{
Name: "Fingerprints by Label Pair",
Purpose: "Index",
Path: baseDirectory + "/fingerprints_by_label_name_and_value_pair",
CacheSizeBytes: *labelPairToFingerprintsCacheSize,
},
)
workers.MayFail(err)
},
},
{
"Metric Membership Index",
func() {
var err error
emission.MetricMembershipIndex, err = NewLevelDBMetricMembershipIndex(
leveldb.LevelDBOptions{
Name: "Metric Membership",
Purpose: "Index",
Path: baseDirectory + "/metric_membership_index",
CacheSizeBytes: *metricMembershipIndexCacheSize,
},
)
workers.MayFail(err)
},
},
{
"Sample Curation Remarks",
func() {
var err error
emission.CurationRemarks, err = NewLevelDBCurationRemarker(
leveldb.LevelDBOptions{
Name: "Sample Curation Remarks",
Purpose: "Ledger of Progress for Various Curators",
Path: baseDirectory + "/curation_remarks",
CacheSizeBytes: *curationRemarksCacheSize,
},
)
workers.MayFail(err)
},
},
}
for _, subsystem := range subsystemOpeners {
opener := subsystem.opener
go opener()
}
if !workers.Wait() {
for _, err := range workers.Errors() {
glog.Error("Could not open storage: ", err)
}
return nil, fmt.Errorf("unable to open metric persistence")
}
emission.Indexer = &TotalIndexer{
FingerprintToMetric: emission.FingerprintToMetrics,
LabelNameToLabelValues: emission.LabelNameToLabelValues,
LabelPairToFingerprint: emission.LabelPairToFingerprints,
MetricMembership: emission.MetricMembershipIndex,
}
return emission, nil
}
// AppendSample implements the Persistence interface.
func (l *LevelDBPersistence) AppendSample(sample *clientmodel.Sample) (err error) {
defer func(begin time.Time) {
recordOutcome(time.Since(begin), err, appendSample)
}(time.Now())
err = l.AppendSamples(clientmodel.Samples{sample})
return
}
// groupByFingerprint collects all of the provided samples and groups them
// together by their respective metric fingerprint.
func groupByFingerprint(samples clientmodel.Samples) map[clientmodel.Fingerprint]clientmodel.Samples {
fingerprintToSamples := map[clientmodel.Fingerprint]clientmodel.Samples{}
for _, sample := range samples {
fingerprint := &clientmodel.Fingerprint{}
fingerprint.LoadFromMetric(sample.Metric)
samples := fingerprintToSamples[*fingerprint]
samples = append(samples, sample)
fingerprintToSamples[*fingerprint] = samples
}
return fingerprintToSamples
}
func (l *LevelDBPersistence) refreshHighWatermarks(groups map[clientmodel.Fingerprint]clientmodel.Samples) (err error) {
defer func(begin time.Time) {
recordOutcome(time.Since(begin), err, refreshHighWatermarks)
}(time.Now())
b := FingerprintHighWatermarkMapping{}
for fp, ss := range groups {
if len(ss) == 0 {
continue
}
b[fp] = ss[len(ss)-1].Timestamp
}
return l.MetricHighWatermarks.UpdateBatch(b)
}
// AppendSamples appends the given Samples to the database and indexes them.
func (l *LevelDBPersistence) AppendSamples(samples clientmodel.Samples) (err error) {
defer func(begin time.Time) {
recordOutcome(time.Since(begin), err, appendSamples)
}(time.Now())
fingerprintToSamples := groupByFingerprint(samples)
indexErrChan := make(chan error, 1)
watermarkErrChan := make(chan error, 1)
go func(groups map[clientmodel.Fingerprint]clientmodel.Samples) {
metrics := FingerprintMetricMapping{}
for fingerprint, samples := range groups {
metrics[fingerprint] = samples[0].Metric
}
indexErrChan <- l.Indexer.IndexMetrics(metrics)
}(fingerprintToSamples)
go func(groups map[clientmodel.Fingerprint]clientmodel.Samples) {
watermarkErrChan <- l.refreshHighWatermarks(groups)
}(fingerprintToSamples)
samplesBatch := leveldb.NewBatch()
defer samplesBatch.Close()
key := &SampleKey{}
keyDto := &dto.SampleKey{}
values := make(metric.Values, 0, *leveldbChunkSize)
for fingerprint, group := range fingerprintToSamples {
for {
values := values[:0]
lengthOfGroup := len(group)
if lengthOfGroup == 0 {
break
}
take := *leveldbChunkSize
if lengthOfGroup < take {
take = lengthOfGroup
}
chunk := group[0:take]
group = group[take:lengthOfGroup]
key.Fingerprint = &fingerprint
key.FirstTimestamp = chunk[0].Timestamp
key.LastTimestamp = chunk[take-1].Timestamp
key.SampleCount = uint32(take)
key.Dump(keyDto)
for _, sample := range chunk {
values = append(values, metric.SamplePair{
Timestamp: sample.Timestamp,
Value: sample.Value,
})
}
val := marshalValues(values, nil)
samplesBatch.PutRaw(keyDto, val)
}
}
err = l.MetricSamples.Commit(samplesBatch)
if err != nil {
return
}
err = <-indexErrChan
if err != nil {
return
}
err = <-watermarkErrChan
if err != nil {
return
}
return
}
func extractSampleKey(i leveldb.Iterator) (*SampleKey, error) {
k := &dto.SampleKey{}
if err := i.Key(k); err != nil {
return nil, err
}
key := &SampleKey{}
key.Load(k)
return key, nil
}
func (l *LevelDBPersistence) hasIndexMetric(m clientmodel.Metric) (value bool, err error) {
defer func(begin time.Time) {
recordOutcome(time.Since(begin), err, hasIndexMetric)
}(time.Now())
return l.MetricMembershipIndex.Has(m)
}
// GetFingerprintsForLabelMatchers returns the Fingerprints for the given
// LabelMatchers by querying the underlying LabelPairFingerprintIndex and
// possibly the LabelNameLabelValuesIndex for each matcher. It implements the
// Persistence interface.
func (l *LevelDBPersistence) GetFingerprintsForLabelMatchers(labelMatchers metric.LabelMatchers) (fps clientmodel.Fingerprints, err error) {
defer func(begin time.Time) {
recordOutcome(time.Since(begin), err, getFingerprintsForLabelMatchers)
}(time.Now())
sets := []utility.Set{}
for _, matcher := range labelMatchers {
set := utility.Set{}
switch matcher.Type {
case metric.Equal:
fps, _, err := l.LabelPairToFingerprints.Lookup(&metric.LabelPair{
Name: matcher.Name,
Value: matcher.Value,
})
if err != nil {
return nil, err
}
for _, fp := range fps {
set.Add(*fp)
}
default:
values, err := l.GetLabelValuesForLabelName(matcher.Name)
if err != nil {
return nil, err
}
matches := matcher.Filter(values)
if len(matches) == 0 {
return nil, nil
}
for _, v := range matches {
fps, _, err := l.LabelPairToFingerprints.Lookup(&metric.LabelPair{
Name: matcher.Name,
Value: v,
})
if err != nil {
return nil, err
}
for _, fp := range fps {
set.Add(*fp)
}
}
}
sets = append(sets, set)
}
numberOfSets := len(sets)
if numberOfSets == 0 {
return nil, nil
}
base := sets[0]
for i := 1; i < numberOfSets; i++ {
base = base.Intersection(sets[i])
}
for _, e := range base.Elements() {
fingerprint := e.(clientmodel.Fingerprint)
fps = append(fps, &fingerprint)
}
return fps, nil
}
// GetLabelValuesForLabelName returns the LabelValues for the given LabelName
// from the underlying LabelNameLabelValuesIndex. It implements the
// Persistence interface.
func (l *LevelDBPersistence) GetLabelValuesForLabelName(labelName clientmodel.LabelName) (clientmodel.LabelValues, error) {
var err error
defer func(begin time.Time) {
recordOutcome(time.Since(begin), err, getLabelValuesForLabelName)
}(time.Now())
values, _, err := l.LabelNameToLabelValues.Lookup(labelName)
return values, err
}
// GetMetricForFingerprint returns the Metric for the given Fingerprint from the
// underlying FingerprintMetricIndex. It implements the Persistence
// interface.
func (l *LevelDBPersistence) GetMetricForFingerprint(f *clientmodel.Fingerprint) (m clientmodel.Metric, err error) {
defer func(begin time.Time) {
recordOutcome(time.Since(begin), err, getMetricForFingerprint)
}(time.Now())
// TODO(matt): Update signature to work with ok.
m, _, err = l.FingerprintToMetrics.Lookup(f)
return m, nil
}
// GetAllValuesForLabel gets all label values that are associated with the
// provided label name.
func (l *LevelDBPersistence) GetAllValuesForLabel(labelName clientmodel.LabelName) (values clientmodel.LabelValues, err error) {
filter := &LabelNameFilter{
labelName: labelName,
}
labelValuesOp := &CollectLabelValuesOp{}
_, err = l.LabelPairToFingerprints.ForEach(&MetricKeyDecoder{}, filter, labelValuesOp)
if err != nil {
return
}
values = labelValuesOp.labelValues
return
}
// Prune compacts each database's keyspace serially.
//
// Beware that it would probably be imprudent to run this on a live user-facing
// server due to latency implications.
func (l *LevelDBPersistence) Prune() {
l.CurationRemarks.Prune()
l.FingerprintToMetrics.Prune()
l.LabelNameToLabelValues.Prune()
l.LabelPairToFingerprints.Prune()
l.MetricHighWatermarks.Prune()
l.MetricMembershipIndex.Prune()
l.MetricSamples.Prune()
}
// Sizes returns the sum of all sizes of the underlying databases.
func (l *LevelDBPersistence) Sizes() (total uint64, err error) {
size := uint64(0)
if size, err = l.CurationRemarks.Size(); err != nil {
return 0, err
}
total += size
if size, err = l.FingerprintToMetrics.Size(); err != nil {
return 0, err
}
total += size
if size, err = l.LabelNameToLabelValues.Size(); err != nil {
return 0, err
}
total += size
if size, err = l.LabelPairToFingerprints.Size(); err != nil {
return 0, err
}
total += size
if size, err = l.MetricHighWatermarks.Size(); err != nil {
return 0, err
}
total += size
if size, err = l.MetricMembershipIndex.Size(); err != nil {
return 0, err
}
total += size
if size, err = l.MetricSamples.Size(); err != nil {
return 0, err
}
total += size
return total, nil
}
// States returns the DatabaseStates of all underlying databases.
func (l *LevelDBPersistence) States() raw.DatabaseStates {
return raw.DatabaseStates{
l.CurationRemarks.State(),
l.FingerprintToMetrics.State(),
l.LabelNameToLabelValues.State(),
l.LabelPairToFingerprints.State(),
l.MetricHighWatermarks.State(),
l.MetricMembershipIndex.State(),
l.MetricSamples.State(),
}
}
// CollectLabelValuesOp implements storage.RecordOperator. It collects the
// encountered LabelValues in a slice.
type CollectLabelValuesOp struct {
labelValues []clientmodel.LabelValue
}
// Operate implements storage.RecordOperator. 'key' is required to be a
// LabelPair. Its Value is appended to a slice of collected LabelValues.
func (op *CollectLabelValuesOp) Operate(key, value interface{}) (err *storage.OperatorError) {
labelPair := key.(metric.LabelPair)
op.labelValues = append(op.labelValues, labelPair.Value)
return
}
// MetricKeyDecoder implements storage.RecordDecoder for LabelPairs.
type MetricKeyDecoder struct{}
// DecodeKey implements storage.RecordDecoder. It requires 'in' to be a
// LabelPair protobuf. 'out' is a metric.LabelPair.
func (d *MetricKeyDecoder) DecodeKey(in interface{}) (out interface{}, err error) {
unmarshaled := dto.LabelPair{}
err = proto.Unmarshal(in.([]byte), &unmarshaled)
if err != nil {
return
}
out = metric.LabelPair{
Name: clientmodel.LabelName(*unmarshaled.Name),
Value: clientmodel.LabelValue(*unmarshaled.Value),
}
return
}
// DecodeValue implements storage.RecordDecoder. It is a no-op and always
// returns (nil, nil).
func (d *MetricKeyDecoder) DecodeValue(in interface{}) (out interface{}, err error) {
return
}
// MetricSamplesDecoder implements storage.RecordDecoder for SampleKeys.
type MetricSamplesDecoder struct{}
// DecodeKey implements storage.RecordDecoder. It requires 'in' to be a
// SampleKey protobuf. 'out' is a metric.SampleKey.
func (d *MetricSamplesDecoder) DecodeKey(in interface{}) (interface{}, error) {
key := &dto.SampleKey{}
err := proto.Unmarshal(in.([]byte), key)
if err != nil {
return nil, err
}
sampleKey := &SampleKey{}
sampleKey.Load(key)
return sampleKey, nil
}
// DecodeValue implements storage.RecordDecoder. It requires 'in' to be a
// SampleValueSeries protobuf. 'out' is of type metric.Values.
func (d *MetricSamplesDecoder) DecodeValue(in interface{}) (interface{}, error) {
return unmarshalValues(in.([]byte), nil), nil
}
// AcceptAllFilter implements storage.RecordFilter and accepts all records.
type AcceptAllFilter struct{}
// Filter implements storage.RecordFilter. It always returns ACCEPT.
func (d *AcceptAllFilter) Filter(_, _ interface{}) storage.FilterResult {
return storage.Accept
}
// LabelNameFilter implements storage.RecordFilter and filters records matching
// a LabelName.
type LabelNameFilter struct {
labelName clientmodel.LabelName
}
// Filter implements storage.RecordFilter. 'key' is expected to be a
// LabelPair. The result is ACCEPT if the Name of the LabelPair matches the
// LabelName of this LabelNameFilter.
func (f LabelNameFilter) Filter(key, value interface{}) (filterResult storage.FilterResult) {
labelPair, ok := key.(metric.LabelPair)
if ok && labelPair.Name == f.labelName {
return storage.Accept
}
return storage.Skip
}
func recordOutcome(duration time.Duration, err error, op string) {
labels := prometheus.Labels{operation: op}
if err == nil {
labels[result] = success
} else {
labels[result] = failure
}
storageLatency.With(labels).Observe(float64(duration / time.Microsecond))
}

View file

@ -1,74 +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 tiered
import (
"encoding/binary"
"math"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
)
const (
// sampleSize is the number of bytes per sample in marshalled format.
sampleSize = 16
// formatVersion is used as a version marker in the marshalled format.
formatVersion = 1
// formatVersionSize is the number of bytes used by the serialized formatVersion.
formatVersionSize = 1
)
// marshal marshals a group of samples for being written to disk into dest or a
// new slice if dest has insufficient capacity.
func marshalValues(v metric.Values, dest []byte) []byte {
sz := formatVersionSize + len(v)*sampleSize
if cap(dest) < sz {
dest = make([]byte, sz)
} else {
dest = dest[0:sz]
}
dest[0] = formatVersion
for i, val := range v {
offset := formatVersionSize + i*sampleSize
binary.LittleEndian.PutUint64(dest[offset:], uint64(val.Timestamp.Unix()))
binary.LittleEndian.PutUint64(dest[offset+8:], math.Float64bits(float64(val.Value)))
}
return dest
}
// unmarshalValues decodes marshalled samples into dest and returns either dest
// or a new slice containing those values if dest has insufficient capacity.
func unmarshalValues(buf []byte, dest metric.Values) metric.Values {
if buf[0] != formatVersion {
panic("unsupported format version")
}
n := (len(buf) - formatVersionSize) / sampleSize
if cap(dest) < n {
dest = make(metric.Values, n)
} else {
dest = dest[0:n]
}
for i := 0; i < n; i++ {
offset := formatVersionSize + i*sampleSize
dest[i].Timestamp = clientmodel.TimestampFromUnix(int64(binary.LittleEndian.Uint64(buf[offset:])))
dest[i].Value = clientmodel.SampleValue(math.Float64frombits(binary.LittleEndian.Uint64(buf[offset+8:])))
}
return dest
}

View file

@ -1,580 +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 tiered
import (
"sort"
"sync"
"github.com/golang/glog"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility"
)
// An initialSeriesArenaSize of 4*60 allows for one hour's worth of storage per
// metric without any major reallocations - assuming a sample rate of 1 / 15Hz.
const initialSeriesArenaSize = 4 * 60
type stream interface {
add(metric.Values)
clone() metric.Values
getOlderThan(age clientmodel.Timestamp) metric.Values
evictOlderThan(age clientmodel.Timestamp)
size() int
clear()
metric() clientmodel.Metric
getValueAtTime(t clientmodel.Timestamp) metric.Values
getBoundaryValues(in metric.Interval) metric.Values
getRangeValues(in metric.Interval) metric.Values
}
type arrayStream struct {
sync.RWMutex
m clientmodel.Metric
values metric.Values
}
func (s *arrayStream) metric() clientmodel.Metric {
return s.m
}
// add implements the stream interface. This implementation requires both
// s.values and the passed in v to be sorted already. Values in v that have a
// timestamp older than the most recent value in s.values are skipped.
func (s *arrayStream) add(v metric.Values) {
s.Lock()
defer s.Unlock()
// Skip over values that are older than the most recent value in s.
if len(s.values) > 0 {
i := 0
mostRecentTimestamp := s.values[len(s.values)-1].Timestamp
for ; i < len(v) && mostRecentTimestamp > v[i].Timestamp; i++ {
}
if i > 0 {
glog.Warningf(
"Skipped out-of-order values while adding to %#v: %#v",
s.m, v[:i],
)
v = v[i:]
}
}
s.values = append(s.values, v...)
}
func (s *arrayStream) clone() metric.Values {
s.RLock()
defer s.RUnlock()
clone := make(metric.Values, len(s.values))
copy(clone, s.values)
return clone
}
func (s *arrayStream) getOlderThan(t clientmodel.Timestamp) metric.Values {
s.RLock()
defer s.RUnlock()
finder := func(i int) bool {
return s.values[i].Timestamp.After(t)
}
i := sort.Search(len(s.values), finder)
return s.values[:i]
}
func (s *arrayStream) evictOlderThan(t clientmodel.Timestamp) {
s.Lock()
defer s.Unlock()
finder := func(i int) bool {
return s.values[i].Timestamp.After(t)
}
i := sort.Search(len(s.values), finder)
s.values = s.values[i:]
}
func (s *arrayStream) getValueAtTime(t clientmodel.Timestamp) metric.Values {
s.RLock()
defer s.RUnlock()
// BUG(all): May be avenues for simplification.
l := len(s.values)
switch l {
case 0:
return metric.Values{}
case 1:
return metric.Values{s.values[0]}
default:
index := sort.Search(l, func(i int) bool {
return !s.values[i].Timestamp.Before(t)
})
if index == 0 {
return metric.Values{s.values[0]}
}
if index == l {
return metric.Values{s.values[l-1]}
}
if s.values[index].Timestamp.Equal(t) {
return metric.Values{s.values[index]}
}
return metric.Values{s.values[index-1], s.values[index]}
}
}
func (s *arrayStream) getBoundaryValues(in metric.Interval) metric.Values {
s.RLock()
defer s.RUnlock()
oldest := sort.Search(len(s.values), func(i int) bool {
return !s.values[i].Timestamp.Before(in.OldestInclusive)
})
newest := sort.Search(len(s.values), func(i int) bool {
return s.values[i].Timestamp.After(in.NewestInclusive)
})
resultRange := s.values[oldest:newest]
switch len(resultRange) {
case 0:
return metric.Values{}
case 1:
return metric.Values{resultRange[0]}
default:
return metric.Values{resultRange[0], resultRange[len(resultRange)-1]}
}
}
func (s *arrayStream) getRangeValues(in metric.Interval) metric.Values {
s.RLock()
defer s.RUnlock()
oldest := sort.Search(len(s.values), func(i int) bool {
return !s.values[i].Timestamp.Before(in.OldestInclusive)
})
newest := sort.Search(len(s.values), func(i int) bool {
return s.values[i].Timestamp.After(in.NewestInclusive)
})
result := make(metric.Values, newest-oldest)
copy(result, s.values[oldest:newest])
return result
}
func (s *arrayStream) size() int {
return len(s.values)
}
func (s *arrayStream) clear() {
s.values = metric.Values{}
}
func newArrayStream(m clientmodel.Metric) *arrayStream {
return &arrayStream{
m: m,
values: make(metric.Values, 0, initialSeriesArenaSize),
}
}
type memorySeriesStorage struct {
sync.RWMutex
wmCache *watermarkCache
fingerprintToSeries map[clientmodel.Fingerprint]stream
labelPairToFingerprints map[metric.LabelPair]utility.Set
labelNameToLabelValues map[clientmodel.LabelName]utility.Set
}
// MemorySeriesOptions bundles options used by NewMemorySeriesStorage to create
// a memory series storage.
type MemorySeriesOptions struct {
// If provided, this WatermarkCache will be updated for any samples that
// are appended to the memorySeriesStorage.
WatermarkCache *watermarkCache
}
func (s *memorySeriesStorage) AppendSamples(samples clientmodel.Samples) error {
for _, sample := range samples {
s.AppendSample(sample)
}
return nil
}
func (s *memorySeriesStorage) AppendSample(sample *clientmodel.Sample) error {
s.Lock()
defer s.Unlock()
fingerprint := &clientmodel.Fingerprint{}
fingerprint.LoadFromMetric(sample.Metric)
series := s.getOrCreateSeries(sample.Metric, fingerprint)
series.add(metric.Values{
metric.SamplePair{
Value: sample.Value,
Timestamp: sample.Timestamp,
},
})
if s.wmCache != nil {
s.wmCache.Put(fingerprint, &watermarks{High: sample.Timestamp})
}
return nil
}
func (s *memorySeriesStorage) CreateEmptySeries(metric clientmodel.Metric) {
s.Lock()
defer s.Unlock()
m := clientmodel.Metric{}
for label, value := range metric {
m[label] = value
}
fingerprint := &clientmodel.Fingerprint{}
fingerprint.LoadFromMetric(m)
s.getOrCreateSeries(m, fingerprint)
}
func (s *memorySeriesStorage) getOrCreateSeries(m clientmodel.Metric, fp *clientmodel.Fingerprint) stream {
series, ok := s.fingerprintToSeries[*fp]
if !ok {
series = newArrayStream(m)
s.fingerprintToSeries[*fp] = series
for k, v := range m {
labelPair := metric.LabelPair{
Name: k,
Value: v,
}
fps, ok := s.labelPairToFingerprints[labelPair]
if !ok {
fps = utility.Set{}
s.labelPairToFingerprints[labelPair] = fps
}
fps.Add(*fp)
values, ok := s.labelNameToLabelValues[k]
if !ok {
values = utility.Set{}
s.labelNameToLabelValues[k] = values
}
values.Add(v)
}
}
return series
}
func (s *memorySeriesStorage) Flush(flushOlderThan clientmodel.Timestamp, queue chan<- clientmodel.Samples) {
s.RLock()
for _, stream := range s.fingerprintToSeries {
toArchive := stream.getOlderThan(flushOlderThan)
queued := make(clientmodel.Samples, 0, len(toArchive))
// NOTE: This duplication will go away soon.
for _, value := range toArchive {
queued = append(queued, &clientmodel.Sample{
Metric: stream.metric(),
Timestamp: value.Timestamp,
Value: value.Value,
})
}
// BUG(all): this can deadlock if the queue is full, as we only ever clear
// the queue after calling this method:
// https://github.com/prometheus/prometheus/issues/275
if len(queued) > 0 {
queue <- queued
}
}
s.RUnlock()
}
func (s *memorySeriesStorage) Evict(flushOlderThan clientmodel.Timestamp) {
emptySeries := []clientmodel.Fingerprint{}
s.RLock()
for fingerprint, stream := range s.fingerprintToSeries {
stream.evictOlderThan(flushOlderThan)
if stream.size() == 0 {
emptySeries = append(emptySeries, fingerprint)
}
}
s.RUnlock()
s.Lock()
for _, fingerprint := range emptySeries {
if series, ok := s.fingerprintToSeries[fingerprint]; ok && series.size() == 0 {
s.dropSeries(&fingerprint)
}
}
s.Unlock()
}
// Drop a label value from the label names to label values index.
func (s *memorySeriesStorage) dropLabelValue(l clientmodel.LabelName, v clientmodel.LabelValue) {
if set, ok := s.labelNameToLabelValues[l]; ok {
set.Remove(v)
if len(set) == 0 {
delete(s.labelNameToLabelValues, l)
}
}
}
// Drop all references to a series, including any samples.
func (s *memorySeriesStorage) dropSeries(fingerprint *clientmodel.Fingerprint) {
series, ok := s.fingerprintToSeries[*fingerprint]
if !ok {
return
}
for k, v := range series.metric() {
labelPair := metric.LabelPair{
Name: k,
Value: v,
}
if set, ok := s.labelPairToFingerprints[labelPair]; ok {
set.Remove(*fingerprint)
if len(set) == 0 {
delete(s.labelPairToFingerprints, labelPair)
s.dropLabelValue(k, v)
}
}
}
delete(s.fingerprintToSeries, *fingerprint)
}
// Append raw samples, bypassing indexing. Only used to add data to views,
// which don't need to lookup by metric.
func (s *memorySeriesStorage) appendSamplesWithoutIndexing(fingerprint *clientmodel.Fingerprint, samples metric.Values) {
s.Lock()
defer s.Unlock()
series, ok := s.fingerprintToSeries[*fingerprint]
if !ok {
series = newArrayStream(clientmodel.Metric{})
s.fingerprintToSeries[*fingerprint] = series
}
series.add(samples)
}
func (s *memorySeriesStorage) GetFingerprintsForLabelMatchers(labelMatchers metric.LabelMatchers) (clientmodel.Fingerprints, error) {
s.RLock()
defer s.RUnlock()
sets := []utility.Set{}
for _, matcher := range labelMatchers {
switch matcher.Type {
case metric.Equal:
set, ok := s.labelPairToFingerprints[metric.LabelPair{
Name: matcher.Name,
Value: matcher.Value,
}]
if !ok {
return nil, nil
}
sets = append(sets, set)
default:
values, err := s.getLabelValuesForLabelName(matcher.Name)
if err != nil {
return nil, err
}
matches := matcher.Filter(values)
if len(matches) == 0 {
return nil, nil
}
set := utility.Set{}
for _, v := range matches {
subset, ok := s.labelPairToFingerprints[metric.LabelPair{
Name: matcher.Name,
Value: v,
}]
if !ok {
return nil, nil
}
for fp := range subset {
set.Add(fp)
}
}
sets = append(sets, set)
}
}
setCount := len(sets)
if setCount == 0 {
return nil, nil
}
base := sets[0]
for i := 1; i < setCount; i++ {
base = base.Intersection(sets[i])
}
fingerprints := clientmodel.Fingerprints{}
for _, e := range base.Elements() {
fingerprint := e.(clientmodel.Fingerprint)
fingerprints = append(fingerprints, &fingerprint)
}
return fingerprints, nil
}
func (s *memorySeriesStorage) GetLabelValuesForLabelName(labelName clientmodel.LabelName) (clientmodel.LabelValues, error) {
s.RLock()
defer s.RUnlock()
return s.getLabelValuesForLabelName(labelName)
}
func (s *memorySeriesStorage) getLabelValuesForLabelName(labelName clientmodel.LabelName) (clientmodel.LabelValues, error) {
set, ok := s.labelNameToLabelValues[labelName]
if !ok {
return nil, nil
}
values := make(clientmodel.LabelValues, 0, len(set))
for e := range set {
val := e.(clientmodel.LabelValue)
values = append(values, val)
}
return values, nil
}
func (s *memorySeriesStorage) GetMetricForFingerprint(f *clientmodel.Fingerprint) (clientmodel.Metric, error) {
s.RLock()
defer s.RUnlock()
series, ok := s.fingerprintToSeries[*f]
if !ok {
return nil, nil
}
metric := clientmodel.Metric{}
for label, value := range series.metric() {
metric[label] = value
}
return metric, nil
}
func (s *memorySeriesStorage) HasFingerprint(f *clientmodel.Fingerprint) bool {
s.RLock()
defer s.RUnlock()
_, has := s.fingerprintToSeries[*f]
return has
}
func (s *memorySeriesStorage) CloneSamples(f *clientmodel.Fingerprint) metric.Values {
s.RLock()
defer s.RUnlock()
series, ok := s.fingerprintToSeries[*f]
if !ok {
return nil
}
return series.clone()
}
func (s *memorySeriesStorage) GetValueAtTime(f *clientmodel.Fingerprint, t clientmodel.Timestamp) metric.Values {
s.RLock()
defer s.RUnlock()
series, ok := s.fingerprintToSeries[*f]
if !ok {
return nil
}
return series.getValueAtTime(t)
}
func (s *memorySeriesStorage) GetBoundaryValues(f *clientmodel.Fingerprint, i metric.Interval) metric.Values {
s.RLock()
defer s.RUnlock()
series, ok := s.fingerprintToSeries[*f]
if !ok {
return nil
}
return series.getBoundaryValues(i)
}
func (s *memorySeriesStorage) GetRangeValues(f *clientmodel.Fingerprint, i metric.Interval) metric.Values {
s.RLock()
defer s.RUnlock()
series, ok := s.fingerprintToSeries[*f]
if !ok {
return nil
}
return series.getRangeValues(i)
}
func (s *memorySeriesStorage) Close() {
s.Lock()
defer s.Unlock()
s.fingerprintToSeries = nil
s.labelPairToFingerprints = nil
s.labelNameToLabelValues = nil
}
func (s *memorySeriesStorage) GetAllValuesForLabel(labelName clientmodel.LabelName) (values clientmodel.LabelValues, err error) {
s.RLock()
defer s.RUnlock()
valueSet := map[clientmodel.LabelValue]bool{}
for _, series := range s.fingerprintToSeries {
if value, ok := series.metric()[labelName]; ok {
if !valueSet[value] {
values = append(values, value)
valueSet[value] = true
}
}
}
return
}
// NewMemorySeriesStorage returns a memory series storage ready to use.
func NewMemorySeriesStorage(o MemorySeriesOptions) *memorySeriesStorage {
return &memorySeriesStorage{
fingerprintToSeries: make(map[clientmodel.Fingerprint]stream),
labelPairToFingerprints: make(map[metric.LabelPair]utility.Set),
labelNameToLabelValues: make(map[clientmodel.LabelName]utility.Set),
wmCache: o.WatermarkCache,
}
}

View file

@ -1,284 +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 tiered
import (
"fmt"
"reflect"
"runtime"
"sync"
"testing"
"time"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
)
func BenchmarkStreamAdd(b *testing.B) {
b.StopTimer()
s := newArrayStream(clientmodel.Metric{})
samples := make(metric.Values, b.N)
for i := 0; i < b.N; i++ {
samples = append(samples, metric.SamplePair{
Timestamp: clientmodel.TimestampFromTime(time.Date(i, 0, 0, 0, 0, 0, 0, time.UTC)),
Value: clientmodel.SampleValue(i),
})
}
b.StartTimer()
s.add(samples)
}
func TestStreamAdd(t *testing.T) {
s := newArrayStream(clientmodel.Metric{})
// Add empty to empty.
v := metric.Values{}
expected := metric.Values{}
s.add(v)
if got := s.values; !reflect.DeepEqual(expected, got) {
t.Fatalf("Expected values %#v in stream, got %#v.", expected, got)
}
// Add something to empty.
v = metric.Values{
metric.SamplePair{Timestamp: 1, Value: -1},
}
expected = append(expected, v...)
s.add(v)
if got := s.values; !reflect.DeepEqual(expected, got) {
t.Fatalf("Expected values %#v in stream, got %#v.", expected, got)
}
// Add something to something.
v = metric.Values{
metric.SamplePair{Timestamp: 2, Value: -2},
metric.SamplePair{Timestamp: 5, Value: -5},
}
expected = append(expected, v...)
s.add(v)
if got := s.values; !reflect.DeepEqual(expected, got) {
t.Fatalf("Expected values %#v in stream, got %#v.", expected, got)
}
// Add something outdated to something.
v = metric.Values{
metric.SamplePair{Timestamp: 3, Value: -3},
metric.SamplePair{Timestamp: 4, Value: -4},
}
s.add(v)
if got := s.values; !reflect.DeepEqual(expected, got) {
t.Fatalf("Expected values %#v in stream, got %#v.", expected, got)
}
// Add something partially outdated to something.
v = metric.Values{
metric.SamplePair{Timestamp: 3, Value: -3},
metric.SamplePair{Timestamp: 6, Value: -6},
}
expected = append(expected, metric.SamplePair{Timestamp: 6, Value: -6})
s.add(v)
if got := s.values; !reflect.DeepEqual(expected, got) {
t.Fatalf("Expected values %#v in stream, got %#v.", expected, got)
}
}
func benchmarkAppendSamples(b *testing.B, labels int) {
b.StopTimer()
s := NewMemorySeriesStorage(MemorySeriesOptions{})
metric := clientmodel.Metric{}
for i := 0; i < labels; i++ {
metric[clientmodel.LabelName(fmt.Sprintf("label_%d", i))] = clientmodel.LabelValue(fmt.Sprintf("value_%d", i))
}
samples := make(clientmodel.Samples, 0, b.N)
for i := 0; i < b.N; i++ {
samples = append(samples, &clientmodel.Sample{
Metric: metric,
Value: clientmodel.SampleValue(i),
Timestamp: clientmodel.TimestampFromTime(time.Date(i, 0, 0, 0, 0, 0, 0, time.UTC)),
})
}
b.StartTimer()
for i := 0; i < b.N; i++ {
s.AppendSample(samples[i])
}
}
func BenchmarkAppendSample1(b *testing.B) {
benchmarkAppendSamples(b, 1)
}
func BenchmarkAppendSample10(b *testing.B) {
benchmarkAppendSamples(b, 10)
}
func BenchmarkAppendSample100(b *testing.B) {
benchmarkAppendSamples(b, 100)
}
func BenchmarkAppendSample1000(b *testing.B) {
benchmarkAppendSamples(b, 1000)
}
// Regression test for https://github.com/prometheus/prometheus/issues/381.
//
// 1. Creates samples for two timeseries with one common labelpair.
// 2. Flushes memory storage such that only one series is dropped from memory.
// 3. Gets fingerprints for common labelpair.
// 4. Checks that exactly one fingerprint remains.
func TestDroppedSeriesIndexRegression(t *testing.T) {
samples := clientmodel.Samples{
&clientmodel.Sample{
Metric: clientmodel.Metric{
clientmodel.MetricNameLabel: "testmetric",
"different": "differentvalue1",
"common": "samevalue",
},
Value: 1,
Timestamp: clientmodel.TimestampFromTime(time.Date(2000, 0, 0, 0, 0, 0, 0, time.UTC)),
},
&clientmodel.Sample{
Metric: clientmodel.Metric{
clientmodel.MetricNameLabel: "testmetric",
"different": "differentvalue2",
"common": "samevalue",
},
Value: 2,
Timestamp: clientmodel.TimestampFromTime(time.Date(2002, 0, 0, 0, 0, 0, 0, time.UTC)),
},
}
s := NewMemorySeriesStorage(MemorySeriesOptions{})
s.AppendSamples(samples)
common := clientmodel.LabelSet{"common": "samevalue"}
fps, err := s.GetFingerprintsForLabelMatchers(labelMatchersFromLabelSet(common))
if err != nil {
t.Fatal(err)
}
if len(fps) != 2 {
t.Fatalf("Got %d fingerprints, expected 2", len(fps))
}
toDisk := make(chan clientmodel.Samples, 2)
flushOlderThan := clientmodel.TimestampFromTime(time.Date(2001, 0, 0, 0, 0, 0, 0, time.UTC))
s.Flush(flushOlderThan, toDisk)
if len(toDisk) != 1 {
t.Fatalf("Got %d disk sample lists, expected 1", len(toDisk))
}
diskSamples := <-toDisk
if len(diskSamples) != 1 {
t.Fatalf("Got %d disk samples, expected 1", len(diskSamples))
}
s.Evict(flushOlderThan)
fps, err = s.GetFingerprintsForLabelMatchers(labelMatchersFromLabelSet(common))
if err != nil {
t.Fatal(err)
}
if len(fps) != 1 {
t.Fatalf("Got %d fingerprints, expected 1", len(fps))
}
}
func TestReaderWriterDeadlockRegression(t *testing.T) {
mp := runtime.GOMAXPROCS(2)
defer func(mp int) {
runtime.GOMAXPROCS(mp)
}(mp)
s := NewMemorySeriesStorage(MemorySeriesOptions{})
lms := metric.LabelMatchers{}
for i := 0; i < 100; i++ {
lm, err := metric.NewLabelMatcher(metric.NotEqual, clientmodel.MetricNameLabel, "testmetric")
if err != nil {
t.Fatal(err)
}
lms = append(lms, lm)
}
wg := sync.WaitGroup{}
wg.Add(2)
start := time.Now()
runDuration := 250 * time.Millisecond
writer := func() {
for time.Since(start) < runDuration {
s.AppendSamples(clientmodel.Samples{
&clientmodel.Sample{
Metric: clientmodel.Metric{
clientmodel.MetricNameLabel: "testmetric",
},
Value: 1,
Timestamp: 0,
},
})
}
wg.Done()
}
reader := func() {
for time.Since(start) < runDuration {
s.GetFingerprintsForLabelMatchers(lms)
}
wg.Done()
}
go reader()
go writer()
allDone := make(chan struct{})
go func() {
wg.Wait()
allDone <- struct{}{}
}()
select {
case <-allDone:
break
case <-time.NewTimer(5 * time.Second).C:
t.Fatalf("Deadlock timeout")
}
}
func BenchmarkGetFingerprintsForNotEqualMatcher1000(b *testing.B) {
numSeries := 1000
samples := make(clientmodel.Samples, 0, numSeries)
for i := 0; i < numSeries; i++ {
samples = append(samples, &clientmodel.Sample{
Metric: clientmodel.Metric{
clientmodel.MetricNameLabel: "testmetric",
"instance": clientmodel.LabelValue(fmt.Sprint("instance_", i)),
},
Value: 1,
Timestamp: clientmodel.TimestampFromTime(time.Date(2000, 0, 0, 0, 0, 0, 0, time.UTC)),
})
}
s := NewMemorySeriesStorage(MemorySeriesOptions{})
if err := s.AppendSamples(samples); err != nil {
b.Fatal(err)
}
m, err := metric.NewLabelMatcher(metric.NotEqual, "instance", "foo")
if err != nil {
b.Fatal(err)
}
b.ResetTimer()
for i := 0; i < b.N; i++ {
s.GetFingerprintsForLabelMatchers(metric.LabelMatchers{m})
}
}

View file

@ -1,212 +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 tiered
import (
"fmt"
"github.com/golang/glog"
"github.com/prometheus/prometheus/storage/raw/leveldb"
dto "github.com/prometheus/prometheus/model/generated"
)
type iteratorSeekerState struct {
// Immutable State
i leveldb.Iterator
obj *SampleKey
first, last *SampleKey
dtoSampleKeys *dtoSampleKeyList
sampleKeys *sampleKeyList
// Mutable State
iteratorInvalid bool
seriesOperable bool
err error
key *SampleKey
keyDto *dto.SampleKey
}
// iteratorSeeker is a function that models a state machine state and
// is responsible for choosing the subsequent state given the present
// disposition.
//
// It returns the next state or nil if no remaining transition is possible.
// It returns an error if one occurred and finally a truth value indicating
// whether the current iterator state is usable and whether it can proceed with
// the current fingerprint.
type iteratorSeeker func() iteratorSeeker
func (s *iteratorSeekerState) initialize() iteratorSeeker {
s.key, _ = s.sampleKeys.Get()
s.keyDto, _ = s.dtoSampleKeys.Get()
return s.start
}
func (s *iteratorSeekerState) destroy() iteratorSeeker {
s.sampleKeys.Give(s.key)
s.dtoSampleKeys.Give(s.keyDto)
return nil
}
func (s *iteratorSeekerState) start() iteratorSeeker {
switch {
case s.obj.Fingerprint.Less(s.first.Fingerprint):
// The fingerprint does not exist in the database.
return s.destroy
case s.last.Fingerprint.Less(s.obj.Fingerprint):
// The fingerprint does not exist in the database.
return s.destroy
case s.obj.Fingerprint.Equal(s.first.Fingerprint) && s.obj.FirstTimestamp.Before(s.first.FirstTimestamp):
// The fingerprint is the first fingerprint, but we've requested a value
// before what exists in the database.
return s.seekBeginning
case s.last.Before(s.obj.Fingerprint, s.obj.FirstTimestamp):
// The requested time for work is after the last sample in the database; we
// can't do anything!
return s.destroy
default:
return s.initialSeek
}
}
func (s *iteratorSeekerState) seekBeginning() iteratorSeeker {
s.i.SeekToFirst()
if !s.i.Valid() {
s.err = s.i.Error()
// If we can't seek to the beginning, there isn't any hope for us.
glog.Warning("iterator went bad: %s", s.err)
s.iteratorInvalid = true
return s.destroy
}
return s.initialMatchFingerprint
}
func (s *iteratorSeekerState) initialSeek() iteratorSeeker {
s.obj.Dump(s.keyDto)
s.i.Seek(s.keyDto)
if !s.i.Valid() {
s.err = s.i.Error()
glog.Warningf("iterator went bad %s", s.err)
s.iteratorInvalid = true
return s.destroy
}
return s.initialMatchFingerprint
}
func (s *iteratorSeekerState) initialMatchFingerprint() iteratorSeeker {
if err := s.i.Key(s.keyDto); err != nil {
s.err = err
return s.destroy
}
s.key.Load(s.keyDto)
switch {
case s.obj.Fingerprint.Less(s.key.Fingerprint):
return s.initialFingerprintOvershot
case s.key.Fingerprint.Less(s.obj.Fingerprint):
panic("violated invariant")
default:
return s.initialMatchTime
}
}
func (s *iteratorSeekerState) initialFingerprintOvershot() iteratorSeeker {
s.i.Previous()
if !s.i.Valid() {
glog.Warningf("Could not backtrack for %s", s)
panic("violated invariant")
}
if err := s.i.Key(s.keyDto); err != nil {
s.err = err
return s.destroy
}
s.key.Load(s.keyDto)
if !s.key.Fingerprint.Equal(s.obj.Fingerprint) {
return s.destroy
}
return s.initialMatchTime
}
func (s *iteratorSeekerState) initialMatchTime() iteratorSeeker {
switch {
case s.key.MayContain(s.obj.FirstTimestamp):
s.seriesOperable = true
return s.destroy
case s.key.Equal(s.first), s.obj.FirstTimestamp.Equal(s.key.FirstTimestamp):
s.seriesOperable = true
return s.destroy
case s.obj.FirstTimestamp.Before(s.key.FirstTimestamp):
return s.reCue
default:
panic("violated invariant " + fmt.Sprintln(s.obj, s.key))
}
}
func (s *iteratorSeekerState) reCue() iteratorSeeker {
s.i.Previous()
if !s.i.Valid() {
glog.Warningf("Could not backtrack for %s", s)
panic("violated invariant")
}
if err := s.i.Key(s.keyDto); err != nil {
s.err = err
return s.destroy
}
s.key.Load(s.keyDto)
if !s.key.Fingerprint.Equal(s.obj.Fingerprint) {
return s.fastForward
}
s.seriesOperable = true
return s.destroy
}
func (s *iteratorSeekerState) fastForward() iteratorSeeker {
s.i.Next()
if !s.i.Valid() {
glog.Warningf("Could not fast-forward for %s", s)
panic("violated invariant")
}
s.seriesOperable = true
return s.destroy
}

View file

@ -1,326 +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 tiered
import (
"fmt"
"sort"
"time"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
)
// durationOperator encapsulates a general operation that occurs over a
// duration.
type durationOperator interface {
metric.Op
Through() clientmodel.Timestamp
}
// ops is a heap of operations, primary sorting key is the fingerprint.
type ops []metric.Op
// Len implements sort.Interface and heap.Interface.
func (o ops) Len() int {
return len(o)
}
// Less implements sort.Interface and heap.Interface. It compares the
// fingerprints. If they are equal, the comparison is delegated to
// currentTimeSort.
func (o ops) Less(i, j int) bool {
fpi := o[i].Fingerprint()
fpj := o[j].Fingerprint()
if fpi.Equal(fpj) {
return currentTimeSort{o}.Less(i, j)
}
return fpi.Less(fpj)
}
// Swap implements sort.Interface and heap.Interface.
func (o ops) Swap(i, j int) {
o[i], o[j] = o[j], o[i]
}
// Push implements heap.Interface.
func (o *ops) Push(x interface{}) {
// Push and Pop use pointer receivers because they modify the slice's
// length, not just its contents.
*o = append(*o, x.(metric.Op))
}
// Push implements heap.Interface.
func (o *ops) Pop() interface{} {
old := *o
n := len(old)
x := old[n-1]
*o = old[0 : n-1]
return x
}
// currentTimeSort is a wrapper for ops with customized sorting order.
type currentTimeSort struct {
ops
}
// currentTimeSort implements sort.Interface and sorts the operations in
// chronological order by their current time.
func (s currentTimeSort) Less(i, j int) bool {
return s.ops[i].CurrentTime().Before(s.ops[j].CurrentTime())
}
// baseOp contains the implementations and fields shared between different op
// types.
type baseOp struct {
fp clientmodel.Fingerprint
current clientmodel.Timestamp
}
func (g *baseOp) Fingerprint() *clientmodel.Fingerprint {
return &g.fp
}
func (g *baseOp) CurrentTime() clientmodel.Timestamp {
return g.current
}
// getValuesAtTimeOp encapsulates getting values at or adjacent to a specific
// time.
type getValuesAtTimeOp struct {
baseOp
consumed bool
}
func (g *getValuesAtTimeOp) String() string {
return fmt.Sprintf("getValuesAtTimeOp at %s", g.current)
}
func (g *getValuesAtTimeOp) ExtractSamples(in metric.Values) (out metric.Values) {
if len(in) == 0 {
return
}
out = extractValuesAroundTime(g.current, in)
g.consumed = true
return
}
func (g getValuesAtTimeOp) Consumed() bool {
return g.consumed
}
// getValuesAlongRangeOp encapsulates getting all values in a given range.
type getValuesAlongRangeOp struct {
baseOp
through clientmodel.Timestamp
}
func (g *getValuesAlongRangeOp) String() string {
return fmt.Sprintf("getValuesAlongRangeOp from %s through %s", g.current, g.through)
}
func (g *getValuesAlongRangeOp) Through() clientmodel.Timestamp {
return g.through
}
func (g *getValuesAlongRangeOp) ExtractSamples(in metric.Values) (out metric.Values) {
if len(in) == 0 {
return
}
// Find the first sample where time >= g.current.
firstIdx := sort.Search(len(in), func(i int) bool {
return !in[i].Timestamp.Before(g.current)
})
if firstIdx == len(in) {
// No samples at or after operator start time. This can only
// happen if we try applying the operator to a time after the
// last recorded sample. In this case, we're finished.
g.current = g.through.Add(clientmodel.MinimumTick)
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 {
g.current = g.through.Add(clientmodel.MinimumTick)
return
}
lastSampleTime := in[lastIdx-1].Timestamp
// Sample times are stored with a maximum time resolution of one second,
// so we have to add exactly that to target the next chunk on the next
// op iteration.
g.current = lastSampleTime.Add(time.Second)
return in[firstIdx:lastIdx]
}
func (g *getValuesAlongRangeOp) Consumed() bool {
return g.current.After(g.through)
}
// getValuesAtIntervalOp encapsulates getting values at a given interval over a
// duration.
type getValuesAtIntervalOp struct {
getValuesAlongRangeOp
interval time.Duration
}
func (g *getValuesAtIntervalOp) String() string {
return fmt.Sprintf("getValuesAtIntervalOp from %s each %s through %s", g.current, g.interval, g.through)
}
func (g *getValuesAtIntervalOp) ExtractSamples(in metric.Values) (out metric.Values) {
if len(in) == 0 {
return
}
lastChunkTime := in[len(in)-1].Timestamp
if g.current.After(lastChunkTime) {
g.current = g.through.Add(clientmodel.MinimumTick)
return metric.Values{in[len(in)-1]}
}
for len(in) > 0 {
out = append(out, extractValuesAroundTime(g.current, in)...)
if g.current.After(lastChunkTime) {
break
}
lastExtractedTime := out[len(out)-1].Timestamp
in = in.TruncateBefore(lastExtractedTime.Add(
clientmodel.MinimumTick))
g.current = g.current.Add(g.interval)
for !g.current.After(lastExtractedTime) {
g.current = g.current.Add(g.interval)
}
if lastExtractedTime.Equal(lastChunkTime) {
break
}
if g.current.After(g.through) {
break
}
}
return
}
// getValueRangeAtIntervalOp encapsulates getting all values from ranges along
// intervals.
//
// Works just like getValuesAlongRangeOp, but when from > through, through is
// incremented by interval and from is reset to through-rangeDuration. Returns
// current time nil when from > totalThrough.
type getValueRangeAtIntervalOp struct {
getValuesAtIntervalOp
rangeThrough clientmodel.Timestamp
rangeDuration time.Duration
}
func (g *getValueRangeAtIntervalOp) String() string {
return fmt.Sprintf("getValueRangeAtIntervalOp range %s from %s each %s through %s", g.rangeDuration, g.current, g.interval, g.through)
}
// Through panics because the notion of 'through' is ambiguous for this op.
func (g *getValueRangeAtIntervalOp) Through() clientmodel.Timestamp {
panic("not implemented")
}
func (g *getValueRangeAtIntervalOp) advanceToNextInterval() {
g.rangeThrough = g.rangeThrough.Add(g.interval)
g.current = g.rangeThrough.Add(-g.rangeDuration)
}
func (g *getValueRangeAtIntervalOp) ExtractSamples(in metric.Values) (out metric.Values) {
if len(in) == 0 {
return
}
// Find the first sample where time >= g.current.
firstIdx := sort.Search(len(in), func(i int) bool {
return !in[i].Timestamp.Before(g.current)
})
if firstIdx == len(in) {
// No samples at or after operator start time. This can only
// happen if we try applying the operator to a time after the
// last recorded sample. In this case, we're finished.
g.current = g.through.Add(clientmodel.MinimumTick)
return
}
// Find the first sample where time > g.rangeThrough.
lastIdx := sort.Search(len(in), func(i int) bool {
return in[i].Timestamp.After(g.rangeThrough)
})
// This only happens when there is only one sample and it is both after
// g.current and after g.rangeThrough. In this case, both indexes are 0.
if lastIdx == firstIdx {
g.advanceToNextInterval()
return
}
lastSampleTime := in[lastIdx-1].Timestamp
// Sample times are stored with a maximum time resolution of one second,
// so we have to add exactly that to target the next chunk on the next
// op iteration.
g.current = lastSampleTime.Add(time.Second)
if g.current.After(g.rangeThrough) {
g.advanceToNextInterval()
}
return in[firstIdx:lastIdx]
}
// getValuesAtIntervalOps contains getValuesAtIntervalOp operations. It
// implements sort.Interface and sorts the operations in ascending order by
// their frequency.
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]
}
func (s getValuesAtIntervalOps) Less(i, j int) bool {
return s[i].interval < s[j].interval
}
// 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 clientmodel.Timestamp, in metric.Values) metric.Values {
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.
return in[len(in)-1:]
}
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.
return in[i : i+1]
}
if i == 0 {
// We hit before the first sample time. Return only the first
// sample.
return in[0:1]
}
// We hit between two samples. Return both surrounding samples.
return in[i-1 : i+1]
}

View file

@ -1,826 +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 tiered
import (
"testing"
"time"
"github.com/prometheus/prometheus/storage/metric"
)
func TestGetValuesAtTimeOp(t *testing.T) {
var scenarios = []struct {
op getValuesAtTimeOp
in metric.Values
out metric.Values
}{
// No values.
{
op: getValuesAtTimeOp{
baseOp: baseOp{current: testInstant},
},
},
// Operator time before single value.
{
op: getValuesAtTimeOp{
baseOp: baseOp{current: testInstant},
},
in: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
},
// Operator time exactly at single value.
{
op: getValuesAtTimeOp{
baseOp: baseOp{current: testInstant.Add(1 * time.Minute)},
},
in: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
},
// Operator time after single value.
{
op: getValuesAtTimeOp{
baseOp: baseOp{current: testInstant.Add(2 * time.Minute)},
},
in: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
},
// Operator time before two values.
{
op: getValuesAtTimeOp{
baseOp: baseOp{current: testInstant},
},
in: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
},
// Operator time at first of two values.
{
op: getValuesAtTimeOp{
baseOp: baseOp{current: testInstant.Add(1 * time.Minute)},
},
in: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
},
// Operator time between first and second of two values.
{
op: getValuesAtTimeOp{
baseOp: baseOp{current: testInstant.Add(90 * time.Second)},
},
in: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
},
// Operator time at second of two values.
{
op: getValuesAtTimeOp{
baseOp: baseOp{current: testInstant.Add(2 * time.Minute)},
},
in: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
},
// Operator time after second of two values.
{
op: getValuesAtTimeOp{
baseOp: baseOp{current: testInstant.Add(3 * time.Minute)},
},
in: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
},
}
for i, scenario := range scenarios {
actual := scenario.op.ExtractSamples(scenario.in)
if len(actual) != len(scenario.out) {
t.Fatalf("%d. expected length %d, got %d: %v", i, len(scenario.out), len(actual), scenario.op)
t.Fatalf("%d. expected length %d, got %d", i, len(scenario.out), len(actual))
}
for j, out := range scenario.out {
if !out.Equal(&actual[j]) {
t.Fatalf("%d. expected output %v, got %v", i, scenario.out, actual)
}
}
}
}
func TestGetValuesAtIntervalOp(t *testing.T) {
var scenarios = []struct {
op getValuesAtIntervalOp
in metric.Values
out metric.Values
}{
// No values.
{
op: getValuesAtIntervalOp{
getValuesAlongRangeOp: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant},
through: testInstant.Add(1 * time.Minute),
},
interval: 30 * time.Second,
},
},
// Entire operator range before first value.
{
op: getValuesAtIntervalOp{
getValuesAlongRangeOp: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant},
through: testInstant.Add(1 * time.Minute),
},
interval: 30 * time.Second,
},
in: metric.Values{
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
},
// Operator range starts before first value, ends within available values.
{
op: getValuesAtIntervalOp{
getValuesAlongRangeOp: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant},
through: testInstant.Add(2 * time.Minute),
},
interval: 30 * time.Second,
},
in: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
},
// Entire operator range is within available values.
{
op: getValuesAtIntervalOp{
getValuesAlongRangeOp: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant.Add(1 * time.Minute)},
through: testInstant.Add(2 * time.Minute),
},
interval: 30 * time.Second,
},
in: metric.Values{
{
Timestamp: testInstant,
Value: 1,
},
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
},
// Operator range begins before first value, ends after last.
{
op: getValuesAtIntervalOp{
getValuesAlongRangeOp: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant},
through: testInstant.Add(3 * time.Minute),
},
interval: 30 * time.Second,
},
in: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
},
// Operator range begins within available values, ends after the last value.
{
op: getValuesAtIntervalOp{
getValuesAlongRangeOp: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant.Add(2 * time.Minute)},
through: testInstant.Add(4 * time.Minute),
},
interval: 30 * time.Second,
},
in: metric.Values{
{
Timestamp: testInstant,
Value: 1,
},
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
},
// Entire operator range after the last available value.
{
op: getValuesAtIntervalOp{
getValuesAlongRangeOp: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant.Add(2 * time.Minute)},
through: testInstant.Add(3 * time.Minute),
},
interval: 30 * time.Second,
},
in: metric.Values{
{
Timestamp: testInstant,
Value: 1,
},
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
},
// Operator interval skips over several values and ends past the last
// available value. This is to verify that we still include the last value
// of a series even if we target a time past it and haven't extracted that
// value yet as part of a previous interval step (thus the necessity to
// skip over values for the test).
{
op: getValuesAtIntervalOp{
getValuesAlongRangeOp: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant.Add(30 * time.Second)},
through: testInstant.Add(4 * time.Minute),
},
interval: 3 * time.Minute,
},
in: metric.Values{
{
Timestamp: testInstant,
Value: 1,
},
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant,
Value: 1,
},
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
},
}
for i, scenario := range scenarios {
actual := scenario.op.ExtractSamples(scenario.in)
if len(actual) != len(scenario.out) {
t.Fatalf("%d. expected length %d, got %d: %v", i, len(scenario.out), len(actual), actual)
}
if len(scenario.in) < 1 {
continue
}
lastExtractedTime := scenario.out[len(scenario.out)-1].Timestamp
if !scenario.op.Consumed() && scenario.op.CurrentTime().Before(lastExtractedTime) {
t.Fatalf("%d. expected op to be consumed or with CurrentTime() after current chunk, %v, %v", i, scenario.op.CurrentTime(), scenario.out)
}
for j, out := range scenario.out {
if !out.Equal(&actual[j]) {
t.Fatalf("%d. expected output %v, got %v", i, scenario.out, actual)
}
}
}
}
func TestGetValuesAlongRangeOp(t *testing.T) {
var scenarios = []struct {
op getValuesAlongRangeOp
in metric.Values
out metric.Values
}{
// No values.
{
op: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant},
through: testInstant.Add(1 * time.Minute),
},
},
// Entire operator range before first value.
{
op: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant},
through: testInstant.Add(1 * time.Minute),
},
in: metric.Values{
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
out: metric.Values{},
},
// Operator range starts before first value, ends within available values.
{
op: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant},
through: testInstant.Add(2 * time.Minute),
},
in: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
},
// Entire operator range is within available values.
{
op: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant.Add(1 * time.Minute)},
through: testInstant.Add(2 * time.Minute),
},
in: metric.Values{
{
Timestamp: testInstant,
Value: 1,
},
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
},
// Operator range begins before first value, ends after last.
{
op: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant},
through: testInstant.Add(3 * time.Minute),
},
in: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
},
},
// Operator range begins within available values, ends after the last value.
{
op: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant.Add(2 * time.Minute)},
through: testInstant.Add(4 * time.Minute),
},
in: metric.Values{
{
Timestamp: testInstant,
Value: 1,
},
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(2 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(3 * time.Minute),
Value: 1,
},
},
},
// Entire operator range after the last available value.
{
op: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant.Add(2 * time.Minute)},
through: testInstant.Add(3 * time.Minute),
},
in: metric.Values{
{
Timestamp: testInstant,
Value: 1,
},
{
Timestamp: testInstant.Add(1 * time.Minute),
Value: 1,
},
},
out: metric.Values{},
},
}
for i, scenario := range scenarios {
actual := scenario.op.ExtractSamples(scenario.in)
if len(actual) != len(scenario.out) {
t.Fatalf("%d. expected length %d, got %d: %v", i, len(scenario.out), len(actual), actual)
}
for j, out := range scenario.out {
if !out.Equal(&actual[j]) {
t.Fatalf("%d. expected output %v, got %v", i, scenario.out, actual)
}
}
}
}
func TestGetValueRangeAtIntervalOp(t *testing.T) {
testOp := getValueRangeAtIntervalOp{
getValuesAtIntervalOp: getValuesAtIntervalOp{
getValuesAlongRangeOp: getValuesAlongRangeOp{
baseOp: baseOp{current: testInstant.Add(-2 * time.Minute)},
through: testInstant.Add(20 * time.Minute),
},
interval: 10 * time.Minute,
},
rangeThrough: testInstant,
rangeDuration: 2 * time.Minute,
}
var scenarios = []struct {
op getValueRangeAtIntervalOp
in metric.Values
out metric.Values
}{
// All values before the first range.
{
op: testOp,
in: metric.Values{
{
Timestamp: testInstant.Add(-4 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(-3 * time.Minute),
Value: 2,
},
},
out: metric.Values{},
},
// metric.Values starting before first range, ending after last.
{
op: testOp,
in: metric.Values{
{
Timestamp: testInstant.Add(-4 * time.Minute),
Value: 1,
},
{
Timestamp: testInstant.Add(-3 * time.Minute),
Value: 2,
},
{
Timestamp: testInstant.Add(-2 * time.Minute),
Value: 3,
},
{
Timestamp: testInstant.Add(-1 * time.Minute),
Value: 4,
},
{
Timestamp: testInstant.Add(0 * time.Minute),
Value: 5,
},
{
Timestamp: testInstant.Add(5 * time.Minute),
Value: 6,
},
{
Timestamp: testInstant.Add(8 * time.Minute),
Value: 7,
},
{
Timestamp: testInstant.Add(9 * time.Minute),
Value: 8,
},
{
Timestamp: testInstant.Add(10 * time.Minute),
Value: 9,
},
{
Timestamp: testInstant.Add(15 * time.Minute),
Value: 10,
},
{
Timestamp: testInstant.Add(18 * time.Minute),
Value: 11,
},
{
Timestamp: testInstant.Add(19 * time.Minute),
Value: 12,
},
{
Timestamp: testInstant.Add(20 * time.Minute),
Value: 13,
},
{
Timestamp: testInstant.Add(21 * time.Minute),
Value: 14,
},
},
out: metric.Values{
{
Timestamp: testInstant.Add(-2 * time.Minute),
Value: 3,
},
{
Timestamp: testInstant.Add(-1 * time.Minute),
Value: 4,
},
{
Timestamp: testInstant.Add(0 * time.Minute),
Value: 5,
},
{
Timestamp: testInstant.Add(8 * time.Minute),
Value: 7,
},
{
Timestamp: testInstant.Add(9 * time.Minute),
Value: 8,
},
{
Timestamp: testInstant.Add(10 * time.Minute),
Value: 9,
},
{
Timestamp: testInstant.Add(18 * time.Minute),
Value: 11,
},
{
Timestamp: testInstant.Add(19 * time.Minute),
Value: 12,
},
{
Timestamp: testInstant.Add(20 * time.Minute),
Value: 13,
},
},
},
// metric.Values starting after last range.
{
op: testOp,
in: metric.Values{
{
Timestamp: testInstant.Add(21 * time.Minute),
Value: 14,
},
},
out: metric.Values{},
},
}
for i, scenario := range scenarios {
actual := metric.Values{}
for !scenario.op.Consumed() {
actual = append(actual, scenario.op.ExtractSamples(scenario.in)...)
}
if len(actual) != len(scenario.out) {
t.Fatalf("%d. expected length %d, got %d: %v", i, len(scenario.out), len(actual), actual)
}
for j, out := range scenario.out {
if !out.Equal(&actual[j]) {
t.Fatalf("%d. expected output %v, got %v", i, scenario.out, actual)
}
}
}
}

View file

@ -1,449 +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 tiered
import (
"fmt"
"code.google.com/p/goprotobuf/proto"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/raw"
"github.com/prometheus/prometheus/storage/raw/leveldb"
dto "github.com/prometheus/prometheus/model/generated"
)
// Processor models a post-processing agent that performs work given a sample
// corpus.
type Processor interface {
// Name emits the name of this processor's signature encoder. It must
// be fully-qualified in the sense that it could be used via a Protocol
// Buffer registry to extract the descriptor to reassemble this message.
Name() string
// Signature emits a byte signature for this process for the purpose of
// remarking how far along it has been applied to the database.
Signature() []byte
// Apply runs this processor against the sample set. sampleIterator
// expects to be pre-seeked to the initial starting position. The
// processor will run until up until stopAt has been reached. It is
// imperative that the provided stopAt is within the interval of the
// series frontier.
//
// Upon completion or error, the last time at which the processor
// finished shall be emitted in addition to any errors.
Apply(sampleIterator leveldb.Iterator, samplesPersistence raw.Persistence, stopAt clientmodel.Timestamp, fingerprint *clientmodel.Fingerprint) (lastCurated clientmodel.Timestamp, err error)
// Close reaps all of the underlying system resources associated with
// this processor.
Close()
}
// CompactionProcessor combines sparse values in the database together such that
// at least MinimumGroupSize-sized chunks are grouped together. It implements
// the Processor interface.
type CompactionProcessor struct {
maximumMutationPoolBatch int
minimumGroupSize int
// signature is the byte representation of the CompactionProcessor's
// settings, used for purely memoization purposes across an instance.
signature []byte
dtoSampleKeys *dtoSampleKeyList
sampleKeys *sampleKeyList
}
// Name implements the Processor interface. It returns
// "io.prometheus.CompactionProcessorDefinition".
func (p *CompactionProcessor) Name() string {
return "io.prometheus.CompactionProcessorDefinition"
}
// Signature implements the Processor interface.
func (p *CompactionProcessor) Signature() []byte {
if len(p.signature) == 0 {
out, err := proto.Marshal(&dto.CompactionProcessorDefinition{
MinimumGroupSize: proto.Uint32(uint32(p.minimumGroupSize)),
})
if err != nil {
panic(err)
}
p.signature = out
}
return p.signature
}
func (p *CompactionProcessor) String() string {
return fmt.Sprintf("compactionProcessor for minimum group size %d", p.minimumGroupSize)
}
// Apply implements the Processor interface.
func (p *CompactionProcessor) Apply(sampleIterator leveldb.Iterator, samplesPersistence raw.Persistence, stopAt clientmodel.Timestamp, fingerprint *clientmodel.Fingerprint) (lastCurated clientmodel.Timestamp, err error) {
var pendingBatch raw.Batch
defer func() {
if pendingBatch != nil {
pendingBatch.Close()
}
}()
var pendingMutations = 0
var pendingSamples metric.Values
var unactedSamples metric.Values
var lastTouchedTime clientmodel.Timestamp
var keyDropped bool
sampleKey, _ := p.sampleKeys.Get()
defer p.sampleKeys.Give(sampleKey)
sampleKeyDto, _ := p.dtoSampleKeys.Get()
defer p.dtoSampleKeys.Give(sampleKeyDto)
if err = sampleIterator.Key(sampleKeyDto); err != nil {
return
}
sampleKey.Load(sampleKeyDto)
unactedSamples = unmarshalValues(sampleIterator.RawValue(), nil)
for lastCurated.Before(stopAt) && lastTouchedTime.Before(stopAt) && sampleKey.Fingerprint.Equal(fingerprint) {
switch {
// Furnish a new pending batch operation if none is available.
case pendingBatch == nil:
pendingBatch = leveldb.NewBatch()
// If there are no sample values to extract from the datastore, let's
// continue extracting more values to use. We know that the time.Before()
// block would prevent us from going into unsafe territory.
case len(unactedSamples) == 0:
if !sampleIterator.Next() {
return lastCurated, fmt.Errorf("illegal condition: invalid iterator on continuation")
}
keyDropped = false
if err = sampleIterator.Key(sampleKeyDto); err != nil {
return
}
sampleKey.Load(sampleKeyDto)
if !sampleKey.Fingerprint.Equal(fingerprint) {
break
}
unactedSamples = unmarshalValues(sampleIterator.RawValue(), nil)
// If the number of pending mutations exceeds the allowed batch amount,
// commit to disk and delete the batch. A new one will be recreated if
// necessary.
case pendingMutations >= p.maximumMutationPoolBatch:
err = samplesPersistence.Commit(pendingBatch)
if err != nil {
return
}
pendingMutations = 0
pendingBatch.Close()
pendingBatch = nil
case len(pendingSamples) == 0 && len(unactedSamples) >= p.minimumGroupSize:
lastTouchedTime = unactedSamples[len(unactedSamples)-1].Timestamp
unactedSamples = metric.Values{}
case len(pendingSamples)+len(unactedSamples) < p.minimumGroupSize:
if !keyDropped {
k := &dto.SampleKey{}
sampleKey.Dump(k)
pendingBatch.Drop(k)
keyDropped = true
}
pendingSamples = append(pendingSamples, unactedSamples...)
lastTouchedTime = unactedSamples[len(unactedSamples)-1].Timestamp
unactedSamples = metric.Values{}
pendingMutations++
// If the number of pending writes equals the target group size
case len(pendingSamples) == p.minimumGroupSize:
k := &dto.SampleKey{}
newSampleKey := buildSampleKey(fingerprint, pendingSamples)
newSampleKey.Dump(k)
b := marshalValues(pendingSamples, nil)
pendingBatch.PutRaw(k, b)
pendingMutations++
lastCurated = newSampleKey.FirstTimestamp
if len(unactedSamples) > 0 {
if !keyDropped {
sampleKey.Dump(k)
pendingBatch.Drop(k)
keyDropped = true
}
if len(unactedSamples) > p.minimumGroupSize {
pendingSamples = unactedSamples[:p.minimumGroupSize]
unactedSamples = unactedSamples[p.minimumGroupSize:]
lastTouchedTime = unactedSamples[len(unactedSamples)-1].Timestamp
} else {
pendingSamples = unactedSamples
lastTouchedTime = pendingSamples[len(pendingSamples)-1].Timestamp
unactedSamples = metric.Values{}
}
}
case len(pendingSamples)+len(unactedSamples) >= p.minimumGroupSize:
if !keyDropped {
k := &dto.SampleKey{}
sampleKey.Dump(k)
pendingBatch.Drop(k)
keyDropped = true
}
remainder := p.minimumGroupSize - len(pendingSamples)
pendingSamples = append(pendingSamples, unactedSamples[:remainder]...)
unactedSamples = unactedSamples[remainder:]
if len(unactedSamples) == 0 {
lastTouchedTime = pendingSamples[len(pendingSamples)-1].Timestamp
} else {
lastTouchedTime = unactedSamples[len(unactedSamples)-1].Timestamp
}
pendingMutations++
default:
err = fmt.Errorf("unhandled processing case")
}
}
if len(unactedSamples) > 0 || len(pendingSamples) > 0 {
pendingSamples = append(pendingSamples, unactedSamples...)
k := &dto.SampleKey{}
newSampleKey := buildSampleKey(fingerprint, pendingSamples)
newSampleKey.Dump(k)
b := marshalValues(pendingSamples, nil)
pendingBatch.PutRaw(k, b)
pendingSamples = metric.Values{}
pendingMutations++
lastCurated = newSampleKey.FirstTimestamp
}
// This is not deferred due to the off-chance that a pre-existing commit
// failed.
if pendingBatch != nil && pendingMutations > 0 {
err = samplesPersistence.Commit(pendingBatch)
if err != nil {
return
}
}
return
}
// Close implements the Processor interface.
func (p *CompactionProcessor) Close() {
p.dtoSampleKeys.Close()
p.sampleKeys.Close()
}
// CompactionProcessorOptions are used for connstruction of a
// CompactionProcessor.
type CompactionProcessorOptions struct {
// MaximumMutationPoolBatch represents approximately the largest pending
// batch of mutation operations for the database before pausing to
// commit before resumption.
//
// A reasonable value would be (MinimumGroupSize * 2) + 1.
MaximumMutationPoolBatch int
// MinimumGroupSize represents the smallest allowed sample chunk size in the
// database.
MinimumGroupSize int
}
// NewCompactionProcessor returns a CompactionProcessor ready to use.
func NewCompactionProcessor(o *CompactionProcessorOptions) *CompactionProcessor {
return &CompactionProcessor{
maximumMutationPoolBatch: o.MaximumMutationPoolBatch,
minimumGroupSize: o.MinimumGroupSize,
dtoSampleKeys: newDtoSampleKeyList(10),
sampleKeys: newSampleKeyList(10),
}
}
// DeletionProcessor deletes sample blocks older than a defined value. It
// implements the Processor interface.
type DeletionProcessor struct {
maximumMutationPoolBatch int
// signature is the byte representation of the DeletionProcessor's settings,
// used for purely memoization purposes across an instance.
signature []byte
dtoSampleKeys *dtoSampleKeyList
sampleKeys *sampleKeyList
}
// Name implements the Processor interface. It returns
// "io.prometheus.DeletionProcessorDefinition".
func (p *DeletionProcessor) Name() string {
return "io.prometheus.DeletionProcessorDefinition"
}
// Signature implements the Processor interface.
func (p *DeletionProcessor) Signature() []byte {
if len(p.signature) == 0 {
out, err := proto.Marshal(&dto.DeletionProcessorDefinition{})
if err != nil {
panic(err)
}
p.signature = out
}
return p.signature
}
func (p *DeletionProcessor) String() string {
return "deletionProcessor"
}
// Apply implements the Processor interface.
func (p *DeletionProcessor) Apply(sampleIterator leveldb.Iterator, samplesPersistence raw.Persistence, stopAt clientmodel.Timestamp, fingerprint *clientmodel.Fingerprint) (lastCurated clientmodel.Timestamp, err error) {
var pendingBatch raw.Batch
defer func() {
if pendingBatch != nil {
pendingBatch.Close()
}
}()
sampleKeyDto, _ := p.dtoSampleKeys.Get()
defer p.dtoSampleKeys.Give(sampleKeyDto)
sampleKey, _ := p.sampleKeys.Get()
defer p.sampleKeys.Give(sampleKey)
if err = sampleIterator.Key(sampleKeyDto); err != nil {
return
}
sampleKey.Load(sampleKeyDto)
sampleValues := unmarshalValues(sampleIterator.RawValue(), nil)
pendingMutations := 0
for lastCurated.Before(stopAt) && sampleKey.Fingerprint.Equal(fingerprint) {
switch {
// Furnish a new pending batch operation if none is available.
case pendingBatch == nil:
pendingBatch = leveldb.NewBatch()
// If there are no sample values to extract from the datastore,
// let's continue extracting more values to use. We know that
// the time.Before() block would prevent us from going into
// unsafe territory.
case len(sampleValues) == 0:
if !sampleIterator.Next() {
return lastCurated, fmt.Errorf("illegal condition: invalid iterator on continuation")
}
if err = sampleIterator.Key(sampleKeyDto); err != nil {
return
}
sampleKey.Load(sampleKeyDto)
sampleValues = unmarshalValues(sampleIterator.RawValue(), nil)
// If the number of pending mutations exceeds the allowed batch
// amount, commit to disk and delete the batch. A new one will
// be recreated if necessary.
case pendingMutations >= p.maximumMutationPoolBatch:
err = samplesPersistence.Commit(pendingBatch)
if err != nil {
return
}
pendingMutations = 0
pendingBatch.Close()
pendingBatch = nil
case !sampleKey.MayContain(stopAt):
k := &dto.SampleKey{}
sampleKey.Dump(k)
pendingBatch.Drop(k)
lastCurated = sampleKey.LastTimestamp
sampleValues = metric.Values{}
pendingMutations++
case sampleKey.MayContain(stopAt):
k := &dto.SampleKey{}
sampleKey.Dump(k)
pendingBatch.Drop(k)
pendingMutations++
sampleValues = sampleValues.TruncateBefore(stopAt)
if len(sampleValues) > 0 {
k := &dto.SampleKey{}
sampleKey = buildSampleKey(fingerprint, sampleValues)
sampleKey.Dump(k)
lastCurated = sampleKey.FirstTimestamp
v := marshalValues(sampleValues, nil)
pendingBatch.PutRaw(k, v)
pendingMutations++
} else {
lastCurated = sampleKey.LastTimestamp
}
default:
err = fmt.Errorf("unhandled processing case")
}
}
// This is not deferred due to the off-chance that a pre-existing commit
// failed.
if pendingBatch != nil && pendingMutations > 0 {
err = samplesPersistence.Commit(pendingBatch)
if err != nil {
return
}
}
return
}
// Close implements the Processor interface.
func (p *DeletionProcessor) Close() {
p.dtoSampleKeys.Close()
p.sampleKeys.Close()
}
// DeletionProcessorOptions are used for connstruction of a DeletionProcessor.
type DeletionProcessorOptions struct {
// MaximumMutationPoolBatch represents approximately the largest pending
// batch of mutation operations for the database before pausing to
// commit before resumption.
MaximumMutationPoolBatch int
}
// NewDeletionProcessor returns a DeletionProcessor ready to use.
func NewDeletionProcessor(o *DeletionProcessorOptions) *DeletionProcessor {
return &DeletionProcessor{
maximumMutationPoolBatch: o.MaximumMutationPoolBatch,
dtoSampleKeys: newDtoSampleKeyList(10),
sampleKeys: newSampleKeyList(10),
}
}

File diff suppressed because it is too large Load diff

View file

@ -1,86 +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 tiered
import (
"testing"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
)
func GetFingerprintsForLabelSetUsesAndForLabelMatchingTests(p metric.Persistence, t testing.TB) {
metrics := []clientmodel.LabelSet{
{clientmodel.MetricNameLabel: "request_metrics_latency_equal_tallying_microseconds", "instance": "http://localhost:9090/metrics.json", "percentile": "0.010000"},
{clientmodel.MetricNameLabel: "requests_metrics_latency_equal_accumulating_microseconds", "instance": "http://localhost:9090/metrics.json", "percentile": "0.010000"},
{clientmodel.MetricNameLabel: "requests_metrics_latency_logarithmic_accumulating_microseconds", "instance": "http://localhost:9090/metrics.json", "percentile": "0.010000"},
{clientmodel.MetricNameLabel: "requests_metrics_latency_logarithmic_tallying_microseconds", "instance": "http://localhost:9090/metrics.json", "percentile": "0.010000"},
{clientmodel.MetricNameLabel: "targets_healthy_scrape_latency_ms", "instance": "http://localhost:9090/metrics.json", "percentile": "0.010000"},
}
for _, metric := range metrics {
m := clientmodel.Metric{}
for k, v := range metric {
m[clientmodel.LabelName(k)] = clientmodel.LabelValue(v)
}
testAppendSamples(p, &clientmodel.Sample{
Value: clientmodel.SampleValue(0.0),
Timestamp: clientmodel.Now(),
Metric: m,
}, t)
}
labelSet := clientmodel.LabelSet{
clientmodel.MetricNameLabel: "targets_healthy_scrape_latency_ms",
"percentile": "0.010000",
}
fingerprints, err := p.GetFingerprintsForLabelMatchers(labelMatchersFromLabelSet(labelSet))
if err != nil {
t.Errorf("could not get labels: %s", err)
}
if len(fingerprints) != 1 {
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

@ -1,954 +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 tiered
import (
"testing"
"time"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility/test"
)
func GetValueAtTimeTests(persistenceMaker func() (metric.ViewablePersistence, test.Closer), t testing.TB) {
type value struct {
year int
month time.Month
day int
hour int
value clientmodel.SampleValue
}
type input struct {
year int
month time.Month
day int
hour int
}
type output []clientmodel.SampleValue
type behavior struct {
name string
input input
output output
}
var contexts = []struct {
name string
values []value
behaviors []behavior
}{
{
name: "no values",
values: []value{},
behaviors: []behavior{
{
name: "random target",
input: input{
year: 1984,
month: 3,
day: 30,
hour: 0,
},
},
},
},
{
name: "singleton",
values: []value{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
},
behaviors: []behavior{
{
name: "exact",
input: input{
year: 1984,
month: 3,
day: 30,
hour: 0,
},
output: output{
0,
},
},
{
name: "before",
input: input{
year: 1984,
month: 3,
day: 29,
hour: 0,
},
output: output{
0,
},
},
{
name: "after",
input: input{
year: 1984,
month: 3,
day: 31,
hour: 0,
},
output: output{
0,
},
},
},
},
{
name: "double",
values: []value{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
{
year: 1985,
month: 3,
day: 30,
hour: 0,
value: 1,
},
},
behaviors: []behavior{
{
name: "exact first",
input: input{
year: 1984,
month: 3,
day: 30,
hour: 0,
},
output: output{
0,
},
},
{
name: "exact second",
input: input{
year: 1985,
month: 3,
day: 30,
hour: 0,
},
output: output{
1,
},
},
{
name: "before first",
input: input{
year: 1983,
month: 9,
day: 29,
hour: 12,
},
output: output{
0,
},
},
{
name: "after second",
input: input{
year: 1985,
month: 9,
day: 28,
hour: 12,
},
output: output{
1,
},
},
{
name: "middle",
input: input{
year: 1984,
month: 9,
day: 28,
hour: 12,
},
output: output{
0,
1,
},
},
},
},
{
name: "triple",
values: []value{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
{
year: 1985,
month: 3,
day: 30,
hour: 0,
value: 1,
},
{
year: 1986,
month: 3,
day: 30,
hour: 0,
value: 2,
},
},
behaviors: []behavior{
{
name: "exact first",
input: input{
year: 1984,
month: 3,
day: 30,
hour: 0,
},
output: output{
0,
},
},
{
name: "exact second",
input: input{
year: 1985,
month: 3,
day: 30,
hour: 0,
},
output: output{
1,
},
},
{
name: "exact third",
input: input{
year: 1986,
month: 3,
day: 30,
hour: 0,
},
output: output{
2,
},
},
{
name: "before first",
input: input{
year: 1983,
month: 9,
day: 29,
hour: 12,
},
output: output{
0,
},
},
{
name: "after third",
input: input{
year: 1986,
month: 9,
day: 28,
hour: 12,
},
output: output{
2,
},
},
{
name: "first middle",
input: input{
year: 1984,
month: 9,
day: 28,
hour: 12,
},
output: output{
0,
1,
},
},
{
name: "second middle",
input: input{
year: 1985,
month: 9,
day: 28,
hour: 12,
},
output: output{
1,
2,
},
},
},
},
}
for i, context := range contexts {
// Wrapping in function to enable garbage collection of resources.
func() {
p, closer := persistenceMaker()
defer closer.Close()
defer p.Close()
m := clientmodel.Metric{
clientmodel.MetricNameLabel: "age_in_years",
}
for _, value := range context.values {
testAppendSamples(p, &clientmodel.Sample{
Value: clientmodel.SampleValue(value.value),
Timestamp: clientmodel.TimestampFromTime(time.Date(value.year, value.month, value.day, value.hour, 0, 0, 0, time.UTC)),
Metric: m,
}, t)
}
for j, behavior := range context.behaviors {
input := behavior.input
time := clientmodel.TimestampFromTime(time.Date(input.year, input.month, input.day, input.hour, 0, 0, 0, time.UTC))
fingerprint := &clientmodel.Fingerprint{}
fingerprint.LoadFromMetric(m)
actual := p.GetValueAtTime(fingerprint, time)
if len(behavior.output) != len(actual) {
t.Fatalf("%d.%d(%s.%s). Expected %d samples but got: %v\n", i, j, context.name, behavior.name, len(behavior.output), actual)
}
for k, samplePair := range actual {
if samplePair.Value != behavior.output[k] {
t.Fatalf("%d.%d.%d(%s.%s). Expected %s but got %s\n", i, j, k, context.name, behavior.name, behavior.output[k], samplePair)
}
}
}
}()
}
}
func GetRangeValuesTests(persistenceMaker func() (metric.ViewablePersistence, test.Closer), onlyBoundaries bool, t testing.TB) {
type value struct {
year int
month time.Month
day int
hour int
value clientmodel.SampleValue
}
type input struct {
openYear int
openMonth time.Month
openDay int
openHour int
endYear int
endMonth time.Month
endDay int
endHour int
}
type output struct {
year int
month time.Month
day int
hour int
value clientmodel.SampleValue
}
type behavior struct {
name string
input input
output []output
}
var contexts = []struct {
name string
values []value
behaviors []behavior
}{
{
name: "no values",
values: []value{},
behaviors: []behavior{
{
name: "non-existent interval",
input: input{
openYear: 1984,
openMonth: 3,
openDay: 30,
openHour: 0,
endYear: 1985,
endMonth: 3,
endDay: 30,
endHour: 0,
},
},
},
},
{
name: "singleton value",
values: []value{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
},
behaviors: []behavior{
{
name: "start on first value",
input: input{
openYear: 1984,
openMonth: 3,
openDay: 30,
openHour: 0,
endYear: 1985,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
},
},
{
name: "end on first value",
input: input{
openYear: 1983,
openMonth: 3,
openDay: 30,
openHour: 0,
endYear: 1984,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
},
},
{
name: "overlap on first value",
input: input{
openYear: 1983,
openMonth: 3,
openDay: 30,
openHour: 0,
endYear: 1985,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
},
},
},
},
{
name: "two values",
values: []value{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
{
year: 1985,
month: 3,
day: 30,
hour: 0,
value: 1,
},
},
behaviors: []behavior{
{
name: "start on first value",
input: input{
openYear: 1984,
openMonth: 3,
openDay: 30,
openHour: 0,
endYear: 1985,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
{
year: 1985,
month: 3,
day: 30,
hour: 0,
value: 1,
},
},
},
{
name: "start on second value",
input: input{
openYear: 1985,
openMonth: 3,
openDay: 30,
openHour: 0,
endYear: 1986,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1985,
month: 3,
day: 30,
hour: 0,
value: 1,
},
},
},
{
name: "end on first value",
input: input{
openYear: 1983,
openMonth: 3,
openDay: 30,
openHour: 0,
endYear: 1984,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
},
},
{
name: "end on second value",
input: input{
openYear: 1985,
openMonth: 1,
openDay: 1,
openHour: 0,
endYear: 1985,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1985,
month: 3,
day: 30,
hour: 0,
value: 1,
},
},
},
{
name: "overlap on values",
input: input{
openYear: 1983,
openMonth: 3,
openDay: 30,
openHour: 0,
endYear: 1986,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
{
year: 1985,
month: 3,
day: 30,
hour: 0,
value: 1,
},
},
},
},
},
{
name: "three values",
values: []value{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
{
year: 1985,
month: 3,
day: 30,
hour: 0,
value: 1,
},
{
year: 1986,
month: 3,
day: 30,
hour: 0,
value: 2,
},
},
behaviors: []behavior{
{
name: "start on first value",
input: input{
openYear: 1984,
openMonth: 3,
openDay: 30,
openHour: 0,
endYear: 1985,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
{
year: 1985,
month: 3,
day: 30,
hour: 0,
value: 1,
},
},
},
{
name: "start on second value",
input: input{
openYear: 1985,
openMonth: 3,
openDay: 30,
openHour: 0,
endYear: 1986,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1985,
month: 3,
day: 30,
hour: 0,
value: 1,
},
{
year: 1986,
month: 3,
day: 30,
hour: 0,
value: 2,
},
},
},
{
name: "end on first value",
input: input{
openYear: 1983,
openMonth: 3,
openDay: 30,
openHour: 0,
endYear: 1984,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
},
},
{
name: "end on second value",
input: input{
openYear: 1985,
openMonth: 1,
openDay: 1,
openHour: 0,
endYear: 1985,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1985,
month: 3,
day: 30,
hour: 0,
value: 1,
},
},
},
{
name: "overlap on values",
input: input{
openYear: 1983,
openMonth: 3,
openDay: 30,
openHour: 0,
endYear: 1986,
endMonth: 3,
endDay: 30,
endHour: 0,
},
output: []output{
{
year: 1984,
month: 3,
day: 30,
hour: 0,
value: 0,
},
{
year: 1985,
month: 3,
day: 30,
hour: 0,
value: 1,
},
{
year: 1986,
month: 3,
day: 30,
hour: 0,
value: 2,
},
},
},
},
},
}
for i, context := range contexts {
// Wrapping in function to enable garbage collection of resources.
func() {
p, closer := persistenceMaker()
defer closer.Close()
defer p.Close()
m := clientmodel.Metric{
clientmodel.MetricNameLabel: "age_in_years",
}
for _, value := range context.values {
testAppendSamples(p, &clientmodel.Sample{
Value: clientmodel.SampleValue(value.value),
Timestamp: clientmodel.TimestampFromTime(time.Date(value.year, value.month, value.day, value.hour, 0, 0, 0, time.UTC)),
Metric: m,
}, t)
}
for j, behavior := range context.behaviors {
input := behavior.input
open := clientmodel.TimestampFromTime(time.Date(input.openYear, input.openMonth, input.openDay, input.openHour, 0, 0, 0, time.UTC))
end := clientmodel.TimestampFromTime(time.Date(input.endYear, input.endMonth, input.endDay, input.endHour, 0, 0, 0, time.UTC))
in := metric.Interval{
OldestInclusive: open,
NewestInclusive: end,
}
actualValues := metric.Values{}
expectedValues := []output{}
fp := &clientmodel.Fingerprint{}
fp.LoadFromMetric(m)
if onlyBoundaries {
actualValues = p.GetBoundaryValues(fp, in)
l := len(behavior.output)
if l == 1 {
expectedValues = behavior.output[0:1]
}
if l > 1 {
expectedValues = append(behavior.output[0:1], behavior.output[l-1])
}
} else {
actualValues = p.GetRangeValues(fp, in)
expectedValues = behavior.output
}
if actualValues == nil && len(expectedValues) != 0 {
t.Fatalf("%d.%d(%s). Expected %v but got: %v\n", i, j, behavior.name, expectedValues, actualValues)
}
if expectedValues == nil {
if actualValues != nil {
t.Fatalf("%d.%d(%s). Expected nil values but got: %s\n", i, j, behavior.name, actualValues)
}
} else {
if len(expectedValues) != len(actualValues) {
t.Fatalf("%d.%d(%s). Expected length %d but got: %d\n", i, j, behavior.name, len(expectedValues), len(actualValues))
}
for k, actual := range actualValues {
expected := expectedValues[k]
if actual.Value != clientmodel.SampleValue(expected.value) {
t.Fatalf("%d.%d.%d(%s). Expected %v but got: %v\n", i, j, k, behavior.name, expected.value, actual.Value)
}
if actual.Timestamp.Time().Year() != expected.year {
t.Fatalf("%d.%d.%d(%s). Expected %d but got: %d\n", i, j, k, behavior.name, expected.year, actual.Timestamp.Time().Year())
}
if actual.Timestamp.Time().Month() != expected.month {
t.Fatalf("%d.%d.%d(%s). Expected %d but got: %d\n", i, j, k, behavior.name, expected.month, actual.Timestamp.Time().Month())
}
// XXX: Find problem here.
// Mismatches occur in this and have for a long time in the LevelDB
// case, however not im-memory.
//
// if actual.Timestamp.Day() != expected.day {
// t.Fatalf("%d.%d.%d(%s). Expected %d but got: %d\n", i, j, k, behavior.name, expected.day, actual.Timestamp.Day())
// }
// if actual.Timestamp.Hour() != expected.hour {
// t.Fatalf("%d.%d.%d(%s). Expected %d but got: %d\n", i, j, k, behavior.name, expected.hour, actual.Timestamp.Hour())
// }
}
}
}
}()
}
}
// Test Definitions Follow
func testMemoryGetValueAtTime(t testing.TB) {
persistenceMaker := func() (metric.ViewablePersistence, test.Closer) {
return NewMemorySeriesStorage(MemorySeriesOptions{}), test.NilCloser
}
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 *testing.T) {
testMemoryGetBoundaryValues(t)
}
func BenchmarkMemoryGetBoundaryValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryGetBoundaryValues(b)
}
}
func testMemoryGetRangeValues(t testing.TB) {
persistenceMaker := func() (metric.ViewablePersistence, test.Closer) {
return NewMemorySeriesStorage(MemorySeriesOptions{}), test.NilCloser
}
GetRangeValuesTests(persistenceMaker, false, t)
}
func testMemoryGetBoundaryValues(t testing.TB) {
persistenceMaker := func() (metric.ViewablePersistence, test.Closer) {
return NewMemorySeriesStorage(MemorySeriesOptions{}), test.NilCloser
}
GetRangeValuesTests(persistenceMaker, true, t)
}
func TestMemoryGetRangeValues(t *testing.T) {
testMemoryGetRangeValues(t)
}
func BenchmarkMemoryGetRangeValues(b *testing.B) {
for i := 0; i < b.N; i++ {
testMemoryGetRangeValues(b)
}
}

View file

@ -1,101 +0,0 @@
package tiered
import (
"math/rand"
"testing"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
)
const numTestValues = 5000
func TestValuesMarshalAndUnmarshal(t *testing.T) {
values := randomValues(numTestValues)
marshalled := marshalValues(values, nil)
unmarshalled := unmarshalValues(marshalled, nil)
for i, expected := range values {
actual := unmarshalled[i]
if !actual.Equal(&expected) {
t.Fatalf("%d. got: %v, expected: %v", i, actual, expected)
}
}
}
func randomValues(numSamples int) metric.Values {
v := make(metric.Values, 0, numSamples)
for i := 0; i < numSamples; i++ {
v = append(v, metric.SamplePair{
Timestamp: clientmodel.Timestamp(rand.Int63()),
Value: clientmodel.SampleValue(rand.NormFloat64()),
})
}
return v
}
func benchmarkMarshal(b *testing.B, n int) {
v := randomValues(n)
b.ResetTimer()
// TODO: Reuse buffer to compare performance.
// - Delta is -30 percent time overhead.
for i := 0; i < b.N; i++ {
marshalValues(v, nil)
}
}
func BenchmarkMarshal1(b *testing.B) {
benchmarkMarshal(b, 1)
}
func BenchmarkMarshal10(b *testing.B) {
benchmarkMarshal(b, 10)
}
func BenchmarkMarshal100(b *testing.B) {
benchmarkMarshal(b, 100)
}
func BenchmarkMarshal1000(b *testing.B) {
benchmarkMarshal(b, 1000)
}
func BenchmarkMarshal10000(b *testing.B) {
benchmarkMarshal(b, 10000)
}
func benchmarkUnmarshal(b *testing.B, n int) {
v := randomValues(numTestValues)
marshalled := marshalValues(v, nil)
b.ResetTimer()
// TODO: Reuse buffer to compare performance.
// - Delta is -15 percent time overhead.
for i := 0; i < b.N; i++ {
unmarshalValues(marshalled, nil)
}
}
func BenchmarkUnmarshal1(b *testing.B) {
benchmarkUnmarshal(b, 1)
}
func BenchmarkUnmarshal10(b *testing.B) {
benchmarkUnmarshal(b, 10)
}
func BenchmarkUnmarshal100(b *testing.B) {
benchmarkUnmarshal(b, 100)
}
func BenchmarkUnmarshal1000(b *testing.B) {
benchmarkUnmarshal(b, 1000)
}
func BenchmarkUnmarshal10000(b *testing.B) {
benchmarkUnmarshal(b, 10000)
}

View file

@ -1,141 +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 tiered
import (
"fmt"
"code.google.com/p/goprotobuf/proto"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/coding/indexable"
"github.com/prometheus/prometheus/storage/metric"
dto "github.com/prometheus/prometheus/model/generated"
)
// SampleKey models the business logic around the data-transfer object
// SampleKey.
type SampleKey struct {
Fingerprint *clientmodel.Fingerprint
FirstTimestamp clientmodel.Timestamp
LastTimestamp clientmodel.Timestamp
SampleCount uint32
}
// Constrain merges the underlying SampleKey to fit within the keyspace of
// the provided first and last keys and returns whether the key was modified.
func (s *SampleKey) Constrain(first, last *SampleKey) bool {
switch {
case s.Before(first.Fingerprint, first.FirstTimestamp):
*s = *first
return true
case last.Before(s.Fingerprint, s.FirstTimestamp):
*s = *last
return true
default:
return false
}
}
// Equal returns true if this SampleKey and o have equal fingerprints,
// timestamps, and sample counts.
func (s *SampleKey) Equal(o *SampleKey) bool {
if s == o {
return true
}
if !s.Fingerprint.Equal(o.Fingerprint) {
return false
}
if !s.FirstTimestamp.Equal(o.FirstTimestamp) {
return false
}
if !s.LastTimestamp.Equal(o.LastTimestamp) {
return false
}
return s.SampleCount == o.SampleCount
}
// MayContain indicates whether the given SampleKey could potentially contain a
// value at the provided time. Even if true is emitted, that does not mean a
// satisfactory value, in fact, exists.
func (s *SampleKey) MayContain(t clientmodel.Timestamp) bool {
switch {
case t.Before(s.FirstTimestamp):
return false
case t.After(s.LastTimestamp):
return false
default:
return true
}
}
// Before returns true if the Fingerprint of this SampleKey is less than fp and
// false if it is greater. If both fingerprints are equal, the FirstTimestamp of
// this SampleKey is checked in the same way against t. If the timestamps are
// eqal, the LastTimestamp of this SampleKey is checked against t (and false is
// returned if they are equal again).
func (s *SampleKey) Before(fp *clientmodel.Fingerprint, t clientmodel.Timestamp) bool {
if s.Fingerprint.Less(fp) {
return true
}
if !s.Fingerprint.Equal(fp) {
return false
}
if s.FirstTimestamp.Before(t) {
return true
}
return s.LastTimestamp.Before(t)
}
// Dump converts this SampleKey into a DTO for use in serialization purposes.
func (s *SampleKey) Dump(d *dto.SampleKey) {
d.Reset()
fp := &dto.Fingerprint{}
dumpFingerprint(fp, s.Fingerprint)
d.Fingerprint = fp
d.Timestamp = indexable.EncodeTime(s.FirstTimestamp)
d.LastTimestamp = proto.Int64(s.LastTimestamp.Unix())
d.SampleCount = proto.Uint32(s.SampleCount)
}
func (s *SampleKey) String() string {
return fmt.Sprintf("SampleKey for %s at %s to %s with %d values.", s.Fingerprint, s.FirstTimestamp, s.LastTimestamp, s.SampleCount)
}
// Load deserializes this SampleKey from a DTO.
func (s *SampleKey) Load(d *dto.SampleKey) {
f := &clientmodel.Fingerprint{}
loadFingerprint(f, d.GetFingerprint())
s.Fingerprint = f
s.FirstTimestamp = indexable.DecodeTime(d.Timestamp)
s.LastTimestamp = clientmodel.TimestampFromUnix(d.GetLastTimestamp())
s.SampleCount = d.GetSampleCount()
}
// buildSampleKey returns the SampleKey for the given Fingerprint and Values.
func buildSampleKey(f *clientmodel.Fingerprint, v metric.Values) *SampleKey {
return &SampleKey{
Fingerprint: f,
FirstTimestamp: v[0].Timestamp,
LastTimestamp: v[len(v)-1].Timestamp,
SampleCount: uint32(len(v)),
}
}

View file

@ -1,632 +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 tiered
import (
"fmt"
"math"
"math/rand"
"sort"
"testing"
"testing/quick"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/coding/indexable"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility/test"
dto "github.com/prometheus/prometheus/model/generated"
)
const stochasticMaximumVariance = 8
func BasicLifecycleTests(p metric.Persistence, t testing.TB) {
if p == nil {
t.Errorf("Received nil Metric Persistence.\n")
return
}
}
func ReadEmptyTests(p metric.Persistence, t testing.TB) {
hasLabelPair := func(x int) (success bool) {
fingerprints, err := p.GetFingerprintsForLabelMatchers(metric.LabelMatchers{{
Type: metric.Equal,
Name: clientmodel.LabelName(string(x)),
Value: clientmodel.LabelValue(string(x)),
}})
if err != nil {
t.Error(err)
return
}
success = len(fingerprints) == 0
if !success {
t.Errorf("unexpected fingerprint length %d, got %d", 0, len(fingerprints))
}
return
}
err := quick.Check(hasLabelPair, nil)
if err != nil {
t.Error(err)
return
}
hasLabelName := func(x int) (success bool) {
labelName := clientmodel.LabelName(string(x))
values, err := p.GetLabelValuesForLabelName(labelName)
if err != nil {
t.Error(err)
return
}
success = len(values) == 0
if !success {
t.Errorf("unexpected values length %d, got %d", 0, len(values))
}
return
}
err = quick.Check(hasLabelName, nil)
if err != nil {
t.Error(err)
return
}
}
func AppendSampleAsPureSparseAppendTests(p metric.Persistence, t testing.TB) {
appendSample := func(x int) (success bool) {
v := clientmodel.SampleValue(x)
ts := clientmodel.TimestampFromUnix(int64(x))
labelName := clientmodel.LabelName(x)
labelValue := clientmodel.LabelValue(x)
l := clientmodel.Metric{labelName: labelValue}
sample := &clientmodel.Sample{
Value: v,
Timestamp: ts,
Metric: l,
}
err := p.AppendSamples(clientmodel.Samples{sample})
success = err == nil
if !success {
t.Error(err)
}
return
}
if err := quick.Check(appendSample, nil); err != nil {
t.Error(err)
}
}
func AppendSampleAsSparseAppendWithReadsTests(p metric.Persistence, t testing.TB) {
appendSample := func(x int) (success bool) {
v := clientmodel.SampleValue(x)
ts := clientmodel.TimestampFromUnix(int64(x))
labelName := clientmodel.LabelName(x)
labelValue := clientmodel.LabelValue(x)
l := clientmodel.Metric{labelName: labelValue}
sample := &clientmodel.Sample{
Value: v,
Timestamp: ts,
Metric: l,
}
err := p.AppendSamples(clientmodel.Samples{sample})
if err != nil {
t.Error(err)
return
}
values, err := p.GetLabelValuesForLabelName(labelName)
if err != nil {
t.Error(err)
return
}
if len(values) != 1 {
t.Errorf("expected label values count of %d, got %d", 1, len(values))
return
}
fingerprints, err := p.GetFingerprintsForLabelMatchers(metric.LabelMatchers{{
Type: metric.Equal,
Name: labelName,
Value: labelValue,
}})
if err != nil {
t.Error(err)
return
}
if len(fingerprints) != 1 {
t.Errorf("expected fingerprint count of %d, got %d", 1, len(fingerprints))
return
}
return true
}
if err := quick.Check(appendSample, nil); err != nil {
t.Error(err)
}
}
func AppendSampleAsPureSingleEntityAppendTests(p metric.Persistence, t testing.TB) {
appendSample := func(x int) bool {
sample := &clientmodel.Sample{
Value: clientmodel.SampleValue(x),
Timestamp: clientmodel.TimestampFromUnix(int64(x)),
Metric: clientmodel.Metric{clientmodel.MetricNameLabel: "my_metric"},
}
err := p.AppendSamples(clientmodel.Samples{sample})
return err == nil
}
if err := quick.Check(appendSample, nil); err != nil {
t.Error(err)
}
}
func levelDBGetRangeValues(l *LevelDBPersistence, fp *clientmodel.Fingerprint, i metric.Interval) (samples metric.Values, err error) {
fpDto := &dto.Fingerprint{}
dumpFingerprint(fpDto, fp)
k := &dto.SampleKey{
Fingerprint: fpDto,
Timestamp: indexable.EncodeTime(i.OldestInclusive),
}
iterator, err := l.MetricSamples.NewIterator(true)
if err != nil {
panic(err)
}
defer iterator.Close()
for valid := iterator.Seek(k); valid; valid = iterator.Next() {
retrievedKey, err := extractSampleKey(iterator)
if err != nil {
return samples, err
}
if retrievedKey.FirstTimestamp.After(i.NewestInclusive) {
break
}
if !retrievedKey.Fingerprint.Equal(fp) {
break
}
retrievedValues := unmarshalValues(iterator.RawValue(), nil)
samples = append(samples, retrievedValues...)
}
return
}
type timeslice []clientmodel.Timestamp
func (t timeslice) Len() int {
return len(t)
}
func (t timeslice) Swap(i, j int) {
t[i], t[j] = t[j], t[i]
}
func (t timeslice) Less(i, j int) bool {
return t[i].Before(t[j])
}
func StochasticTests(persistenceMaker func() (metric.Persistence, test.Closer), t testing.TB) {
stochastic := func(x int) (success bool) {
p, closer := persistenceMaker()
defer closer.Close()
defer p.Close()
seed := rand.NewSource(int64(x))
random := rand.New(seed)
numberOfMetrics := random.Intn(stochasticMaximumVariance) + 1
numberOfSharedLabels := random.Intn(stochasticMaximumVariance)
numberOfUnsharedLabels := random.Intn(stochasticMaximumVariance)
numberOfSamples := random.Intn(stochasticMaximumVariance) + 2
numberOfRangeScans := random.Intn(stochasticMaximumVariance)
metricTimestamps := map[int]map[int64]bool{}
metricEarliestSample := map[int]int64{}
metricNewestSample := map[int]int64{}
for metricIndex := 0; metricIndex < numberOfMetrics; metricIndex++ {
sample := &clientmodel.Sample{
Metric: clientmodel.Metric{},
}
v := clientmodel.LabelValue(fmt.Sprintf("metric_index_%d", metricIndex))
sample.Metric[clientmodel.MetricNameLabel] = v
for sharedLabelIndex := 0; sharedLabelIndex < numberOfSharedLabels; sharedLabelIndex++ {
l := clientmodel.LabelName(fmt.Sprintf("shared_label_%d", sharedLabelIndex))
v := clientmodel.LabelValue(fmt.Sprintf("label_%d", sharedLabelIndex))
sample.Metric[l] = v
}
for unsharedLabelIndex := 0; unsharedLabelIndex < numberOfUnsharedLabels; unsharedLabelIndex++ {
l := clientmodel.LabelName(fmt.Sprintf("metric_index_%d_private_label_%d", metricIndex, unsharedLabelIndex))
v := clientmodel.LabelValue(fmt.Sprintf("private_label_%d", unsharedLabelIndex))
sample.Metric[l] = v
}
timestamps := map[int64]bool{}
metricTimestamps[metricIndex] = timestamps
var newestSample int64 = math.MinInt64
var oldestSample int64 = math.MaxInt64
var nextTimestamp func() int64
nextTimestamp = func() int64 {
var candidate int64
candidate = random.Int63n(math.MaxInt32 - 1)
if _, has := timestamps[candidate]; has {
// WART
candidate = nextTimestamp()
}
timestamps[candidate] = true
if candidate < oldestSample {
oldestSample = candidate
}
if candidate > newestSample {
newestSample = candidate
}
return candidate
}
// BUG(matt): Invariant of the in-memory database assumes this.
sortedTimestamps := timeslice{}
for sampleIndex := 0; sampleIndex < numberOfSamples; sampleIndex++ {
sortedTimestamps = append(sortedTimestamps, clientmodel.TimestampFromUnix(nextTimestamp()))
}
sort.Sort(sortedTimestamps)
for sampleIndex := 0; sampleIndex < numberOfSamples; sampleIndex++ {
sample.Timestamp = sortedTimestamps[sampleIndex]
sample.Value = clientmodel.SampleValue(sampleIndex)
err := p.AppendSamples(clientmodel.Samples{sample})
if err != nil {
t.Error(err)
return
}
}
metricEarliestSample[metricIndex] = oldestSample
metricNewestSample[metricIndex] = newestSample
for sharedLabelIndex := 0; sharedLabelIndex < numberOfSharedLabels; sharedLabelIndex++ {
matchers := metric.LabelMatchers{{
Type: metric.Equal,
Name: clientmodel.LabelName(fmt.Sprintf("shared_label_%d", sharedLabelIndex)),
Value: clientmodel.LabelValue(fmt.Sprintf("label_%d", sharedLabelIndex)),
}}
fingerprints, err := p.GetFingerprintsForLabelMatchers(matchers)
if err != nil {
t.Error(err)
return
}
if len(fingerprints) == 0 {
t.Errorf("expected fingerprint count of %d, got %d", 0, len(fingerprints))
return
}
}
}
for metricIndex := 0; metricIndex < numberOfMetrics; metricIndex++ {
for unsharedLabelIndex := 0; unsharedLabelIndex < numberOfUnsharedLabels; unsharedLabelIndex++ {
labelName := clientmodel.LabelName(fmt.Sprintf("metric_index_%d_private_label_%d", metricIndex, unsharedLabelIndex))
labelValue := clientmodel.LabelValue(fmt.Sprintf("private_label_%d", unsharedLabelIndex))
matchers := metric.LabelMatchers{{
Type: metric.Equal,
Name: labelName,
Value: labelValue,
}}
fingerprints, err := p.GetFingerprintsForLabelMatchers(matchers)
if err != nil {
t.Error(err)
return
}
if len(fingerprints) != 1 {
t.Errorf("expected fingerprint count of %d, got %d", 1, len(fingerprints))
return
}
}
m := clientmodel.Metric{}
m[clientmodel.MetricNameLabel] = clientmodel.LabelValue(fmt.Sprintf("metric_index_%d", metricIndex))
for i := 0; i < numberOfSharedLabels; i++ {
l := clientmodel.LabelName(fmt.Sprintf("shared_label_%d", i))
v := clientmodel.LabelValue(fmt.Sprintf("label_%d", i))
m[l] = v
}
for i := 0; i < numberOfUnsharedLabels; i++ {
l := clientmodel.LabelName(fmt.Sprintf("metric_index_%d_private_label_%d", metricIndex, i))
v := clientmodel.LabelValue(fmt.Sprintf("private_label_%d", i))
m[l] = v
}
for i := 0; i < numberOfRangeScans; i++ {
timestamps := metricTimestamps[metricIndex]
var first int64
var second int64
for {
firstCandidate := random.Int63n(int64(len(timestamps)))
secondCandidate := random.Int63n(int64(len(timestamps)))
smallest := int64(-1)
largest := int64(-1)
if firstCandidate == secondCandidate {
continue
} else if firstCandidate > secondCandidate {
largest = firstCandidate
smallest = secondCandidate
} else {
largest = secondCandidate
smallest = firstCandidate
}
j := int64(0)
for i := range timestamps {
if j == smallest {
first = i
} else if j == largest {
second = i
break
}
j++
}
break
}
begin := first
end := second
if second < first {
begin, end = second, first
}
interval := metric.Interval{
OldestInclusive: clientmodel.TimestampFromUnix(begin),
NewestInclusive: clientmodel.TimestampFromUnix(end),
}
samples := metric.Values{}
fp := &clientmodel.Fingerprint{}
fp.LoadFromMetric(m)
switch persistence := p.(type) {
case metric.View:
samples = persistence.GetRangeValues(fp, interval)
if len(samples) < 2 {
t.Fatalf("expected sample count greater than %d, got %d", 2, len(samples))
}
case *LevelDBPersistence:
var err error
samples, err = levelDBGetRangeValues(persistence, fp, interval)
if err != nil {
t.Fatal(err)
}
if len(samples) < 2 {
t.Fatalf("expected sample count greater than %d, got %d", 2, len(samples))
}
default:
t.Error("Unexpected type of metric.Persistence.")
}
}
}
return true
}
if err := quick.Check(stochastic, nil); err != nil {
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 testing.TB) {
persistenceMaker := func() (metric.Persistence, test.Closer) {
temporaryDirectory := test.NewTemporaryDirectory("test_leveldb_stochastic", t)
p, err := NewLevelDBPersistence(temporaryDirectory.Path())
if err != nil {
t.Errorf("Could not start up LevelDB: %q\n", err)
}
return p, temporaryDirectory
}
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 testing.TB) {
persistenceMaker := func() (metric.Persistence, test.Closer) {
return NewMemorySeriesStorage(MemorySeriesOptions{}), test.NilCloser
}
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

@ -1,131 +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 tiered
import (
"fmt"
"testing"
"time"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/utility/test"
)
var (
// ``hg clone https://code.google.com/p/go ; cd go ; hg log | tail -n 20``
usEastern, _ = time.LoadLocation("US/Eastern")
testInstant = clientmodel.TimestampFromTime(time.Date(1972, 7, 18, 19, 5, 45, 0, usEastern).In(time.UTC))
)
func testAppendSamples(p metric.Persistence, s *clientmodel.Sample, t testing.TB) {
err := p.AppendSamples(clientmodel.Samples{s})
if err != nil {
t.Fatal(err)
}
}
func buildLevelDBTestPersistencesMaker(name string, t testing.TB) func() (metric.Persistence, test.Closer) {
return func() (metric.Persistence, test.Closer) {
temporaryDirectory := test.NewTemporaryDirectory("get_value_at_time", t)
p, err := NewLevelDBPersistence(temporaryDirectory.Path())
if err != nil {
t.Errorf("Could not start up LevelDB: %q\n", err)
}
return p, temporaryDirectory
}
}
func buildLevelDBTestPersistence(name string, f func(p metric.Persistence, t testing.TB)) func(t testing.TB) {
return func(t testing.TB) {
temporaryDirectory := test.NewTemporaryDirectory(fmt.Sprintf("test_leveldb_%s", name), t)
defer temporaryDirectory.Close()
p, err := NewLevelDBPersistence(temporaryDirectory.Path())
if err != nil {
t.Errorf("Could not create LevelDB Metric Persistence: %q\n", err)
}
defer p.Close()
f(p, t)
}
}
func buildMemoryTestPersistence(f func(p metric.Persistence, t testing.TB)) func(t testing.TB) {
return func(t testing.TB) {
p := NewMemorySeriesStorage(MemorySeriesOptions{})
defer p.Close()
f(p, t)
}
}
type testTieredStorageCloser struct {
storage *TieredStorage
directory test.Closer
}
func (t *testTieredStorageCloser) Close() {
t.storage.Close()
t.directory.Close()
}
func NewTestTieredStorage(t testing.TB) (*TieredStorage, test.Closer) {
directory := test.NewTemporaryDirectory("test_tiered_storage", t)
storage, err := NewTieredStorage(2500, 1000, 5*time.Second, 0, directory.Path())
if err != nil {
if storage != nil {
storage.Close()
}
directory.Close()
t.Fatalf("Error creating storage: %s", err)
}
if storage == nil {
directory.Close()
t.Fatalf("storage == nil")
}
started := make(chan bool)
go storage.Serve(started)
<-started
closer := &testTieredStorageCloser{
storage: storage,
directory: directory,
}
return storage, closer
}
func labelMatchersFromLabelSet(l clientmodel.LabelSet) metric.LabelMatchers {
m := make(metric.LabelMatchers, 0, len(l))
for k, v := range l {
m = append(m, &metric.LabelMatcher{
Type: metric.Equal,
Name: k,
Value: v,
})
}
return m
}

View file

@ -1,813 +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 tiered
import (
"fmt"
"os"
"sort"
"sync"
"time"
"github.com/golang/glog"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/stats"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/raw/leveldb"
"github.com/prometheus/prometheus/utility"
)
// Constants for instrumentation.
const (
namespace = "prometheus"
operation = "operation"
success = "success"
failure = "failure"
result = "result"
appendSample = "append_sample"
appendSamples = "append_samples"
flushMemory = "flush_memory"
getLabelValuesForLabelName = "get_label_values_for_label_name"
getFingerprintsForLabelMatchers = "get_fingerprints_for_label_matchers"
getMetricForFingerprint = "get_metric_for_fingerprint"
hasIndexMetric = "has_index_metric"
refreshHighWatermarks = "refresh_high_watermarks"
renderView = "render_view"
queue = "queue"
appendToDisk = "append_to_disk"
viewGeneration = "view_generation"
)
var (
storageLatency = prometheus.NewSummaryVec(
prometheus.SummaryOpts{
Namespace: namespace,
Name: "metric_disk_latency_milliseconds",
Help: "Latency for metric disk operations (includes any storage drive even if it is not strictly a disk, e.g. SSD).",
Objectives: []float64{0.01, 0.05, 0.5, 0.90, 0.99},
},
[]string{operation, result},
)
storedSamplesCount = prometheus.NewCounter(prometheus.CounterOpts{
Namespace: namespace,
Name: "stored_samples_total",
Help: "The number of samples that have been stored.",
})
)
func init() {
prometheus.MustRegister(storageLatency)
prometheus.MustRegister(storedSamplesCount)
}
type chunk metric.Values
// TruncateBefore returns a subslice of the original such that extraneous
// samples in the collection that occur before the provided time are
// dropped. The original slice is not mutated. It works with the assumption
// that consumers of these values could want preceding values if none would
// exist prior to the defined time.
func (c chunk) TruncateBefore(t clientmodel.Timestamp) chunk {
index := sort.Search(len(c), func(i int) bool {
timestamp := c[i].Timestamp
return !timestamp.Before(t)
})
switch index {
case 0:
return c
case len(c):
return c[len(c)-1:]
default:
return c[index-1:]
}
}
type tieredStorageState uint
const (
tieredStorageStarting tieredStorageState = iota
tieredStorageServing
tieredStorageDraining
tieredStorageStopping
)
// Ignore timeseries in queries that are more stale than this limit.
const stalenessLimit = time.Minute * 5
// TieredStorage both persists samples and generates materialized views for
// queries.
type TieredStorage struct {
// mu is purely used for state transitions.
mu sync.RWMutex
// BUG(matt): This introduces a Law of Demeter violation. Ugh.
DiskStorage *LevelDBPersistence
appendToDiskQueue chan clientmodel.Samples
memoryArena *memorySeriesStorage
memoryTTL time.Duration
flushMemoryInterval time.Duration
ViewQueue chan viewJob
draining chan chan<- bool
state tieredStorageState
memorySemaphore chan bool
wmCache *watermarkCache
Indexer MetricIndexer
flushSema chan bool
dtoSampleKeys *dtoSampleKeyList
sampleKeys *sampleKeyList
queueLength *prometheus.GaugeVec
queueCapacity *prometheus.GaugeVec
}
// viewJob encapsulates a request to extract sample values from the datastore.
type viewJob struct {
builder metric.ViewRequestBuilder
output chan metric.View
abort chan bool
err chan error
stats *stats.TimerGroup
}
const (
tieredMemorySemaphores = 5
watermarkCacheLimit = 1024 * 1024
)
// NewTieredStorage returns a TieredStorage object ready to use.
func NewTieredStorage(
appendToDiskQueueDepth,
viewQueueDepth uint,
flushMemoryInterval time.Duration,
memoryTTL time.Duration,
rootDirectory string,
) (*TieredStorage, error) {
if isDir, _ := utility.IsDir(rootDirectory); !isDir {
if err := os.MkdirAll(rootDirectory, 0755); err != nil {
return nil, fmt.Errorf("could not find or create metrics directory %s: %s", rootDirectory, err)
}
}
diskStorage, err := NewLevelDBPersistence(rootDirectory)
if err != nil {
return nil, err
}
wmCache := &watermarkCache{
C: utility.NewSynchronizedCache(utility.NewLRUCache(watermarkCacheLimit)),
}
memOptions := MemorySeriesOptions{
WatermarkCache: wmCache,
}
s := &TieredStorage{
appendToDiskQueue: make(chan clientmodel.Samples, appendToDiskQueueDepth),
DiskStorage: diskStorage,
draining: make(chan chan<- bool),
flushMemoryInterval: flushMemoryInterval,
memoryArena: NewMemorySeriesStorage(memOptions),
memoryTTL: memoryTTL,
ViewQueue: make(chan viewJob, viewQueueDepth),
memorySemaphore: make(chan bool, tieredMemorySemaphores),
wmCache: wmCache,
flushSema: make(chan bool, 1),
dtoSampleKeys: newDtoSampleKeyList(10),
sampleKeys: newSampleKeyList(10),
queueLength: prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: namespace,
Name: "storage_queue_length",
Help: "The number of items in the storage queues.",
},
[]string{queue},
),
queueCapacity: prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: namespace,
Name: "storage_queue_capacity",
Help: "The capacity of the storage queues.",
},
[]string{queue},
),
}
s.queueCapacity.WithLabelValues(appendToDisk).Set(float64(appendToDiskQueueDepth))
s.queueCapacity.WithLabelValues(viewGeneration).Set(float64(viewQueueDepth))
for i := 0; i < tieredMemorySemaphores; i++ {
s.memorySemaphore <- true
}
return s, nil
}
// AppendSamples enqueues Samples for storage.
func (t *TieredStorage) AppendSamples(samples clientmodel.Samples) (err error) {
t.mu.RLock()
defer t.mu.RUnlock()
if t.state != tieredStorageServing {
return fmt.Errorf("storage is not serving")
}
t.memoryArena.AppendSamples(samples)
storedSamplesCount.Add(float64(len(samples)))
return
}
// Drain stops the storage subsystem, flushing all pending operations.
func (t *TieredStorage) Drain(drained chan<- bool) {
t.mu.Lock()
defer t.mu.Unlock()
t.drain(drained)
}
func (t *TieredStorage) drain(drained chan<- bool) {
if t.state >= tieredStorageDraining {
panic("Illegal State: Supplemental drain requested.")
}
t.state = tieredStorageDraining
glog.Info("Triggering drain...")
t.draining <- (drained)
}
// NewViewRequestBuilder furnishes a ViewRequestBuilder for remarking what types
// of queries to perform.
func (t *TieredStorage) NewViewRequestBuilder() metric.ViewRequestBuilder {
return &viewRequestBuilder{storage: t}
}
// makeView materializes a View according to a ViewRequestBuilder, subject to a
// timeout.
func (t *TieredStorage) makeView(builder metric.ViewRequestBuilder, deadline time.Duration, queryStats *stats.TimerGroup) (metric.View, error) {
t.mu.RLock()
defer t.mu.RUnlock()
if t.state != tieredStorageServing {
return nil, fmt.Errorf("storage is not serving")
}
// The result channel needs a one-element buffer in case we have timed
// out in makeView, but the view rendering still completes afterwards
// and writes to the channel.
result := make(chan metric.View, 1)
// The abort channel needs a one-element buffer in case the view
// rendering has already exited and doesn't consume from the channel
// anymore.
abortChan := make(chan bool, 1)
errChan := make(chan error)
queryStats.GetTimer(stats.ViewQueueTime).Start()
t.ViewQueue <- viewJob{
builder: builder,
output: result,
abort: abortChan,
err: errChan,
stats: queryStats,
}
select {
case view := <-result:
return view, nil
case err := <-errChan:
return nil, err
case <-time.After(deadline):
abortChan <- true
return nil, fmt.Errorf("fetching query data timed out after %s", deadline)
}
}
// Serve starts serving requests.
func (t *TieredStorage) Serve(started chan<- bool) {
t.mu.Lock()
if t.state != tieredStorageStarting {
panic("Illegal State: Attempted to restart TieredStorage.")
}
t.state = tieredStorageServing
t.mu.Unlock()
flushMemoryTicker := time.NewTicker(t.flushMemoryInterval)
defer flushMemoryTicker.Stop()
started <- true
for {
select {
case <-flushMemoryTicker.C:
select {
case t.flushSema <- true:
go func() {
t.flushMemory(t.memoryTTL)
<-t.flushSema
}()
default:
glog.Warning("Backlogging on flush...")
}
case viewRequest := <-t.ViewQueue:
<-t.memorySemaphore
viewRequest.stats.GetTimer(stats.ViewQueueTime).Stop()
go t.renderView(viewRequest)
case drainingDone := <-t.draining:
t.Flush()
drainingDone <- true
return
}
}
}
// Flush flushes all samples to disk.
func (t *TieredStorage) Flush() {
t.flushSema <- true
t.flushMemory(0)
<-t.flushSema
}
func (t *TieredStorage) flushMemory(ttl time.Duration) {
flushOlderThan := clientmodel.Now().Add(-1 * ttl)
glog.Info("Flushing samples to disk...")
t.memoryArena.Flush(flushOlderThan, t.appendToDiskQueue)
queueLength := len(t.appendToDiskQueue)
if queueLength > 0 {
samples := clientmodel.Samples{}
for i := 0; i < queueLength; i++ {
chunk := <-t.appendToDiskQueue
samples = append(samples, chunk...)
}
glog.Infof("Writing %d samples...", len(samples))
t.DiskStorage.AppendSamples(samples)
}
t.memoryArena.Evict(flushOlderThan)
glog.Info("Done flushing.")
}
// Close stops serving, flushes all pending operations, and frees all resources.
func (t *TieredStorage) Close() {
t.mu.Lock()
defer t.mu.Unlock()
t.close()
}
func (t *TieredStorage) close() {
if t.state == tieredStorageStopping {
panic("Illegal State: Attempted to restop TieredStorage.")
}
drained := make(chan bool)
t.drain(drained)
<-drained
t.memoryArena.Close()
t.DiskStorage.Close()
// BUG(matt): There is a probability that pending items may hang here
// and not get flushed.
close(t.appendToDiskQueue)
close(t.ViewQueue)
t.wmCache.Clear()
t.dtoSampleKeys.Close()
t.sampleKeys.Close()
t.state = tieredStorageStopping
}
func (t *TieredStorage) seriesTooOld(f *clientmodel.Fingerprint, i clientmodel.Timestamp) (bool, error) {
// BUG(julius): Make this configurable by query layer.
i = i.Add(-stalenessLimit)
wm, cacheHit, _ := t.wmCache.Get(f)
if !cacheHit {
if t.memoryArena.HasFingerprint(f) {
samples := t.memoryArena.CloneSamples(f)
if len(samples) > 0 {
newest := samples[len(samples)-1].Timestamp
t.wmCache.Put(f, &watermarks{High: newest})
return newest.Before(i), nil
}
}
highTime, diskHit, err := t.DiskStorage.MetricHighWatermarks.Get(f)
if err != nil {
return false, err
}
if diskHit {
t.wmCache.Put(f, &watermarks{High: highTime})
return highTime.Before(i), nil
}
t.wmCache.Put(f, &watermarks{})
return true, nil
}
return wm.High.Before(i), nil
}
func (t *TieredStorage) renderView(viewJob viewJob) {
// Telemetry.
var err error
begin := time.Now()
defer func() {
t.memorySemaphore <- true
if err == nil {
storageLatency.With(
prometheus.Labels{operation: renderView, result: success},
).Observe(
float64(time.Since(begin) / time.Millisecond),
)
} else {
storageLatency.With(
prometheus.Labels{operation: renderView, result: failure},
).Observe(
float64(time.Since(begin) / time.Millisecond),
)
}
}()
view := newView()
var iterator leveldb.Iterator
diskPresent := true
firstBlock, _ := t.sampleKeys.Get()
defer t.sampleKeys.Give(firstBlock)
lastBlock, _ := t.sampleKeys.Get()
defer t.sampleKeys.Give(lastBlock)
sampleKeyDto, _ := t.dtoSampleKeys.Get()
defer t.dtoSampleKeys.Give(sampleKeyDto)
defer func() {
// Give back all ops not yet popped.
for viewJob.builder.HasOp() {
giveBackOp(viewJob.builder.PopOp())
}
}()
extractionTimer := viewJob.stats.GetTimer(stats.ViewDataExtractionTime).Start()
for viewJob.builder.HasOp() {
op := viewJob.builder.PopOp()
defer giveBackOp(op)
fp := op.Fingerprint()
old, err := t.seriesTooOld(fp, op.CurrentTime())
if err != nil {
glog.Errorf("Error getting watermark from cache for %s: %s", fp, err)
continue
}
if old {
continue
}
memValues := t.memoryArena.CloneSamples(fp)
for !op.Consumed() {
// Abort the view rendering if the caller (makeView) has timed out.
if len(viewJob.abort) > 0 {
return
}
// Load data value chunk(s) around the current time.
targetTime := op.CurrentTime()
currentChunk := chunk{}
// If we aimed before the oldest value in memory, load more data from disk.
if (len(memValues) == 0 || memValues.FirstTimeAfter(targetTime)) && diskPresent {
if iterator == nil {
// Get a single iterator that will be used for all data extraction
// below.
iterator, _ = t.DiskStorage.MetricSamples.NewIterator(true)
defer iterator.Close()
if diskPresent = iterator.SeekToLast(); diskPresent {
if err := iterator.Key(sampleKeyDto); err != nil {
panic(err)
}
lastBlock.Load(sampleKeyDto)
if !iterator.SeekToFirst() {
diskPresent = false
} else {
if err := iterator.Key(sampleKeyDto); err != nil {
panic(err)
}
firstBlock.Load(sampleKeyDto)
}
}
}
if diskPresent {
diskTimer := viewJob.stats.GetTimer(stats.ViewDiskExtractionTime).Start()
diskValues, expired := t.loadChunkAroundTime(
iterator,
fp,
targetTime,
firstBlock,
lastBlock,
)
if expired {
diskPresent = false
}
diskTimer.Stop()
// If we aimed past the newest value on disk,
// combine it with the next value from memory.
if len(diskValues) == 0 {
currentChunk = chunk(memValues)
} else {
if len(memValues) > 0 && diskValues.LastTimeBefore(targetTime) {
latestDiskValue := diskValues[len(diskValues)-1:]
currentChunk = append(chunk(latestDiskValue), chunk(memValues)...)
} else {
currentChunk = chunk(diskValues)
}
}
} else {
currentChunk = chunk(memValues)
}
} else {
currentChunk = chunk(memValues)
}
// There's no data at all for this fingerprint, so stop processing.
if len(currentChunk) == 0 {
break
}
currentChunk = currentChunk.TruncateBefore(targetTime)
lastChunkTime := currentChunk[len(currentChunk)-1].Timestamp
if lastChunkTime.After(targetTime) {
targetTime = lastChunkTime
}
if op.CurrentTime().After(targetTime) {
break
}
// Extract all needed data from the current chunk and append the
// extracted samples to the materialized view.
for !op.Consumed() && !op.CurrentTime().After(targetTime) {
view.appendSamples(fp, op.ExtractSamples(metric.Values(currentChunk)))
}
}
}
extractionTimer.Stop()
viewJob.output <- view
return
}
func (t *TieredStorage) loadChunkAroundTime(
iterator leveldb.Iterator,
fingerprint *clientmodel.Fingerprint,
ts clientmodel.Timestamp,
firstBlock,
lastBlock *SampleKey,
) (chunk metric.Values, expired bool) {
if fingerprint.Less(firstBlock.Fingerprint) {
return nil, false
}
if lastBlock.Fingerprint.Less(fingerprint) {
return nil, true
}
seekingKey, _ := t.sampleKeys.Get()
defer t.sampleKeys.Give(seekingKey)
seekingKey.Fingerprint = fingerprint
if fingerprint.Equal(firstBlock.Fingerprint) && ts.Before(firstBlock.FirstTimestamp) {
seekingKey.FirstTimestamp = firstBlock.FirstTimestamp
} else if fingerprint.Equal(lastBlock.Fingerprint) && ts.After(lastBlock.FirstTimestamp) {
seekingKey.FirstTimestamp = lastBlock.FirstTimestamp
} else {
seekingKey.FirstTimestamp = ts
}
dto, _ := t.dtoSampleKeys.Get()
defer t.dtoSampleKeys.Give(dto)
seekingKey.Dump(dto)
if !iterator.Seek(dto) {
return chunk, true
}
var foundValues metric.Values
if err := iterator.Key(dto); err != nil {
panic(err)
}
seekingKey.Load(dto)
if seekingKey.Fingerprint.Equal(fingerprint) {
// 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.
if !seekingKey.MayContain(ts) {
postValues := unmarshalValues(iterator.RawValue(), nil)
if !seekingKey.Equal(firstBlock) {
if !iterator.Previous() {
panic("This should never return false.")
}
if err := iterator.Key(dto); err != nil {
panic(err)
}
seekingKey.Load(dto)
if !seekingKey.Fingerprint.Equal(fingerprint) {
return postValues, false
}
foundValues = unmarshalValues(iterator.RawValue(), nil)
foundValues = append(foundValues, postValues...)
return foundValues, false
}
}
foundValues = unmarshalValues(iterator.RawValue(), nil)
return foundValues, false
}
if fingerprint.Less(seekingKey.Fingerprint) {
if !seekingKey.Equal(firstBlock) {
if !iterator.Previous() {
panic("This should never return false.")
}
if err := iterator.Key(dto); err != nil {
panic(err)
}
seekingKey.Load(dto)
if !seekingKey.Fingerprint.Equal(fingerprint) {
return nil, false
}
foundValues = unmarshalValues(iterator.RawValue(), nil)
return foundValues, false
}
}
panic("illegal state: violated sort invariant")
}
// GetAllValuesForLabel gets all label values that are associated with the
// provided label name.
func (t *TieredStorage) GetAllValuesForLabel(labelName clientmodel.LabelName) (clientmodel.LabelValues, error) {
t.mu.RLock()
defer t.mu.RUnlock()
if t.state != tieredStorageServing {
panic("Illegal State: Attempted to query non-running TieredStorage.")
}
diskValues, err := t.DiskStorage.GetAllValuesForLabel(labelName)
if err != nil {
return nil, err
}
memoryValues, err := t.memoryArena.GetAllValuesForLabel(labelName)
if err != nil {
return nil, err
}
valueSet := map[clientmodel.LabelValue]bool{}
values := clientmodel.LabelValues{}
for _, value := range append(diskValues, memoryValues...) {
if !valueSet[value] {
values = append(values, value)
valueSet[value] = true
}
}
return values, nil
}
// GetFingerprintsForLabelMatchers gets all of the metric fingerprints that are
// associated with the provided label matchers.
func (t *TieredStorage) GetFingerprintsForLabelMatchers(matchers metric.LabelMatchers) (clientmodel.Fingerprints, error) {
t.mu.RLock()
defer t.mu.RUnlock()
if t.state != tieredStorageServing {
panic("Illegal State: Attempted to query non-running TieredStorage.")
}
memFingerprints, err := t.memoryArena.GetFingerprintsForLabelMatchers(matchers)
if err != nil {
return nil, err
}
diskFingerprints, err := t.DiskStorage.GetFingerprintsForLabelMatchers(matchers)
if err != nil {
return nil, err
}
fingerprintSet := map[clientmodel.Fingerprint]bool{}
for _, fingerprint := range append(memFingerprints, diskFingerprints...) {
fingerprintSet[*fingerprint] = true
}
fingerprints := clientmodel.Fingerprints{}
for fingerprint := range fingerprintSet {
fpCopy := fingerprint
fingerprints = append(fingerprints, &fpCopy)
}
return fingerprints, nil
}
// Get all of the label values that are associated with a given label name.
func (t *TieredStorage) GetLabelValuesForLabelName(clientmodel.LabelName) (clientmodel.LabelValues, error) {
// TODO(julius): Implement this or decide what to do with this
// Persistence interface method. It's currently unused on the
// TieredStorage, but used on the LevelDBPersistence and the
// memorySeriesStorage.
panic("not implemented")
}
// GetMetricForFingerprint gets the metric associated with the provided
// fingerprint.
func (t *TieredStorage) GetMetricForFingerprint(f *clientmodel.Fingerprint) (clientmodel.Metric, error) {
t.mu.RLock()
defer t.mu.RUnlock()
if t.state != tieredStorageServing {
panic("Illegal State: Attempted to query non-running TieredStorage.")
}
m, err := t.memoryArena.GetMetricForFingerprint(f)
if err != nil {
return nil, err
}
if m == nil {
m, err = t.DiskStorage.GetMetricForFingerprint(f)
t.memoryArena.CreateEmptySeries(m)
}
return m, err
}
// Describe implements prometheus.Collector.
func (t *TieredStorage) Describe(ch chan<- *prometheus.Desc) {
t.queueLength.Describe(ch)
t.queueCapacity.Describe(ch)
}
// Collect implements prometheus.Collector.
func (t *TieredStorage) Collect(ch chan<- prometheus.Metric) {
t.queueLength.WithLabelValues(appendToDisk).Set(float64(len(t.appendToDiskQueue)))
t.queueLength.WithLabelValues(viewGeneration).Set(float64(len(t.ViewQueue)))
t.queueLength.Collect(ch)
t.queueCapacity.Collect(ch)
}

File diff suppressed because it is too large Load diff

View file

@ -1,100 +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 tiered
import (
"container/heap"
"time"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/stats"
"github.com/prometheus/prometheus/storage/metric"
)
// viewRequestBuilder contains the various requests for data.
type viewRequestBuilder struct {
storage *TieredStorage
operations ops
}
var getValuesAtTimes = newValueAtTimeList(10 * 1024)
// GetMetricAtTime implements ViewRequestBuilder.
func (v *viewRequestBuilder) GetMetricAtTime(fp *clientmodel.Fingerprint, time clientmodel.Timestamp) {
heap.Push(&v.operations, getValuesAtTimes.Get(fp, time))
}
var getValuesAtIntervals = newValueAtIntervalList(10 * 1024)
// GetMetricAtInterval implements ViewRequestBuilder.
func (v *viewRequestBuilder) GetMetricAtInterval(fp *clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval time.Duration) {
heap.Push(&v.operations, getValuesAtIntervals.Get(fp, from, through, interval))
}
var getValuesAlongRanges = newValueAlongRangeList(10 * 1024)
// GetMetricRange implements ViewRequestBuilder.
func (v *viewRequestBuilder) GetMetricRange(fp *clientmodel.Fingerprint, from, through clientmodel.Timestamp) {
heap.Push(&v.operations, getValuesAlongRanges.Get(fp, from, through))
}
var getValuesAtIntervalAlongRanges = newValueAtIntervalAlongRangeList(10 * 1024)
// GetMetricRangeAtInterval implements ViewRequestBuilder.
func (v *viewRequestBuilder) GetMetricRangeAtInterval(fp *clientmodel.Fingerprint, from, through clientmodel.Timestamp, interval, rangeDuration time.Duration) {
heap.Push(&v.operations, getValuesAtIntervalAlongRanges.Get(fp, from, through, interval, rangeDuration))
}
// Execute implements ViewRequestBuilder.
func (v *viewRequestBuilder) Execute(deadline time.Duration, queryStats *stats.TimerGroup) (metric.View, error) {
return v.storage.makeView(v, deadline, queryStats)
}
// PopOp implements ViewRequestBuilder.
func (v *viewRequestBuilder) PopOp() metric.Op {
return heap.Pop(&v.operations).(metric.Op)
}
// HasOp implements ViewRequestBuilder.
func (v *viewRequestBuilder) HasOp() bool {
return v.operations.Len() > 0
}
type view struct {
*memorySeriesStorage
}
func (v view) appendSamples(fingerprint *clientmodel.Fingerprint, samples metric.Values) {
v.memorySeriesStorage.appendSamplesWithoutIndexing(fingerprint, samples)
}
func newView() view {
return view{NewMemorySeriesStorage(MemorySeriesOptions{})}
}
func giveBackOp(op interface{}) bool {
switch v := op.(type) {
case *getValuesAtTimeOp:
return getValuesAtTimes.Give(v)
case *getValuesAtIntervalOp:
return getValuesAtIntervals.Give(v)
case *getValuesAlongRangeOp:
return getValuesAlongRanges.Give(v)
case *getValueRangeAtIntervalOp:
return getValuesAtIntervalAlongRanges.Give(v)
default:
panic("unrecognized operation")
}
}

View file

@ -1,194 +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 tiered
import (
"testing"
"time"
clientmodel "github.com/prometheus/client_golang/model"
)
func testBuilder(t testing.TB) {
type atTime struct {
fingerprint string
time clientmodel.Timestamp
}
type atInterval struct {
fingerprint string
from clientmodel.Timestamp
through clientmodel.Timestamp
interval time.Duration
}
type atRange struct {
fingerprint string
from clientmodel.Timestamp
through clientmodel.Timestamp
}
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: clientmodel.TimestampFromUnix(100),
},
{
fingerprint: "0000000000000000000-a-4-a",
time: clientmodel.TimestampFromUnix(100),
},
},
},
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: clientmodel.TimestampFromUnix(100),
},
{
fingerprint: "1111-a-4-a",
time: clientmodel.TimestampFromUnix(200),
},
{
fingerprint: "0-a-4-a",
time: clientmodel.TimestampFromUnix(100),
},
{
fingerprint: "0-a-4-a",
time: clientmodel.TimestampFromUnix(0),
},
},
},
out: out{
{
fingerprint: "00000000000000000000-a-4-a",
},
{
fingerprint: "00000000000000000000-a-4-a",
},
{
fingerprint: "00000000000000001111-a-4-a",
},
{
fingerprint: "00000000000000001111-a-4-a",
},
},
},
// Ensure grouping of operations
{
in: in{
atTimes: []atTime{
{
fingerprint: "1111-a-4-a",
time: clientmodel.TimestampFromUnix(100),
},
},
atRanges: []atRange{
{
fingerprint: "1111-a-4-a",
from: clientmodel.TimestampFromUnix(100),
through: clientmodel.TimestampFromUnix(1000),
},
{
fingerprint: "1111-a-4-a",
from: clientmodel.TimestampFromUnix(100),
through: clientmodel.TimestampFromUnix(9000),
},
},
},
out: out{
{
fingerprint: "00000000000000001111-a-4-a",
},
{
fingerprint: "00000000000000001111-a-4-a",
},
{
fingerprint: "00000000000000001111-a-4-a",
},
},
},
}
for i, scenario := range scenarios {
builder := &viewRequestBuilder{}
for _, atTime := range scenario.in.atTimes {
fingerprint := &clientmodel.Fingerprint{}
fingerprint.LoadFromString(atTime.fingerprint)
builder.GetMetricAtTime(fingerprint, atTime.time)
}
for _, atInterval := range scenario.in.atIntervals {
fingerprint := &clientmodel.Fingerprint{}
fingerprint.LoadFromString(atInterval.fingerprint)
builder.GetMetricAtInterval(fingerprint, atInterval.from, atInterval.through, atInterval.interval)
}
for _, atRange := range scenario.in.atRanges {
fingerprint := &clientmodel.Fingerprint{}
fingerprint.LoadFromString(atRange.fingerprint)
builder.GetMetricRange(fingerprint, atRange.from, atRange.through)
}
for j, job := range scenario.out {
got := builder.PopOp()
if got.Fingerprint().String() != job.fingerprint {
t.Errorf("%d.%d. expected fingerprint %s, got %s", i, j, job.fingerprint, got.Fingerprint())
}
}
if builder.HasOp() {
t.Error("Expected builder to have no scan jobs left.")
}
}
}
func TestBuilder(t *testing.T) {
testBuilder(t)
}
func BenchmarkBuilder(b *testing.B) {
for i := 0; i < b.N; i++ {
testBuilder(b)
}
}

View file

@ -1,189 +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 tiered
import (
"code.google.com/p/goprotobuf/proto"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/raw"
"github.com/prometheus/prometheus/storage/raw/leveldb"
"github.com/prometheus/prometheus/utility"
dto "github.com/prometheus/prometheus/model/generated"
)
type watermarks struct {
High clientmodel.Timestamp
}
func (w *watermarks) load(d *dto.MetricHighWatermark) {
w.High = clientmodel.TimestampFromUnix(d.GetTimestamp())
}
func (w *watermarks) dump(d *dto.MetricHighWatermark) {
d.Reset()
d.Timestamp = proto.Int64(w.High.Unix())
}
// A FingerprintHighWatermarkMapping is used for batch updates of many high
// watermarks in a database.
type FingerprintHighWatermarkMapping map[clientmodel.Fingerprint]clientmodel.Timestamp
// HighWatermarker models a high-watermark database.
type HighWatermarker interface {
raw.Database
raw.ForEacher
raw.Pruner
UpdateBatch(FingerprintHighWatermarkMapping) error
Get(*clientmodel.Fingerprint) (t clientmodel.Timestamp, ok bool, err error)
}
// LevelDBHighWatermarker is an implementation of HighWatermarker backed by
// leveldb.
type LevelDBHighWatermarker struct {
*leveldb.LevelDBPersistence
}
// Get implements HighWatermarker.
func (w *LevelDBHighWatermarker) Get(f *clientmodel.Fingerprint) (t clientmodel.Timestamp, ok bool, err error) {
k := &dto.Fingerprint{}
dumpFingerprint(k, f)
v := &dto.MetricHighWatermark{}
ok, err = w.LevelDBPersistence.Get(k, v)
if err != nil {
return t, ok, err
}
if !ok {
return clientmodel.TimestampFromUnix(0), ok, nil
}
t = clientmodel.TimestampFromUnix(v.GetTimestamp())
return t, true, nil
}
// UpdateBatch implements HighWatermarker.
func (w *LevelDBHighWatermarker) UpdateBatch(m FingerprintHighWatermarkMapping) error {
batch := leveldb.NewBatch()
defer batch.Close()
for fp, t := range m {
existing, present, err := w.Get(&fp)
if err != nil {
return err
}
k := &dto.Fingerprint{}
dumpFingerprint(k, &fp)
v := &dto.MetricHighWatermark{}
if !present {
v.Timestamp = proto.Int64(t.Unix())
batch.Put(k, v)
continue
}
// BUG(matt): Replace this with watermark management.
if t.After(existing) {
v.Timestamp = proto.Int64(t.Unix())
batch.Put(k, v)
}
}
return w.LevelDBPersistence.Commit(batch)
}
// NewLevelDBHighWatermarker returns a LevelDBHighWatermarker ready to use.
func NewLevelDBHighWatermarker(o leveldb.LevelDBOptions) (*LevelDBHighWatermarker, error) {
s, err := leveldb.NewLevelDBPersistence(o)
if err != nil {
return nil, err
}
return &LevelDBHighWatermarker{
LevelDBPersistence: s,
}, nil
}
// CurationRemarker models a curation remarker database.
type CurationRemarker interface {
raw.Database
raw.Pruner
Update(*curationKey, clientmodel.Timestamp) error
Get(*curationKey) (t clientmodel.Timestamp, ok bool, err error)
}
// LevelDBCurationRemarker is an implementation of CurationRemarker backed by
// leveldb.
type LevelDBCurationRemarker struct {
*leveldb.LevelDBPersistence
}
// Get implements CurationRemarker.
func (w *LevelDBCurationRemarker) Get(c *curationKey) (t clientmodel.Timestamp, ok bool, err error) {
k := &dto.CurationKey{}
c.dump(k)
v := &dto.CurationValue{}
ok, err = w.LevelDBPersistence.Get(k, v)
if err != nil || !ok {
return clientmodel.TimestampFromUnix(0), ok, err
}
return clientmodel.TimestampFromUnix(v.GetLastCompletionTimestamp()), true, nil
}
// Update implements CurationRemarker.
func (w *LevelDBCurationRemarker) Update(pair *curationKey, t clientmodel.Timestamp) error {
k := &dto.CurationKey{}
pair.dump(k)
return w.LevelDBPersistence.Put(k, &dto.CurationValue{
LastCompletionTimestamp: proto.Int64(t.Unix()),
})
}
// NewLevelDBCurationRemarker returns a LevelDBCurationRemarker ready to use.
func NewLevelDBCurationRemarker(o leveldb.LevelDBOptions) (*LevelDBCurationRemarker, error) {
s, err := leveldb.NewLevelDBPersistence(o)
if err != nil {
return nil, err
}
return &LevelDBCurationRemarker{
LevelDBPersistence: s,
}, nil
}
type watermarkCache struct {
C utility.Cache
}
func (c *watermarkCache) Get(f *clientmodel.Fingerprint) (*watermarks, bool, error) {
v, ok, err := c.C.Get(*f)
if ok {
return v.(*watermarks), ok, err
}
return nil, ok, err
}
func (c *watermarkCache) Put(f *clientmodel.Fingerprint, v *watermarks) (bool, error) {
return c.C.Put(*f, v)
}
func (c *watermarkCache) Clear() (bool, error) {
return c.C.Clear()
}

View file

@ -1,89 +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 raw
import (
"code.google.com/p/goprotobuf/proto"
"github.com/prometheus/prometheus/storage"
)
// Database provides a few very basic methods to manage a database and inquire
// its state.
type Database interface {
// Close reaps all of the underlying system resources associated with
// this database. For databases that don't need that kind of clean-up,
// it is implemented as a no-op (so that clients don't need to reason
// and always call Close 'just in case').
Close() error
// State reports the state of the database as a DatabaseState object.
State() *DatabaseState
// Size returns the total size of the database in bytes. The number may
// be an approximation, depending on the underlying database type.
Size() (uint64, error)
}
// ForEacher is implemented by databases that can be iterated through.
type ForEacher interface {
// ForEach is responsible for iterating through all records in the
// database until one of the following conditions are met:
//
// 1.) A system anomaly in the database scan.
// 2.) The last record in the database is reached.
// 3.) A FilterResult of STOP is emitted by the Filter.
//
// Decoding errors for an entity cause that entity to be skipped.
ForEach(storage.RecordDecoder, storage.RecordFilter, storage.RecordOperator) (scannedEntireCorpus bool, err error)
}
// Pruner is implemented by a database that can be pruned in some way.
type Pruner interface {
Prune()
}
// Persistence models a key-value store for bytes that supports various
// additional operations.
type Persistence interface {
Database
ForEacher
// Has informs the user whether a given key exists in the database.
Has(key proto.Message) (bool, error)
// Get populates 'value' with the value of 'key', if present, in which
// case 'present' is returned as true.
Get(key, value proto.Message) (present bool, err error)
// Drop removes the key from the database.
Drop(key proto.Message) error
// Put sets the key to a given value.
Put(key, value proto.Message) error
// PutRaw sets the key to a given raw bytes value.
PutRaw(key proto.Message, value []byte) error
// Commit applies the Batch operations to the database.
Commit(Batch) error
}
// Batch models a pool of mutations for the database that can be committed
// en masse. The interface implies no protocol around the atomicity of
// effectuation.
type Batch interface {
// Close reaps all of the underlying system resources associated with
// this batch mutation.
Close()
// Put follows the same protocol as Persistence.Put.
Put(key, value proto.Message)
// PutRaw follows the same protocol as Persistence.PutRaw.
PutRaw(key proto.Message, value []byte)
// Drop follows the same protocol as Persistence.Drop.
Drop(key proto.Message)
}

View file

@ -1,88 +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"
"code.google.com/p/goprotobuf/proto"
"github.com/jmhodges/levigo"
)
type batch struct {
batch *levigo.WriteBatch
drops uint32
puts uint32
}
// NewBatch returns a fully allocated batch object.
func NewBatch() *batch {
return &batch{
batch: levigo.NewWriteBatch(),
}
}
func (b *batch) Drop(key proto.Message) {
buf, _ := buffers.Get()
defer buffers.Give(buf)
if err := buf.Marshal(key); err != nil {
panic(err)
}
b.batch.Delete(buf.Bytes())
b.drops++
}
func (b *batch) Put(key, value proto.Message) {
keyBuf, _ := buffers.Get()
defer buffers.Give(keyBuf)
if err := keyBuf.Marshal(key); err != nil {
panic(err)
}
valBuf, _ := buffers.Get()
defer buffers.Give(valBuf)
if err := valBuf.Marshal(value); err != nil {
panic(err)
}
b.batch.Put(keyBuf.Bytes(), valBuf.Bytes())
b.puts++
}
func (b *batch) PutRaw(key proto.Message, value []byte) {
keyBuf, _ := buffers.Get()
defer buffers.Give(keyBuf)
if err := keyBuf.Marshal(key); err != nil {
panic(err)
}
b.batch.Put(keyBuf.Bytes(), value)
b.puts++
}
func (b *batch) Close() {
b.batch.Close()
}
func (b *batch) String() string {
return fmt.Sprintf("LevelDB batch with %d puts and %d drops.", b.puts, b.drops)
}

View file

@ -1,46 +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/utility"
"code.google.com/p/goprotobuf/proto"
)
var buffers = newBufferList(50)
type bufferList struct {
l utility.FreeList
}
func (l *bufferList) Get() (*proto.Buffer, bool) {
if v, ok := l.l.Get(); ok {
return v.(*proto.Buffer), ok
}
return proto.NewBuffer(make([]byte, 0, 4096)), false
}
func (l *bufferList) Give(v *proto.Buffer) bool {
v.Reset()
return l.l.Give(v)
}
func newBufferList(cap int) *bufferList {
return &bufferList{
l: utility.NewFreeList(cap),
}
}

View file

@ -1,24 +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 (
"testing"
"github.com/prometheus/prometheus/storage/raw"
)
func TestInterfaceAdherence(t *testing.T) {
var _ raw.Persistence = &LevelDBPersistence{}
}

View file

@ -1,41 +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"
)
// TODO: Evaluate whether to use coding.Encoder for the key and values instead
// raw bytes for consistency reasons.
// Iterator provides method to iterate through a leveldb.
type Iterator interface {
Error() error
Valid() bool
SeekToFirst() bool
SeekToLast() bool
Seek(proto.Message) bool
Next() bool
Previous() bool
Key(proto.Message) error
RawValue() []byte
Close() error
rawKey() []byte
}

View file

@ -1,519 +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"
"time"
"code.google.com/p/goprotobuf/proto"
"github.com/jmhodges/levigo"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/storage/raw"
)
// LevelDBPersistence is a disk-backed sorted key-value store. It implements the
// interfaces raw.Database, raw.ForEacher, raw.Pruner, raw.Persistence.
type LevelDBPersistence struct {
path string
name string
purpose string
cache *levigo.Cache
filterPolicy *levigo.FilterPolicy
options *levigo.Options
storage *levigo.DB
readOptions *levigo.ReadOptions
writeOptions *levigo.WriteOptions
}
// levigoIterator wraps the LevelDB resources in a convenient manner for uniform
// resource access and closing through the raw.Iterator protocol.
type levigoIterator struct {
// iterator is the receiver of most proxied operation calls.
iterator *levigo.Iterator
// readOptions is only set if the iterator is a snapshot of an
// underlying database. This signals that it needs to be explicitly
// reaped upon the end of this iterator's life.
readOptions *levigo.ReadOptions
// snapshot is only set if the iterator is a snapshot of an underlying
// database. This signals that it needs to be explicitly reaped upon
// the end of this this iterator's life.
snapshot *levigo.Snapshot
// storage is only set if the iterator is a snapshot of an underlying
// database. This signals that it needs to be explicitly reaped upon
// the end of this this iterator's life. The snapshot must be freed in
// the context of an actual database.
storage *levigo.DB
// closed indicates whether the iterator has been closed before.
closed bool
// valid indicates whether the iterator may be used. If a LevelDB
// iterator ever becomes invalid, it must be disposed of and cannot be
// reused.
valid bool
// creationTime provides the time at which the iterator was made.
creationTime time.Time
}
func (i levigoIterator) String() string {
valid := "valid"
open := "open"
snapshotted := "snapshotted"
if i.closed {
open = "closed"
}
if !i.valid {
valid = "invalid"
}
if i.snapshot == nil {
snapshotted = "unsnapshotted"
}
return fmt.Sprintf("levigoIterator created at %s that is %s and %s and %s", i.creationTime, open, valid, snapshotted)
}
func (i *levigoIterator) Close() error {
if i.closed {
return nil
}
if i.iterator != nil {
i.iterator.Close()
}
if i.readOptions != nil {
i.readOptions.Close()
}
if i.snapshot != nil {
i.storage.ReleaseSnapshot(i.snapshot)
}
// Explicitly dereference the pointers to prevent cycles, however unlikely.
i.iterator = nil
i.readOptions = nil
i.snapshot = nil
i.storage = nil
i.closed = true
i.valid = false
return nil
}
func (i *levigoIterator) Seek(m proto.Message) bool {
buf, _ := buffers.Get()
defer buffers.Give(buf)
if err := buf.Marshal(m); err != nil {
panic(err)
}
i.iterator.Seek(buf.Bytes())
i.valid = i.iterator.Valid()
return i.valid
}
func (i *levigoIterator) SeekToFirst() bool {
i.iterator.SeekToFirst()
i.valid = i.iterator.Valid()
return i.valid
}
func (i *levigoIterator) SeekToLast() bool {
i.iterator.SeekToLast()
i.valid = i.iterator.Valid()
return i.valid
}
func (i *levigoIterator) Next() bool {
i.iterator.Next()
i.valid = i.iterator.Valid()
return i.valid
}
func (i *levigoIterator) Previous() bool {
i.iterator.Prev()
i.valid = i.iterator.Valid()
return i.valid
}
func (i *levigoIterator) rawKey() (key []byte) {
return i.iterator.Key()
}
func (i *levigoIterator) Error() (err error) {
return i.iterator.GetError()
}
func (i *levigoIterator) Key(m proto.Message) error {
buf, _ := buffers.Get()
defer buffers.Give(buf)
buf.SetBuf(i.iterator.Key())
return buf.Unmarshal(m)
}
func (i *levigoIterator) RawValue() []byte {
return i.iterator.Value()
}
func (i *levigoIterator) Valid() bool {
return i.valid
}
// Compression defines the compression mode.
type Compression uint
// Possible compression modes.
const (
Snappy Compression = iota
Uncompressed
)
// LevelDBOptions bundles options needed to create a LevelDBPersistence object.
type LevelDBOptions struct {
Path string
Name string
Purpose string
CacheSizeBytes int
OpenFileAllowance int
FlushOnMutate bool
UseParanoidChecks bool
Compression Compression
}
// NewLevelDBPersistence returns an initialized LevelDBPersistence object,
// created with the given options.
func NewLevelDBPersistence(o LevelDBOptions) (*LevelDBPersistence, error) {
options := levigo.NewOptions()
options.SetCreateIfMissing(true)
options.SetParanoidChecks(o.UseParanoidChecks)
compression := levigo.SnappyCompression
if o.Compression == Uncompressed {
compression = levigo.NoCompression
}
options.SetCompression(compression)
cache := levigo.NewLRUCache(o.CacheSizeBytes)
options.SetCache(cache)
filterPolicy := levigo.NewBloomFilter(10)
options.SetFilterPolicy(filterPolicy)
options.SetMaxOpenFiles(o.OpenFileAllowance)
storage, err := levigo.Open(o.Path, options)
if err != nil {
return nil, err
}
readOptions := levigo.NewReadOptions()
writeOptions := levigo.NewWriteOptions()
writeOptions.SetSync(o.FlushOnMutate)
return &LevelDBPersistence{
path: o.Path,
name: o.Name,
purpose: o.Purpose,
cache: cache,
filterPolicy: filterPolicy,
options: options,
readOptions: readOptions,
writeOptions: writeOptions,
storage: storage,
}, nil
}
// Close implements raw.Persistence (and raw.Database).
func (l *LevelDBPersistence) Close() error {
// These are deferred to take advantage of forced closing in case of
// stack unwinding due to anomalies.
defer func() {
if l.filterPolicy != nil {
l.filterPolicy.Close()
}
}()
defer func() {
if l.cache != nil {
l.cache.Close()
}
}()
defer func() {
if l.options != nil {
l.options.Close()
}
}()
defer func() {
if l.readOptions != nil {
l.readOptions.Close()
}
}()
defer func() {
if l.writeOptions != nil {
l.writeOptions.Close()
}
}()
defer func() {
if l.storage != nil {
l.storage.Close()
}
}()
return nil
}
// Get implements raw.Persistence.
func (l *LevelDBPersistence) Get(k, v proto.Message) (bool, error) {
buf, _ := buffers.Get()
defer buffers.Give(buf)
if err := buf.Marshal(k); err != nil {
panic(err)
}
raw, err := l.storage.Get(l.readOptions, buf.Bytes())
if err != nil {
return false, err
}
if raw == nil {
return false, nil
}
if v == nil {
return true, nil
}
buf.SetBuf(raw)
if err := buf.Unmarshal(v); err != nil {
return true, err
}
return true, nil
}
// Has implements raw.Persistence.
func (l *LevelDBPersistence) Has(k proto.Message) (has bool, err error) {
return l.Get(k, nil)
}
// Drop implements raw.Persistence.
func (l *LevelDBPersistence) Drop(k proto.Message) error {
buf, _ := buffers.Get()
defer buffers.Give(buf)
if err := buf.Marshal(k); err != nil {
panic(err)
}
return l.storage.Delete(l.writeOptions, buf.Bytes())
}
// Put implements raw.Persistence.
func (l *LevelDBPersistence) Put(k, v proto.Message) error {
keyBuf, _ := buffers.Get()
defer buffers.Give(keyBuf)
if err := keyBuf.Marshal(k); err != nil {
panic(err)
}
valBuf, _ := buffers.Get()
defer buffers.Give(valBuf)
if err := valBuf.Marshal(v); err != nil {
panic(err)
}
return l.storage.Put(l.writeOptions, keyBuf.Bytes(), valBuf.Bytes())
}
// PutRaw implements raw.Persistence.
func (l *LevelDBPersistence) PutRaw(key proto.Message, value []byte) error {
keyBuf, _ := buffers.Get()
defer buffers.Give(keyBuf)
if err := keyBuf.Marshal(key); err != nil {
panic(err)
}
return l.storage.Put(l.writeOptions, keyBuf.Bytes(), value)
}
// Commit implements raw.Persistence.
func (l *LevelDBPersistence) Commit(b raw.Batch) (err error) {
// XXX: This is a wart to clean up later. Ideally, after doing
// extensive tests, we could create a Batch struct that journals pending
// operations which the given Persistence implementation could convert
// to its specific commit requirements.
batch, ok := b.(*batch)
if !ok {
panic("leveldb.batch expected")
}
return l.storage.Write(l.writeOptions, batch.batch)
}
// Prune implements raw.Pruner. It compacts the entire keyspace of the database.
//
// Beware that it would probably be imprudent to run this on a live user-facing
// server due to latency implications.
func (l *LevelDBPersistence) Prune() {
// Magic values per https://code.google.com/p/leveldb/source/browse/include/leveldb/db.h#131.
keyspace := levigo.Range{
Start: nil,
Limit: nil,
}
l.storage.CompactRange(keyspace)
}
// Size returns the approximate size the entire database takes on disk (in
// bytes). It implements the raw.Database interface.
func (l *LevelDBPersistence) Size() (uint64, error) {
iterator, err := l.NewIterator(false)
if err != nil {
return 0, err
}
defer iterator.Close()
if !iterator.SeekToFirst() {
return 0, fmt.Errorf("could not seek to first key")
}
keyspace := levigo.Range{}
keyspace.Start = iterator.rawKey()
if !iterator.SeekToLast() {
return 0, fmt.Errorf("could not seek to last key")
}
keyspace.Limit = iterator.rawKey()
sizes := l.storage.GetApproximateSizes([]levigo.Range{keyspace})
total := uint64(0)
for _, size := range sizes {
total += size
}
return total, nil
}
// NewIterator creates a new levigoIterator, which follows the Iterator
// interface.
//
// Important notes:
//
// For each of the iterator methods that have a return signature of (ok bool),
// if ok == false, the iterator may not be used any further and must be closed.
// Further work with the database requires the creation of a new iterator. This
// is due to LevelDB and Levigo design. Please refer to Jeff and Sanjay's notes
// in the LevelDB documentation for this behavior's rationale.
//
// The returned iterator must explicitly be closed; otherwise non-managed memory
// will be leaked.
//
// The iterator is optionally snapshotable.
func (l *LevelDBPersistence) NewIterator(snapshotted bool) (Iterator, error) {
var (
snapshot *levigo.Snapshot
readOptions *levigo.ReadOptions
iterator *levigo.Iterator
)
if snapshotted {
snapshot = l.storage.NewSnapshot()
readOptions = levigo.NewReadOptions()
readOptions.SetSnapshot(snapshot)
iterator = l.storage.NewIterator(readOptions)
} else {
iterator = l.storage.NewIterator(l.readOptions)
}
return &levigoIterator{
creationTime: time.Now(),
iterator: iterator,
readOptions: readOptions,
snapshot: snapshot,
storage: l.storage,
}, nil
}
// ForEach implements raw.ForEacher.
func (l *LevelDBPersistence) ForEach(decoder storage.RecordDecoder, filter storage.RecordFilter, operator storage.RecordOperator) (scannedEntireCorpus bool, err error) {
iterator, err := l.NewIterator(true)
if err != nil {
return false, err
}
defer iterator.Close()
for valid := iterator.SeekToFirst(); valid; valid = iterator.Next() {
if err = iterator.Error(); err != nil {
return false, err
}
decodedKey, decodeErr := decoder.DecodeKey(iterator.rawKey())
if decodeErr != nil {
continue
}
decodedValue, decodeErr := decoder.DecodeValue(iterator.RawValue())
if decodeErr != nil {
continue
}
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
}
}
}
return true, nil
}

View file

@ -1,49 +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/raw"
"github.com/prometheus/prometheus/utility"
)
const (
statsKey = "leveldb.stats"
sstablesKey = "leveldb.sstables"
)
// State returns the DatabaseState. It implements the raw.Database interface and
// sets the following Supplemental entries:
// "Low Level": leveldb property value for "leveldb.stats"
// "SSTable": leveldb property value for "leveldb.sstables"
// "Errors": only set if an error has occurred determining the size
func (l *LevelDBPersistence) State() *raw.DatabaseState {
databaseState := &raw.DatabaseState{
Location: l.path,
Name: l.name,
Purpose: l.purpose,
Supplemental: map[string]string{},
}
if size, err := l.Size(); err != nil {
databaseState.Supplemental["Errors"] = err.Error()
} else {
databaseState.Size = utility.ByteSize(size)
}
databaseState.Supplemental["Low Level"] = l.storage.PropertyValue(statsKey)
databaseState.Supplemental["SSTable"] = l.storage.PropertyValue(sstablesKey)
return databaseState
}

View file

@ -1,124 +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 test
import (
"testing"
"code.google.com/p/goprotobuf/proto"
"github.com/prometheus/prometheus/storage/raw/leveldb"
"github.com/prometheus/prometheus/utility/test"
)
const cacheCapacity = 0
type (
// Pair models a prospective (key, value) double that will be committed
// to a database.
Pair interface {
Get() (key proto.Message, value interface{})
}
// Pairs models a list of Pair for disk committing.
Pairs []Pair
// Preparer readies a LevelDB store for a given raw state given the
// fixtures definitions passed into it.
Preparer interface {
// Prepare furnishes the database and returns its path along
// with any encountered anomalies.
Prepare(namespace string, f FixtureFactory) test.TemporaryDirectory
}
// FixtureFactory is an iterator emitting fixture data.
FixtureFactory interface {
// HasNext indicates whether the FixtureFactory has more pending
// fixture data to build.
HasNext() (has bool)
// Next emits the next (key, value) double for storage.
Next() (key proto.Message, value interface{})
}
preparer struct {
tester testing.TB
}
cassetteFactory struct {
index int
count int
pairs Pairs
}
)
func (p preparer) Prepare(n string, f FixtureFactory) (t test.TemporaryDirectory) {
t = test.NewTemporaryDirectory(n, p.tester)
persistence, err := leveldb.NewLevelDBPersistence(leveldb.LevelDBOptions{
Path: t.Path(),
CacheSizeBytes: cacheCapacity,
})
if err != nil {
defer t.Close()
p.tester.Fatal(err)
}
defer persistence.Close()
for f.HasNext() {
key, value := f.Next()
switch v := value.(type) {
case proto.Message:
err = persistence.Put(key, v)
case []byte:
err = persistence.PutRaw(key, v)
default:
panic("illegal value type")
}
if err != nil {
defer t.Close()
p.tester.Fatal(err)
}
}
return
}
// HasNext implements FixtureFactory.
func (f cassetteFactory) HasNext() bool {
return f.index < f.count
}
// Next implements FixtureFactory.
func (f *cassetteFactory) Next() (key proto.Message, value interface{}) {
key, value = f.pairs[f.index].Get()
f.index++
return
}
// NewPreparer creates a new Preparer for use in testing scenarios.
func NewPreparer(t testing.TB) Preparer {
return preparer{t}
}
// NewCassetteFactory builds a new FixtureFactory that uses Pairs as the basis
// for generated fixture data.
func NewCassetteFactory(pairs Pairs) FixtureFactory {
return &cassetteFactory{
pairs: pairs,
count: len(pairs),
}
}

View file

@ -1,60 +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 raw
import (
"github.com/prometheus/prometheus/utility"
)
// DatabaseState contains some fundamental attributes of a database.
type DatabaseState struct {
Name string
Size utility.ByteSize
Location string
Purpose string
Supplemental map[string]string
}
// DatabaseStates is a sortable slice of DatabaseState pointers. It implements
// sort.Interface.
type DatabaseStates []*DatabaseState
// Len implements sort.Interface.
func (s DatabaseStates) Len() int {
return len(s)
}
// Less implements sort.Interface. The primary sorting criterion is the Name,
// the secondary criterion is the Size.
func (s DatabaseStates) Less(i, j int) bool {
l := s[i]
r := s[j]
if l.Name > r.Name {
return false
}
if l.Name < r.Name {
return true
}
return l.Size < r.Size
}
// Swap implements sort.Interface.
func (s DatabaseStates) Swap(i, j int) {
s[i], s[j] = s[j], s[i]
}

View file

@ -30,7 +30,7 @@ import (
"github.com/prometheus/prometheus/rules"
"github.com/prometheus/prometheus/rules/ast"
"github.com/prometheus/prometheus/stats"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/local"
)
// A version of vector that's easier to use from templates.
@ -57,7 +57,7 @@ func (q queryResultByLabelSorter) Swap(i, j int) {
q.results[i], q.results[j] = q.results[j], q.results[i]
}
func query(q string, timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (queryResult, error) {
func query(q string, timestamp clientmodel.Timestamp, storage storage_ng.Storage) (queryResult, error) {
exprNode, err := rules.LoadExprFromString(q)
if err != nil {
return nil, err
@ -91,7 +91,7 @@ type templateExpander struct {
funcMap text_template.FuncMap
}
func NewTemplateExpander(text string, name string, data interface{}, timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) *templateExpander {
func NewTemplateExpander(text string, name string, data interface{}, timestamp clientmodel.Timestamp, storage storage_ng.Storage) *templateExpander {
return &templateExpander{
text: text,
name: name,

View file

@ -18,7 +18,7 @@ import (
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric/tiered"
"github.com/prometheus/prometheus/storage/local"
)
type testTemplatesScenario struct {
@ -152,8 +152,9 @@ func TestTemplateExpansion(t *testing.T) {
time := clientmodel.Timestamp(0)
ts, _ := tiered.NewTestTieredStorage(t)
ts.AppendSamples(clientmodel.Samples{
storage, closer := storage_ng.NewTestStorage(t)
defer closer.Close()
storage.AppendSamples(clientmodel.Samples{
{
Metric: clientmodel.Metric{
clientmodel.MetricNameLabel: "metric",
@ -171,7 +172,7 @@ func TestTemplateExpansion(t *testing.T) {
for _, s := range scenarios {
var result string
var err error
expander := NewTemplateExpander(s.text, "test", s.input, time, ts)
expander := NewTemplateExpander(s.text, "test", s.input, time, storage)
if s.html {
result, err = expander.ExpandHTML(nil)
} else {

View file

@ -11,7 +11,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
all: dumper pruner rule_checker
all: dumper rule_checker
SUFFIXES:
@ -20,15 +20,11 @@ include ../Makefile.INCLUDE
dumper:
$(MAKE) -C dumper
pruner:
$(MAKE) -C pruner
rule_checker:
$(MAKE) -C rule_checker
clean:
$(MAKE) -C dumper clean
$(MAKE) -C pruner clean
$(MAKE) -C rule_checker clean
.PHONY: clean dumper pruner rule_checker
.PHONY: clean dumper rule_checker

View file

@ -18,6 +18,7 @@
// <fingerprint>,<chunk_first_time>,<chunk_last_time>,<chunk_sample_count>,<chunk_index>,<timestamp>,<value>
package main
/*
import (
"encoding/csv"
"flag"
@ -98,3 +99,7 @@ func main() {
glog.Fatal("Error flushing CSV: ", err)
}
}
*/
func main() {
}

View file

@ -1 +0,0 @@
pruner

View file

@ -1,28 +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.
MAKE_ARTIFACTS = pruner
all: pruner
SUFFIXES:
include ../../Makefile.INCLUDE
pruner: $(shell find . -iname '*.go')
$(GO) build -o pruner .
clean:
rm -rf $(MAKE_ARTIFACTS)
.PHONY: clean

View file

@ -1,52 +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.
// Pruner is responsible for cleaning all Prometheus disk databases, which
// minimally includes 1. applying pending commit logs, 2. compacting SSTables,
// 3. purging stale SSTables, and 4. removing old tombstones.
package main
import (
"flag"
"github.com/prometheus/prometheus/storage/metric/tiered"
"time"
"github.com/golang/glog"
)
var (
storageRoot = flag.String("storage.root", "", "The path to the storage root for Prometheus.")
)
func main() {
flag.Parse()
if storageRoot == nil || *storageRoot == "" {
glog.Fatal("Must provide a path...")
}
persistences, err := tiered.NewLevelDBPersistence(*storageRoot)
if err != nil {
glog.Fatal(err)
}
defer persistences.Close()
start := time.Now()
glog.Info("Starting compaction...")
size, _ := persistences.Sizes()
glog.Info("Original Size: ", size)
persistences.Prune()
glog.Info("Finished in ", time.Since(start))
size, _ = persistences.Sizes()
glog.Info("New Size: ", size)
}

View file

@ -20,7 +20,7 @@ import (
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/retrieval"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/local"
"github.com/prometheus/prometheus/utility"
"github.com/prometheus/prometheus/web/http_utils"
)
@ -29,7 +29,7 @@ type MetricsService struct {
time utility.Time
Config *config.Config
TargetManager retrieval.TargetManager
Storage metric.PreloadingPersistence
Storage storage_ng.Storage
}
func (msrv *MetricsService) RegisterHandler() {

View file

@ -137,13 +137,7 @@ func (serv MetricsService) QueryRange(w http.ResponseWriter, r *http.Request) {
func (serv MetricsService) Metrics(w http.ResponseWriter, r *http.Request) {
setAccessControlHeaders(w)
metricNames, err := serv.Storage.GetAllValuesForLabel(clientmodel.MetricNameLabel)
w.Header().Set("Content-Type", "application/json")
if err != nil {
glog.Error("Error loading metric names: ", err)
http.Error(w, err.Error(), http.StatusInternalServerError)
return
}
metricNames := serv.Storage.GetAllValuesForLabel(clientmodel.MetricNameLabel)
sort.Sort(metricNames)
resultBytes, err := json.Marshal(metricNames)
if err != nil {
@ -151,5 +145,6 @@ func (serv MetricsService) Metrics(w http.ResponseWriter, r *http.Request) {
http.Error(w, err.Error(), http.StatusInternalServerError)
return
}
w.Header().Set("Content-Type", "application/json")
w.Write(resultBytes)
}

View file

@ -22,7 +22,7 @@ import (
"path/filepath"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/storage/local"
"github.com/prometheus/prometheus/templates"
)
@ -32,7 +32,7 @@ var (
)
type ConsolesHandler struct {
Storage metric.PreloadingPersistence
Storage storage_ng.Storage
}
func (h *ConsolesHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {

View file

@ -1,60 +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 web
import (
"net/http"
"sync"
"time"
"github.com/prometheus/prometheus/storage/raw"
)
type DatabaseStatesProvider interface {
States() raw.DatabaseStates
}
type DatabasesHandler struct {
RefreshInterval time.Duration
NextRefresh time.Time
Current raw.DatabaseStates
Provider DatabaseStatesProvider
mutex sync.RWMutex
}
func (h *DatabasesHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
h.Refresh()
h.mutex.RLock()
defer h.mutex.RUnlock()
executeTemplate(w, "databases", h)
}
func (h *DatabasesHandler) Refresh() {
h.mutex.RLock()
if !time.Now().After(h.NextRefresh) {
h.mutex.RUnlock()
return
}
h.mutex.RUnlock()
h.mutex.Lock()
defer h.mutex.Unlock()
h.Current = h.Provider.States()
h.NextRefresh = time.Now().Add(h.RefreshInterval)
}

View file

@ -20,7 +20,6 @@ import (
"github.com/prometheus/prometheus/retrieval"
"github.com/prometheus/prometheus/rules/manager"
"github.com/prometheus/prometheus/storage/metric"
)
type PrometheusStatusHandler struct {
@ -28,7 +27,6 @@ type PrometheusStatusHandler struct {
BuildInfo map[string]string
Config string
Curation metric.CurationState
Flags map[string]string
RuleManager manager.RuleManager
TargetPools map[string]*retrieval.TargetPool
@ -36,16 +34,6 @@ type PrometheusStatusHandler struct {
Birth time.Time
}
func (h *PrometheusStatusHandler) UpdateCurationState(c *metric.CurationState) {
h.mu.Lock()
defer h.mu.Unlock()
h.Curation = *c
}
func (h *PrometheusStatusHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
h.mu.RLock()
defer h.mu.RUnlock()
executeTemplate(w, "status", h)
}

View file

@ -1,32 +0,0 @@
{{define "head"}}<!-- nix -->{{end}}
{{define "content"}}
<div class="container-fluid">
<h2>Database Information</h2>
{{range $database := .Current }}
<h3>{{$database.Name}}</h3>
<table class="table table-bordered table-condensed table-hover literal_output">
<tbody>
<tr>
<th>Location</th>
<td>{{$database.Location}}</td>
</tr>
<tr>
<th>Purpose</th>
<td>{{$database.Purpose}}</td>
</tr>
<tr>
<th>Size</th>
<td>{{$database.Size}}</td>
</tr>
{{range $subject, $state := $database.Supplemental }}
<tr>
<th>{{$subject}}</th>
<td><pre>{{$state}}</pre></td>
</tr>
{{end}}
</tbody>
</table>
{{end}}
</div>
{{end}}

View file

@ -69,28 +69,6 @@
</table>
{{end}}
<h2>Curation</h2>
<table class="table table-condensed table-bordered table-striped table-hover">
<tr>
<th>Active</th>
<td>{{.Curation.Active}}</td>
</tr>
{{if .Curation.Active}}
<tr>
<th>Processor Name</th>
<td>{{.Curation.Name}}</td>
</tr>
<tr>
<th>Recency Limit</th>
<td>{{.Curation.Limit}}</td>
</tr>
<tr>
<th>Current Fingerprint</th>
<td>{{.Curation.Fingerprint}}</td>
</tr>
{{end}}
</table>
<h2>Startup Flags</h2>
<table class="table table-condensed table-bordered table-striped table-hover">
<tbody>

View file

@ -42,11 +42,10 @@ var (
)
type WebService struct {
StatusHandler *PrometheusStatusHandler
DatabasesHandler *DatabasesHandler
MetricsHandler *api.MetricsService
AlertsHandler *AlertsHandler
ConsolesHandler *ConsolesHandler
StatusHandler *PrometheusStatusHandler
MetricsHandler *api.MetricsService
AlertsHandler *AlertsHandler
ConsolesHandler *ConsolesHandler
QuitDelegate func()
}
@ -59,9 +58,6 @@ func (w WebService) ServeForever() error {
http.Handle("/", prometheus.InstrumentHandler(
"/", w.StatusHandler,
))
http.Handle("/databases", prometheus.InstrumentHandler(
"/databases", w.DatabasesHandler,
))
http.Handle("/alerts", prometheus.InstrumentHandler(
"/alerts", w.AlertsHandler,
))