mirror of
https://github.com/prometheus/prometheus.git
synced 2024-12-24 05:04:05 -08:00
Initial experimental snapshot of next-gen storage.
Change-Id: Ifb8709960dbedd1d9f5efd88cdd359ee9fa9d26d
This commit is contained in:
parent
134bd8fe34
commit
e7ed39c9a6
202
main.go
202
main.go
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
222
rules/ast/ast.go
222
rules/ast/ast.go
|
@ -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{},
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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)))
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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
67
storage/local/chunk.go
Normal 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
399
storage/local/delta.go
Normal 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
44
storage/local/freelist.go
Normal 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()
|
||||
}
|
88
storage/local/instrumentation.go
Normal file
88
storage/local/instrumentation.go
Normal 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))
|
||||
}
|
98
storage/local/interface.go
Normal file
98
storage/local/interface.go
Normal 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
|
||||
}
|
388
storage/local/persistence.go
Normal file
388
storage/local/persistence.go
Normal 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
|
||||
}
|
107
storage/local/persistence_test.go
Normal file
107
storage/local/persistence_test.go
Normal 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
102
storage/local/preload.go
Normal 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
453
storage/local/series.go
Normal 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
503
storage/local/storage.go
Normal 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
|
||||
}
|
112
storage/local/storage_test.go
Normal file
112
storage/local/storage_test.go
Normal 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)
|
||||
}
|
47
storage/local/test_helpers.go
Normal file
47
storage/local/test_helpers.go
Normal 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
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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())
|
||||
}
|
|
@ -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))
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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),
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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{})
|
||||
}
|
|
@ -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))
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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,
|
||||
}
|
||||
}
|
|
@ -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})
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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]
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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)),
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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
|
@ -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")
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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),
|
||||
}
|
||||
}
|
|
@ -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{}
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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),
|
||||
}
|
||||
}
|
|
@ -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]
|
||||
}
|
|
@ -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,
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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() {
|
||||
}
|
||||
|
|
1
tools/pruner/.gitignore
vendored
1
tools/pruner/.gitignore
vendored
|
@ -1 +0,0 @@
|
|||
pruner
|
|
@ -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
|
|
@ -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)
|
||||
}
|
|
@ -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() {
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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}}
|
|
@ -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>
|
||||
|
|
12
web/web.go
12
web/web.go
|
@ -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,
|
||||
))
|
||||
|
|
Loading…
Reference in a new issue